From 9d0ad7ba67b6855344512398b5f924bdad4ece9e Mon Sep 17 00:00:00 2001 From: copperybean Date: Sun, 14 Jan 2024 11:25:12 +0800 Subject: [PATCH 01/36] original parquet reader Change-Id: I83a8ec8271edefcd96cb5b3bcd12f6b545d9dec0 --- .../Impl/Parquet/ParquetColumnReader.h | 29 + .../Formats/Impl/Parquet/ParquetDataBuffer.h | 179 ++++++ .../Impl/Parquet/ParquetDataValuesReader.cpp | 553 ++++++++++++++++++ .../Impl/Parquet/ParquetDataValuesReader.h | 263 +++++++++ .../Impl/Parquet/ParquetLeafColReader.cpp | 506 ++++++++++++++++ .../Impl/Parquet/ParquetLeafColReader.h | 63 ++ .../Impl/Parquet/ParquetRecordReader.cpp | 225 +++++++ .../Impl/Parquet/ParquetRecordReader.h | 48 ++ 8 files changed, 1866 insertions(+) create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp create mode 100644 src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h diff --git a/src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h b/src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h new file mode 100644 index 00000000000..cfd9d3ba5bd --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h @@ -0,0 +1,29 @@ +#pragma once + +#include + +namespace parquet +{ + +class PageReader; +class ColumnChunkMetaData; +class DataPageV1; +class DataPageV2; + +} + +namespace DB +{ + +class ParquetColumnReader +{ +public: + virtual ColumnWithTypeAndName readBatch(UInt32 rows_num, const String & name) = 0; + + virtual ~ParquetColumnReader() = default; +}; + +using ParquetColReaderPtr = std::unique_ptr; +using ParquetColReaders = std::vector; + +} diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h new file mode 100644 index 00000000000..1f83c74f9ad --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h @@ -0,0 +1,179 @@ +#pragma once + +#include + +#include +#include +#include + +namespace DB +{ + +template struct ToArrowDecimal; + +template <> struct ToArrowDecimal>> +{ + using ArrowDecimal = arrow::Decimal128; +}; + +template <> struct ToArrowDecimal>> +{ + using ArrowDecimal = arrow::Decimal256; +}; + + +class ParquetDataBuffer +{ +private: + +public: + ParquetDataBuffer(const uint8_t * data_, UInt64 avaible_, UInt8 datetime64_scale_ = DataTypeDateTime64::default_scale) + : data(reinterpret_cast(data_)), avaible(avaible_), datetime64_scale(datetime64_scale_) {} + + template + void ALWAYS_INLINE readValue(TValue & dst) + { + checkAvaible(sizeof(TValue)); + dst = *reinterpret_cast(data); + consume(sizeof(TValue)); + } + + void ALWAYS_INLINE readBytes(void * dst, size_t bytes) + { + checkAvaible(bytes); + memcpy(dst, data, bytes); + consume(bytes); + } + + void ALWAYS_INLINE readDateTime64(DateTime64 & dst) + { + static const int max_scale_num = 9; + static const UInt64 pow10[max_scale_num + 1] + = {1000000000, 100000000, 10000000, 1000000, 100000, 10000, 1000, 100, 10, 1}; + static const UInt64 spd = 60 * 60 * 24; + static const UInt64 scaled_day[max_scale_num + 1] + = {spd, + 10 * spd, + 100 * spd, + 1000 * spd, + 10000 * spd, + 100000 * spd, + 1000000 * spd, + 10000000 * spd, + 100000000 * spd, + 1000000000 * spd}; + + checkAvaible(sizeof(parquet::Int96)); + auto decoded = parquet::DecodeInt96Timestamp(*reinterpret_cast(data)); + + uint64_t scaled_nano = decoded.nanoseconds / pow10[datetime64_scale]; + dst = static_cast(decoded.days_since_epoch * scaled_day[datetime64_scale] + scaled_nano); + + consume(sizeof(parquet::Int96)); + } + + /** + * This method should only be used to read string whose elements size is small. + * Because memcpySmallAllowReadWriteOverflow15 instead of memcpy is used according to ColumnString::indexImpl + */ + void ALWAYS_INLINE readString(ColumnString & column, size_t cursor) + { + // refer to: PlainByteArrayDecoder::DecodeArrowDense in encoding.cc + // deserializeBinarySSE2 in SerializationString.cpp + checkAvaible(4); + auto value_len = ::arrow::util::SafeLoadAs(getArrowData()); + if (unlikely(value_len < 0 || value_len > INT32_MAX - 4)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid or corrupted value_len '{}'", value_len); + } + consume(4); + checkAvaible(value_len); + + auto chars_cursor = column.getChars().size(); + column.getChars().resize(chars_cursor + value_len + 1); + + memcpySmallAllowReadWriteOverflow15(&column.getChars()[chars_cursor], data, value_len); + column.getChars().back() = 0; + + column.getOffsets().data()[cursor] = column.getChars().size(); + consume(value_len); + } + + template + void ALWAYS_INLINE readOverBigDecimal(TDecimal * out, Int32 elem_bytes_num) + { + using TArrowDecimal = typename ToArrowDecimal::ArrowDecimal; + + checkAvaible(elem_bytes_num); + + // refer to: RawBytesToDecimalBytes in reader_internal.cc, Decimal128::FromBigEndian in decimal.cc + auto status = TArrowDecimal::FromBigEndian(getArrowData(), elem_bytes_num); + if (unlikely(!status.ok())) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Read parquet decimal failed: {}", status.status().ToString()); + } + status.ValueUnsafe().ToBytes(reinterpret_cast(out)); + consume(elem_bytes_num); + } + +private: + const Int8 * data; + UInt64 avaible; + const UInt8 datetime64_scale; + + void ALWAYS_INLINE checkAvaible(UInt64 num) + { + if (unlikely(avaible < num)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Consuming {} bytes while {} avaible", num, avaible); + } + } + + const uint8_t * ALWAYS_INLINE getArrowData() { return reinterpret_cast(data); } + + void ALWAYS_INLINE consume(UInt64 num) + { + data += num; + avaible -= num; + } +}; + + +class LazyNullMap +{ +public: + LazyNullMap(UInt32 size_) : size(size_), col_nullable(nullptr) {} + + void setNull(UInt32 cursor) + { + initialize(); + null_map[cursor] = 1; + } + + void setNull(UInt32 cursor, UInt32 count) + { + initialize(); + memset(null_map + cursor, 1, count); + } + + ColumnPtr getNullableCol() { return col_nullable; } + +private: + UInt32 size; + UInt8 * null_map; + ColumnPtr col_nullable; + + void initialize() + { + if (likely(col_nullable)) + { + return; + } + auto col = ColumnVector::create(size); + null_map = col->getData().data(); + col_nullable = std::move(col); + memset(null_map, 0, size); + } +}; + +} diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp new file mode 100644 index 00000000000..659a7a11969 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -0,0 +1,553 @@ +#include "ParquetDataValuesReader.h" + +#include +#include + +#include + +namespace DB +{ + +void RleValuesReader::nextGroup() +{ + // refer to: + // RleDecoder::NextCounts in rle_encoding.h and VectorizedRleValuesReader::readNextGroup in Spark + UInt32 indicator_value = 0; + [[maybe_unused]] auto read_res = bit_reader->GetVlqInt(&indicator_value); + assert(read_res); + + cur_group_is_packed = indicator_value & 1; + cur_group_size = indicator_value >> 1; + + if (cur_group_is_packed) + { + cur_group_size *= 8; + cur_packed_bit_values.resize(cur_group_size); + bit_reader->GetBatch(bit_width, cur_packed_bit_values.data(), cur_group_size); + } + else + { + cur_value = 0; + read_res = bit_reader->GetAligned((bit_width + 7) / 8, &cur_value); + assert(read_res); + } + cur_group_cursor = 0; + +} + +template +void RleValuesReader::visitValues( + UInt32 num_values, IndividualVisitor && individual_visitor, RepeatedVisitor && repeated_visitor) +{ + // refer to: VisitNullBitmapInline in visitor_inline.h + while (num_values) + { + nextGroupIfNecessary(); + auto cur_count = std::min(num_values, curGroupLeft()); + + if (cur_group_is_packed) + { + for (auto i = cur_group_cursor; i < cur_group_cursor + cur_count; i++) + { + individual_visitor(cur_packed_bit_values[i]); + } + } + else + { + repeated_visitor(cur_count, cur_value); + } + cur_group_cursor += cur_count; + num_values -= cur_count; + } +} + +template +void RleValuesReader::visitNullableValues( + size_t cursor, + UInt32 num_values, + Int32 max_def_level, + LazyNullMap & null_map, + IndividualVisitor && individual_visitor, + RepeatedVisitor && repeated_visitor) +{ + while (num_values) + { + nextGroupIfNecessary(); + auto cur_count = std::min(num_values, curGroupLeft()); + + if (cur_group_is_packed) + { + for (auto i = cur_group_cursor; i < cur_group_cursor + cur_count; i++) + { + if (cur_packed_bit_values[i] == max_def_level) + { + individual_visitor(cursor); + } + else + { + null_map.setNull(cursor); + } + cursor++; + } + } + else + { + if (cur_value == max_def_level) + { + repeated_visitor(cursor, cur_count); + } + else + { + null_map.setNull(cursor, cur_count); + } + cursor += cur_count; + } + cur_group_cursor += cur_count; + num_values -= cur_count; + } +} + +template +void RleValuesReader::visitNullableBySteps( + size_t cursor, + UInt32 num_values, + Int32 max_def_level, + IndividualNullVisitor && individual_null_visitor, + SteppedValidVisitor && stepped_valid_visitor, + RepeatedVisitor && repeated_visitor) +{ + // refer to: + // RleDecoder::GetBatch in rle_encoding.h and TypedColumnReaderImpl::ReadBatchSpaced in column_reader.cc + // VectorizedRleValuesReader::readBatchInternal in Spark + while (num_values > 0) + { + nextGroupIfNecessary(); + auto cur_count = std::min(num_values, curGroupLeft()); + + if (cur_group_is_packed) + { + valid_index_steps.resize(cur_count + 1); + valid_index_steps[0] = 0; + auto step_idx = 0; + auto null_map_cursor = cursor; + + for (auto i = cur_group_cursor; i < cur_group_cursor + cur_count; i++) + { + if (cur_packed_bit_values[i] == max_def_level) + { + valid_index_steps[++step_idx] = 1; + } + else + { + individual_null_visitor(null_map_cursor); + if (unlikely(valid_index_steps[step_idx] == UINT8_MAX)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "unsupported packed values number"); + } + valid_index_steps[step_idx]++; + } + null_map_cursor++; + } + valid_index_steps.resize(step_idx + 1); + stepped_valid_visitor(cursor, valid_index_steps); + } + else + { + repeated_visitor(cur_value == max_def_level, cursor, cur_count); + } + + cursor += cur_count; + cur_group_cursor += cur_count; + num_values -= cur_count; + } +} + +template +void RleValuesReader::setValues(TValue * res_values, UInt32 num_values, ValueGetter && val_getter) +{ + visitValues( + num_values, + /* individual_visitor */ [&](Int32 val) + { + *(res_values++) = val_getter(val); + }, + /* repeated_visitor */ [&](UInt32 count, Int32 val) + { + std::fill(res_values, res_values + count, val_getter(val)); + res_values += count; + } + ); +} + +template +void RleValuesReader::setValueBySteps( + TValue * res_values, + const std::vector & col_data_steps, + ValueGetter && val_getter) +{ + auto step_iterator = col_data_steps.begin(); + res_values += *(step_iterator++); + + visitValues( + col_data_steps.size() - 1, + /* individual_visitor */ [&](Int32 val) + { + *res_values = val_getter(val); + res_values += *(step_iterator++); + }, + /* repeated_visitor */ [&](UInt32 count, Int32 val) + { + auto getted_val = val_getter(val); + for (UInt32 i = 0; i < count; i++) + { + *res_values = getted_val; + res_values += *(step_iterator++); + } + } + ); +} + + +namespace +{ + +template +TValue * getResizedPrimitiveData(TColumn & column, size_t size) +{ + auto old_size = column.size(); + column.getData().resize(size); + memset(column.getData().data() + old_size, 0, sizeof(TValue) * (size - old_size)); + return column.getData().data(); +} + +} // anoynomous namespace + + +template <> +void ParquetPlainValuesReader::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto & column = *assert_cast(col_ptr.get()); + auto cursor = column.size(); + + column.getOffsets().resize(cursor + num_values); + auto * offset_data = column.getOffsets().data(); + auto & chars = column.getChars(); + + def_level_reader->visitValues( + num_values, + /* individual_visitor */ [&](Int32 val) + { + if (val == max_def_level) + { + plain_data_buffer.readString(column, cursor); + } + else + { + chars.push_back(0); + offset_data[cursor] = chars.size(); + null_map.setNull(cursor); + } + cursor++; + }, + /* repeated_visitor */ [&](UInt32 count, Int32 val) + { + if (val == max_def_level) + { + for (UInt32 i = 0; i < count; i++) + { + plain_data_buffer.readString(column, cursor); + cursor++; + } + } + else + { + null_map.setNull(cursor, count); + + auto chars_size_bak = chars.size(); + chars.resize(chars_size_bak + count); + memset(&chars[chars_size_bak], 0, count); + + auto idx = cursor; + cursor += count; + // the type of offset_data is PaddedPODArray, which makes sure that the -1 index is avaible + for (auto val_offset = offset_data[idx - 1]; idx < cursor; idx++) + { + offset_data[idx] = ++val_offset; + } + } + } + ); +} + + +template <> +void ParquetPlainValuesReader>::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto cursor = col_ptr->size(); + auto * column_data = getResizedPrimitiveData( + *assert_cast *>(col_ptr.get()), cursor + num_values); + + def_level_reader->visitNullableValues( + cursor, + num_values, + max_def_level, + null_map, + /* individual_visitor */ [&](size_t nest_cursor) + { + plain_data_buffer.readDateTime64(column_data[nest_cursor]); + }, + /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) + { + auto col_data_pos = column_data + nest_cursor; + for (UInt32 i = 0; i < count; i++) + { + plain_data_buffer.readDateTime64(col_data_pos[i]); + } + } + ); +} + +template +void ParquetPlainValuesReader::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto cursor = col_ptr->size(); + auto * column_data = getResizedPrimitiveData(*assert_cast(col_ptr.get()), cursor + num_values); + using TValue = std::decay_t; + + def_level_reader->visitNullableValues( + cursor, + num_values, + max_def_level, + null_map, + /* individual_visitor */ [&](size_t nest_cursor) + { + plain_data_buffer.readValue(column_data[nest_cursor]); + }, + /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) + { + plain_data_buffer.readBytes(column_data + nest_cursor, count * sizeof(TValue)); + } + ); +} + + +template +void ParquetFixedLenPlainReader::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + if constexpr (std::same_as> || std::same_as>) + { + readOverBigDecimal(col_ptr, null_map, num_values); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "unsupported type"); + } +} + +template +void ParquetFixedLenPlainReader::readOverBigDecimal( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto cursor = col_ptr->size(); + auto * column_data = getResizedPrimitiveData( + *assert_cast(col_ptr.get()), cursor + num_values); + + def_level_reader->visitNullableValues( + cursor, + num_values, + max_def_level, + null_map, + /* individual_visitor */ [&](size_t nest_cursor) + { + plain_data_buffer.readOverBigDecimal(column_data + nest_cursor, elem_bytes_num); + }, + /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) + { + auto col_data_pos = column_data + nest_cursor; + for (UInt32 i = 0; i < count; i++) + { + plain_data_buffer.readOverBigDecimal(col_data_pos + i, elem_bytes_num); + } + } + ); +} + + +template +void ParquetRleLCReader::readBatch( + MutableColumnPtr & index_col, LazyNullMap & null_map, UInt32 num_values) +{ + auto cursor = index_col->size(); + auto * column_data = getResizedPrimitiveData(*assert_cast(index_col.get()), cursor + num_values); + + bool has_null = false; + + // in ColumnLowCardinality, first element in dictionary is null + // so we should increase each value by 1 in parquet index + auto val_getter = [&](Int32 val) { return val + 1; }; + + def_level_reader->visitNullableBySteps( + cursor, + num_values, + max_def_level, + /* individual_null_visitor */ [&](UInt32 nest_cursor) { + column_data[nest_cursor] = 0; + has_null = true; + }, + /* stepped_valid_visitor */ [&](UInt32 nest_cursor, const std::vector & valid_index_steps) { + rle_data_reader->setValueBySteps(column_data + nest_cursor, valid_index_steps, val_getter); + }, + /* repeated_visitor */ [&](bool is_valid, UInt32 nest_cursor, UInt32 count) { + if (is_valid) + { + rle_data_reader->setValues(column_data + nest_cursor, count, val_getter); + } + else + { + auto data_pos = column_data + nest_cursor; + std::fill(data_pos, data_pos + count, 0); + has_null = true; + } + } + ); + if (has_null) + { + null_map.setNull(0); + } +} + +template <> +void ParquetRleDictReader::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto & column = *assert_cast(col_ptr.get()); + auto cursor = column.size(); + std::vector value_cache; + + const auto & dict_chars = static_cast(page_dictionary).getChars(); + const auto & dict_offsets = static_cast(page_dictionary).getOffsets(); + + column.getOffsets().resize(cursor + num_values); + auto * offset_data = column.getOffsets().data(); + auto & chars = column.getChars(); + + auto append_nulls = [&](UInt8 num) { + for (auto limit = cursor + num; cursor < limit; cursor++) + { + chars.push_back(0); + offset_data[cursor] = chars.size(); + null_map.setNull(cursor); + } + }; + + auto append_string = [&](Int32 dict_idx) { + auto dict_chars_cursor = dict_offsets[dict_idx - 1]; + auto value_len = dict_offsets[dict_idx] - dict_chars_cursor; + auto chars_cursor = chars.size(); + chars.resize(chars_cursor + value_len); + + memcpySmallAllowReadWriteOverflow15(&chars[chars_cursor], &dict_chars[dict_chars_cursor], value_len); + offset_data[cursor] = chars.size(); + cursor++; + }; + + auto val_getter = [&](Int32 val) { return val + 1; }; + + def_level_reader->visitNullableBySteps( + cursor, + num_values, + max_def_level, + /* individual_null_visitor */ [&](UInt32) {}, + /* stepped_valid_visitor */ [&](UInt32, const std::vector & valid_index_steps) { + value_cache.resize(valid_index_steps.size()); + rle_data_reader->setValues(value_cache.data() + 1, valid_index_steps.size() - 1, val_getter); + + append_nulls(valid_index_steps[0]); + for (size_t i = 1; i < valid_index_steps.size(); i++) + { + append_string(value_cache[i]); + append_nulls(valid_index_steps[i] - 1); + } + }, + /* repeated_visitor */ [&](bool is_valid, UInt32, UInt32 count) { + if (is_valid) + { + value_cache.resize(count); + rle_data_reader->setValues(value_cache.data(), count, val_getter); + for (UInt32 i = 0; i < count; i++) + { + append_string(value_cache[i]); + } + } + else + { + append_nulls(count); + } + } + ); +} + +template +void ParquetRleDictReader::readBatch( + MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) +{ + auto cursor = col_ptr->size(); + auto * column_data = getResizedPrimitiveData(*assert_cast(col_ptr.get()), cursor + num_values); + const auto & dictionary_array = static_cast(page_dictionary).getData(); + + auto val_getter = [&](Int32 val) { return dictionary_array[val]; }; + def_level_reader->visitNullableBySteps( + cursor, + num_values, + max_def_level, + /* individual_null_visitor */ [&](UInt32 nest_cursor) { + null_map.setNull(nest_cursor); + }, + /* stepped_valid_visitor */ [&](UInt32 nest_cursor, const std::vector & valid_index_steps) { + rle_data_reader->setValueBySteps(column_data + nest_cursor, valid_index_steps, val_getter); + }, + /* repeated_visitor */ [&](bool is_valid, UInt32 nest_cursor, UInt32 count) { + if (is_valid) + { + rle_data_reader->setValues(column_data + nest_cursor, count, val_getter); + } + else + { + null_map.setNull(nest_cursor, count); + } + } + ); +} + + +template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader>; +template class ParquetPlainValuesReader>; +template class ParquetPlainValuesReader; + +template class ParquetFixedLenPlainReader>; +template class ParquetFixedLenPlainReader>; + +template class ParquetRleLCReader; +template class ParquetRleLCReader; +template class ParquetRleLCReader; + +template class ParquetRleDictReader; +template class ParquetRleDictReader; +template class ParquetRleDictReader; +template class ParquetRleDictReader; +template class ParquetRleDictReader>; +template class ParquetRleDictReader>; +template class ParquetRleDictReader>; +template class ParquetRleDictReader>; +template class ParquetRleDictReader>; +template class ParquetRleDictReader; + +} diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h new file mode 100644 index 00000000000..2c95f495339 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -0,0 +1,263 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include "ParquetDataBuffer.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int PARQUET_EXCEPTION; +} + +class RleValuesReader +{ +public: + RleValuesReader(std::unique_ptr bit_reader_, Int32 bit_width_) + : bit_reader(std::move(bit_reader_)), bit_width(bit_width_) {} + + /** + * @brief Used when the bit_width is 0, so all elements have same value. + */ + RleValuesReader(UInt32 total_size, Int32 val = 0) + : bit_reader(nullptr), bit_width(0), cur_group_size(total_size), cur_value(val), cur_group_is_packed(false) + {} + + void nextGroup(); + + void nextGroupIfNecessary() { if (cur_group_cursor >= cur_group_size) nextGroup(); } + + UInt32 curGroupLeft() const { return cur_group_size - cur_group_cursor; } + + /** + * @brief Visit num_values elements. + * For RLE encoding, for same group, the value is same, so they can be visited repeatedly. + * For BitPacked encoding, the values may be different with each other, so they must be visited individual. + * + * @tparam IndividualVisitor A callback with signature: void(Int32 val) + * @tparam RepeatedVisitor A callback with signature: void(UInt32 count, Int32 val) + */ + template + void visitValues(UInt32 num_values, IndividualVisitor && individual_visitor, RepeatedVisitor && repeated_visitor); + + /** + * @brief Visit num_values elements by parsed nullability. + * If the parsed value is same as max_def_level, then it is processed as null value. + * + * @tparam IndividualVisitor A callback with signature: void(size_t cursor) + * @tparam RepeatedVisitor A callback with signature: void(size_t cursor, UInt32 count) + * + * Because the null map is processed, so only the callbacks only need to process the valid data. + */ + template + void visitNullableValues( + size_t cursor, + UInt32 num_values, + Int32 max_def_level, + LazyNullMap & null_map, + IndividualVisitor && individual_visitor, + RepeatedVisitor && repeated_visitor); + + /** + * @brief Visit num_values elements by parsed nullability. + * It may be inefficient to process the valid data individually like in visitNullableValues, + * so a valid_index_steps index array is generated first, in order to process valid data continuously. + * + * @tparam IndividualNullVisitor A callback with signature: void(size_t cursor), used to process null value + * @tparam SteppedValidVisitor A callback with signature: + * void(size_t cursor, const std::vector & valid_index_steps) + * for n valid elements with null value interleaved in a BitPacked group, + * i-th item in valid_index_steps describes how many elements in column there are after (i-1)-th valid element. + * + * take following BitPacked group with 2 valid elements for example: + * null valid null null valid null + * then the valid_index_steps has values [1, 3, 2]. + * Please note that the the sum of valid_index_steps is same as elements number in this group. + * + * @tparam RepeatedVisitor A callback with signature: void(bool is_valid, UInt32 cursor, UInt32 count) + */ + template + void visitNullableBySteps( + size_t cursor, + UInt32 num_values, + Int32 max_def_level, + IndividualNullVisitor && null_visitor, + SteppedValidVisitor && stepped_valid_visitor, + RepeatedVisitor && repeated_visitor); + + /** + * @brief Set the Values to column_data directly + * + * @tparam TValue The type of column data. + * @tparam ValueGetter A callback with signature: TValue(Int32 val) + */ + template + void setValues(TValue * column_data, UInt32 num_values, ValueGetter && val_getter); + + /** + * @brief Set the value by valid_index_steps generated in visitNullableBySteps. + * According to visitNullableBySteps, the elements number is valid_index_steps.size()-1, + * so valid_index_steps.size()-1 elements are read, and set to column_data with steps in valid_index_steps + */ + template + void setValueBySteps( + TValue * column_data, + const std::vector & col_data_steps, + ValueGetter && val_getter); + +private: + std::unique_ptr bit_reader; + + std::vector cur_packed_bit_values; + std::vector valid_index_steps; + + Int32 bit_width; + + UInt32 cur_group_size = 0; + UInt32 cur_group_cursor = 0; + Int32 cur_value; + bool cur_group_is_packed; +}; + +using RleValuesReaderPtr = std::unique_ptr; + + +class ParquetDataValuesReader +{ +public: + virtual void readBatch(MutableColumnPtr & column, LazyNullMap & null_map, UInt32 num_values) = 0; + + virtual ~ParquetDataValuesReader() = default; +}; + +using ParquetDataValuesReaderPtr = std::unique_ptr; + + +/** + * The definition level is RLE or BitPacked encoding, while data is read directly + */ +template +class ParquetPlainValuesReader : public ParquetDataValuesReader +{ +public: + + ParquetPlainValuesReader( + Int32 max_def_level_, + std::unique_ptr def_level_reader_, + ParquetDataBuffer data_buffer_) + : max_def_level(max_def_level_) + , def_level_reader(std::move(def_level_reader_)) + , plain_data_buffer(std::move(data_buffer_)) + {} + + void readBatch(MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) override; + +private: + Int32 max_def_level; + std::unique_ptr def_level_reader; + ParquetDataBuffer plain_data_buffer; +}; + +/** + * The data and definition level encoding are same as ParquetPlainValuesReader. + * But the element size is const and bigger than primitive data type. + */ +template +class ParquetFixedLenPlainReader : public ParquetDataValuesReader +{ +public: + + ParquetFixedLenPlainReader( + Int32 max_def_level_, + Int32 elem_bytes_num_, + std::unique_ptr def_level_reader_, + ParquetDataBuffer data_buffer_) + : max_def_level(max_def_level_) + , elem_bytes_num(elem_bytes_num_) + , def_level_reader(std::move(def_level_reader_)) + , plain_data_buffer(std::move(data_buffer_)) + {} + + void readOverBigDecimal(MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values); + + void readBatch(MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) override; + +private: + Int32 max_def_level; + Int32 elem_bytes_num; + std::unique_ptr def_level_reader; + ParquetDataBuffer plain_data_buffer; +}; + +/** + * Read data according to the format of ColumnLowCardinality format. + * + * Only index and null column are processed in this class. + * And all null value is mapped to first index in dictionary, + * so the result index valued is added by one. +*/ +template +class ParquetRleLCReader : public ParquetDataValuesReader +{ +public: + ParquetRleLCReader( + Int32 max_def_level_, + std::unique_ptr def_level_reader_, + std::unique_ptr rle_data_reader_) + : max_def_level(max_def_level_) + , def_level_reader(std::move(def_level_reader_)) + , rle_data_reader(std::move(rle_data_reader_)) + {} + + void readBatch(MutableColumnPtr & index_col, LazyNullMap & null_map, UInt32 num_values) override; + +private: + Int32 max_def_level; + std::unique_ptr def_level_reader; + std::unique_ptr rle_data_reader; +}; + +/** + * The definition level is RLE or BitPacked encoded, + * and the index of dictionary is also RLE or BitPacked encoded. + * + * while the result is not parsed as a low cardinality column, + * instead, a normal column is generated. + */ +template +class ParquetRleDictReader : public ParquetDataValuesReader +{ +public: + ParquetRleDictReader( + Int32 max_def_level_, + std::unique_ptr def_level_reader_, + std::unique_ptr rle_data_reader_, + const IColumn & page_dictionary_) + : max_def_level(max_def_level_) + , def_level_reader(std::move(def_level_reader_)) + , rle_data_reader(std::move(rle_data_reader_)) + , page_dictionary(page_dictionary_) + {} + + void readBatch(MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) override; + +private: + Int32 max_def_level; + std::unique_ptr def_level_reader; + std::unique_ptr rle_data_reader; + const IColumn & page_dictionary; +}; + +} diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp new file mode 100644 index 00000000000..00dee9074fe --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -0,0 +1,506 @@ +#include "ParquetLeafColReader.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; + extern const int PARQUET_EXCEPTION; +} + +namespace +{ + +template +void visitColStrIndexType(size_t data_size, TypeVisitor && visitor) +{ + // refer to: DataTypeLowCardinality::createColumnUniqueImpl + if (data_size < (1ull << 8)) + { + visitor(static_cast(nullptr)); + } + else if (data_size < (1ull << 16)) + { + visitor(static_cast(nullptr)); + } + else if (data_size < (1ull << 32)) + { + visitor(static_cast(nullptr)); + } + else + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "unsupported data size {}", data_size); + } +} + +void reserveColumnStrRows(MutableColumnPtr & col, UInt32 rows_num) +{ + col->reserve(rows_num); + + /// Never reserve for too big size according to SerializationString::deserializeBinaryBulk + if (rows_num < 256 * 1024 * 1024) + { + try + { + static_cast(col.get())->getChars().reserve(rows_num); + } + catch (Exception & e) + { + e.addMessage("(limit = " + toString(rows_num) + ")"); + throw; + } + } +}; + + +template +ColumnPtr readDictPage( + const parquet::DictionaryPage & page, + const parquet::ColumnDescriptor & col_des, + const DataTypePtr & /* data_type */); + +template <> +ColumnPtr readDictPage( + const parquet::DictionaryPage & page, + const parquet::ColumnDescriptor & /* col_des */, + const DataTypePtr & /* data_type */) +{ + auto col = ColumnString::create(); + col->getOffsets().resize(page.num_values() + 1); + col->getChars().reserve(page.num_values()); + ParquetDataBuffer buffer(page.data(), page.size()); + + // will be read as low cardinality column + // in which case, the null key is set to first position, so the first string should be empty + col->getChars().push_back(0); + col->getOffsets()[0] = 1; + for (auto i = 1; i <= page.num_values(); i++) + { + buffer.readString(*col, i); + } + return col; +} + +template <> +ColumnPtr readDictPage>( + const parquet::DictionaryPage & page, + const parquet::ColumnDescriptor & /* col_des */, + const DataTypePtr & data_type) +{ + auto & datetime_type = assert_cast(*data_type); + auto dict_col = ColumnDecimal::create(page.num_values(), datetime_type.getScale()); + auto * col_data = dict_col->getData().data(); + ParquetDataBuffer buffer(page.data(), page.size(), datetime_type.getScale()); + for (auto i = 0; i < page.num_values(); i++) + { + buffer.readDateTime64(col_data[i]); + } + return dict_col; +} + +template +ColumnPtr readDictPage( + const parquet::DictionaryPage & page, + const parquet::ColumnDescriptor & col_des, + const DataTypePtr & /* data_type */) +{ + auto dict_col = TColumnDecimal::create(page.num_values(), col_des.type_scale()); + auto * col_data = dict_col->getData().data(); + ParquetDataBuffer buffer(page.data(), page.size()); + for (auto i = 0; i < page.num_values(); i++) + { + buffer.readOverBigDecimal(col_data + i, col_des.type_length()); + } + return dict_col; +} + +template requires (!std::is_same_v) +ColumnPtr readDictPage( + const parquet::DictionaryPage & page, + const parquet::ColumnDescriptor & col_des, + const DataTypePtr & /* data_type */) +{ + auto dict_col = TColumnDecimal::create(page.num_values(), col_des.type_scale()); + ParquetDataBuffer buffer(page.data(), page.size()); + buffer.readBytes(dict_col->getData().data(), page.num_values() * sizeof(typename TColumnDecimal::ValueType)); + return dict_col; +} + +template +ColumnPtr readDictPage( + const parquet::DictionaryPage & page, + const parquet::ColumnDescriptor & /* col_des */, + const DataTypePtr & /* data_type */) +{ + auto dict_col = TColumnVector::create(page.num_values()); + ParquetDataBuffer buffer(page.data(), page.size()); + buffer.readBytes(dict_col->getData().data(), page.num_values() * sizeof(typename TColumnVector::ValueType)); + return dict_col; +} + + +template +std::unique_ptr createPlainReader( + const parquet::ColumnDescriptor & col_des, + RleValuesReaderPtr def_level_reader, + ParquetDataBuffer buffer); + +template +std::unique_ptr createPlainReader( + const parquet::ColumnDescriptor & col_des, + RleValuesReaderPtr def_level_reader, + ParquetDataBuffer buffer) +{ + return std::make_unique>( + col_des.max_definition_level(), + col_des.type_length(), + std::move(def_level_reader), + std::move(buffer)); +} + +template +std::unique_ptr createPlainReader( + const parquet::ColumnDescriptor & col_des, + RleValuesReaderPtr def_level_reader, + ParquetDataBuffer buffer) +{ + return std::make_unique>( + col_des.max_definition_level(), std::move(def_level_reader), std::move(buffer)); +} + + +} // anonymous namespace + + +template +ParquetLeafColReader::ParquetLeafColReader( + const parquet::ColumnDescriptor & col_descriptor_, + DataTypePtr base_type_, + std::unique_ptr meta_, + std::unique_ptr reader_) + : col_descriptor(col_descriptor_) + , base_data_type(base_type_) + , col_chunk_meta(std::move(meta_)) + , parquet_page_reader(std::move(reader_)) + , log(&Poco::Logger::get("ParquetLeafColReader")) +{ +} + +template +ColumnWithTypeAndName ParquetLeafColReader::readBatch(UInt32 rows_num, const String & name) +{ + reading_rows_num = rows_num; + auto readPageIfEmpty = [&]() { + while (!cur_page_values) readPage(); + }; + + // make sure the dict page has been read, and the status is updated + readPageIfEmpty(); + resetColumn(rows_num); + + while (rows_num) + { + // if dictionary page encountered, another page should be read + readPageIfEmpty(); + + auto read_values = std::min(rows_num, cur_page_values); + data_values_reader->readBatch(column, *null_map, read_values); + + cur_page_values -= read_values; + rows_num -= read_values; + } + + return releaseColumn(name); +} + +template <> +void ParquetLeafColReader::resetColumn(UInt32 rows_num) +{ + if (reading_low_cardinality) + { + assert(dictionary); + visitColStrIndexType(dictionary->size(), [&](TColVec *) { + column = TColVec::create(); + }); + + // only first position is used + null_map = std::make_unique(1); + column->reserve(rows_num); + } + else + { + null_map = std::make_unique(rows_num); + column = ColumnString::create(); + reserveColumnStrRows(column, rows_num); + } +} + +template +void ParquetLeafColReader::resetColumn(UInt32 rows_num) +{ + assert(!reading_low_cardinality); + + column = base_data_type->createColumn(); + column->reserve(rows_num); + null_map = std::make_unique(rows_num); +} + +template +void ParquetLeafColReader::degradeDictionary() +{ + assert(dictionary && column->size()); + null_map = std::make_unique(reading_rows_num); + auto col_existing = std::move(column); + column = ColumnString::create(); + + ColumnString & col_dest = *static_cast(column.get()); + const ColumnString & col_dict_str = *static_cast(dictionary.get()); + + visitColStrIndexType(dictionary->size(), [&](TColVec *) { + const TColVec & col_src = *static_cast(col_existing.get()); + reserveColumnStrRows(column, reading_rows_num); + + col_dest.getOffsets().resize(col_src.size()); + for (size_t i = 0; i < col_src.size(); i++) + { + auto src_idx = col_src.getData()[i]; + if (0 == src_idx) + { + null_map->setNull(i); + } + auto dict_chars_cursor = col_dict_str.getOffsets()[src_idx - 1]; + auto str_len = col_dict_str.getOffsets()[src_idx] - dict_chars_cursor; + auto dst_chars_cursor = col_dest.getChars().size(); + col_dest.getChars().resize(dst_chars_cursor + str_len); + + memcpySmallAllowReadWriteOverflow15( + &col_dest.getChars()[dst_chars_cursor], &col_dict_str.getChars()[dict_chars_cursor], str_len); + col_dest.getOffsets()[i] = col_dest.getChars().size(); + } + }); + LOG_INFO(log, "degraded dictionary to normal column"); +} + +template +ColumnWithTypeAndName ParquetLeafColReader::releaseColumn(const String & name) +{ + DataTypePtr data_type = base_data_type; + if (reading_low_cardinality) + { + MutableColumnPtr col_unique; + if (null_map->getNullableCol()) + { + data_type = std::make_shared(data_type); + col_unique = ColumnUnique::create(dictionary->assumeMutable(), true); + } + else + { + col_unique = ColumnUnique::create(dictionary->assumeMutable(), false); + } + column = ColumnLowCardinality::create(std::move(col_unique), std::move(column), true); + data_type = std::make_shared(data_type); + } + else + { + if (null_map->getNullableCol()) + { + column = ColumnNullable::create(std::move(column), null_map->getNullableCol()->assumeMutable()); + data_type = std::make_shared(data_type); + } + } + ColumnWithTypeAndName res = {std::move(column), data_type, name}; + column = nullptr; + null_map = nullptr; + + return res; +} + +template +void ParquetLeafColReader::readPage() +{ + // refer to: ColumnReaderImplBase::ReadNewPage in column_reader.cc + auto cur_page = parquet_page_reader->NextPage(); + switch (cur_page->type()) + { + case parquet::PageType::DATA_PAGE: + readPageV1(*std::static_pointer_cast(cur_page)); + break; + case parquet::PageType::DATA_PAGE_V2: + readPageV2(*std::static_pointer_cast(cur_page)); + break; + case parquet::PageType::DICTIONARY_PAGE: + { + const parquet::DictionaryPage & dict_page = *std::static_pointer_cast(cur_page); + if (unlikely( + dict_page.encoding() != parquet::Encoding::PLAIN_DICTIONARY + && dict_page.encoding() != parquet::Encoding::PLAIN)) + { + throw new Exception( + ErrorCodes::NOT_IMPLEMENTED, "Unsupported dictionary page encoding {}", dict_page.encoding()); + } + LOG_INFO(log, "{} values in dictionary page of column {}", dict_page.num_values(), col_descriptor.name()); + + dictionary = readDictPage(dict_page, col_descriptor, base_data_type); + if (std::is_same_v) + { + reading_low_cardinality = true; + } + break; + } + default: + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unsupported page type: {}", cur_page->type()); + } +} + +template +void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) +{ + static parquet::LevelDecoder repetition_level_decoder; + + cur_page_values = page.num_values(); + + // refer to: VectorizedColumnReader::readPageV1 in Spark and LevelDecoder::SetData in column_reader.cc + if (page.definition_level_encoding() != parquet::Encoding::RLE && col_descriptor.max_definition_level() != 0) + { + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Unsupported encoding: {}", page.definition_level_encoding()); + } + const auto * buffer = page.data(); + auto max_size = page.size(); + + if (col_descriptor.max_repetition_level() > 0) + { + auto rep_levels_bytes = repetition_level_decoder.SetData( + page.repetition_level_encoding(), col_descriptor.max_repetition_level(), 0, buffer, max_size); + buffer += rep_levels_bytes; + max_size -= rep_levels_bytes; + } + + assert(col_descriptor.max_definition_level() >= 0); + std::unique_ptr def_level_reader; + if (col_descriptor.max_definition_level() > 0) { + auto bit_width = arrow::BitUtil::Log2(col_descriptor.max_definition_level() + 1); + auto num_bytes = ::arrow::util::SafeLoadAs(buffer); + auto bit_reader = std::make_unique(buffer + 4, num_bytes); + num_bytes += 4; + buffer += num_bytes; + max_size -= num_bytes; + def_level_reader = std::make_unique(std::move(bit_reader), bit_width); + } + else + { + def_level_reader = std::make_unique(page.num_values()); + } + + switch (page.encoding()) + { + case parquet::Encoding::PLAIN: + { + if (reading_low_cardinality) + { + reading_low_cardinality = false; + degradeDictionary(); + } + + ParquetDataBuffer parquet_buffer = [&]() { + if constexpr (!std::is_same_v, TColumn>) + return ParquetDataBuffer(buffer, max_size); + + auto scale = assert_cast(*base_data_type).getScale(); + return ParquetDataBuffer(buffer, max_size, scale); + }(); + data_values_reader = createPlainReader( + col_descriptor, std::move(def_level_reader), std::move(parquet_buffer)); + break; + } + case parquet::Encoding::RLE_DICTIONARY: + case parquet::Encoding::PLAIN_DICTIONARY: + { + if (unlikely(!dictionary)) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "dictionary should be existed"); + } + + // refer to: DictDecoderImpl::SetData in encoding.cc + auto bit_width = *buffer; + auto bit_reader = std::make_unique(++buffer, --max_size); + data_values_reader = createDictReader( + std::move(def_level_reader), std::make_unique(std::move(bit_reader), bit_width)); + break; + } + case parquet::Encoding::BYTE_STREAM_SPLIT: + case parquet::Encoding::DELTA_BINARY_PACKED: + case parquet::Encoding::DELTA_LENGTH_BYTE_ARRAY: + case parquet::Encoding::DELTA_BYTE_ARRAY: + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Unsupported encoding: {}", page.encoding()); + + default: + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Unknown encoding type: {}", page.encoding()); + } +} + +template +void ParquetLeafColReader::readPageV2(const parquet::DataPageV2 & /*page*/) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "read page V2 is not implemented yet"); +} + +template +std::unique_ptr ParquetLeafColReader::createDictReader( + std::unique_ptr def_level_reader, std::unique_ptr rle_data_reader) +{ + if (reading_low_cardinality && std::same_as) + { + std::unique_ptr res; + visitColStrIndexType(dictionary->size(), [&](TCol *) { + res = std::make_unique>( + col_descriptor.max_definition_level(), + std::move(def_level_reader), + std::move(rle_data_reader)); + }); + return res; + } + return std::make_unique>( + col_descriptor.max_definition_level(), + std::move(def_level_reader), + std::move(rle_data_reader), + *assert_cast(dictionary.get())); +} + + +template class ParquetLeafColReader; +template class ParquetLeafColReader; +template class ParquetLeafColReader; +template class ParquetLeafColReader; +template class ParquetLeafColReader; +template class ParquetLeafColReader>; +template class ParquetLeafColReader>; +template class ParquetLeafColReader>; +template class ParquetLeafColReader>; +template class ParquetLeafColReader>; + +} diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h new file mode 100644 index 00000000000..f730afe40ed --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h @@ -0,0 +1,63 @@ +#pragma once + +#include +#include +#include + +#include "ParquetColumnReader.h" +#include "ParquetDataValuesReader.h" + +namespace parquet +{ + +class ColumnDescriptor; + +} + + +namespace DB +{ + +template +class ParquetLeafColReader : public ParquetColumnReader +{ +public: + ParquetLeafColReader( + const parquet::ColumnDescriptor & col_descriptor_, + DataTypePtr base_type_, + std::unique_ptr meta_, + std::unique_ptr reader_); + + ColumnWithTypeAndName readBatch(UInt32 rows_num, const String & name) override; + +private: + const parquet::ColumnDescriptor & col_descriptor; + DataTypePtr base_data_type; + std::unique_ptr col_chunk_meta; + std::unique_ptr parquet_page_reader; + std::unique_ptr data_values_reader; + + MutableColumnPtr column; + std::unique_ptr null_map; + + ColumnPtr dictionary; + + UInt32 cur_page_values = 0; + UInt32 reading_rows_num = 0; + bool reading_low_cardinality = false; + + Poco::Logger * log; + + void resetColumn(UInt32 rows_num); + void degradeDictionary(); + ColumnWithTypeAndName releaseColumn(const String & name); + + void readPage(); + void readPageV1(const parquet::DataPageV1 & page); + void readPageV2(const parquet::DataPageV2 & page); + + std::unique_ptr createDictReader( + std::unique_ptr def_level_reader, std::unique_ptr rle_data_reader); +}; + +} diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp new file mode 100644 index 00000000000..a5744b85174 --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -0,0 +1,225 @@ +#include "ParquetRecordReader.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include "ParquetLeafColReader.h" + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int PARQUET_EXCEPTION; +} + +// #define THROW_ARROW_NOT_OK(status) \ +// do \ +// { \ +// if (::arrow::Status _s = (status); !_s.ok()) \ +// throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ +// } while (false) + + +#define THROW_PARQUET_EXCEPTION(s) \ + do \ + { \ + try { (s); } \ + catch (const ::parquet::ParquetException & e) \ + { \ + throw Exception(e.what(), ErrorCodes::PARQUET_EXCEPTION); \ + } \ + } while (false) + +namespace +{ + +Int64 getTotalRows(const parquet::FileMetaData & meta_data) +{ + Int64 res = 0; + for (int i = 0; i < meta_data.num_row_groups(); i++) + { + res += meta_data.RowGroup(i)->num_rows(); + } + return res; +} + +std::unique_ptr createReader( + const parquet::ColumnDescriptor & col_descriptor, + DataTypePtr ch_type, + std::unique_ptr meta, + std::unique_ptr reader) +{ + if (col_descriptor.logical_type()->is_date() && parquet::Type::INT32 == col_descriptor.physical_type()) + { + return std::make_unique>( + col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); + } + else if (col_descriptor.logical_type()->is_decimal()) + { + switch (col_descriptor.physical_type()) + { + case parquet::Type::INT32: + { + auto data_type = std::make_shared( + col_descriptor.type_precision(), col_descriptor.type_scale()); + return std::make_unique>>( + col_descriptor, data_type, std::move(meta), std::move(reader)); + } + case parquet::Type::INT64: + { + auto data_type = std::make_shared( + col_descriptor.type_precision(), col_descriptor.type_scale()); + return std::make_unique>>( + col_descriptor, data_type, std::move(meta), std::move(reader)); + } + case parquet::Type::FIXED_LEN_BYTE_ARRAY: + { + if (col_descriptor.type_length() <= static_cast(DecimalUtils::max_precision)) + { + auto data_type = std::make_shared( + col_descriptor.type_precision(), col_descriptor.type_scale()); + return std::make_unique>>( + col_descriptor, data_type, std::move(meta), std::move(reader)); + } + else + { + auto data_type = std::make_shared( + col_descriptor.type_precision(), col_descriptor.type_scale()); + return std::make_unique>>( + col_descriptor, data_type, std::move(meta), std::move(reader)); + } + } + default: + throw Exception( + ErrorCodes::PARQUET_EXCEPTION, + "Type not supported for decimal: {}", + col_descriptor.physical_type()); + } + } + else + { + switch (col_descriptor.physical_type()) + { + case parquet::Type::INT32: + return std::make_unique>( + col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); + case parquet::Type::INT64: + return std::make_unique>( + col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); + case parquet::Type::FLOAT: + return std::make_unique>( + col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); + case parquet::Type::INT96: + { + DataTypePtr read_type = ch_type; + if (!isDateTime64(ch_type)) + { + read_type = std::make_shared(ParquetRecordReader::default_datetime64_scale); + } + return std::make_unique>>( + col_descriptor, read_type, std::move(meta), std::move(reader)); + } + case parquet::Type::DOUBLE: + return std::make_unique>( + col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); + case parquet::Type::BYTE_ARRAY: + return std::make_unique>( + col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); + default: + throw Exception( + ErrorCodes::PARQUET_EXCEPTION, "Type not supported: {}", col_descriptor.physical_type()); + } + } +} + +} // anonymouse namespace + +ParquetRecordReader::ParquetRecordReader( + Block header_, + std::shared_ptr<::arrow::io::RandomAccessFile> file, + const parquet::ReaderProperties& properties) + : header(std::move(header_)) +{ + // Only little endian system is supported currently + static_assert(std::endian::native == std::endian::little); + + log = &Poco::Logger::get("ParquetRecordReader"); + THROW_PARQUET_EXCEPTION(file_reader = parquet::ParquetFileReader::Open(std::move(file), properties)); + left_rows = getTotalRows(*file_reader->metadata()); + + parquet_col_indice.reserve(header.columns()); + column_readers.reserve(header.columns()); + for (const auto & col_with_name : header) + { + auto idx = file_reader->metadata()->schema()->ColumnIndex(col_with_name.name); + if (idx < 0) + { + throw Exception("can not find column with name: " + col_with_name.name, ErrorCodes::BAD_ARGUMENTS); + } + parquet_col_indice.push_back(idx); + } +} + +Chunk ParquetRecordReader::readChunk(UInt32 num_rows) +{ + if (!left_rows) + { + return Chunk{}; + } + if (!cur_row_group_left_rows) + { + loadNextRowGroup(); + } + + Columns columns(header.columns()); + auto num_rows_read = std::min(static_cast(num_rows), cur_row_group_left_rows); + for (size_t i = 0; i < header.columns(); i++) + { + columns[i] = castColumn( + column_readers[i]->readBatch(num_rows_read, header.getByPosition(i).name), + header.getByPosition(i).type); + } + left_rows -= num_rows_read; + cur_row_group_left_rows -= num_rows_read; + + return Chunk{std::move(columns), num_rows_read}; +} + +void ParquetRecordReader::loadNextRowGroup() +{ + Stopwatch watch(CLOCK_MONOTONIC); + cur_row_group_reader = file_reader->RowGroup(next_row_group_idx); + + column_readers.clear(); + for (size_t i = 0; i < parquet_col_indice.size(); i++) + { + column_readers.emplace_back(createReader( + *file_reader->metadata()->schema()->Column(parquet_col_indice[i]), + header.getByPosition(i).type, + cur_row_group_reader->metadata()->ColumnChunk(parquet_col_indice[i]), + cur_row_group_reader->GetColumnPageReader(parquet_col_indice[i]))); + } + LOG_DEBUG(log, "reading row group {} consumed {} ms", next_row_group_idx, watch.elapsedNanoseconds() / 1e6); + ++next_row_group_idx; + cur_row_group_left_rows = cur_row_group_reader->metadata()->num_rows(); +} + +} diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h new file mode 100644 index 00000000000..d77cab6553b --- /dev/null +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include + +#include +#include +#include + +#include "ParquetColumnReader.h" + +namespace DB +{ + +class ParquetRecordReader +{ +public: + ParquetRecordReader( + Block header_, + std::shared_ptr<::arrow::io::RandomAccessFile> file, + const parquet::ReaderProperties& properties); + + Chunk readChunk(UInt32 num_rows); + + // follow the scale generated by spark + static constexpr UInt8 default_datetime64_scale = 9; + +private: + std::unique_ptr file_reader; + + Block header; + + std::shared_ptr cur_row_group_reader; + ParquetColReaders column_readers; + + std::vector parquet_col_indice; + UInt64 left_rows; + UInt64 cur_row_group_left_rows = 0; + int next_row_group_idx = 0; + + Poco::Logger * log; + + void loadNextRowGroup(); +}; + +} From 8fb89cec9f28d6a12c2216ccd849fe0ead3ccd33 Mon Sep 17 00:00:00 2001 From: copperybean Date: Sun, 14 Jan 2024 12:01:23 +0800 Subject: [PATCH 02/36] fix build Change-Id: I57f025b17a04e2c5dded3f18e7f477841287a2c2 --- base/base/Decimal_fwd.h | 4 ++++ src/Columns/ColumnDecimal.h | 8 +++++++ src/Columns/ColumnVector.h | 3 +++ src/Common/ErrorCodes.cpp | 1 + .../Impl/Parquet/ParquetColumnReader.h | 3 ++- .../Formats/Impl/Parquet/ParquetDataBuffer.h | 12 ++++++---- .../Impl/Parquet/ParquetDataValuesReader.cpp | 23 ++++++++++--------- .../Impl/Parquet/ParquetDataValuesReader.h | 23 +++++++++---------- .../Impl/Parquet/ParquetLeafColReader.cpp | 17 +++++++------- .../Impl/Parquet/ParquetLeafColReader.h | 7 +++--- .../Impl/Parquet/ParquetRecordReader.cpp | 19 ++++++--------- .../Impl/Parquet/ParquetRecordReader.h | 7 +++--- 12 files changed, 71 insertions(+), 56 deletions(-) diff --git a/base/base/Decimal_fwd.h b/base/base/Decimal_fwd.h index beb228cea3c..a11e13a479b 100644 --- a/base/base/Decimal_fwd.h +++ b/base/base/Decimal_fwd.h @@ -44,6 +44,10 @@ concept is_over_big_int = || std::is_same_v || std::is_same_v || std::is_same_v; + +template +concept is_over_big_decimal = is_decimal && is_over_big_int; + } template <> struct is_signed { static constexpr bool value = true; }; diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index e0ea26744dc..e606aaaff0f 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -141,6 +141,14 @@ protected: UInt32 scale; }; +template +concept is_col_over_big_decimal = std::is_same_v> + && is_decimal && is_over_big_int; + +template +concept is_col_int_decimal = std::is_same_v> + && is_decimal && std::is_integral_v; + template class ColumnVector; template struct ColumnVectorOrDecimalT { using Col = ColumnVector; }; template struct ColumnVectorOrDecimalT { using Col = ColumnDecimal; }; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 39ee1d931bd..91bceaa4534 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -441,6 +441,9 @@ ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_ return res; } +template +concept is_col_vector = std::is_same_v>; + /// Prevent implicit template instantiation of ColumnVector for common types extern template class ColumnVector; diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 44c051401ef..106f443d532 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -600,6 +600,7 @@ M(719, QUERY_CACHE_USED_WITH_SYSTEM_TABLE) \ M(720, USER_EXPIRED) \ M(721, DEPRECATED_FUNCTION) \ + M(722, PARQUET_EXCEPTION) \ \ M(900, DISTRIBUTED_CACHE_ERROR) \ M(901, CANNOT_USE_DISTRIBUTED_CACHE) \ diff --git a/src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h b/src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h index cfd9d3ba5bd..2c78949e8e1 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetColumnReader.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace parquet { @@ -18,7 +19,7 @@ namespace DB class ParquetColumnReader { public: - virtual ColumnWithTypeAndName readBatch(UInt32 rows_num, const String & name) = 0; + virtual ColumnWithTypeAndName readBatch(UInt64 rows_num, const String & name) = 0; virtual ~ParquetColumnReader() = default; }; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h index 1f83c74f9ad..be9710e1726 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h @@ -142,15 +142,19 @@ private: class LazyNullMap { public: - LazyNullMap(UInt32 size_) : size(size_), col_nullable(nullptr) {} + LazyNullMap(UInt64 size_) : size(size_), col_nullable(nullptr) {} - void setNull(UInt32 cursor) + template + requires std::is_integral_v + void setNull(T cursor) { initialize(); null_map[cursor] = 1; } - void setNull(UInt32 cursor, UInt32 count) + template + requires std::is_integral_v + void setNull(T cursor, UInt32 count) { initialize(); memset(null_map + cursor, 1, count); @@ -159,7 +163,7 @@ public: ColumnPtr getNullableCol() { return col_nullable; } private: - UInt32 size; + UInt64 size; UInt8 * null_map; ColumnPtr col_nullable; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 659a7a11969..3afc66dcb36 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -189,7 +189,7 @@ void RleValuesReader::setValueBySteps( res_values += *(step_iterator++); visitValues( - col_data_steps.size() - 1, + static_cast(col_data_steps.size() - 1), /* individual_visitor */ [&](Int32 val) { *res_values = val_getter(val); @@ -394,14 +394,14 @@ void ParquetRleLCReader::readBatch( cursor, num_values, max_def_level, - /* individual_null_visitor */ [&](UInt32 nest_cursor) { + /* individual_null_visitor */ [&](size_t nest_cursor) { column_data[nest_cursor] = 0; has_null = true; }, - /* stepped_valid_visitor */ [&](UInt32 nest_cursor, const std::vector & valid_index_steps) { + /* stepped_valid_visitor */ [&](size_t nest_cursor, const std::vector & valid_index_steps) { rle_data_reader->setValueBySteps(column_data + nest_cursor, valid_index_steps, val_getter); }, - /* repeated_visitor */ [&](bool is_valid, UInt32 nest_cursor, UInt32 count) { + /* repeated_visitor */ [&](bool is_valid, size_t nest_cursor, UInt32 count) { if (is_valid) { rle_data_reader->setValues(column_data + nest_cursor, count, val_getter); @@ -461,10 +461,11 @@ void ParquetRleDictReader::readBatch( cursor, num_values, max_def_level, - /* individual_null_visitor */ [&](UInt32) {}, - /* stepped_valid_visitor */ [&](UInt32, const std::vector & valid_index_steps) { + /* individual_null_visitor */ [&](size_t) {}, + /* stepped_valid_visitor */ [&](size_t, const std::vector & valid_index_steps) { value_cache.resize(valid_index_steps.size()); - rle_data_reader->setValues(value_cache.data() + 1, valid_index_steps.size() - 1, val_getter); + rle_data_reader->setValues( + value_cache.data() + 1, static_cast(valid_index_steps.size() - 1), val_getter); append_nulls(valid_index_steps[0]); for (size_t i = 1; i < valid_index_steps.size(); i++) @@ -473,7 +474,7 @@ void ParquetRleDictReader::readBatch( append_nulls(valid_index_steps[i] - 1); } }, - /* repeated_visitor */ [&](bool is_valid, UInt32, UInt32 count) { + /* repeated_visitor */ [&](bool is_valid, size_t, UInt32 count) { if (is_valid) { value_cache.resize(count); @@ -504,13 +505,13 @@ void ParquetRleDictReader::readBatch( cursor, num_values, max_def_level, - /* individual_null_visitor */ [&](UInt32 nest_cursor) { + /* individual_null_visitor */ [&](size_t nest_cursor) { null_map.setNull(nest_cursor); }, - /* stepped_valid_visitor */ [&](UInt32 nest_cursor, const std::vector & valid_index_steps) { + /* stepped_valid_visitor */ [&](size_t nest_cursor, const std::vector & valid_index_steps) { rle_data_reader->setValueBySteps(column_data + nest_cursor, valid_index_steps, val_getter); }, - /* repeated_visitor */ [&](bool is_valid, UInt32 nest_cursor, UInt32 count) { + /* repeated_visitor */ [&](bool is_valid, size_t nest_cursor, UInt32 count) { if (is_valid) { rle_data_reader->setValues(column_data + nest_cursor, count, val_getter); diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index 2c95f495339..66a1f4877e4 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -3,7 +3,6 @@ #include #include -#include #include #include #include @@ -25,7 +24,7 @@ namespace ErrorCodes class RleValuesReader { public: - RleValuesReader(std::unique_ptr bit_reader_, Int32 bit_width_) + RleValuesReader(std::unique_ptr bit_reader_, Int32 bit_width_) : bit_reader(std::move(bit_reader_)), bit_width(bit_width_) {} /** @@ -45,7 +44,7 @@ public: * @brief Visit num_values elements. * For RLE encoding, for same group, the value is same, so they can be visited repeatedly. * For BitPacked encoding, the values may be different with each other, so they must be visited individual. - * + * * @tparam IndividualVisitor A callback with signature: void(Int32 val) * @tparam RepeatedVisitor A callback with signature: void(UInt32 count, Int32 val) */ @@ -55,10 +54,10 @@ public: /** * @brief Visit num_values elements by parsed nullability. * If the parsed value is same as max_def_level, then it is processed as null value. - * + * * @tparam IndividualVisitor A callback with signature: void(size_t cursor) * @tparam RepeatedVisitor A callback with signature: void(size_t cursor, UInt32 count) - * + * * Because the null map is processed, so only the callbacks only need to process the valid data. */ template @@ -74,18 +73,18 @@ public: * @brief Visit num_values elements by parsed nullability. * It may be inefficient to process the valid data individually like in visitNullableValues, * so a valid_index_steps index array is generated first, in order to process valid data continuously. - * + * * @tparam IndividualNullVisitor A callback with signature: void(size_t cursor), used to process null value * @tparam SteppedValidVisitor A callback with signature: * void(size_t cursor, const std::vector & valid_index_steps) * for n valid elements with null value interleaved in a BitPacked group, * i-th item in valid_index_steps describes how many elements in column there are after (i-1)-th valid element. - * + * * take following BitPacked group with 2 valid elements for example: * null valid null null valid null * then the valid_index_steps has values [1, 3, 2]. * Please note that the the sum of valid_index_steps is same as elements number in this group. - * + * * @tparam RepeatedVisitor A callback with signature: void(bool is_valid, UInt32 cursor, UInt32 count) */ template @@ -99,7 +98,7 @@ public: /** * @brief Set the Values to column_data directly - * + * * @tparam TValue The type of column data. * @tparam ValueGetter A callback with signature: TValue(Int32 val) */ @@ -118,7 +117,7 @@ public: ValueGetter && val_getter); private: - std::unique_ptr bit_reader; + std::unique_ptr bit_reader; std::vector cur_packed_bit_values; std::vector valid_index_steps; @@ -203,7 +202,7 @@ private: /** * Read data according to the format of ColumnLowCardinality format. - * + * * Only index and null column are processed in this class. * And all null value is mapped to first index in dictionary, * so the result index valued is added by one. @@ -232,7 +231,7 @@ private: /** * The definition level is RLE or BitPacked encoded, * and the index of dictionary is also RLE or BitPacked encoded. - * + * * while the result is not parsed as a low cardinality column, * instead, a normal column is generated. */ diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index 00dee9074fe..2e3d329bcd2 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -58,7 +59,7 @@ void visitColStrIndexType(size_t data_size, TypeVisitor && visitor) } } -void reserveColumnStrRows(MutableColumnPtr & col, UInt32 rows_num) +void reserveColumnStrRows(MutableColumnPtr & col, UInt64 rows_num) { col->reserve(rows_num); @@ -212,7 +213,7 @@ ParquetLeafColReader::ParquetLeafColReader( } template -ColumnWithTypeAndName ParquetLeafColReader::readBatch(UInt32 rows_num, const String & name) +ColumnWithTypeAndName ParquetLeafColReader::readBatch(UInt64 rows_num, const String & name) { reading_rows_num = rows_num; auto readPageIfEmpty = [&]() { @@ -228,7 +229,7 @@ ColumnWithTypeAndName ParquetLeafColReader::readBatch(UInt32 rows_num, // if dictionary page encountered, another page should be read readPageIfEmpty(); - auto read_values = std::min(rows_num, cur_page_values); + auto read_values = static_cast(std::min(rows_num, static_cast(cur_page_values))); data_values_reader->readBatch(column, *null_map, read_values); cur_page_values -= read_values; @@ -239,7 +240,7 @@ ColumnWithTypeAndName ParquetLeafColReader::readBatch(UInt32 rows_num, } template <> -void ParquetLeafColReader::resetColumn(UInt32 rows_num) +void ParquetLeafColReader::resetColumn(UInt64 rows_num) { if (reading_low_cardinality) { @@ -261,7 +262,7 @@ void ParquetLeafColReader::resetColumn(UInt32 rows_num) } template -void ParquetLeafColReader::resetColumn(UInt32 rows_num) +void ParquetLeafColReader::resetColumn(UInt64 rows_num) { assert(!reading_low_cardinality); @@ -403,9 +404,9 @@ void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) assert(col_descriptor.max_definition_level() >= 0); std::unique_ptr def_level_reader; if (col_descriptor.max_definition_level() > 0) { - auto bit_width = arrow::BitUtil::Log2(col_descriptor.max_definition_level() + 1); + auto bit_width = arrow::bit_util::Log2(col_descriptor.max_definition_level() + 1); auto num_bytes = ::arrow::util::SafeLoadAs(buffer); - auto bit_reader = std::make_unique(buffer + 4, num_bytes); + auto bit_reader = std::make_unique(buffer + 4, num_bytes); num_bytes += 4; buffer += num_bytes; max_size -= num_bytes; @@ -447,7 +448,7 @@ void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) // refer to: DictDecoderImpl::SetData in encoding.cc auto bit_width = *buffer; - auto bit_reader = std::make_unique(++buffer, --max_size); + auto bit_reader = std::make_unique(++buffer, --max_size); data_values_reader = createDictReader( std::move(def_level_reader), std::make_unique(std::move(bit_reader), bit_width)); break; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h index f730afe40ed..c5b14132f17 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include @@ -28,7 +27,7 @@ public: std::unique_ptr meta_, std::unique_ptr reader_); - ColumnWithTypeAndName readBatch(UInt32 rows_num, const String & name) override; + ColumnWithTypeAndName readBatch(UInt64 rows_num, const String & name) override; private: const parquet::ColumnDescriptor & col_descriptor; @@ -42,13 +41,13 @@ private: ColumnPtr dictionary; + UInt64 reading_rows_num = 0; UInt32 cur_page_values = 0; - UInt32 reading_rows_num = 0; bool reading_low_cardinality = false; Poco::Logger * log; - void resetColumn(UInt32 rows_num); + void resetColumn(UInt64 rows_num); void degradeDictionary(); ColumnWithTypeAndName releaseColumn(const String & name); diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index a5744b85174..9ff4a7a16aa 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -30,21 +31,14 @@ namespace ErrorCodes extern const int PARQUET_EXCEPTION; } -// #define THROW_ARROW_NOT_OK(status) \ -// do \ -// { \ -// if (::arrow::Status _s = (status); !_s.ok()) \ -// throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \ -// } while (false) - - #define THROW_PARQUET_EXCEPTION(s) \ do \ { \ try { (s); } \ catch (const ::parquet::ParquetException & e) \ { \ - throw Exception(e.what(), ErrorCodes::PARQUET_EXCEPTION); \ + auto msg = PreformattedMessage::create("Excepted when reading parquet: {}", e.what()); \ + throw Exception(std::move(msg), ErrorCodes::PARQUET_EXCEPTION); \ } \ } while (false) @@ -172,13 +166,14 @@ ParquetRecordReader::ParquetRecordReader( auto idx = file_reader->metadata()->schema()->ColumnIndex(col_with_name.name); if (idx < 0) { - throw Exception("can not find column with name: " + col_with_name.name, ErrorCodes::BAD_ARGUMENTS); + auto msg = PreformattedMessage::create("can not find column with name: {}", col_with_name.name); + throw Exception(std::move(msg), ErrorCodes::BAD_ARGUMENTS); } parquet_col_indice.push_back(idx); } } -Chunk ParquetRecordReader::readChunk(UInt32 num_rows) +Chunk ParquetRecordReader::readChunk(size_t num_rows) { if (!left_rows) { @@ -190,7 +185,7 @@ Chunk ParquetRecordReader::readChunk(UInt32 num_rows) } Columns columns(header.columns()); - auto num_rows_read = std::min(static_cast(num_rows), cur_row_group_left_rows); + auto num_rows_read = std::min(num_rows, cur_row_group_left_rows); for (size_t i = 0; i < header.columns(); i++) { columns[i] = castColumn( diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h index d77cab6553b..69cdaa5ccb7 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h @@ -1,9 +1,8 @@ #pragma once -#include #include #include -#include +#include #include #include @@ -22,8 +21,8 @@ public: std::shared_ptr<::arrow::io::RandomAccessFile> file, const parquet::ReaderProperties& properties); - Chunk readChunk(UInt32 num_rows); - + Chunk readChunk(size_t num_rows); + // follow the scale generated by spark static constexpr UInt8 default_datetime64_scale = 9; From dbdff6c038834f973d803f44ef096b6015d09e3b Mon Sep 17 00:00:00 2001 From: copperybean Date: Sun, 28 Jan 2024 09:56:36 +0800 Subject: [PATCH 03/36] support reading simple types by native parquet reader Change-Id: I38b8368b022263d9a71cb3f3e9fdad5d6ca26753 --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/Parquet/ParquetDataBuffer.h | 2 +- .../Impl/Parquet/ParquetLeafColReader.cpp | 12 +- .../Impl/Parquet/ParquetRecordReader.cpp | 73 +++++++---- .../Impl/Parquet/ParquetRecordReader.h | 14 ++- .../Formats/Impl/ParquetBlockInputFormat.cpp | 118 ++++++++++++------ .../Formats/Impl/ParquetBlockInputFormat.h | 4 + 9 files changed, 153 insertions(+), 73 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a0de354a03..2465164e912 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1013,6 +1013,7 @@ class IColumn; M(Bool, input_format_parquet_case_insensitive_column_matching, false, "Ignore case when matching Parquet columns with CH columns.", 0) \ M(Bool, input_format_parquet_preserve_order, false, "Avoid reordering rows when reading from Parquet files. Usually makes it much slower.", 0) \ M(Bool, input_format_parquet_filter_push_down, true, "When reading Parquet files, skip whole row groups based on the WHERE/PREWHERE expressions and min/max statistics in the Parquet metadata.", 0) \ + M(Bool, input_format_parquet_use_native_reader, false, "When reading Parquet files, to use native reader instead of arrow reader.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ M(Bool, input_format_orc_allow_missing_columns, true, "Allow missing columns while reading ORC input formats", 0) \ M(Bool, input_format_orc_use_fast_decoder, true, "Use a faster ORC decoder implementation.", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 43ccee173f0..557b49d2a0a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -154,6 +154,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.parquet.case_insensitive_column_matching = settings.input_format_parquet_case_insensitive_column_matching; format_settings.parquet.preserve_order = settings.input_format_parquet_preserve_order; format_settings.parquet.filter_push_down = settings.input_format_parquet_filter_push_down; + format_settings.parquet.use_native_reader = settings.input_format_parquet_use_native_reader; format_settings.parquet.allow_missing_columns = settings.input_format_parquet_allow_missing_columns; format_settings.parquet.skip_columns_with_unsupported_types_in_schema_inference = settings.input_format_parquet_skip_columns_with_unsupported_types_in_schema_inference; format_settings.parquet.output_string_as_string = settings.output_format_parquet_string_as_string; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index d5fedf99adb..0ac4ea5e0fb 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -258,6 +258,7 @@ struct FormatSettings bool skip_columns_with_unsupported_types_in_schema_inference = false; bool case_insensitive_column_matching = false; bool filter_push_down = true; + bool use_native_reader = false; std::unordered_set skip_row_groups = {}; bool output_string_as_string = false; bool output_fixed_string_as_fixed_byte_array = true; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h index be9710e1726..d4956f83092 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h @@ -34,7 +34,7 @@ public: void ALWAYS_INLINE readValue(TValue & dst) { checkAvaible(sizeof(TValue)); - dst = *reinterpret_cast(data); + dst = *(reinterpret_cast(data)); consume(sizeof(TValue)); } diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index 2e3d329bcd2..e2677d7cae3 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -274,7 +274,14 @@ void ParquetLeafColReader::resetColumn(UInt64 rows_num) template void ParquetLeafColReader::degradeDictionary() { + // if last batch read all dictionary indices, then degrade is not needed this time + if (!column) + { + dictionary = nullptr; + return; + } assert(dictionary && column->size()); + null_map = std::make_unique(reading_rows_num); auto col_existing = std::move(column); column = ColumnString::create(); @@ -304,7 +311,8 @@ void ParquetLeafColReader::degradeDictionary() col_dest.getOffsets()[i] = col_dest.getChars().size(); } }); - LOG_INFO(log, "degraded dictionary to normal column"); + dictionary = nullptr; + LOG_DEBUG(log, "degraded dictionary to normal column"); } template @@ -364,7 +372,7 @@ void ParquetLeafColReader::readPage() throw new Exception( ErrorCodes::NOT_IMPLEMENTED, "Unsupported dictionary page encoding {}", dict_page.encoding()); } - LOG_INFO(log, "{} values in dictionary page of column {}", dict_page.num_values(), col_descriptor.name()); + LOG_DEBUG(log, "{} values in dictionary page of column {}", dict_page.num_values(), col_descriptor.name()); dictionary = readDictPage(dict_page, col_descriptor, base_data_type); if (std::is_same_v) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index 9ff4a7a16aa..42f131ff794 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -31,31 +31,29 @@ namespace ErrorCodes extern const int PARQUET_EXCEPTION; } -#define THROW_PARQUET_EXCEPTION(s) \ - do \ - { \ - try { (s); } \ - catch (const ::parquet::ParquetException & e) \ - { \ +#define THROW_PARQUET_EXCEPTION(s) \ + do \ + { \ + try { (s); } \ + catch (const ::parquet::ParquetException & e) \ + { \ auto msg = PreformattedMessage::create("Excepted when reading parquet: {}", e.what()); \ throw Exception(std::move(msg), ErrorCodes::PARQUET_EXCEPTION); \ - } \ + } \ } while (false) namespace { -Int64 getTotalRows(const parquet::FileMetaData & meta_data) +std::unique_ptr createFileReader( + std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file) { - Int64 res = 0; - for (int i = 0; i < meta_data.num_row_groups(); i++) - { - res += meta_data.RowGroup(i)->num_rows(); - } + std::unique_ptr res; + THROW_PARQUET_EXCEPTION(res = parquet::ParquetFileReader::Open(std::move(arrow_file))); return res; } -std::unique_ptr createReader( +std::unique_ptr createColReader( const parquet::ColumnDescriptor & col_descriptor, DataTypePtr ch_type, std::unique_ptr meta, @@ -86,7 +84,7 @@ std::unique_ptr createReader( } case parquet::Type::FIXED_LEN_BYTE_ARRAY: { - if (col_descriptor.type_length() <= static_cast(DecimalUtils::max_precision)) + if (col_descriptor.type_length() <= static_cast(sizeof(Decimal128))) { auto data_type = std::make_shared( col_descriptor.type_precision(), col_descriptor.type_scale()); @@ -148,16 +146,21 @@ std::unique_ptr createReader( ParquetRecordReader::ParquetRecordReader( Block header_, - std::shared_ptr<::arrow::io::RandomAccessFile> file, - const parquet::ReaderProperties& properties) - : header(std::move(header_)) + parquet::ArrowReaderProperties reader_properties_, + std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file, + const FormatSettings & format_settings, + std::vector row_groups_indices_) + : file_reader(createFileReader(std::move(arrow_file))) + , reader_properties(reader_properties_) + , header(std::move(header_)) + , max_block_size(format_settings.parquet.max_block_size) + , row_groups_indices(std::move(row_groups_indices_)) + , left_rows(getTotalRows(*file_reader->metadata())) { // Only little endian system is supported currently static_assert(std::endian::native == std::endian::little); log = &Poco::Logger::get("ParquetRecordReader"); - THROW_PARQUET_EXCEPTION(file_reader = parquet::ParquetFileReader::Open(std::move(file), properties)); - left_rows = getTotalRows(*file_reader->metadata()); parquet_col_indice.reserve(header.columns()); column_readers.reserve(header.columns()); @@ -167,13 +170,18 @@ ParquetRecordReader::ParquetRecordReader( if (idx < 0) { auto msg = PreformattedMessage::create("can not find column with name: {}", col_with_name.name); - throw Exception(std::move(msg), ErrorCodes::BAD_ARGUMENTS); + throw Exception(std::move(msg), ErrorCodes::PARQUET_EXCEPTION); } parquet_col_indice.push_back(idx); } + if (reader_properties.pre_buffer()) + { + THROW_PARQUET_EXCEPTION(file_reader->PreBuffer( + row_groups_indices, parquet_col_indice, reader_properties.io_context(), reader_properties.cache_options())); + } } -Chunk ParquetRecordReader::readChunk(size_t num_rows) +Chunk ParquetRecordReader::readChunk() { if (!left_rows) { @@ -185,7 +193,7 @@ Chunk ParquetRecordReader::readChunk(size_t num_rows) } Columns columns(header.columns()); - auto num_rows_read = std::min(num_rows, cur_row_group_left_rows); + auto num_rows_read = std::min(max_block_size, cur_row_group_left_rows); for (size_t i = 0; i < header.columns(); i++) { columns[i] = castColumn( @@ -201,20 +209,33 @@ Chunk ParquetRecordReader::readChunk(size_t num_rows) void ParquetRecordReader::loadNextRowGroup() { Stopwatch watch(CLOCK_MONOTONIC); - cur_row_group_reader = file_reader->RowGroup(next_row_group_idx); + cur_row_group_reader = file_reader->RowGroup(row_groups_indices[next_row_group_idx]); column_readers.clear(); for (size_t i = 0; i < parquet_col_indice.size(); i++) { - column_readers.emplace_back(createReader( + column_readers.emplace_back(createColReader( *file_reader->metadata()->schema()->Column(parquet_col_indice[i]), header.getByPosition(i).type, cur_row_group_reader->metadata()->ColumnChunk(parquet_col_indice[i]), cur_row_group_reader->GetColumnPageReader(parquet_col_indice[i]))); } - LOG_DEBUG(log, "reading row group {} consumed {} ms", next_row_group_idx, watch.elapsedNanoseconds() / 1e6); + + auto duration = watch.elapsedNanoseconds() / 1e6; + LOG_DEBUG(log, "reading row group {} consumed {} ms", row_groups_indices[next_row_group_idx], duration); + ++next_row_group_idx; cur_row_group_left_rows = cur_row_group_reader->metadata()->num_rows(); } +Int64 ParquetRecordReader::getTotalRows(const parquet::FileMetaData & meta_data) +{ + Int64 res = 0; + for (size_t i = 0; i < row_groups_indices.size(); i++) + { + res += meta_data.RowGroup(row_groups_indices[i])->num_rows(); + } + return res; +} + } diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h index 69cdaa5ccb7..4789be59ec8 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -18,23 +19,29 @@ class ParquetRecordReader public: ParquetRecordReader( Block header_, - std::shared_ptr<::arrow::io::RandomAccessFile> file, - const parquet::ReaderProperties& properties); + parquet::ArrowReaderProperties reader_properties_, + std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file, + const FormatSettings & format_settings, + std::vector row_groups_indices_); - Chunk readChunk(size_t num_rows); + Chunk readChunk(); // follow the scale generated by spark static constexpr UInt8 default_datetime64_scale = 9; private: std::unique_ptr file_reader; + parquet::ArrowReaderProperties reader_properties; Block header; std::shared_ptr cur_row_group_reader; ParquetColReaders column_readers; + UInt64 max_block_size; + std::vector parquet_col_indice; + std::vector row_groups_indices; UInt64 left_rows; UInt64 cur_row_group_left_rows = 0; int next_row_group_idx = 0; @@ -42,6 +49,7 @@ private: Poco::Logger * log; void loadNextRowGroup(); + Int64 getTotalRows(const parquet::FileMetaData & meta_data); }; } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index d41cb3447de..e35d53dc4f4 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -23,6 +23,7 @@ #include #include #include +#include namespace CurrentMetrics { @@ -392,6 +393,8 @@ void ParquetBlockInputFormat::initializeIfNeeded() { if (std::exchange(is_initialized, true)) return; + if (format_settings.parquet.use_native_reader) + LOG_INFO(&Poco::Logger::get("ParquetBlockInputFormat"), "using native parquet reader"); // Create arrow file adapter. // TODO: Make the adapter do prefetching on IO threads, based on the full set of ranges that @@ -479,23 +482,35 @@ void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_bat if (metadata->writer_version().VersionLt(parquet::ApplicationVersion::PARQUET_816_FIXED_VERSION())) properties.set_pre_buffer(false); - parquet::arrow::FileReaderBuilder builder; - THROW_ARROW_NOT_OK( - builder.Open(arrow_file, /* not to be confused with ArrowReaderProperties */ parquet::default_reader_properties(), metadata)); - builder.properties(properties); - // TODO: Pass custom memory_pool() to enable memory accounting with non-jemalloc allocators. - THROW_ARROW_NOT_OK(builder.Build(&row_group_batch.file_reader)); + if (format_settings.parquet.use_native_reader) + { + row_group_batch.native_record_reader = std::make_shared( + getPort().getHeader(), + std::move(properties), + arrow_file, + format_settings, + row_group_batch.row_groups_idxs); + } + else + { + parquet::arrow::FileReaderBuilder builder; + THROW_ARROW_NOT_OK( + builder.Open(arrow_file, /* not to be confused with ArrowReaderProperties */ parquet::default_reader_properties(), metadata)); + builder.properties(properties); + // TODO: Pass custom memory_pool() to enable memory accounting with non-jemalloc allocators. + THROW_ARROW_NOT_OK(builder.Build(&row_group_batch.file_reader)); - THROW_ARROW_NOT_OK( - row_group_batch.file_reader->GetRecordBatchReader(row_group_batch.row_groups_idxs, column_indices, &row_group_batch.record_batch_reader)); + THROW_ARROW_NOT_OK( + row_group_batch.file_reader->GetRecordBatchReader(row_group_batch.row_groups_idxs, column_indices, &row_group_batch.record_batch_reader)); - row_group_batch.arrow_column_to_ch_column = std::make_unique( - getPort().getHeader(), - "Parquet", - format_settings.parquet.allow_missing_columns, - format_settings.null_as_default, - format_settings.date_time_overflow_behavior, - format_settings.parquet.case_insensitive_column_matching); + row_group_batch.arrow_column_to_ch_column = std::make_unique( + getPort().getHeader(), + "Parquet", + format_settings.parquet.allow_missing_columns, + format_settings.null_as_default, + format_settings.date_time_overflow_behavior, + format_settings.parquet.case_insensitive_column_matching); + } } void ParquetBlockInputFormat::scheduleRowGroup(size_t row_group_batch_idx) @@ -561,6 +576,7 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un lock.unlock(); auto end_of_row_group = [&] { + row_group_batch.native_record_reader.reset(); row_group_batch.arrow_column_to_ch_column.reset(); row_group_batch.record_batch_reader.reset(); row_group_batch.file_reader.reset(); @@ -573,35 +589,55 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un // reached. Wake up read() instead. condvar.notify_all(); }; - - if (!row_group_batch.record_batch_reader) - initializeRowGroupBatchReader(row_group_batch_idx); - - auto batch = row_group_batch.record_batch_reader->Next(); - if (!batch.ok()) - throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", batch.status().ToString()); - - if (!*batch) + auto get_pending_chunk = [&](size_t num_rows, Chunk chunk = {}) { - end_of_row_group(); - return; - } - - auto tmp_table = arrow::Table::FromRecordBatches({*batch}); - - size_t approx_chunk_original_size = static_cast(std::ceil(static_cast(row_group_batch.total_bytes_compressed) / row_group_batch.total_rows * (*tmp_table)->num_rows())); - PendingChunk res = { - .chunk = {}, - .block_missing_values = {}, - .chunk_idx = row_group_batch.next_chunk_idx, - .row_group_batch_idx = row_group_batch_idx, - .approx_original_chunk_size = approx_chunk_original_size + size_t approx_chunk_original_size = static_cast(std::ceil( + static_cast(row_group_batch.total_bytes_compressed) / row_group_batch.total_rows * num_rows)); + return PendingChunk{ + .chunk = std::move(chunk), + .block_missing_values = {}, + .chunk_idx = row_group_batch.next_chunk_idx, + .row_group_batch_idx = row_group_batch_idx, + .approx_original_chunk_size = approx_chunk_original_size + }; }; - /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. - /// Otherwise fill the missing columns with zero values of its type. - BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &res.block_missing_values : nullptr; - res.chunk = row_group_batch.arrow_column_to_ch_column->arrowTableToCHChunk(*tmp_table, (*tmp_table)->num_rows(), block_missing_values_ptr); + if (!row_group_batch.record_batch_reader && !row_group_batch.native_record_reader) + initializeRowGroupBatchReader(row_group_batch_idx); + + PendingChunk res; + if (format_settings.parquet.use_native_reader) + { + auto chunk = row_group_batch.native_record_reader->readChunk(); + if (!chunk) + { + end_of_row_group(); + return; + } + + auto num_rows = chunk.getNumRows(); + res = get_pending_chunk(num_rows, std::move(chunk)); + } + else + { + auto batch = row_group_batch.record_batch_reader->Next(); + if (!batch.ok()) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading Parquet data: {}", batch.status().ToString()); + + if (!*batch) + { + end_of_row_group(); + return; + } + + auto tmp_table = arrow::Table::FromRecordBatches({*batch}); + res = get_pending_chunk((*tmp_table)->num_rows()); + + /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. + /// Otherwise fill the missing columns with zero values of its type. + BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &res.block_missing_values : nullptr; + res.chunk = row_group_batch.arrow_column_to_ch_column->arrowTableToCHChunk(*tmp_table, (*tmp_table)->num_rows(), block_missing_values_ptr); + } lock.lock(); diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index b5b884b5efa..a737c695fd6 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -16,6 +16,7 @@ namespace DB { class ArrowColumnToCHColumn; +class ParquetRecordReader; // Parquet files contain a metadata block with the following information: // * list of columns, @@ -210,6 +211,9 @@ private: std::vector row_groups_idxs; // These are only used by the decoding thread, so don't require locking the mutex. + // If use_native_reader, only native_record_reader is used; + // otherwise, only native_record_reader is not used. + std::shared_ptr native_record_reader; std::unique_ptr file_reader; std::shared_ptr record_batch_reader; std::unique_ptr arrow_column_to_ch_column; From 8172f6cec023df144ef20a7cfd49b43548cefd41 Mon Sep 17 00:00:00 2001 From: copperybean Date: Wed, 21 Feb 2024 00:17:30 +0800 Subject: [PATCH 04/36] log duration while reading parquet Change-Id: If79741b7456667a8dde3e355d9dc684c2dd84f4f --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 11 +++++++++++ src/Processors/Formats/Impl/ParquetBlockInputFormat.h | 4 ++++ 2 files changed, 15 insertions(+) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index e35d53dc4f4..7faa7300416 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -673,6 +673,15 @@ void ParquetBlockInputFormat::scheduleMoreWorkIfNeeded(std::optional row } } +Chunk ParquetBlockInputFormat::generate() +{ + auto res = IInputFormat::generate(); + if (!res) + LOG_INFO(&Poco::Logger::get("ParquetBlockInputFormat"), "{} ms consumed by reading parquet file", consumed_nanosecs / 1e6); + + return res; +} + Chunk ParquetBlockInputFormat::read() { initializeIfNeeded(); @@ -683,6 +692,8 @@ Chunk ParquetBlockInputFormat::read() if (need_only_count) return getChunkForCount(row_group_batches[row_group_batches_completed++].total_rows); + Stopwatch watch(CLOCK_MONOTONIC); + SCOPE_EXIT({ consumed_nanosecs += watch.elapsedNanoseconds(); }); std::unique_lock lock(mutex); while (true) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index a737c695fd6..a94637da942 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -65,6 +65,8 @@ public: size_t getApproxBytesReadForChunk() const override { return previous_approx_bytes_read_for_chunk; } + Chunk generate() override; + private: Chunk read() override; @@ -286,6 +288,8 @@ private: std::exception_ptr background_exception = nullptr; std::atomic is_stopped{0}; bool is_initialized = false; + + UInt64 consumed_nanosecs = 0; }; class ParquetSchemaReader : public ISchemaReader From e0179150c1671f75f9480ebca17c4ea2595ae811 Mon Sep 17 00:00:00 2001 From: copperybean Date: Fri, 23 Feb 2024 01:09:02 +0800 Subject: [PATCH 05/36] Revert "log duration while reading parquet" This reverts commit 5df94b7f8955b541ae37e4bbdc13a1fec9ddbbd9. --- .../Formats/Impl/ParquetBlockInputFormat.cpp | 11 ----------- src/Processors/Formats/Impl/ParquetBlockInputFormat.h | 4 ---- 2 files changed, 15 deletions(-) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7faa7300416..e35d53dc4f4 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -673,15 +673,6 @@ void ParquetBlockInputFormat::scheduleMoreWorkIfNeeded(std::optional row } } -Chunk ParquetBlockInputFormat::generate() -{ - auto res = IInputFormat::generate(); - if (!res) - LOG_INFO(&Poco::Logger::get("ParquetBlockInputFormat"), "{} ms consumed by reading parquet file", consumed_nanosecs / 1e6); - - return res; -} - Chunk ParquetBlockInputFormat::read() { initializeIfNeeded(); @@ -692,8 +683,6 @@ Chunk ParquetBlockInputFormat::read() if (need_only_count) return getChunkForCount(row_group_batches[row_group_batches_completed++].total_rows); - Stopwatch watch(CLOCK_MONOTONIC); - SCOPE_EXIT({ consumed_nanosecs += watch.elapsedNanoseconds(); }); std::unique_lock lock(mutex); while (true) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index a94637da942..a737c695fd6 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -65,8 +65,6 @@ public: size_t getApproxBytesReadForChunk() const override { return previous_approx_bytes_read_for_chunk; } - Chunk generate() override; - private: Chunk read() override; @@ -288,8 +286,6 @@ private: std::exception_ptr background_exception = nullptr; std::atomic is_stopped{0}; bool is_initialized = false; - - UInt64 consumed_nanosecs = 0; }; class ParquetSchemaReader : public ISchemaReader From 18b3ebcda363eb7e9b8f52c7170d8bc208bb9b07 Mon Sep 17 00:00:00 2001 From: copperybean Date: Fri, 23 Feb 2024 01:10:22 +0800 Subject: [PATCH 06/36] add test Change-Id: I53ade40ba24a742a21f9e09dbab7fff90b032b4b --- .../02998_native_parquet_reader.parquet | Bin 0 -> 76392 bytes .../02998_native_parquet_reader.reference | 2000 +++++++++++++++++ .../02998_native_parquet_reader.sh | 210 ++ 3 files changed, 2210 insertions(+) create mode 100644 tests/queries/0_stateless/02998_native_parquet_reader.parquet create mode 100644 tests/queries/0_stateless/02998_native_parquet_reader.reference create mode 100755 tests/queries/0_stateless/02998_native_parquet_reader.sh diff --git a/tests/queries/0_stateless/02998_native_parquet_reader.parquet b/tests/queries/0_stateless/02998_native_parquet_reader.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c0d222342e31969fd5e6b4fb0fd8d0ecd4a822bc GIT binary patch literal 76392 zcmeFa2UHZ<*7jdjLRG1#7*NcZF=GH4V*>`vn7UfKN)ZzV6f^rubDC=l>mf^Z- zqw=2B(?(r=cDA9rxJhB;L3FaI0sDdNA@0=`ljj(2m|~TW>|lDI&YqoX=qYYiEMx_Dy%Hgw@r=N&ukx2)p2rw;f`sW@`W8j z$LpG~3k;s(c7>OR(sxZw*{^J0alfv(ywGsZv_tvEdeH}T&DceTe&SB0gglH+F*Rqu zv;D<`x{~r@!+len@`LrJ59u7(C58dwF2zb7PCqcUU@2~(n4l{qFEugLPZjg9P zS5^)-q?-0BmYffLT<64w7zT^`lydSI+H7)WSW)MY1 zsUVM|(@icc&q-pU&PEP5WSCT?I5(a?rE9}R7>0=PN=4b1&NQ`UOK?NQ)4ED>q~WP) zzfzK$K%ddIW0xDe!~=@0Jdu88YR_76!^E??%JK@sbJIbk6z4~u({*518ob3rN)>q$ zon`9Cmga_w=XF)(RfZR)1f>i&nZBUYva1av#KTH8c?z9va%HW#kz$grx*TQ5F&$CL za#QJxIyW}jFiJeC)R3pqxh8kE95-6Lq^l{%81hWVl=9qk`m(MQyT;%n9#`z-8T3n2 zXSM=2M!ce{C9gHSGM!LtIDh)8t_!=)FjhRN)Rt$``KGRHMQ)sUO;<-=Z+LA=R4Q?^ z=<7NUc7tKOcuJ`&&!*p)y0NyLub8Z>CvP;oHJw%}b93k$y6)^I!vyh+QeU1+zcck< zt8f#=o4N*atl_=stWuSmN8i%*WH%f9#B++hJfHqx>cv*$CW*Io4dpF{kEZiVbuNIu zqwCFXHB1&SD2?O=bb+Z4TZ5Y--qkghw;4W}l9Za_@XG~j~iRGpqR8s>;Ml-67u!jwS;$uZCN7JR1k!))&NX*r_%0~=klqZTK7enXiMzKc?!D6c7Cavd z7b3pYxy#24WfimH%&nzg>3rDZhEOq0=_IeC%PC`64HqWn>pIIP4CR${#f4i>zt)Xq zPa49-45f>_fv%v8W7}{M;u~F8IniLFWGZdBjr3dHc=nVbQhcg-$eZYjiZ9!aTQ0uS zb(2pUDk;yD_FOFeUN?a~V^|?RSGvoaX@wc#IL%(@+Ct}<)zY@i=)5krn8p~YsFVe zKY15zr_5lxaO=eHy8iMNLoFp=>B{YCWn?(y@kbKk7K>48b;!L!qFps@u*dl&Z2FnUAqPvGQ5z%k8I&3k%tMh8^M;Wr%!$ZlWw=`*AzP z62ee9#n4pws`Td$(j|q(?0rL=_)YPW57Et(CF}rhmuMvnlOGtGE8mrYTmoH6Sjs*$ z>=u70-tuAEL0QJ?xIJQNVYvLr&_bb9fjdH%5dzuA23e%l5%N*Gr4q#IIiqMTjFg`k zS}BY=h&x7?6@uAR!(OqNI!ZoHw^l;f!Q4KvoG@B88ypo2B+v=Eyb#K!8BC(3>LZ_| zos=+E8RFPB1%BN_J62T7P;>C)>I62ecqVVca?lfIV zh-9A{_KU^U@$wnEjk28e;tq(mg0K9{&{ip-4&%$YYH)Jp5d5SR-Gzerrne^tPgiwv=gSu zFAeTWIdu$og{~#6WnURih~?Gk@>RN%vW^|gofK;eGvs_jXQhHVj=M(J5!SP>4T++S z>MviXyC@sj@!Tn~t}s)6W9X_>RDHQ*x}LC+eQP)^R#IolH)s!K6FY%BBi0vY%kK=` z6kBy7cav@)#Io-VXT{3u9QhXAUD?d~apy#PVXpka&_k)BPU3FU4TUZ2N5grssya`; zL-$m+vXi+BVk2R`Twv&>R8yyLcj?B$HujStNvy60$oJ^p%64`tcTsF2ERa7N`Y1Kj zX|*`7tD=LjME-8*uhdp&a*ya1!fy75;hI=ST`E7O2Pk{kS=@E8rLat<#DPj(bvE~e zZY9VpEhdZg)Id3v)+t7I4tGOrEd{9Nb*g;q+TZzLI2Xz^jO?MO$*izynv4y%y&Y``P z!)ze;Skwxu<rHXDsv}`SoR9dSc+)LVB zIL4M0&7z|kBfp|YDaYARE=}wttdYx!qZKDLjLWAx3n$p}V!G(8u9aWYKFUcpoXZfq z2U#MtJytozMsiO@4`G8`Q5>hVQI~V?=x)MkwvzZv zY^!dR-_zrjGwcfPx!7IUB-@I!NR{ls&P|mTdxEEqiVY6IC zoTzkAS91k)FX22}Rm>JUs$1kww4ZW;jpA~|-ojS7nm9?(s?pqMx{r{=Ru^+cS9P2G zg`TWjWMjBI(Nowi*AS;DZt5EDE8SPP#MTsFitg$T`5Qe|xy-KRUWxsLowA)cP3fer zqeE>B?1hJ@;B1AncNBi!+oi>IRNt1`5~MI^r9#tGZjJ8Gq$E zyODb<>V!RVU2&%3p>E7erpj_LW|ne;jpg2pdci2y7iTNo)y8WnzSjHgSW*dqHVlUMsbId&D4!e!}BqEgM zM&f*>x4NC<8A-UyHWojNeNO};EksHhs3yo|n5D{NR_17Fgm74HDK1lVs*$s1MhZ{ZRw9O_9+As3fl4a7 zmn$ZX5{}BP#UMqm?&HcaqXjeTC|XE^)MIjaCRj;hO`N6VBOI5V#1Lh$s&EyUF+w`) zEV7b8Jt5mLp-Kj;a-1|)I4NtyFhx}3xr)p^fr{p$b zgfc`uz}Ygs!ZWt5SV9`Ao|Y>!k;-%SAXieFAe@ogiOUr)^$=HunJ8qj?L{kTn0i*O z%B)acunAl#$xk>ZcMw-9-s)kl8Z$}AW;=?drQzy%xjM5-$zhLhWu(c%1z9VuRz|2t zxf;wAA(wR(t)-D_l3bICQu5eiTv=(Va8Y&>qm@zWan6pJCcI?b#d6YU^^#nRiBVp$ zC%E#`bm6kxNnE4&s3*DF%nTu)?JQQ1#;8~1I?P(-HJixUNdCfAxr?|?8LOV+>M}D0 zSlCrAHc=Xzn2QgBN*6V?m#WzfhN+!@?A4bO*~KWTj$3=`-P=;TiM5Tgpox*KD2-FE z$@LgwWKVOIq*=msnHbse>KU#+Gg}}=)>iUWljQ~sF|ud5%F-O+hD?m?1oa$e&&(Bw zk*y+4RBy@+8DeD5b5*5z!Y!E?SwHmx*NB-f5F=Ynnxx*A8#BbnCUMoJ0O5{IjO=9f zBG-gjAP^&4Lz<%Am76le$X?=VN(+U1GBL7K)yrHnW|2UQterGXO_7^3#K>OZYDtTQ z`!X@I)77h-1G7XRMz*#zLwz8(V2F{u#?_IQ3J+ysWc}6aTuWw|K#Xi%X{P!}Zp9EI zo6Oad0)@viF|xDN8(eE9NFYYGzBF5XB0Dm~$ll}{NWns?OpNRt^%m#Egb2jQ+Dmg) zv+T?eBYT@`D1{1XGBL9A)H|Gp2@{BsZ6wWC(`6Ti7}>jAV<}w7kcp8EQ15YVm5Dt)(C}S9WKJku`IUQjCx%6C)d}rg5E^H3BiRPEv^aQtr$UBb&}S zOKXK!GBL8DY6jPZStk%9tC7Oge7P$_jBF<7BCQu*%f!frt4}!(W`jVCY#S*;eIs{c zh>?B9wUsssZ)IX+Bh}|zcV?47jBGn;x%y7-!4M;x#kH4Wh4(TsvMbaVTu)}RK#Xh$ zX{Gu>?!^!zo6U8Uwg?|(Vq{mTIb3gMt3ZsbR$8qV$bA@MWOF%JX`Ap#CPp?&&Eq_o z?E*2fZc?=RS?}F5F?w0A`m!jBFQao%&rK$Pgp@hU+Tr7JkUY$gWr4ayn*@K#Z)1v_Yke0z-`KJFc6A zH6$Z3vK!U+oSrcX#K?A+HmQtp5JQaY2d;;-S14vAMmARc$PH%p3B<_uls2mtMgv2P zYysCxG6|MOVq~|dpE!|G1Y%@+OIuaeC^5vye&+f}=-Z9N$Zk`=a6_1Qff!j&X}iiB zhcd*-e&zZ~`-S30Vq|xy-#9PkfIy6FKWV30!Z?f}M)o_`Upgq1G!i2lr~cr)nL`3G zvIC@Ds+Dm#LyRoN50nywQbuBAcdIl%f;lV@Bde44sHKe~8DeA^IF}WvRaHE z#T*rgk=08^)!I0kAx74MA0!$nKDC^23`2}8%NwKesTGXl7-D33%od4)jgc5xRV~hsXHE&k$PSU>)rv-6h8Wor{7~t% zP{~M)?0&T*KY=+T5F_g)9Z+qJ6B%M;t@vTmS)sC#7}=2a{OrNl2Fq~jO;PBJU^YeED$5>BOO=mj58QwWGnDvq$@%#BQdfkR2$x( zxhfDNJ61ZW);7*$h>@+xkCUzmb&SNwCaRVAS88-YNQ~@RwJJZ4xg`)I>nEL4?TzypVq~lF zlcd{1LnAS==hfY^2`!Ao$X-+H@qx@^ff(6Y(si|^F^C~Xwmv^wdLpzk5+j?e zHsFJqRDl@TInoWawK0StM%JF6E13mHBQdf!)rNd1lO_-&J5Rc$IvK+lVq_cf^QCmb z*+`7+ZM88U&SVJ0$OcGvRE;r$Ax5?dzd*_qT#UrX-c_6Ok<3$p7}>l{FO#x`jz(f+AE~YQC?-cBMmA7-tZI$X3^B5;`5-A*a5WMm`$To*W0*XF z7};PcRdqA2VTh4+;zOjDg1eCzS+nZQuVr2d#K?w9X=*3qI))fo4Id`u3!RO`$fm0< z{CeiKK#Xj-l%aMpZeWO!ZNo=MZ-lN!Vq`Pbw){rstw4-yr1VtvFm7Uqk!{B>m);58 zjKs)3Q`_^g%zJ?t*%i`rwYzaMLyT+(ex>w5=wT#AHcRcuZ(%+P#K^9aUZ_2dTNz?x zwft(SK8+|W8X00_yYrhQ25x~E*>`FW zelJr@PmFA=^j_5)_c6rC_T)E97WzR(Vq`z4y?7I2sV7Eui}X<)Y*ZLxWP9^lC01`R z5+hrn_Tg2A(-R}RP5PvY#(0JpSxBl}tH%kO83>xq%wA$?JY7!NSS$oAuR zN+t9|jl{@)Rr~V?nUZ>9WaFf7s+aK)LyYVIewSpWA7&&*_PaWePhd*viILqc{ZPG) zhZ$mIb^IQww0^je7+ES_;Eyn6^u);G`;7Pz#-j`|vU=VqS?fm{iIHXE2l2<4vU*}< z_e#a$M;VVZ#K;cj_etgSqm9JKTErXp6HIwMF|sDfGTz5{k|9P`jQSw4O!f10VJCq{O^R6KsX@eD(ZtQUVkveo+< ziIFW4Ka4-iRMrzCdr&GFKf!p8Ax74lKO|MrPc#xEYZX77KhIRv6C;}-m5TQ>USNok z9l;-#s_7>giIFWGKax*ks_Ti7JtCEfpKQFy5FgkD*JuOv=pJlwk5F@}%Q{8Hm%h8Wpd{B^0NewmRN*}Czw`6o;(Ju$M$QoZ;>U1v z)LI{8Bu2J={9NA5IO>U!y(u+_4>qPT#K_L$Z%I!25F;_N_VM%ibjDdvjO=ZxVSK1D zgCRyXfWISY^kGI~WE;gV;4>K)Ju$L(rN;5$#-|K1vJ3fpQX74Qkr>$~@r(FpOj|uM zvMEy2_(UrCH&pi}y=$_TALpQ8$TQ&4< z;=SNdeecDXKV;`U?a-Vv@A|Il@-y52Jn!HjM!VY8R9`5F~|&$7^rTKxQmHNrA~OeMI{gBEa~-^o|3`-u-- zVM)v=O;3vEOxWZOQcIOqt-Z7oY9$k4;>#Wl+n*7i;i-kv`!)$Exf?+ifw6 zRt%O8+tp~89~&1nenK!;qMH5MdMnrEE}*!eXGz=YkFyK>-h=x2&%qv)#oV#qo6`UI zXLnP&?ysBE6ommtmatjR*sT3xZdP=>ftLBhtjTywV1OUU_rl%r(+C6o1MGh>?thH) zNDqpd`;Rf=U@wOG+ecD3Ox4%sO&y!tiQ67o6*c+yoiybz%uF)OeDaB2p)KnM-~cCm zrc9tYe)0%^yw03D3a?Wqj>YT9nN#sPZL~LD{l-nl>#%8~@#-^t242U{n2guyKEr9w zVuZh?hVm~yfErV305uLj!{|Er^>H*ujW0F8!h1Sh+F}@vEJLZ`Q2CY`P&}0PpE8ne z#E%&<5h_3Y&L~{LpU{lP(WFAn@EK!h4e#qSh2~0`E&Rvg!qh^^I2=tYl#KTFr8O3l z##_Fp{Bf5FxPU)#mx(x?mu5^JL3gBl8j|0eG7=YYK;A^*?|D!6!Nu`~ z7iWz5^_v)e;l)wI#?u{p&BgC|`~Lbp47>2s*pYtty{U!2H)_(aix@GvXfa{TNT%>$ z?6hB3FxB4%fIj*{Qm4cP!s>R_x-cc{(dx+A9>inKleX<7YKkcySo1T z?Z2PrA3xK-pXdKPvi|sK{`E6|=jU%3{&#-<{3NLVtM#_-xapHfXM>3rom-r*O+a0n zP4}Ws6ffEm-FDJcg=Q^jG8dC}a}`~pLM&-!uDz6uxU6Z(>H0KhT+wF`Xs!Lnlv7-erSO>iy4#ACSfwChNDp#iJz%x zlSmuHQKRrTOzPxMv@V#|snLb4$z)%DG(e;c;;6CE%)}M^2@Pg=(rTfRnm%bbI-t?s zmhaIcOdf&r6ABgRNvMg13evZAv=}$4P%+(yv?FMgFj+t`rBFedwT>1O#}>9o!^RT@ zCKGfgq+k>_X+ImJk$%4}p*JbKM0&HIm&Q!}bqVjI@Y1C5GtkR;H{3uP1PfpEhlMQ? z-a_H!X@1_gJgTtOnS{2m@FGTEc#-s>KU<_}=mrZfVc3P2M);5YMVyH)u<#PbOD8p@+A+$nm@bi+a7v+-v|CS&#CC3{0lp%KU)&|xmYv-4Od~8 zRAf!4@Cr5gS1*J{v2fJGru}M?rvJKt{*YWi!TNh~=no5Me=jEdjE(=-umAs_D4aIX zrOf}`m#|>`#$S4&{8|#FZ=0qYioQ}~E;jhrb49QJr(%0E%zQ_OaasAty{rS84_s^*=7F$RU;_E*JO$@&A+E&}< zeo{*3qN_Xio$8V=yta__oJ87EbDQ)p+4vhy7G0m2IThbO{al~n$u~Oq!VBvXTw#;K z@l!^Ub)HFuUw9QR)bOJU*LgW-1 zJY9`1Y?%tb9>coI&p)9eUs&O*F|uYu`9b9~9N)%~bsGvxEaXpou|*mc@)g-+{7l7K z&-l^B-{X)h7-9Ja0Q&5r53F8)azEbT7<8~ARwzn9q}j% zSBjc;!|DvCfBe1h%e}%2WPzyg0;Yd*p>S=88jhlZFY$g|!aE@M7&D@CXKFOLLKcP! ze+i=|SA2bNWgM;`nAhM087#RO>CcW+6UhbN!V5@&8Teb#6!;krQ2#za{|6_i34fjo zBu@VrbwvP$hlMY{6_$UVDKfl&{00fue=7dMQIVSz9mqd!_}g3m&nJrh`p;v`kbhbM z{!v$oz>D1fpQHZQ>%Wzhe=7dMQIVVcJp6B;`17Y!^kzkmfA^q@uzvpz|M8&yt)gfo z{?~7_$Suer8s)-Q{L*iS|J8|~PwDr{qPO{X$N#uJSzja*a;aahzHR<_{PX-@IrG~c zs4+pm?)2M4|J!|kQ;@s=@s35W{{Gl^H@U-qekIe+e^&jKJ7IFiH^G06`q%FFb6lj= z|4(-%EyMrrUHp8W|GRtqbCkb5&!6V)zw>VYwVuYM*LFOBb$d@M^-V=b!{ zezR1XA5GR(sBySL`j*lb!)IV^#eXbWOc^%{i!kIHAwROdVK;UpVPGWame!ZBiDPWu zB^^pARdgRyxTU5dG|x!8ltNmiT-q{!U~L?nExPNf3O^xobm^Xr)cT9oWd;=s368t? zw7$Q0qId1?la=DX?YdfEW@kQ2+*W&B%Y<$lHKw@0wc><@4cI32x74k)?)ST{{#GhNQ{)dDA$L1^b zzu0_bSvcHPM`@dl_}L_9q!cUKjb4)8U2H~%ToGeDLe|p?_lhO5I0`OCJ2*X`u(TsZ zdFbj^vcdSSGgwAar15IG7dX&9W`O~dgidto;M`d-K z+o(h>iaOn;9;Hvce|`Xla>cqHd8e#WKU^tlAXu7= z0ehmc)K>{C>%1qAb&eD@60Dfed5^5!C~7rWdZ!A*jQDS#YzE6#JMZGcIuvystiaR< zBd>B4bswzA(A~{XLX!rTXH^$4tl)Q1*bN+3MJN(|j|%^8}PUj^80dJ9&5sO)Qk<_lP+sET)&K+_Pce5agjpKcU2 z2dtzFA&{<3Q60cac#(D_u^mN)f~5^waCjCp$H2;Rs??zpwh1PI6{lUWBexMneE`eW z{MgO848o!;o>7?5sv$JI3!ZyarD@-xaRSTEIX5T`krxD(wOQX_R$Yqf0an(F^g%9& zhc#gN1ufLSh2|nyw$8SXBVc{*fR&pQrgr+P<{PfKe#tVMS-O?4?6z_pRFD&+w6=21u?S&tk|GMAICv+7c39w%5!)d zib@BIcU_eExH&~xgXM2d4Y033QB}YS2;0}a3N*FB(neQ~e+7*vSOLzPoH{TRwGAvk z^Wc%=8Y1?<^2pANZiIM<2P-+q(b5x|OtAc&t5jWpt)Z{L(zq^Oq(fd*2P@8OuDKbh z)c`D0nCZm|XqtoN7hR>F4>S^3Cg<2Y2Q4T{0V~#Qu*htT*aOQy`>E4b#7rVsRIt;w zBxqiP73W;_$fwE_^%X21*Cn;Dx2C8@V5OMTPE^ME)?k^#l)#e6opxZwMpu1phGr~S z1+6wuZH~M+0amhE+$uIh?12@R{p?Ov#KTpv9D<$QU7`67R*G}AL6fUdRB;h;>$>z* zSLB5=SbSQ#v;rFxT*0yqR~sybrW;tv(bXacK{Ep^`_@~^#}%ij%V1GylG_~z#2#2F z+0Uo0LOeVG%PUxuei)jPt#MCH^@8jgs6t>Rxi0fcbfKtDU}@4azLvoG-e76NkKR zu)NZS2AoG-+5wg(XKk(5G)27!D>1l@-5zM_f#svA>GTLS#{n#Rx1fzm2Z|a3R!rK{ zHeZpOK48U$AFy}{jUQP4t7`h5fo3CE39Yy1mqlK@0V^`it7b{kcV__dqCHr8x8U}TkQd{?N=kcne=0t*X<%gqAKW?$nz>-bt+KOf z2hAR^vRdzm2rf@iKfp>%8+PIl>QW+DF*)mprn9J-VA+MV8?qjnj$kEeYE8Y0n$sPu zK(~-bTihwiAFP73=l*#OC~6^CwjqbyQlSY3D`i!!J^P_a0L#{KXR~6+3o8NRPxF=> z+9USBO3K;reK6vo8d#no?Y~ukrYBehn%d>NATI>465K*(S4Cbd0n0u;tI{acq6n}& zLK4yhXrjU5SJ&>|6q<8jc{s*hosBuGGFW!$!@tI%EyLsum&`#V8uD^F6oOos})!Q=_A{$MO}IamOeN3 z;aSvbcd*PM+6VKXnF3a!>lviz8rBsktjXP>arh#fKh~+CXy|tfbZVTF-{&Gg!Qn+-(T*q8C`^ z^wItys7s~6O32+(`5qZiD5O`tpVgYSCk`d_qsGYYfd(u0C&S3jvC78Akp%cy$Ri^0bP zmPg+94>b`Fi@{0`?fm{eH2cAdbZS(xB=X`USf1L>={=AaFTqMo%`ej!wdf;QNui1N zt3mSvtiY&;CJCBmU?n-3&S{Ys3&4uas5bmQ>XJKH{&_oA#~>b-gGGgPS=A4kQ(z@J zHQv1lHRlRg0opDL&Xz(C16F?O>%CjG6lGBk&nWbi|7vKg+TgiIJ?e88nzmpSI4L6z zH>0R&6iM@B(PHQ;%)^Y z9`=CcALMb<9h!$=**Q1Onu?n93@o$OHo#VIc#+(%gmP2Mut-BRs4=lUvU6UEaLjqWFLER=DgC+|sPv>UK@=$Z$ zf@SU6ZAdfZMQyMG%jP@{DI?EV0lERHqk=k4OXD@e%Ct4ivwW!o9$AM zqAtw<%QJg-+1ZGPb6}+eb+>ke<~>*e&duwOLd~Hh#I0-hZ#ycXhXKoEe&1+mcZ$-0 zl@NCB-3(~7VEIRzFDF1V5v&B~1Lk#@vrdB*XRbA`HRh?sU>v z2j^EnUR(z&#az3(5p`)fSf*^b&v?W`Dp;D}p1tcsQwA)vbNbEGs5w=^igoSTHl`YS z7_h9<;_jv6?e+)DHr#*nZDmy%2bVC2H$N1CguZciiItP|tj_Go5#6x+o3W7Z^ z@yH!_uwpgOdT&R~=>wLRo9Cff_0hwCm7FH`KUadHmVuQLKBxL2Xd=LhiFvjp6q@5; z<+nb%c4Alb|6nDhHPG;Oh&`}kbCj_cP^<00vJdGyW&t#wU?ppw=T%2uh+sv!_0_$# zM-KxQm2P~~9<^u{SbW6XW6hyi50-h=^SYMEol9WZIUaM#>W2Owtb#QAdjY6RsbD4N zsKvF2ho)fZL;CTbp^?C%T(Ux=P;TG%n@7A1jY4q2cn2Td|qv8!`ppF{HH$m)y6`6Z*No~Z#FtBVxxwl=RSpim}OD;bXHD?1@F&%Z2dLu6` zgT<$+rPjcI-2uxwbm6^VXdZ)=ygIj|7c?d7h611=7QSrAdvK%i?@GSlZA<(~BW@vcaOF z@}8tXV+)qnX^M9Po;Y1j`}n<=no|)C0@UX(}CSMNu=s^2liJ za20i_4On)0hrL64V_gWWxX=>G3N(ko@^O0gG#xePELis1LA?{&p@#u0CiOrzgY#d( ziVa<|{~eAk+TyuKy{dKznwDV2I!%kUMP4ig%Rj^6dS}d2UBU9q8|UYTc!&ZkCA8$k zM$nuGE5<2*u>pB;11!CEaGnG5;s;nssRx5**MYY!k7pFRw7)MjWx3SaBIGCY(TB8UR*6-uN;vEKoDS;=`;;Z-wR-SV>N=>)b}oc>-3T))2bO z6+KKhe4eR??BC(>R0N9(TlVHDG&R6VjCwuzC^S96qMTa1R;{Sa#8GV`OMXf#u~qv)UKr-4U=f znXP-+W1gA~mi5btx&06im%xe%E0be|-1!Wa-uWHX0eN9L1aa#+WK4PFMJuoZ%||SU zpcb_UD>5wjdLL*ygXJ0h&KX}KQ&Yf-be?r|3g)Z}VEJS^X2+r~1%jo|_6xaO8Z{Fv zbC7lL5@nHyuRHDF>E^>rg(l2YMK=63j=p-G%?^4pw4V$b=+l`hgV?{r>I- zXac}Wbe=sp7kPIZte8xvpvs-GCJ9zx_M}Fg5D%GPSqGPG_#K)GU?n(z=+GQ_Q4=hG zSFfEG$ctWJWtoq;_dzWZz{(E`E#(T$P_RtVAEs4=CJd~6=Q*E7V9rVd%WQVGUxB)` z1+0YZ$rq2IR=);I8(c1FIyAMw%5whrcqeL36R_f3%N@v~(Zhgcn|3_?0{qtqu%g+p%#C?m8~s06)@hmxCAwox60EH3DL#&fhwotd1(zTF3YunM z*=h>r*dZ@mz)Eo~KVTU0!Ursmv=a-os6~^&@(d5}(Gr@OU|GlPsyzjoIIui5^PF`E z{V!l?(_BU^LR~rrmTk^dZZBe{qz3sOT!Hn0rY%?=nom{NpyqS|i+8K=_CAXq1}y)y zlXi#Uzvh7z5FU}e8=9qHX=8Su%7NwpSTU{V-@VoV{r@10KP_j{C7x`7S`-47DLgW$4m2yl@{8Hy zH3FK`U?sHe4;1{Bx#v-h_Cl2No4lvC|-E27(o*`7$UNHOC7q zAGeAvE|)|P16E4fsUf@Jzhb~LhcCBZ4^1psu`%+GN6=gctDyCQ3W-h7|AUpBmZ#_~ z5qn_8<;=*egm`EPmP1ITj2qB+gO#HBTEHMLe8GxwtCZ}HyjTwwpMLsVZPcReU|C13 zxKIHaBUs5X#*G7^c?6cdi2%|x(x zmv7OtQFCU2mE>kS@t8Gw7_c)eD2KJ#UbqpR%A%!T4$h{1D3|+yVE<=oMm7YxK%Fgh`cxmmQVWG zwiR&xBv^hCt1Q{Bs1IOytlsyi3p5|V3Upljx*_tS8(3cH`QfKgmr7Q~J#%NBT7Y;M z1Xf~5mBhBt1cBw_^5f2U)SOjd*}GRU8g093o%#XMyTmQU_%ug?upGr`IasXBB&G*Mv1xKPucqULM{ zOYdH_XMN;FGFVCJ=Vw~t{C%*pLZaM1;y4YgxYbG(4K&tZWjQWwUKM#U9IV9jH&^0O zm+FBPlRM|fWW>W1u=n6wnbM}H2=w2;nV-@r;U=^fau$|Wn-WDv| z(CGAu(7XgIWwmIhar?!5LR5D#Hs1%%dUR|A^kVA;3XpLhf{=OS2?wnoFqn&@G` z@=Kj{?g9K4T>;N1w0p!AXo|PPbC25paWgc|VEH=*SuN3`{|C!6obVtsn&K}VpwWu^$aiKk$b%v%Q zSUyn)qN+gC39LA$;62~m(Eo!KknzDI3Uz4^So*wx*#i*|JHaxC+RZA1+_?u1E>#$mt1EI+TE7|GLhPud$e6S+5wb;+~(Zhg6ndffl zgj!?|79ZAYyc0AIU?oKzx?K{vGXyN&IrPQLuIT^4GG`PlU52_e1+0X;g@?N!9!`O! z4Xd3%A$MMbML8$5+kl$$4Xi|M?d_A17mdMkFwb*6UJBk8EKOMNlKY@(4^}}`!j#3( zi~~#K92PhlbJhv4tTR8=qtN38fR&ZENPiNw`Wje%VReL=(0m8W!TIn*8{|cap@>^; zolZH8(Zhh{Wu9+#L@jay%O|YQ?gr3w2g^SCaJg^L_=Dx+9Nxg(6a7C}+RV>q{W0_~ zuxwv0{^*Q&cnDT(Slth=p(%-Z)64nDtZ>wvN?_T!*3B4>yyy&8q&Z;zUihy*V8w)a zc8i5Z081ZzM41N760l;NwT@>gtZ$YrG$X(Yj6Ul55}FlYB{{p^Zil>h z23Bn5S7t2cshwc?zg+6r5b^LHEGqoomfq0R2P@I}Si*eNoEBgOxYn;V5P2~atbFsr z^1J?njtT!!5IJ3?^*-qsZ(I(X9rHHf_NwcmVfYlb4O^jVA*L-eDFif=>eA6 z)qZFfKCXvda$hB8h$^~20aW|0clIt7RUKWumZye zj`)J(7_dBIPF_id<~&$|n$Dgzkr!3K@=v4Y_ryH)5G>D}wste{b{m0}68x}63up#| z6`)CM?Twl<5-hD-qYci;ik?gXJHSIIbHsx4=r!ba@qx zIjaF!acOjobEr#MUVn0GJUYG$n$cjHG^cJoK+Ty9mY-YWcBYQ#VZbt{ zElc@~+}sUTR=BVw7aA3;xR_HVPD5h`i*oE5ZjHQX4OU7TbG$v~srO)+a@r5^K|FK< zOB3=~tPRanu*{m%llvnt=7SaM*5si*^1=v~b$Z~8DX3S6z_N|dyN-nBBv>gir+2l1 zCJ!tJM~|jKn6q5L;?s+X2T_-pD!6A(hi_&MH4`kKkSAZ(LbCuYYnL-+ub}3Hf|cyn zbk^oh=wZOprUzBbwTHI_%Ohft`7tzCz~a}O@rZ}!Ggw}Z-LBBci{4;q(k;F;!#q_6 zEbH8k>jxnohJY0llDf7$G~r-rUCx?1AurZ|MY%VtTm^Y?0W81t-~%I2i*A7BA2E2o z4x0O5Y1W*5-WVFr9(BO6`?y({vkYMQq+71tsYUF8rOnm0y=jG-306{w*(DU3bzu3q zoa=KOHD?D{4(`npR&+xT16FK$$biR9;cdZ+i!ju<4$U*LeAb*>x(%8NU?n>CDDf6~ z=L1$uI_qNRjx|ZJ{Bm6%bVoe+gH;fcmSTb2i32Ov<@`%W09eT`7sB?U<|Kj@>E7bQqTc9Xz@jq3R$PPsDgeta6#p|8XsGsh?yLP< zp$(y$gJtK`%Olec{XbX*>HLhcn5W#qO3rn+ZHIVR0hT^Aqv8i>PJ>0YNovvnd2to2 zME8thKFEu&U^!%jx9EymWNCxk3?1@C15GKgtfOY$oC!@kusof5KNpb~5n$P86t5VJ zy5tEKmDlOee$?tsU(nfYl?V~!Ab~yI^a1p zkHGS3b7`zC^5QvIcAcKO4nkg31S>Kla-s&cs3usZP%oE8&@=$cFKV{+IB0ZWnVdZ9 z^+sOo0xK}%*y(5ZY{!A+mDlC{cErqKu(Cp*y&Ve83$WsxE*D>pn)41UAMG>qjpFEG zz)DG7URr_wY66xybXdw3Xj+368#QNN1~kLLGCTD>d%h9+f3OlVjt?7J46z4RWM0=* zH4qQy!Lki|z9I#h4`8J@U5P8MMIY#exYa(N-yM1302XgvVQh$6mT(71w? z95uJM2#p_D*3SKgcSK&C0V^xx1eIcm*aIsu&!bf|;^7up9${H6`a)9-^Cs_nwd+FE zoYG(=X|rmaDUEOgOJiQya|`@e7qGNp!z-?WrZ-qr^t`;=(98u(>)fA7z?_u~mTl(A zO`ber53KyWZa2#y9?W3*hrLL?2#qyZ8s}@zKBMMT1*<^&;&@x+MR%}#%&T&$;Cz3u z{K7^Yuuml<(u)LxJ8lQ(I8mw67 zf!bY|v$DbR&ph?mwG?6xEYFuctZBrS_^sc5v;h(({qlXE*%Cd;APK! zGZ7D58$6@%Tw^O}oWV+RzHw#*YK|LNfv&l0x7(tJ0jt0qowT?Iye(K%_~;eWp;-u4 zV)Vi>hoDiwqBO$1wJzxY4H$psnX0WYPn`$L^s-m)H%(A8!P121^^~FM1Xh9bq8lE_ z3s0~TT=QINATQ>DWuF!^`Wtd{8CbUApYHfU<5wKdJ$e!E0L=-o95njs<&hUW)aqtnnSx&h zZ-8b9SbEKpGHopJ9RygmZm(vPL0-gw6`0ne!eG>*EnxYFe<|+`O)yxVF-yACf+hv5 zNX=k_1@fXXSOID03)Z79<$Is4$f~_5t?ye1!|V=RZ(+hfaT$qUoo&Y zdKj=0(t5_z?9R#AE5BaRj{(sdiCppT67Gol<;qT`$Mx0 zEK|&~#aP;=-hh>_5!bx$jQ$@ib6S#9IO-Bt75B{PH>MZjVIWxAkT;__7LCGSYvty%O`?rejJ+1U}@I`51Ru` zWw2r#hfW`gyciFbU;5?p1^8^6f#s1qu*)GW)`h@I4td{c5;QVc{w^UGOCc{3z|y$C zZ;{aqJq%cJ>3y#?MJ>t)D<*=z<_Jv+Sbl3lsQ1v+2P?_3>|%H1#VoL5)32z0s7r0Z z^3T;hZH{;d0*eaW*>4~;N5G163H4cunsW{;ANLQ*2IR$Cuu{_d`R=fRw*@OHg7Mu8 z%~P;q*Mx5Jg{Bo)1&-zF94m%=2P--KYTp<5Y(2n=%M}*KA!ee$atMukeHEGuV5PW( zaX(OVZh{r#{&C`M(!=z`=UHUGZb3vbs|LuOH(+J04(sp?8ZB5w_v=y_*YbxoMeKo< zlB=&%3-PcOtfKpM`8MG@W}xPzf|cZ6P`VfLq7+y~_v_k6TI|^b%@43_qrx8#fu~bv11wI_IP2gZb^e|u*-LLBxYT4Zu zxnorV&pj$)&OK-bgH?3Du2)8Kv;LT;CW58O8$7HW;^8=0MfdCaw2930LCyIDmVKws zJv$&T8h}-Fziwm_CZeqsG@trVF z1%VZlSE}?x#6t>LMfd9_wOLVnFha9DSh3o#!OgYkVZbW7UzZBw8!UvT6<8@zD+akj zvjD83`*rg(Zk0Tax)cRgQeJ7}b4$c6Sijq^>%8*RP1GDau#&ak8pgSyhXJeTeqD#K z;^#j@qXmnPURiJ$nsBg+?$@=;yuI5I^VC+b3i8VI^x_e>VEt~tuJbCd9>|NPU{S8$ zlk7U9hXD)!9s$|vz8*jMmkY9g_n+0eB-KT>QR~Sj_x-fxyQ9ryvN``d+3H@AF1pj# zE3CxGDY$ztj8YT5D!L`^z7C@-y3^M)^G>(DsAwjP(*C7&PD%+xJVyDuoxaYif1E+J zYJ*Waxc(TurYpK1jI!uX-^j3%7P-(22FoXUwbLGG;=n4p(>EaVZdQHFaL2*YzbqTv zw-h42EvkIcoxaXd8`~kHx`O59N^#|oQPaRGy3;o?%xdchXhwq-6CHJ@4m1bAD!S9x zlzDI2CU>l6f))6(T*FJK@|D5*-A-TU==KLttpVEt~VuV&5c z9jI0lz{+>s-|snt)ex|X?)3ExFB5PLnsBh}W4hP+1kGcxithB)K7BBvIJ(2vVA;O1 zVOyg^(t=fVr>~c0ZIznHsM%oIxgB^t)Do*9U=`iz`~Q*l-a%2bZ{IhFnk?Op$!^46 zDR!EQii*1y?5LZF9qT3p6*~$jmrAi?C3ftPpkl|8Y}SgMVnLeN5zLDHIk?|>?};TKB%|PZIu+*k6U0Ptb8YbEwUdk;FoTCtzm^U0a$9&H``}-s4MivR$xW0H2-8^eum#nEnzbT2o+UzjrI%15Tm+E(m&|_RCj>7 z1(w?ME%e=AzXWDZF|fG)m*2L6zNil@wdre3=1D4-Tgxz*3vO-pRLGRY0a5u+W{O z$DM}EBw(pcU!UTwnvSqKj0BeJ$7TWVU{+rTmfG|U^c!E;n_Kli{DG|U5**3=77BX{zr8a%Ti?^j6fVs2^Sb-ni3#Pyd={~U3rZ4R`mO5Vt z?p{oR_arD=N_2QG`y9gfTcEl(S&>4L&yvO zR^hhLM;qaP-V7|Y>04dABhCp<t9hlWtV5v=C8~^eCEuk;E04qD_#)0a# za2f(Ewdw1VaKDcMG9!WIx_x|vJ@gI-EVb#|u4HGcH87XX0L$i6t8oKhg=7~1vsz{P zdizhv-vu+L53ta_H~TTr7xRFnHhsera$eqm%p_n1Zl6#u1~Mmsr8a#-OLpD(1Mls1 zV0nLPZM_V$+7(z?YSXvp#KjGvFNOfir?2?>Z70|r0ZVQArX)OA9srrSz@oQLO#ca) z3&2vFzVRiyr!0cGbRSq@pW1Bg(HBmXfc1~*+tcWl05fMiu)_OF(Nmx=)&fgy`pO9p zwPzr+7+BnPeX~mT*jB)M`y5!wpW1di-5*Yrfc1~*+mk$32l`?Luu}S( zn-#zsW(%;?rf+q^ql+GpSp%$`?c}Eqka-L&wdq@4vX=|*2(<^S>`(0;wH*j2O2GQZ z^!1^}u7H`t0!!{IUl|2`u@6{k)3;sX;|Yf$vl&=6DOAFA$h-!Y+VpiRP3`j<=8_dy z6`$HiaKSJdfc1~*>oZBQ!pw;VR&`$sc?Z@or+}q4eM1wUd?%oHIAD3FOzQL!GM|8@ zHhqIjxtHU;q4spp3!gjGYYOX}5y1M#^z{kbVuG2I0IYWXZv6^^zPJc1wdot5_%vlZ zWKIGrEG6vrSjhYYmfG}+R3;DpvrJ%R^}CyQDiEX_V5v=Cx1_uUw$M9|fmM+*H7y%5 zEr6vqebKUm69V9Lz9+D%-@p1fqb}4fu+*k+JKt%pBcU&@0js>{J71wtx4=@HzM;O;Pv3%>a}QW< z{qHy72Eg4BV5v>tsH6g+5;C8F6|!S`Sq5YRfTcElql*rwc);oW2w;VNcl_YP zYSTB~cgC>(&=&>33htkC)(!eXGYsBOmFb(7^nBzT$ovFW^o|*E9UwCpSZdQZz352K z!!Va9V8wqZ%FS_tx&@Zn^v&{}`SBvmoN{1A^?xvCqaN;#084H9<|e)PQVbc?5#Ey> zGg}>i%vfNlP2a+zw5J4|&d&i>)`#LngJGR>0a$9&x7;^;LpSIP1z2hQAAZ$9U$g|4 z+Vn+}UnUzN(+F6FJHl_^keLcBwdw0xe00Hfm`hQ>D*sT@_&UsLGqBXAubbbjjwfN} z)S3!?*FUz_CmXoa4=lCm>y!M->j`990LyhJ_TVpM7+|SQ-@xKy4P6JrX(q7TK9-*G zgLTehV5v>tV87Y7T0>vB01FL@+cyLHqBpSArf+z1;k}`d@d8%hPMnT{%nD$sO<%hB zxDXF>X%n!5KbDOaU{=2dmfG}<@|!bjFU%Y*uzZ5n1iq>Rclv>)Hhoi)U(b^u696oF zr`F{QWYz;qZTfP>Cq~*1gVRi4MSXlz)e+V?pMa${ebfBrqD`PL+5;;*XziHGy? z#mD^p-2^i4fTcElQUCb^QRs{Qz>3PhRWuU*Zx02Q+Vl-fDA9XEW-73vJ!aoq0Vl2J zfTcEly-ULOK83kt23A&#&7Ne)d;^x+^!4#Kl}5wN2?17G{_Rz_VI4IASZdRkPAGkw z1{oSy=^k_DeS;di2`shg8&)#8*KiHg9~f+VtfT%Ho?t?<@yap~qZD8ctvz084H9CYMZkZicz^8dz?zwF*{2rXH}= zrf-Tr(=q~P&TL?z1$VDrf;Cbku+*k+PQshEJ0Y_USgxM)j+aA?y#$up^vy1rx-<~# z;S;ceW9^pwfD8ewe@x$=5!c_r%vl7ikZ1R(?l6m10ZVQA+9Z}s^`UpR11r#TzJ4;C zz`h5T+Vrg`nTDN*x%3lQQL(j~vXJQrtba^jA9nIom^rb)ihg$g_eoeIB?3!r`g$k6 zoxUD22Z2RVP287yF*|20!wZBx|Yp;yaDQAFtAeM>>mw;%wk}vOm6*6mpr8a%Td^@(sp)U%7m6Q8)?2mr1{s)%Y^o>s1=g}WBHcl{eycYIr1AQ?H zSZdQZzG!a!6;KaTf#tf|v7QfPHUmp-`X>8!k~hQk`XjJx@}4OXFt4n@Qk%Z%N&D|y zgp3ogD!dlG-V1kq!hoeVeY1+@Wmwz5%mh~8YNzyc$Z)_?o4(n;9y1-GFMa^aJ1=+3 z`$4e&_k^FRGJOk^4$SEW84OtN-civ_pfBbDOKtj=7tIebK|L%77QNbeczej41eV(L zt?>1%8wb~GWEy-%d3jx@!Mt(>mfG}nO+ILU5;ARp72>_PSu))9SpqDz>FZW(`uwIN z%uHZ$t6e_rgUkhBsZC#ZKd(c-o5I~tU?u0}KYTs}*8jj#o4$d`hmN*}-suLcXzwLg zQ0R-bz*3vO!Nqi97}UddVCAfKO~4>y2A10N4e|31S_pfq=D^C%+fbMdCD0LAYSWia zJ{-IkGJSz1cUnrWg1bIjfTcElqly`iJa3qpz_N*{*J(3k9s^5l`bPV8F8)No-B4gv z^*^{Hht5IBkqiVddL8lcTD};4$wQV zfu%Nm)BU=vp4lAkh62kyKat?;z{~`e+VssyPFs@%nTfz^=dpZO1oXuzV5v>t++udl zt?qC~8CYR44Q9te<`b~grf;EN*XD&S;ch6fLh_R?cYs+m5?E@}*Jk6<)+qE&IIu!J zR`|USg1slO)TS?55^3Kb>ftu9l4BZbenaLbu+*lntADrbQ0R*>z>3aKHXg8tnF%bl z>Fd4m*v)9jEC5!#$I85^&=)eW)TXab$%3O7{ou|Vu(D$sr7eIA3fE1w=^NG>Of;xLQm0!wZBMkO4d`W-T>ftBSE9rb)L>^*^{HhsfO76x~NdME`}MNH#C zpCHo+SZdRk_V4j$F!Y59ScUnUc5ZQjnF%bl>6?~tqE-ZCl7Ln2v8u@!=!+s?sZHOM zl0|P$K+RMF%RRP9*$l|E0G8VH<@|f@x$XgXLxJU5u-VrMX3-j8sZHP9gp>Q;L1s6w z+&sIshD#hgT>)Od%Jh{>qSmyAdiV>hkXTCt9Dknxf9Q+4zhKD7h%g?%8rewFFlu5`)G#!wGU zft4P6yK?|!^uYSZ^!4#u(h2(F0I;&2ZEL&;=G94HsZHPT#Pk)TA#(#*X`Vf59YDY` z2bS9O4J}yfc%O`M)BkVnar8a%DN|*189tL|HUs zpCll&0$Bf;zP`P!&d?XnfEAUybJKX3SFeGkHhtSAW&Y>^nQy=f_wrj22YV|6u+*ln zTbcK;7qww#0*i~w8L}NR>w)!;>Fe8P>)O6>Yz?fm++Cd?Lb-nemfG|UO**sVG-PT! z!_4vW*ZqODbSSXYrf+ar=Z|Bc9>Rf@6Zhb~3-nG3u>LW9eFMAMKwo?TR&MU@96ijU zpTJU^zVS(CecC~$9t6yLs& zq6We}4q%~qdrUVvz|8c5pQ$o^vy#p|;~+x-tJ@>Fbt! zzR@Pg_y8-^yU&h7SW9mJmfG}nFYZ3e2kK!ru+mmPnOO;$JHS$#zU};i(u;<|Jq}={ z11+OKtiFCtt|4hu#?gta$H0pYhNaJAkD&eM5?SpczmzM}d{Q`f1%ckjVp9 zy2|to^|M}9C0ZVQA#`}FU_JO_#1Xgw4{&fV*qQSsYo4#qu^Cr)POc=1rz56ZH z!$&0mOKtk57yAs{4>fZGSUxeiK}N__0ZVQAX8BeBJXaUaG=SAE|3LfgaB&(7EVb#I zn>^p95HjVP0>;kiBDzMb1FS^l`8VZ>uzzXvS%Jhf6xC<<`>FZkJ7rPni;RUc#V)9oHg^Uxh z)TXbS{}11z4sfOctoZyxGvi_ZMFUH1`uc37dq058T3{u63>cN!2`qD9sZHO&691Nt zP!I2bCC3yrmmq@yOKtiF`~S>s4t)^`tgQUQb!}l5EeDp`^bOz0yci6bEx^k57+Bc} z`r-|+)TS?85^y~h>fsx(s$-sK_l8VcV5v>tDF0uJcEa9j6|lwjRWP2a)<7WaqDDPXyK2743G7q)%j^{Y(ZoRZ!)3!xtB0V_22S8^t|AD18eFGC0jPC>)8CcPt z`uQ%<7dWugrmuHtVDFhw4;_J(6SH#fq(x&kb< z=}RXrtUd^tC%{VgG&sfkgJljZwdosH+V{mPPnem&Dvy0#umdvT!1~AZ^|9Toz~1UQ zuqvLBUV$)+?f^?|`f`bjw(FpGihxzGNqxvH z0MNV7`I1nszV5v>t?9%@D zgRZd00akEaantpXSq-dzOkW?n2W8FSOaoXUxs&E~hFMewEVb!tleGAWGxW|cU3Zdb3BH^%mkL& z^z}|!61f&K8W)&3Uc>*J4Sit*mfG}nFB>rax)1DefRz?kI&K+cb_44l)7Q6-OKuxD z(*Ra_?&RYwU>4bU!_QQizF|pA8`wbaGzJ#u71FMF5d5|wu+*k+NZG(@Kd6Vrz{-s) zvwnrlQDFUJ`uf&A6#{)>Hv>MS+$s89PB1fpr8a$&la}#OkZB359Ip}PiO?4dfu%Nm zqssPYWZTc3L4SsSEYK8+=_?n&lZb9Y&u+*lnjh|-y z1=w4402ZA$y=@(sMcsgY@SLtpF#mfG}neWQ=mK|P!VR{H8) zulqsfC9u?{ueYClhpx~UJ%PpL&5)MESx{eKsZHOI={kQoQ8WbeBC(K=w6150iC zh87zdY=e5Z0Ib5*yQ42c<~^{~rf-;^gXK8vt%87+lQ(m^71of$fTcElqmx(NeFT|l zz{>VUMznyw$O4wy^o=hb!aGAfn1SUQv!_`%$W#MMZTcqrIfl1_z8C>4oBZ(FCYVpgeY1*(j#vxz@EBNuF?+9^fJ_~@ZmLb+Y(FPt z5A3ZdV0q`yI`9VeU*W(~o4$p~F%BYRRshSrvnHtl^uAkz$3YG2pg-(^4|?5(1Jm7G84b#6mU5z2r30P`hH^kqyq%QPD9I&$U=SGi! zd6fh#wXaKWTvP4|nM1&mJsjsQhrQKzU~wv6H>#voz!L;!W^?$AV)y$cLBiVcwkj~I5}8hEnT-CyndChn^t01Fcj*c zF|fR259A@xI|G5G_I1$j*L2=_REO<4kJc>A~@~!kT0E!QOe*Bk`<1z3i);u!Vb(jDBmXP_M zM-dM=y6kHQIBWt|TF}ACJK#0cJTkfbOWE5<7?d@SB1Q)sPF)AhS@X!@z+D0Jcf;Rn z9xd!v`6m7&Z0c$rA?!2YNN+o6s+vay)4O^(HG;p_JQ6s#vb^mK7)&*f{srYf+P=}c zT8%EO-Tl!~{WSjQIu)TXWo_c^!SK(j1JB=gkNLa)_*p9&DR8k!k;bhr2*DYw$83J z{L3j@O~pv5<{J*`bwB^S_Hf70Awh#Zd-QdUX~>3*T;JXGx%+)>%IIlB>-MxXKpHn1 zvHAAUiks15(3X-Z4Qh7|?u@iP6KZ$Osq)UM!2_O7qc;aIVI$y2{qM{EzkX-H|NL+V zcu_U)YUX#_Mpxj^&)e6(BI@r9$Y1t#e9bRZ+Q>D(O7WrQyAB@L{1Qbl{K>Ypt?m-> zQnyC#9PvUICwoUcN3bV*+x8do4~G9DP0 zBds%XjLYneB4QG-iDF$7k4+HkIBk*pM7-B+mU~CM(`}M_Ij%z5XFN7WBkeLC8CN20 z6?@ZJj1o1bGZ-nNrp)So5ubG1yR!PFI=u+v&4p`Y%E-CV7iLUto2>DPaYCc zt>a`v#5Y~)mm*^#(j()wF#+kGQE1#?KUZvGx`E9R8=J0Uv&BYE2j$@rKXnJ>VG%!c z`{kjI8xh}(5@RyrlTmC;LV7BgNy6rfO-&*;PjoZgtR5NhM|W5r5%F7hNDhhkg#={0 zF>XfuGs=vc5WkF47YoLST9b^?B5pEcCOcc*QF%0Lqf3)VMf}wr`SQ-V4e6cn*0>ev zl~HcoVjn3sGu_5mkucrDB1E0j33(iAr#mi>Wozk<$zvRMAbm4F7*mkIjQ7UvNFSwz z=^nOFY;L-XEfC#Jcd93_b#?v?mr2Kc$l#2xMh*$is4}J^gOm=Yr`QUyz3Bxy6UpzsjQ3c z+?OB5!^p6V@5V#O(2Q#1LHk(I%T$2Hh@Pf=Y_;g&bV;7cHqc#^XR!5k7v$-VN0AX3 zzl>=}NXAd&5oEa1#q<(eBX%~uz~V%2)AQ=tY-8OOc^2D9cUcZ+8zQ4K{u+-XqcZ*& zk0B#7e!INJ{u8^I3bA;xtLYWC)~>1Unmm_v(`CzZ*e1HGm9>ngkg>e2@gy>aw=tfu z-yrrh6=Ung9;PB}o!H&!rfgzy-3@s@i|MY*^Bgmf@qBG#Iugp;8BZhQ6hBiLmMHp~ zO0fjd$5c`s!4f)2W>}q0lxbFr7N0(|c^Q*xU3D+a&gKx+5=QTk3Ah3)vRB zTk-}&dnZ50DeAF7wIZFKkK#cXTcT{((vg-qjJ zj2DrqytDBFGKF_?`GV~bgG?1zirC-u8QX5xUiUy=#_7>&bmA~mi5-<$}y~$?pY;fl#uy+Qy7EucsJus`-9?e(@*SxIL!0|+b<4v zdM>YJyXp$$HEb7Mz8vRhK^R_Zlo6W8jb_B8j5Phh4vQm9zp+DNi0N1LI<|-IrTib; zUH3wcXS*Shd^6*1gyjk2EhK{1+1O%7#nH44mL`re{l$*h`RZQF8(1G*p}d~$se4t~ z!gvo^$Tv6MMHcYx#yj>W#BsD8c3d1w*TRm8W1LFlB-US7EGM#lx*|Ej@d2`!Z)MCu zqWG4^`^X|?0$mq7C61@-U?;^;x_0#@wzsZK-pKaSmCDI%0J4m4YkY((<=Yq^B1`zz zHX68y8EF(t7bnsPcG|A5?ybCq4b+v(o7q0PHC(1bbaiCINj;1oXQ$>Rq|d|ud9^z zIKD>y4r?D7W z&-XAEA?x_=Hchc>;#}Gd%NFO*O|YwWBXr;8gKUVdT0X!I*L|z>GnOHVysxnoN#K2q zCH6N(6OCgx#Q8LaT^Hv${gRKcqjW#z!|X`i5BZSeTVx|2U@S+Hd4J;@BuR;&2}}|h zT8D`uO>3)vl9tB z`4l^Ws3o6t{DSP@gNzkO3g6%O8QHEZq1#~h#Km-L?5-F^x2on@il{4Ruq07OPG^nC zZhnxl3faXEG*%)z`2p}yoCo4Ex*e7yE~VRI_w6PV8aa~49F9B(jwLsFI1 zv?umdTt$0ePsC`tQ}qRQ2H`B9XQvZRauz!cImiz;{zMM&!;C+W{rpgy&RCupOM7Fv zVhrttJ+qrd)RQl<;e@Mvk)27nRE{+MK@RgHjK7gXe2DRv{c~|G-4!bk*U(+Cd@;_s zp?sB{Lo|@Du(OH!@?|Gm5oI0S1A8g{M|a0wi1Bo{>g()$qOp99 zokuj1v)Q@G34R=DhaBg}lC_Xy{1_Ww?6tUo_Q49p^>k0{l^sJgm2a{%;U?c;O+=H* z31nU56hEG{6noe2-m1w2<#Q)kDtn(@0k&i=RrmAm@~w^g!&BxPu;meH2sZpz4S03Zjkt zfCWuT&S94!m-v}v1LPt0qov+(i$>9#T&ST>U4>^~OMQ-vY5=U_xQzRYve8;MYeLNEuN+);&$RGdIDZcJn0-DzhO5M{&E?+iSUz4o!TQ0 z_+?}}B!^#0wngqMJWb(s#SEIn>xk*JvHBgmjp!}EWw#Q&oo`TzpXPkrN&+IOuzx;{aN%WIHI(0_!_*l{#$>n26FXWkWfu4aoi|6U-xRaPg zPphtE_Ywo;FYF#-fLy`uMxOI)$*xENzlQ9BCOd_39Bp`mz{o`W|Kuh6sc`r>8hA@Vo&0AY}=?0!No zD^9-1YkmXigB0@X$)3n75B4xIO#aRuB8JM<>_Mc2Pa^%1 zVm^`dLyGtW8wPJG-lS>VO}s&y@FsRgi4pQIHjN08f3io2;gy@n-bfk0k?e((^2ua? zgH|-tERKs39f89oz`Mdr@qKrehV3hl=GX(KFAy87QGNBL<_wD zuUQaAey_!zBF0*5*^|T=iw%1M`M_@{`y=o9ZDc>>9lzCPG2UFfLr3B6;%#~n-pnq8 z7;mY~rW2tSJN7g&u5u?i5c$OKAO|2H`4lq9p_O=_UW&IA@6k)}7UErJqXl6FVxpxk zEMX>C>NpKXzVN%rV5EZIMGitfD-Y=vcpLEny&P{X=FrQ&+p}j0%A#S<5TpfVGm)=+ zDrrEf_`ReaspR+Atis!ikLhToPfWHrvROo!#eqFXOsd>Z4nwT`K5{6c z@EkeBp_BNGj=?*MPwCZo2l0vXG>Z#+k(g?6W-ky^EKW`%knj9KG6bpS50Jx=Z%RJB z2KN;6=s4U%%mwqkK6{0jVX4PnCZ=0l*-OYT{s=h=`NYI$avbtkDWVhc9^z{{0q-sr(i^^GtVGPWG=&Y^ zJc}E96RpjkCMTkH{3&t*T8lqvvk~_dOXy_WM=Yk3@Sb)Sg0W~>nV>B=YbH#UJV~K- z`3#ao>+tEM(IG&5LvP0Y#WH#m?kAQyM_QV(w+Yrlu(yZ^i_U2>s^K%qFcjqlauSLt z@91rKZ}Ba?74IdM(_6l`VDAwNEzQ}x!~%;udk1yo&yiD62mUNM1-0kT*zCajiXZ3{ zJWza3Z^!%CJs=iaTCq7ql%*wmpIB6Rft-Ol^XJLws1u(>PICwnKhwML{^BQkC*Du| z=)BC*mVHDlwX|U$5=$(too1o+_)BCs>dIdvXQD1jCA}9PD1M>$-~+@8diVDZ>{DWe zr9JzESZ-;@K1LhzSIIeO1O5s*8?Db@w%Lad7QfOQ9xPVTsrVqfJYtol6Prs!TRO7O zh?SMs$@yqw{u((CZNz7ja~+0=-{=FlLA28QalNQG$6CDD0wTuZ$>tNQEgnt`+LXUZ z(x@AMgEXN{lppkAe3+MK^bMxpNfZE_LX%vy^+g^!hN>67>v$%a1h z-H$CJ5-q-LDUo3DVN1|f{C#pM+LFIVE!PIA$5+v;*zSe;`xPK>j_s9qnUn zLEpm{O3mrJ_yWnDzVm%TWL@HvWqf2E;-n=svNjsTewx;IrG7=@yEfXUV;&f#txfdPCe;!C8~ z&Y6};kq(4lp(5>T%HC-oI+*`Ta%eDLMW&*ItR3j5_zI~#{RCewwWA+@pAzXzoV84j zbRy1J!Xh2fA^bP;0BYc^ixfDyb8li$_Zx>1X)L+VzO@mT8f$M3!Z0qziGb z@&|bs9manr51~W(YVx2%tmH)(;4zXXosX}UJe)6CW=1w3E?Q5+`n0=0{@0b;~@b40OCun@mST1v~OII?n1xm*I($FI|c! zNIrDQ_lQV>kSt83uI8d2sYT)NhZN9>LR~n{nIP1$d5dq90_bu)S@Ngf;7Lle)LZ%m z3Xy8jM=H8hxyqhAi&BDyJcGi2MrJx}k^0c@@y$|i`W?PW>h<@oK1ykxdPl!VaZkOi zU+8omoh&$#S!kHxK%PSZ`A1)U+(BQK-V z1y}Nt!)|F1U4`$G2GW)IPHDj3C;Dimed=TVN~K-uBmD}eYv?SYA(@Sa3k}Gt=uE4g zw&HuG!L)+!k%H;3_AyGQ)MxtDO2^cv`c+B?bgs~Zyn)UU8k5)2+29a;$M;D?=xUsk z4D>fVRq;yA*T*TIsd@TX#UnMh3WG5=UuX)4+4BT9@}|QZOg|9;vVO8_j&ON7?8 znwnb~42q{q6Bz_Qt@KHKr{AjdPJOH2qV!5Fuj)WPMOO&z$tUP?p&j|yAyb;fIN*Xr zG4?eV!M`8%DN5he5Blv&VCsAQHm5vvmC%XIMWcm|hUc$PGkaly|igHkK?dz68x zU-Y|`0jU*LUC5W{8lf}!0*w>A$>$E2q?t?u{Gv32sgGZfrvFv+oH97|t3Fi;POZ}K zb$X5dCv+nV(RiUN`3hZYoy|1HuSm0)M)+kZoM~u(KpB$yO}}3;q+0d+6g|2@=t&l% z>xCX<5xP$3Zrc>UCe3Bs@N8)g(*(b&3{U-`Kcoyx{jNW#3{9=B@*~U8M8TIVMH2)c zvc%!0WMXjqhBTkS@axjNzrXcq%81lo`Xfq6>QDV)r?==vA%HALlLdeB4Vq+)UKx`b(i-;)+It?|236w}IHP~h+$2Ju9$E*!^C zKz9p+$SQP~Fp#W7cM1b++v5+UWlTFfM_S6X#qTSVID7pWh2k{&Oa;b%m7cVsdxgQI zg63~0!mRC6Gvy{o4qyC%{#yRNEI(JIe@U6nxmJxN|DcD35#(?5kPt%ta(FJSWxC=8(i)}#r%Z(G$Wr$__m)jHPO!$AmGqzW8ft1LK1iO6!@P_$$T4VXzFH&o$NGROWGR zRTHSX=qX`5RR=vOgi^H~N~9#lA1{^?89%&8N~qAmrj+5ddb0wHUcKb3K{Es+g`(-g zL<&JqTQ@Pi@iJ*6(+e+^l9>Se+e#$YOn*y(EwJ9AM4*|%B+3C51d6h+*#g-1#otO> zm_WQ-+RXI9-zbZ?7W%u&Law>~j>Qaz(R*sgw(P&bpHsh<}oHFaz+9QVJ7f|43QJ zwbegVmU3iQrkO{%7r31`x{F}0t>!N?5tl>KApDS^kcU1&Mppw8)x|#zJO6%}TI>L;? ze@cg$k@yekP(^orq4FQsP5(-X=ep`&IyXlxLL}v0a~w)FL(SG>%vk)7bd(u`|CZ92 z(e}m42Ck>RNLkPI(7#sJp?8EuR7>=>u#jqj-Vzqrj@Q~sCzw#JjdYwDhyPWQI6r-< zlF0e$OOyo8r)mk+2E8XNrdp$Sg(#|(V{Pd)Gf`_Nonj_vYe^?7dg;rRja-2Kjgrjy z>&u+mqYs2-R68_BSW2};?^}6>($F7U z={z%C>m+3{)9kC1-P|C3rLv0~sQ;quM4t<5sjg^&u!ib_<_mGQv$XZ3OH8=dRl3N` z)Ve6CoL;Xed%3~-ugV@SxN05M1AQs{M|DSE2=P=m$A;2XW{$RjbcLC%tuI}!7^<&U z_Hje>-xQ8B=&jDa=xbpE<%1Rq>#3gTE9-S;zP7P+jhUxyBxN&m?SCoHkEENwAM|!!I-p7lq1|o{cq(kH$wkQImCrjZK8Uk zWx__P7g{PLQvr@z$;`0YnoB^07H$Moj4{|KN4e4Zze*Z6O8>{XFZx#4LItAb!e*)u z`o?;TS*Wd95H8Tx+!l;<)qG@wcbXEY#6Iycc!*Lg7dMc7RRqZPs~Y7qL_`jAiroup?>jJBinlv%CqAU&y=Zg5r3bJGkiN)|WO;OsmC{Vp7& zLeOgA05u%_X3b~TXg#GoCQj=iW61q|%B726x56MH-sb+|Du( z&XYA7A(INz)La54X%Oo>W}CLR^p@GG?Io2nTO3*{ceq7{7RqgIp`p2QOXDb)IF_&oYRf;sNg!|M|O-tb(wM5fGxNDoC z9dEA9q-#UXcFbw*IOPqu$LIm4(?)ne!Gq9+9BP>(+{A_r%|vru zxUD_GT&H5IAy9eCZ87vw%DK&k-p;Eu?S;oww5FZ#h+3&>`^}y?tEJ2u=8P7$KTM`W zf8_(W-Ox{Y&uugGRo-bj3D2k)O-JD=wOZ3bcw&2AJK5~WWNE|94$L|2B;_-=(=b5! z#O*KyDId9%s(fmV##6|n;xryYE*0xI&FsQl)J`=!GZ(Z|%uW@148h75Znt5OQo-#q z40Qfa(^+^<#cRBU0&1sx*1igt#%9&=ec-R#O-axf@gxm1H*sp9q;1}l}CZo(^S zy{4=1l3J(fBD}D@rk!PO$Yg87%?+5V+L_8XZogruV&(Q3hA40es4AioH9dsaRD!0v zP)KcXoNI2v+|bT3H)gJDXPX;U95RF`-?@W^;Yu}kz%a~tqsCV#p^`N|LNS%3>G=(V zu1 zYPID~7{*y`xZ{Sg&O0=Hg%4DUCQx`!ZP)bq)`GdGU1)C3+|@2HyEAtjCR%HArwtRV zcHAk$cxx?9know>rRgtxqIPQf2_J1AXcwDXF*({Ob4%vFc9FH_Ml5Nq%Vijh);e5z zRVB4oGf?SnDyZF#%gk+=N7|+4Hq1lq5_9W{Glnp$hRZZeveqo&Dd&Bf!NOPA z4g?ETRH|msw+_rx?Fw^y=81N>xgGP^VXD=UJ7<_;b>Pk#CR^<_LxgYC0gXYhQu{S} zL9xx#t}=IGa<$Rsj?6RdN~;TZ!7$zG%$+w(vpR8ERX?c1nqk6s>X2rrP)!|lj5T{P z1=<+1CzG#TZT6_RYzVj3<1QIyT3xw|h8fOBH6w&yRGKD4_(>hn4FA@Jd8u7v?##T< z#+kjD=MHnM4Y{j^+13W!6~ioRea&d$FLhiqO87$^(~J~;+rHNRXYR%nYU9manOEAi z)+XF_!#rza?wVn)wGo&7wH9?sGd9zfI;k0xX+xcG++gm>6l>R;doV@Xb>{9BqJg$H zAC3nxT*xEufEz^a%sF|ASOkL1S$#k;) zqTOK*Vk)#L=KjoQ?RINh?xA71wGH>cu*}+;%lTTLx}uqpS&zD`nV#uNU2@!Q9>i2> zcbNw=mD-)=0ToXS(bo3dW5Y^oJMNKTh4VGdtjvZ~wkABY0d-X~^P8TrYWJE4Gm3VP zIhgtC5M%AcJu|GfcI2KKR#`h}=4LjbZfNFYHm0s?W@k3C{jS|-9>P@Dyot;=ZK~Ca z%QwVXJ-IwXtkr|d{ffb1_WaDIpft|Qbfa!M9yAYUergYxhcQ32`^`fuUKrx7ow?_R zwN`Jgz_7;IqG2+%l&qmMaTtT9ZzGvM+Qa4%%x~=>a|rXxVZF5*_sX!&+Le20_|Mu! z6PekJx~*X|3F?+6B2!n(R(sSuI>JVqW*)`-)gH0-Yip{ zW^?MUWlQ^Q%m1p+ZXU{tO-1|TG ze`b=$k+qhWU4iwj@BI{i)jdWonsUWGS}u}u**(hW%4PBH33uT#dv}LBbD7{}dfm;h z{Y&*an_u~t=(XMBdkI#y@q>k*lUWanF{kq&#-dGWv6g z-eci@T!Qy#xG$F*7Ikd1;TLqLnV;8NyXVU_Ql7i#$<ay?fe?;qjeT)}8s2v@)x);sYo|8Bh_AM4+xci>|p zjZ;3kSIUi2KDt-P4O2cK7s8{sBHr`ikz8T#x$ub8%>I3P7e14JuilxD^Y4jIb+3_| zrhIX)mYbw}cCRwVa>c!u!eh8%-izVUTv7Npy*r=P3CDNivp7kaUES;D7AfiOb#n8R zH1}G$nX8odYIr*0x9Y45f0gjCcY(itA{YdX!tel^~5^D>rVMo%FrhI^ASl`HSP6`sPC^WF?k z=E}l{^`5-Pe@GAT@&1E4>y~6P<*Qqii4@x{$hfPL_ilJPSJ8VXJdLa1y{-4*bNY|! zz4;veBYH1Bd!%j35BChi7t?z4yX1QWN|q^nQG9|8c!9 zpUZ#D6YJhCx2Iy<+vIjBKiyl6xmFH$TWo@AHu7+Cf@hqm0V-*yYPzC zV*V@o7`~|gvObzG;=km{;XWw$q_Vq1a)9!`}%agg8!aAjW6%Nt50>Gk%v&fxKGQ2seJBJ z@*r0muN9WKWUm<(InrzBv-ryXhx$yulK+7|gRdCjsQl3Y2<3~W!BAf0TX-wi&iggI zg=_1z!@<;Q{wMkzzN-JRKAW%Nf8;6T{zD#46^tguQ3c##V+YsK`y;%a>)`z!-o~|u zpX>Aa8vbYcJifaBsXo_zK^{dFai5n*Qia{;=HEFX8L@U+atc zI{sIlQtqqrc&eoPiad@g;l6Af;8J{9!uz=%zRcl$TzB}rzMOC1f2S|w>-*p8OWil* zNmLp4b$KFH+I>x);9`94u*}iEY~kNH%9mAN#W(hU)K~J2{2%lce8b38s=WJ_JcTOf zz9~hC^IWI9*@QxA3Rw>-gsWRDG@cfjo<<;=V7>q$<1b$unI2d^y5LxW2yZ z;lo@XpGV)sxAte~8~IlLh`xbu8JSB}cR!NnP}STI<=IqKG)9N<=gtd>s$CX{%?AaPmV05 z>bqabOQ?G8m-1q&F6s@3xnaIv!vEknU%v3~sZMz!eLLUY|5M+_xAXt-G;+U{ms1Vh zZ{%fE1NUp=95>RJKl}$b!siR0<%T11`Yyf`=F)fa9kEz_hx>!PifZD1FR!E;yWhzx zTw{EN!WX#FzJlTN+$diGeJ|e?%dGF=yI`60-F)ZBTB^DGle~s%=Kd(JrkbKf!~ST=n>-yO@U@8i2+Sv;-Wsq%WNrTdG#j%wllY+U0e`ih6Iaua;T z!dJNQ2&~IIg+b8}4(8SmxFhl=%I{8>H&U(LY4Qfw6kn@1F%F-7bsTVMRkT^=`0OMJD$FSy0Nn&Ic%B43T@ zp$qdx1K5~1n#LG8K&3!gWV#i$OMSJ&FH=Wg1@%Ap;aGua zQZts{!$8qf01ZVa0w^e}@s3;Rs~>*Lt?<8k`7qT7@)&8{Mql%A zD!0McEc}IAkCf4G@RP98`gMLHR!YAH<(7|A1E5^;F{(e5Q$FhAeXYU~ZnLjtIGx+% zYoXufr()&wTl^HPtbUW9963b|h7#qI)F3E9K0yscv9Q63KIinw2|i~MC3OZ?QNPPi z$13P|_-R;q&#zEk`3yA_N|H}gL!dl{&28}!VT%j;@UY2ANEQ77KMSj@-{)symGpa% z*ZBfG4En{H2IZi9^6#$gzBb`++%{iw_$#;7N9vFHxmY#*5kCj3sz2msN6t|rq5Mu# z#0bde(3s(9`|wY0r>|Z32e-r5HvBzx0ajCg%FoAY=uh~0Sar`BsE~Yt8Vwbc&r_qI z0;Y@G96=DSY7=kzc_M@ng|t_uTm4BV)7MgJnB#eXIi@3*(?rD z=z2DD>I$r({+3^kHPGMi%dq;MDNrf-1~nNfDPN~1K_yIxQ+>|m52yH?NpMa^n&=<+ zRaj&FJ--rbq`!m8%D1U$P#O6aH5Do?-*g@HF=r7jBz}a~2kdHl!rf$Gm>Z$yCtcCuCUxzjK%z-M& z52)EtMfpB83#wq|!h#W+1%U-U*EI2xZ}P)y1@$=u18M+nC?sjM4nL#q3ZHeY5`PDenQPh2cFHt zo%Rhlo5-E=^*@`C8pM)yi8-PXESl#oFp$`7Kx*-G=JQuc>8F9r+ct6sj%1bp7FTf^;JR8M|GEk}o+Ex?`kISGN>IiJ%kbvM@8h~;--os1ZM zC)Uxk25KaKpjJZ-<@eMosDW9SyW|^rwh(vGH{xtT?gG-y$i(l(x*Bo(9;}Puf||;o zsdZ2j`4hDkYAk%v8-2D2ciA_}h^FKBFtYIbvF=7@eqSVw+6Xn5Q>hJ5 zGx-a(9vy$S1b5vx?rd@Hns4mcVyOy78_}=ulo6fCP4VzhD>>p!3b&NgsZCG|vov?h zH}PyK?xt_T*^=B1Bw!%ChOq|Bs~BTI5GEUx2ssxIlmIzX$gaD-DQC-acYKr2mf>#u zCK(?75Z23x=MQ2%4U`W>Y-$Tc$QBiZaM`3Jbo$u}+WVJE3;+4{8V0 zR{jq9P9qP03L9i3@+Yx@Mgo5#!$s|ZIx4Z$Zm5G2L+t{6XX@|RP$MsY1{-1|@u#uD zp8Zf4B@?v|>a4_3d!bIC??i?f(EtW#MAKM)HS$5-m8{fnP&XwDbpYzBWCnewQGh>( zjWD7~ZNm+RHfBJS3OT_aN`agdAW8;(XX-_4v{8t^fQ>Q=^5?OUo`VphAXEsV6`0Z> z3iO@GIHM?k85?U9;V)rhjKWY)#X}v20!ln}2x1i!^qoct{u(yHD9&HS#v8@>D;YVd zV^D7;2Xz$crDUg$fW9;JCN|k9#oxdt872Aa*hJ4ssGpLXIsx@na#6>jKA`VJrWs}V z+t^g241WuoVw8pkDtV~W&;TWoItBGt5S0s57ANOnr#WHY)KCuvtb${ysL-^9SUFzNxd&P~{gY3=IK&Co<2d%0I^D8ddm5 z*c_uWG+Ze_ori`g`Kfb|bI=5Rr%{7{hAl9v^G~t)Mm7FPMq%m_G*T%MP5wEy&~p_UqZFmCK%MkV2}()o2IxCeKVqwm zM*IhCmC=xYkFE6Fg{CNFsXNeQr3`f&ngsezWS!BJ|BS6Qn(&{nHAZ7-x>A9<4^30b zQ}>{$N;%MX8ZG!VY=hC9PsP?7&G;`Fm8nP2Or;X_5SpP>q#l62Gd05r=d|J@PEt=x zJ{{ZWc?!)@s!~s&*-91aF*FPGorq{)ynzXZ6Z~-=w1(y@HK^y%Jf%AI44SJ{1AV7K z@-`MU2;RaZ1Lw_*+SDs(p;C)_2`x}+QZGQ?>CkiapY(i_@^bTEWlbK7WG%gIS;?Z#-@J9wi#{sZ`f8Nng5Dy@w|nWD0Qhf&|;+y z^%`0P(okfl(VqW_?J(N$Kd|jaTWGn`fO-!tQ|eRipruMZkcJwa1Q)j3=qSWuyNnJ( zOh#ks6SPukM16!-C=IC(APr5+jO{bJ2$`_GMrR=o+v7=v)+kM>FVJeG3H2FT1=3LD zH^VW!PCjQhA&aAWyF%-g7F0U4PH9f1L2H#}APqH~BR-}WPTb!)2RPZE8LcTD+NiXm zGN27gODY1=&@>ba8H|8nnn4RNrg|)hS1`(iHY< z|A=uz`<3o=HfW#Hjm`?~1!*X9-WVAlv|`v5V~kJ~yKIaWieQ&wazF=_o^*C7 zqy%UWq=7UPxo(UXN?_NFaYAwIsxcNiqV%D2L5G#zbWZ4y(hH=a#w4LMcGH+Bl)`Qp z6NHi({pm#Ln9`3D(XS1q;B|=^7wXs;JgT0C=1zlCf(GF=v= zp=r&qFUA_7DfZb|Ei}PC#Z-dsD%0tT&>dwOT>-ic(oiJbSTD4|(u{RNb1cG7yHVUnf4K^lt28M}l|xXainbi`wg9nc4572O1SudJjSL+_LoAPqJ43SIHc z#vY*yp2^rPbk0~yH-|ncYv^XsM`bnL6r`bPJ@9PCexW;_)z~L=!?VP+f>M?BbW7-q zvW{*6eFkYL3LCOO;gIp0kb=9715iZSMEjw1Wh31hN>es~G}O=p7Do+LU~t4x1Uf^Y z2`Cyv!l7s;5eCxGv|f02@O;K;VKAQ8I0a`? z_R`(pIAssr6?Q4RK^kg=1rGNa{}6t~y~gjt(2N6g3YxG7A|I7JVJ6P1(nAUHueK@S9JXxa?CqH$N4j#n`52-EQLF~7ojl{55E zI7vB84}tT5G!(63JP>B#m5uwtOuUkD501`&4uhk4@Eq(^LjY;0@mQFPS2G?7bMUIh zLt%EtIeH`5p zqIHZH!Xmu3@myGl*D{{LMU~6+IJk&%i5?3VRxW}x)Oal{#p@Zbge7=g^ZgI!Hs)HsCFdRAD{d!uTSr!<)y14?!Ag7=nP~hA!|pW@HGPGoH~4;p)m$dI4Nbc|y+zX=qvy zPc|$;!b!svM4X6O0@qSr(2L=k%5!=VTmz(`XglMZuoZ7>d=<9fZ44W(tGuR{!F7~Z z^isIC@)D$>#!q1f-of}GY{%Oh--T@%@933qedR5^0z*K zhMLiO3_Z*&!hXEFnOWGEkw$NXn=7gG2Dq8>gn5kbRQ6BbmQF(KHwc%VhtnSFLH2ONU{(ooYQ9Kw5<@xnp8r-=%o z44d8p6N*I#VO%k338bNENAbR94&eyi$ILDq#(T$XhubLM=xuPa@|E5SlOPR62bj5q z<9L5Fmv9X4XXb?4D?jO-a69D(y#sEmdA*lllVY0K{%ni=sj>pHJ08D zcTi*KT_6oj`yC%@<`vH1L(C-MG(I?HKioylMDK$;t8w&RxD!Z2(P3sZgM~Arj_Fr3 zAKYEdO8*9TQ?t+q;I3+BkcOHCgmd@^GpZVgn@(*j9imm($sTedK|R!Lv<%YFw2Sy? zvygBBA7vI4&f_Cv4#JFz&>`4qMr$wy(ol4qSyZ@;k2Q-3m+&!WVYsL2p%23WHJ&~M zvnmSGP_u+^4WD2Z7p~&t&0@k8Jtut(?ycsakHWpw?DP?khNj)bC!3{&8~7x%q;MUd z7;_Ttr{<hqq>r^%yq$bg4KpL9%5T9*U5+2~Q%!T>XtfA^ z86E}FP;{ADS9pytHR}kk@Fiw#c)VJIz7CI5i__QOv1&1phMEn8clZjkzVH@bZq^gt z=%wjf@IM;JKpKj! zGn)#Z@wH|X;S;{bYz$9VE714hX=-`;9z0bo2hvcpg^-4CFq;dh_O7<4f;7ePpwWrgXgN%KpJY2f{h1FLa=bjbeie4=~wVVwHEynUZB>bUw||; z?K{5BY$JTbx0=bqSA0v%TX>0Dmwp2;R_oBO;YA<~MR%I*g`fBivz_n*-)^>rm#Yov z_wX{cKK%|}s@4N(sM$$$;k(U_Vl2MP>>$SIjpbLWh1aVs=yZ6U+MG^< z*Q(7x8frRmf9F8#WPdu5e8&Lmt!W+JsJ5ar;0 z*-Ol>x21o;Thuo6cQ~jf)89ZEnx254F#CzQ@#AJ+F&BO;HWuElwr67CZE8FEC%hG; zq3CIIpqK|gWeyM%@snnMc$eCViGz2l9T^wAL+t?4P;-cwf;Sj>x`F$alBdRHb3 zyhrWAWQKREotaD^4Ndns6-u1kcV`~csepk0Bi0S?SGzOW;C*U0CM&!bq@n0}bGTRl zKW7dT^W%S*TyzeYfni4iFc5q|?E%tIbCg&Zzi5sW3*i^c5n@4|W#VB~Wf(Mi3BVvA z4NWhGUoppsMe)n#Xt4->DK-auQ0>WNheK+B@xU5LL(%Kzc(DY2%^W8d$FG`W;Uj7v zCKr5I?akzb52?LC8fs1wOXD}qiDD`IhB-kjsrP3R;bUq)CILRG_GNN|G&H>&e#e|5 zmc?(Ilf^Rlt=PQqNp&ET1fNg`FnQqPAPq(Do72S#_&sx)SRTJ?PKD2?LzrLS)9PR* zAACw3lmXCCbCy^ce`wAWE8!2!8Dd49WAaBG6yt+`SBElQkcOsL!=IRQ#H#pXbGBFo ze-v8?{zDzk6pYr!VhX@vkcOhq&G}*t{FymVtd2i5=fW4%QA`o|ygHI844+d+WB@eO zTqM@UUz!WWTKEfdfml-?%M^z%sbiR8@I`erQxv43>Gkk8<`S_k{@PqD*1=!JmV&RU zaCm}|tQ_-Avq*aZI+TM53aPG>5@ zchqT21^6~dL(z0|z1RXzGuMgD@lRDtiSGnvZpJ#_|1L(NTMYdpi;D7L~Q z<_58)K9{KuKT_u~)!>KfY^EwmL(_5GjLre$MpOdedTcHDsXCvj2|rQiF*V@FAPq&o znvzK3wke7PZkYo7LS4kvfuE}jncDC(bpc32&8=cv{JXhDY=eI@gJQD2l&KHDQkO9G z;Fs!RrY=ZB(>o9`=610?{?pthw!?qKHiF-(%bAAo8+94e0DcY9(D*oWm)MDLnLEXf zM69_3{-Ca6n!xYXl}uy!ow@>~q2^w(E0NjUBX%J&nY+c#`dX$r{7GHIG=o2?tC^-C z4NdPsWHa}R-HEK`KCv5-CAJlus;*~R!e7*NObhrkNJHb{XbuknnU3Tl+~xr|qHbdR zaJssYX$_~T8!`YIYHA`&pr$G^1QHG5=mJB)(M%#7&QLcq7)V3YdlA{qgJMs@V}`^4 z5g*$Iwp58phNE4OumRH0_*~`@u@8~cJS_Gma+rtUZ|YX29sE_@!nB2LH3-sB^SIcb zNHC9y{fOM=QL(SSgXswWP`5K3;P2`-raeeQ(+3er=1FlNk;gnC4j>X^yCAXZE~YaQ zqwZun!9PJ78vl!VMjS%qGf#_yiM-}1B$K+A>4wCqdzh|>OWh69&}arD;fp#}!W*q- zrXOHZkSywcrU#N)-N$qXX=wTgqM-SQIGiXDt!+r;k7W?IlL<;A*&Gi{Az7jKf2X1G zMa&E0D59`=UK~jjGS4B1sxbj1dJ13>XCeTkq2^_AEK$t7B#t49nis{<`XQz_60aU) zdLgJ9VtRr!G<^b5(!45;CrX%C#BoIN*nUV3^$61!$*vw|`XC;VhQ^mMZ-|qK(&lw> zB2mh`hU8X{GXs!Z>M^E2l2bhj(oplZIF%@8-V&z}WzC!7Wc?H~7)ex5GJ}u=^#n5z zq@n3Eh>GT2aXL}KydzE{%E$hSJ_Ko~ z84cv)W;7ENGaV1CKVudm)zzoW0;HPygqaW0(DWdYY+9m3kY@B0K*TOVYN;=n#Yj!{ zIkO0<0n*U;cIG#6E78{cDsCa#m^M;Zea$RG>Zq@nrATe{B}hZfpW+UpgZV?;PP8|_ zi`(>f%u1xb`j%ON)KlLu%Rw5NzMJT5#frO#PF9S#ljs<`25F>zU{)gy)%VOQqyb1n zTkq z)l_B!(oFrrtOsdmx<| z3qml}a4rY@s_sk*w^lPi8ftmOLqsntUOY(jv`{gm+sqb(P%S2i;Ht?;APr4FO7yjI zh)0M%R(A0)(K~iK(nkHpY(tXOugq421Zime04uk6oak@m5|0u6tei-D^(V6vX{Y{R zb|7ul?;s7e@`$I1K~|!8k{DLRz9S=mX-Mp>85324j^5%%peW5 z3W(>35mtVuw$*UUC!RGRC!FILFee|Vhn9_zK^mHVkr-_i5-$*=tb*csVr1+=gwYTt zggC(;=K%%M(D-pyQSmY{)+!=iBF0#Sk)E1|IgA9fc;*noYA8rUtrFrjVuDp%yh@C> ziiuZ@oXjz#x0ZuBiuBU5Ge`A1bmYX?&^wn}P$B{lD z4UL~>l@)IjQ>`-MEn(sX&q$M$DKpL9%5+4w=tcv1&VrJ|g$gi5$$%Y)N{oDJPl@?fHSvj2n7M?E)Cw^d zkr7%!<^o7V(_a#ctyVv$u-d`>Kky^4&{iZWM_(OMDaGBOIJq4CSCy5ehMsZ~dO zMJ%yuBjdFa%ynd(R-CzpjMa*PG}LM!z9Uvx^~JZua;u*B#wg9)LMCdZn48E1tt4{; zq@n2_iPcsk@dL5SYAC)ZR>s~%rf6lEJIG|M409Wq1k%v>byidHGqKicB7P#)SdEeC zS_S4lGEFPb+(V{ntrGJPnW0r=9)L77J;PC5 zt;C2^+oz?NPHc>Qip5i@K8!)Eb$u)nJ|@^R(*B zGi0t-4WywKDcVHPB1DUjEL=2=+RQ6tp;n7|i7e1+GA}?Hn*N>GX0;K&5nHWf@hh<< z_ARnRtINDW7Hf5w*T^D}hQ{x-+KWGl9acN>2eI91i!9d~Fz=CNT7BjnvQ(?57x+63 zwK_>IVz<>%iY0bg9i$keG4lyosWoCgA}h3p%m&CLLL)HD}V0wOTWfhFX*aIpO#e z$?fEG_K>m}tr;EJsI_7;kPTW(CIZsX2ug%3MnVY9q9vG6W36Z=DPtm=H9un@n?M>G zf7t3Nd5A++K#C_0S}Y=IB=a?T3SsQ%M1l^`P^*uWlQ?SimU0kBtX@)fqb>6T*`l>! zz9T^`nfV6N&`1Jt!s;jGCXQQurCh`@S1ht!YtP0Y+q8DfPh=}dL*q|d1EoB~DQkd~ zNSw6#BfGRtY#g#v>&Uv09a;yFhFU|Ud`@M`!BSr0j5SC~GP<%^kUd%#HZ!tY>&#{X zX=udf)T#Yd@;cecv zHCie{Tyo_=4r)Ew>_|upupUGMX=wa)YrIr~xMq!$iW66@vB(jv51R`)to3GdB8RkI zAPu!9Nu`OK)JYD7rI4%Ic(x>RMH|PK zKrX|-{hfx!zqgi44TyKvGO0fC)>?|(&?d2Ekn7q+wls20n*h>KYn9ZP_-L(^8WA6? z6;eZEDq9}8rA=YWAvd+jY*~z00l5v* z(D-y~z0`t8v(`z?iBxMX@<5x#RzdD-Gug_>J#7X^L#<6xYa+wiD77LY)&{AiF_*25 zJksW{)sTnUY_=*$LnAn0MkN4YSeqq3p}T4!Pqq1MP2`C-kF9|`25D&gS4)ye!j4`7 z5SAq%FSJE$9pt&Tkgbh8(-wd<)Y>YwCB9o*q&CDiD<~x!OWFF!D{To|4|%CAX6u49 zG}3{Lv9?R?iJ#UssU7jd)d+d3EoU1dZ?t7>1LQSGLp^cUE~yjgvUW-x$yjR#@L#@41S2DA;N9samvUW?IjkRoZsNJAsN$n4fZsVC{NLQ;T?ceO#HT_hvX zKoW@x49bOmuZYW3^puXEa9J$#z11f;80g zi*-gCLguqhOM}V0)+scTwwLXO#%X)luBc1f4bsr4VCK^p2QVqK6%k%g`E(nzw9 zbq+-|jSZl%rm`#wX$nX~t;^C_vY2&A8bcPfE=r?~Lu_v}UOUM4LQyTm_5^8YWCB^z zx+;w)OITN=ab$5C9bzPcBma?v)xwYf$05q3& zjO~x+)Q*BQ)VeKACCgd2q$y-s>!vi>IK>V|6Sb45yb2vtZF@!W*g_&k!XJH4|W9V)6TNPK^huaK-RRLO7qDY))Q$SS=}`T zEu>vwN23L`^Xw?J07yeUb*vZCBC@viTv|xhvYw$uwae@{w1{?z9g7y$E`l`FdMzy_ z>sha)C1hRerL@?%#!f_wYggF`Xff>yJ07H=kriY^>#ek$Y+${SmXY;cQ_xb{4R$hG zQoGJhLQ8-&)YHWJAgv-BTkoZnWFzYxT2{NwPD9IRx7ev@Y3*ioSN?YzYJHa0lFh77 z(i*a<^-)@F++$~=<+Z!)478kfhn)`6(8va|rIjkJCtFxwq;+I-*BrEx_JEy@R@CmZ zv(O444fXh~h_s1pZKX>a$yQbxT2*_@&O@tckJ!0rW$htIL#=2!CvHVOFlJ>)n~i7e zLbST}lwE*U)1I*NK^htflF3%|6hKBn`wdd?2v<65+J?*S-(pIvq z^;OzJwy|uquJ)Q;hSt$uu}jg~+DnjzT0f;7WC!bqw4H2keV4Ww@7R@SeeEr~0vS z+En|@u0xw>pV+l%W9=hIL+z~60kVgkMcPkxw=+xoj5Ky5+FVOzH=xb5FYJ1dhDH>U zwxjhNC_9_<8=2zb(N(%DUD3y_9-0(SH(JZncMav2*!9SFdRs9)1r0d1{i zfHc(hNQcN?cD!_u>}jJ?$gtThsB`ha22p1UnUz2q8aYb#wR1>E$Ub&<=`h*bwH|&B3^~M3l1`I@UHj24p-k*Pv~wtq z-HUbtX{cwI?RBczaP}`wP2*qfd}#MjR`xfvTPO>A0PPye4AM}$fOL)=Vdr-$oDa8s z(peL7l5(6%K#uBi>Or$INJAqR$j>*E-0NRN4gH8PVk2fp-u{r6V9QKCV!`) zo^f_j=`uOiE+Snb$Jm9@o*@r=7!8Et*+Zz)45XoU3F#U+!7eUcCCA&vq$_4l_88hb zl!HBr_6lWZkAO5Za+92Fmy&LflkAexb#kKXB-$^On>~T{4dr5wqkRzn-)X34nq5}9 zO-{ATNVmu-c4>5AC=YuY9S}-nPoe!o2_Ox%D@gaq8FqQ;E;-#UC*3jgIrZrWhw`$& zqk}?8>=}@TMjn#0?Ml)Ea+Y0Dx=+q@{ee1l&>Yk1RKs^Fn+-wQ{hfw-=Gj%H$K+hQ ziu8z_V^>CphYGOgQRm(m>^amCv>*+&Ye>(?1$K4mDLLP+COt6=vzO43p+f9MbVR5i zdjX`Ok(cCRyO#8VTx8djo|6k*SJ5${qU;rPbf^e>865@EP|q^EuJoE*YS)ookxT5_ z==e|x_BuK)RGhtrjtvz9X{g;mdPlCX>q~FR<#s*kjaiz#g-#5WVsD}oLM7Q7APtRt zBv;#wqz~jOyP@=+Td>phbpl5 z(P^Rb>^*d9s2oT`?G{oRxxsEOrIPFIX3`h4GW!Uf8LGrSL}!F5vJXHS8p$9x+pVOC zGmYO;N+&nEo}zO?RoN%#>`)cX8MK|?kMqDu0Wxc|uh4~|TI@@7L8vDC0;Hjl@8mYSjr5J&Y9~uy$t|w8 z=#o%f_6@o?REK?yE&^$&XQ$m>`bqAv+etsj?RHyqd8h&V9$gly&%Q&KhU$Sd)b12? zk-P1V!B}#a-60rbHfBGeD?^RgkLZd}L-qqmLo+gy`|K{kOypj>b1;tF<4Q%>gqpHn z(AA+P>}PZpNJBlp+1-O#$pdz`U>0(}-4$IQYQd(X>q5=hG<0pK8AwBI$MBMhof34D zvfU$?&1}u;=*CbhHUr%dYRN`G8k&KUA=?SwlbY?MaFVLaLirHJn&{?`pEb};$h5!H zP|snzXV60)vID_*@}SK+rvQ@u>P!R>tnH+O;vfyR`vh~6NA2Fh9OMzZS1`NTmi>Wl z3AJIrqrp%z`wgU_842VGyI(LjdED+B%taoHi$%AG+6Q9LZJ~DTPjo9tLp`VMfx$fF zDSJRLkvwVlM|Xuf1>(@1p^gC;x+By9q@nhZU_SD9dvGu>dBz?TOftI$vY>lHT>_cW z-J#BbOdt)-@R4WjUxQvUY!40oLjEJpjqVS14`f64g}Mc@qI*Fa>N#%@4;CQL*~5bQ z$vvEJ0qg#|4X%SM9Orkx-vN zF7$AycOWNvDAWt2q4uO;Y4WB$F<6ScVNVE_H2Vh<(PN>0fduqusBa)QNJBHqk$3DV z!LsCSdvdT0c`Gh2dNMRHkc6HH4G82xk0SsL_1w3o2P=^G>}kRBcV^Th-`Gonb;;NE;$R)} zRa`0bYG{0*Bzh$@E>Hr!4Ej#bdwY4X0r}2e7OYRcwU?qdLX!ez(CeXzfzs%;&;-zT z+N*+%$&dERU?cK_y&~AqoEj*P-U>|#ltXWZCI`xbzB8j4`NdunY)XE%R|lJrpW-T^ zcSF+y712APX@LspZP0gm((U!Z7G#>eF4&w*wb!B#LbC!@(EFj8fy(H;&_An}cV^(EY4bshH0;emKdHynLZ61_2Wp~E zLh}MO(8r+f^nA6YAW7P`7$iu`7SI==MS(i#^U%UTZS+}a0q8sJt--eBcY8~)4f)Lu z29wRDf%@pH(2_ts^kryqpf2b;Gdd*4*xQ5c$)EPNU_0_hTqE>tXnCL^`X;n2&;Wf6 z`p%d*dsnbivdi8X?3f&D??69OfP_cV_fR&Sviqc2CY~?+bQI&Jx!OO%1INv_!vz)&*LipF!Ul z13UG2sAS0gEtrz*why3@(58SNO%H7hv_{iH8$jRbgkacY)Jeo)k`aev(WVd}&}da4 z9F5k7#z5bh(JMK-eK6QF*<*)-f#mqOHfS^ul#E8xfk@NKA`$(`)a!&hjuy=9} z`w;pqv^CHU{TkX5Xp1^sfWFf{9_*i-U>^(iOU`W{4fZv61UjNWLfZo!(C?vbf%c&9 z%ovoMWS4<93KkmP*!>EPhxy!NU1 zOrgDjZt-!UJ%O(AuF!7KcjhhRa^)?VX-CJOJ2Q>#ba|Ka_(ax(9+_rlyWbL;Y8mWfQX{?5L9%6O(wL zPQtmGiB~)^F?q`-UKo+^%oSa_XLQ0P*MBYjHa1~P(dbfPQ*44<^uHG0IvtxZw@7sP zlUK0`w+jE)@+0$H2@n44W%WW=!lQq6qc3(P+&UZ^ZE)d2T*3|K;XhyY!2HY!$Np^b zWnt!oPycH1Xi4S--TJe|jd@xBd>C!vyovQKkzqp_ZU8kb(6`qhCLz1Cu$U_;=g@&n zzrj?m!GR$?`wi|x^{Z0(uP1W!>=zggEJyxBgJHwDVlkN4DLIf3eLDM~exv`p74Rg` z=f6Fh#c7%g7)sCK)vG#14Wh64_p$u-M0`TQ1XogCuu-o-zd_8vN|kFCgPej0SlRB; z-_Ga~Hy6n3%{#w%Hm~zxREx=$CGUTF?+Gqvy~NOn#K^GX+3F_!xmhRuxwR)gah3eH zKV$NGohl4537MRp56y8WlukT1!=2DNruc;sf4)GrESd5q0q@uOZ2#*2QDWlctj?On zzYX&y{kc~p{kfkc#{adW{PmZ!c_n0W{&W5LI)A>>3+I)}CN7`pPG}xe{M%S>0`kAS z73ZV<>!nWJk4?PS(pk4|_3VG{WJ%He?cYbgc&?GcVEdTr*{?j(}*Qi$;uEZUNtGKheGwgr=05;BZCsd6o?(9;3 z_Ved3|HBsdS2vsUCLVJ~*Dbzzc4lXs|6z33oc=1t6nAz-Z$kV($MwH&rhj#NFFh{t zLwa1aQS?LoA4j32|8ea4_jfRLzVi+em(O-5Y>ACYoc$PfHsa#h5>L%==Z+qvl9GmS zeabUK8E4Ng|K~xV{K#J1uzo`ZCuOcuzH<4BF-fKV#~QUITj$D_pzVxWa>(LD8px-O}0g zoz?!?tl!|_z54XxoCSkN3>?U?1Dys=Prb?({;wA)TzlMv31$D)+242n|GU0`fjMvC zzX#_0;T!}1&&LHG>;C`u^!Io8e>EuJ4Mq=^|2?kZ{RZ_KHk=tW^#A-#{(Wfw*WH1` z``7XMzX#d#uVp2x)GS-2)c@B&drXK)YW>d*wcH@C;6JzQf|bfws$9NO!IJEVegk_J ztjtskRIDDTR +# #include +# #include +# #include +# #include +# #include +# #include +# #include +# #include +# #include +# #include +# #include +# +# namespace +# { +# +# using namespace DB; +# +# const UInt32 ROW_NUM = 2000; +# const UInt32 MIN_STRING_LEN = 3; +# const UInt32 MAX_STRING_LEN = 5; +# +# const UInt32 PLAIN_ENCODING_CARDINALITY = ROW_NUM * 2; +# const UInt32 MIX_ENCODING_CARDINALITY = 800; +# const UInt32 DICT_ENCODING_CARDINALITY = 20; +# +# UInt16 nextNum() +# { +# static UInt16 idx = 0; +# static UInt16 nums[] = {0, 21845, 43690}; +# static size_t nums_len = sizeof(nums) / sizeof(nums[0]); +# return nums[(idx++) % nums_len]++; +# } +# +# template +# void generateValues(MutableColumnPtr & col, size_t num) +# { +# using FieldType = typename NumericDataType::FieldType; +# +# const size_t next_num_bytes = sizeof(nextNum()); +# char bytewise_val[sizeof(FieldType)]; +# +# while (col->size() < num) +# { +# for (auto bytes = 0; bytes < sizeof(FieldType); bytes += next_num_bytes) +# { +# auto tmp = nextNum(); +# memcpy(bytewise_val + bytes, &tmp, std::min(next_num_bytes, sizeof(FieldType) - bytes)); +# } +# if (is_decimal) +# { +# // clean highest 3 bits, make sure the result doest not exceed the limits of the decimal type +# if (bytewise_val[sizeof(FieldType) - 1] > 0) +# bytewise_val[sizeof(FieldType) - 1] &= 0x0f; +# else +# bytewise_val[sizeof(FieldType) - 1] |= 0xf0; +# } +# FieldType val; +# memcpy(&val, &bytewise_val, sizeof(FieldType)); +# col->insert(val); +# } +# } +# +# template <> +# void generateValues(MutableColumnPtr & col, size_t num) +# { +# std::string str; +# while (col->size() < num) +# { +# auto len = MIN_STRING_LEN + nextNum() % (MAX_STRING_LEN - MIN_STRING_LEN); +# str.clear(); +# for (size_t i = 0; i < len; i++) +# { +# str.push_back('a' + nextNum() % ('z' - 'a')); +# } +# col->insert(str); +# } +# } +# +# template +# ColumnWithTypeAndName generateColumn( +# std::shared_ptr ch_type, +# size_t cardinality, +# const std::string & col_name, +# const std::set & null_indice) +# { +# DataTypePtr col_type = ch_type; +# if (!null_indice.empty()) +# { +# col_type = std::make_shared(ch_type); +# } +# +# auto values = ch_type->createColumn(); +# values->reserve(cardinality); +# generateValues(values, cardinality); +# +# auto col = col_type->createColumn(); +# col->reserve(ROW_NUM); +# for (size_t i = 0; i < ROW_NUM; i++) +# { +# if (!null_indice.empty() && null_indice.contains(i)) +# { +# col->insert(Null()); +# } +# else +# { +# col->insert(values->operator[](nextNum() % cardinality)); +# } +# } +# return {std::move(col), col_type, col_name}; +# } +# +# Block generateBlock() +# { +# ColumnsWithTypeAndName cols; +# +# // test Int32 type +# std::set null_indice{512, 1001, 211, 392, 553, 1725}; +# // Nullability is expressed by definition level, and encoded by bit packed with smallest group size of 8 +# // when null value appeared. Here we make a big bit packed group with more than 1000 values. +# for (size_t i = 0; i < 170; i++) +# { +# null_indice.emplace(622 + i * 6); +# } +# cols.emplace_back(generateColumn( +# std::make_shared(), PLAIN_ENCODING_CARDINALITY, "plain_encoding_i32", null_indice)); +# null_indice = {917, 482, 283, 580, 1926, 1667, 1971}; +# cols.emplace_back(generateColumn( +# std::make_shared(), DICT_ENCODING_CARDINALITY, "dict_encoding_i32", null_indice)); +# +# // test string type +# null_indice = {818, 928, 1958, 1141, 1553, 1407, 690, 1769}; +# cols.emplace_back(generateColumn( +# std::make_shared(), PLAIN_ENCODING_CARDINALITY, "plain_encoding_str", null_indice)); +# null_indice = {1441, 1747, 216, 1209, 89, 52, 536, 625}; +# cols.emplace_back(generateColumn( +# std::make_shared(), MIX_ENCODING_CARDINALITY, "mix_encoding_str", null_indice)); +# null_indice = {1478, 1862, 894, 1314, 1844, 243, 869, 551}; +# cols.emplace_back(generateColumn( +# std::make_shared(), DICT_ENCODING_CARDINALITY, "dict_encoding_str", null_indice)); +# +# // test DateTime64 type +# auto dt_type = std::make_shared(ParquetRecordReader::default_datetime64_scale); +# null_indice = {1078, 112, 1981, 795, 371, 1176, 1526, 11}; +# cols.emplace_back(generateColumn(dt_type, PLAIN_ENCODING_CARDINALITY, "plain_encoding_dt64", null_indice)); +# null_indice = {1734, 1153, 1893, 1205, 644, 1670, 1482, 1479}; +# cols.emplace_back(generateColumn(dt_type, DICT_ENCODING_CARDINALITY, "dict_encoding_dt64", null_indice)); +# +# // test Decimal128 type +# auto d128_type = std::make_shared(DecimalUtils::max_precision, 3); +# null_indice = {852, 1448, 1569, 896, 1866, 1655, 100, 418}; +# cols.emplace_back(generateColumn(d128_type, PLAIN_ENCODING_CARDINALITY, "plain_encoding_decimal128", null_indice)); +# +# return {cols}; +# } +# +# void dumpBlock(const Block & block) +# { +# WriteBufferFromFile output_buf("/tmp/ut-out.csv"); +# auto out = getContext().context->getOutputFormat("CSVWithNames", output_buf, block); +# out->write(block); +# out->finalize(); +# std::cerr << block.dumpStructure() << std::endl << std::endl; +# } +# +# } +# +# EndOfCodes +# +# How to generate the parquet file: +# 1. Use above C++ codes. +# Put above codes in src/Common/tests/gtest_main.cpp, add following two inlines in main function: +# tryRegisterFormats(); +# dumpBlock(generateBlock()); +# 2. Genetate /tmp/ut-out.csv. +# After compiled, run any test, such as "./src/unit_tests_dbms --gtest_filter=IColumn.dumpStructure", +# 3. Generate the parquet file by following spark sql +# create temporary view tv using csv options('path' '/tmp/ut-out.csv', 'header' 'true', 'nullValue' '\\N'); +# insert overwrite directory "/tmp/test-parquet" using Parquet +# options('parquet.dictionary.page.size' '500') +# select /*+ COALESCE(1) */ cast(plain_encoding_i32 as int), cast(dict_encoding_i32 as int), +# plain_encoding_str, mix_encoding_str, dict_encoding_str, +# cast(plain_encoding_dt64 as timestamp), cast(dict_encoding_dt64 as timestamp), +# cast(plain_encoding_decimal128 as decimal(38, 3)) +# from tv; +# + +CH_SCHEMA="\ + plain_encoding_i32 Nullable(Int32), \ + dict_encoding_i32 Nullable(Int32), \ + plain_encoding_str Nullable(String), \ + mix_encoding_str Nullable(String), \ + dict_encoding_str LowCardinality(Nullable(String)), \ + plain_encoding_dt64 Nullable(DateTime64(9)), \ + dict_encoding_dt64 Nullable(DateTime64(9)), \ + plain_encoding_decimal128 Nullable(Decimal(38, 3))" +QUERY="SELECT * from file('$PAR_PATH', 'Parquet', '$CH_SCHEMA')" + +# there may be more than on group in parquet files, unstable results may generated by multithreads +$CLICKHOUSE_LOCAL --multiquery --max_threads 1 --input_format_parquet_use_native_reader true --query "$QUERY" From e1fcdba4dd51a4b4af500c1a09663820004a4a76 Mon Sep 17 00:00:00 2001 From: copperybean Date: Sat, 24 Feb 2024 22:47:53 +0800 Subject: [PATCH 07/36] fix style Change-Id: I8f7ebd173558b16d94d3161cb0b5300e7e78833d --- .../Formats/Impl/Parquet/ParquetDataBuffer.h | 21 ++++++---- .../Impl/Parquet/ParquetDataValuesReader.cpp | 40 +++++++++++++------ .../Impl/Parquet/ParquetDataValuesReader.h | 6 --- .../Impl/Parquet/ParquetLeafColReader.cpp | 18 ++++++--- .../Impl/Parquet/ParquetRecordReader.cpp | 3 +- 5 files changed, 54 insertions(+), 34 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h index d4956f83092..f21216d5b5d 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int PARQUET_EXCEPTION; +} + template struct ToArrowDecimal; template <> struct ToArrowDecimal>> @@ -27,8 +32,8 @@ class ParquetDataBuffer private: public: - ParquetDataBuffer(const uint8_t * data_, UInt64 avaible_, UInt8 datetime64_scale_ = DataTypeDateTime64::default_scale) - : data(reinterpret_cast(data_)), avaible(avaible_), datetime64_scale(datetime64_scale_) {} + ParquetDataBuffer(const uint8_t * data_, UInt64 available_, UInt8 datetime64_scale_ = DataTypeDateTime64::default_scale) + : data(reinterpret_cast(data_)), available(available_), datetime64_scale(datetime64_scale_) {} template void ALWAYS_INLINE readValue(TValue & dst) @@ -84,7 +89,7 @@ public: auto value_len = ::arrow::util::SafeLoadAs(getArrowData()); if (unlikely(value_len < 0 || value_len > INT32_MAX - 4)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid or corrupted value_len '{}'", value_len); + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Invalid or corrupted value_len '{}'", value_len); } consume(4); checkAvaible(value_len); @@ -110,7 +115,7 @@ public: auto status = TArrowDecimal::FromBigEndian(getArrowData(), elem_bytes_num); if (unlikely(!status.ok())) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Read parquet decimal failed: {}", status.status().ToString()); + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Read parquet decimal failed: {}", status.status().ToString()); } status.ValueUnsafe().ToBytes(reinterpret_cast(out)); consume(elem_bytes_num); @@ -118,14 +123,14 @@ public: private: const Int8 * data; - UInt64 avaible; + UInt64 available; const UInt8 datetime64_scale; void ALWAYS_INLINE checkAvaible(UInt64 num) { - if (unlikely(avaible < num)) + if (unlikely(available < num)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Consuming {} bytes while {} avaible", num, avaible); + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Consuming {} bytes while {} available", num, available); } } @@ -134,7 +139,7 @@ private: void ALWAYS_INLINE consume(UInt64 num) { data += num; - avaible -= num; + available -= num; } }; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 3afc66dcb36..4ebe3d6a636 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -8,6 +8,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int PARQUET_EXCEPTION; +} + void RleValuesReader::nextGroup() { // refer to: @@ -142,7 +148,7 @@ void RleValuesReader::visitNullableBySteps( individual_null_visitor(null_map_cursor); if (unlikely(valid_index_steps[step_idx] == UINT8_MAX)) { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "unsupported packed values number"); + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "unsupported packed values number"); } valid_index_steps[step_idx]++; } @@ -270,7 +276,7 @@ void ParquetPlainValuesReader::readBatch( auto idx = cursor; cursor += count; - // the type of offset_data is PaddedPODArray, which makes sure that the -1 index is avaible + // the type of offset_data is PaddedPODArray, which makes sure that the -1 index is available for (auto val_offset = offset_data[idx - 1]; idx < cursor; idx++) { offset_data[idx] = ++val_offset; @@ -394,14 +400,17 @@ void ParquetRleLCReader::readBatch( cursor, num_values, max_def_level, - /* individual_null_visitor */ [&](size_t nest_cursor) { + /* individual_null_visitor */ [&](size_t nest_cursor) + { column_data[nest_cursor] = 0; has_null = true; }, - /* stepped_valid_visitor */ [&](size_t nest_cursor, const std::vector & valid_index_steps) { + /* stepped_valid_visitor */ [&](size_t nest_cursor, const std::vector & valid_index_steps) + { rle_data_reader->setValueBySteps(column_data + nest_cursor, valid_index_steps, val_getter); }, - /* repeated_visitor */ [&](bool is_valid, size_t nest_cursor, UInt32 count) { + /* repeated_visitor */ [&](bool is_valid, size_t nest_cursor, UInt32 count) + { if (is_valid) { rle_data_reader->setValues(column_data + nest_cursor, count, val_getter); @@ -435,7 +444,8 @@ void ParquetRleDictReader::readBatch( auto * offset_data = column.getOffsets().data(); auto & chars = column.getChars(); - auto append_nulls = [&](UInt8 num) { + auto append_nulls = [&](UInt8 num) + { for (auto limit = cursor + num; cursor < limit; cursor++) { chars.push_back(0); @@ -444,7 +454,8 @@ void ParquetRleDictReader::readBatch( } }; - auto append_string = [&](Int32 dict_idx) { + auto append_string = [&](Int32 dict_idx) + { auto dict_chars_cursor = dict_offsets[dict_idx - 1]; auto value_len = dict_offsets[dict_idx] - dict_chars_cursor; auto chars_cursor = chars.size(); @@ -462,7 +473,8 @@ void ParquetRleDictReader::readBatch( num_values, max_def_level, /* individual_null_visitor */ [&](size_t) {}, - /* stepped_valid_visitor */ [&](size_t, const std::vector & valid_index_steps) { + /* stepped_valid_visitor */ [&](size_t, const std::vector & valid_index_steps) + { value_cache.resize(valid_index_steps.size()); rle_data_reader->setValues( value_cache.data() + 1, static_cast(valid_index_steps.size() - 1), val_getter); @@ -474,7 +486,8 @@ void ParquetRleDictReader::readBatch( append_nulls(valid_index_steps[i] - 1); } }, - /* repeated_visitor */ [&](bool is_valid, size_t, UInt32 count) { + /* repeated_visitor */ [&](bool is_valid, size_t, UInt32 count) + { if (is_valid) { value_cache.resize(count); @@ -505,13 +518,16 @@ void ParquetRleDictReader::readBatch( cursor, num_values, max_def_level, - /* individual_null_visitor */ [&](size_t nest_cursor) { + /* individual_null_visitor */ [&](size_t nest_cursor) + { null_map.setNull(nest_cursor); }, - /* stepped_valid_visitor */ [&](size_t nest_cursor, const std::vector & valid_index_steps) { + /* stepped_valid_visitor */ [&](size_t nest_cursor, const std::vector & valid_index_steps) + { rle_data_reader->setValueBySteps(column_data + nest_cursor, valid_index_steps, val_getter); }, - /* repeated_visitor */ [&](bool is_valid, size_t nest_cursor, UInt32 count) { + /* repeated_visitor */ [&](bool is_valid, size_t nest_cursor, UInt32 count) + { if (is_valid) { rle_data_reader->setValues(column_data + nest_cursor, count, val_getter); diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index 66a1f4877e4..8bc381aa8d2 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -15,12 +15,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int PARQUET_EXCEPTION; -} - class RleValuesReader { public: diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index e2677d7cae3..17feea80b9f 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -216,7 +216,8 @@ template ColumnWithTypeAndName ParquetLeafColReader::readBatch(UInt64 rows_num, const String & name) { reading_rows_num = rows_num; - auto readPageIfEmpty = [&]() { + auto readPageIfEmpty = [&]() + { while (!cur_page_values) readPage(); }; @@ -245,7 +246,8 @@ void ParquetLeafColReader::resetColumn(UInt64 rows_num) if (reading_low_cardinality) { assert(dictionary); - visitColStrIndexType(dictionary->size(), [&](TColVec *) { + visitColStrIndexType(dictionary->size(), [&](TColVec *) + { column = TColVec::create(); }); @@ -289,7 +291,8 @@ void ParquetLeafColReader::degradeDictionary() ColumnString & col_dest = *static_cast(column.get()); const ColumnString & col_dict_str = *static_cast(dictionary.get()); - visitColStrIndexType(dictionary->size(), [&](TColVec *) { + visitColStrIndexType(dictionary->size(), [&](TColVec *) + { const TColVec & col_src = *static_cast(col_existing.get()); reserveColumnStrRows(column, reading_rows_num); @@ -411,7 +414,8 @@ void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) assert(col_descriptor.max_definition_level() >= 0); std::unique_ptr def_level_reader; - if (col_descriptor.max_definition_level() > 0) { + if (col_descriptor.max_definition_level() > 0) + { auto bit_width = arrow::bit_util::Log2(col_descriptor.max_definition_level() + 1); auto num_bytes = ::arrow::util::SafeLoadAs(buffer); auto bit_reader = std::make_unique(buffer + 4, num_bytes); @@ -435,7 +439,8 @@ void ParquetLeafColReader::readPageV1(const parquet::DataPageV1 & page) degradeDictionary(); } - ParquetDataBuffer parquet_buffer = [&]() { + ParquetDataBuffer parquet_buffer = [&]() + { if constexpr (!std::is_same_v, TColumn>) return ParquetDataBuffer(buffer, max_size); @@ -485,7 +490,8 @@ std::unique_ptr ParquetLeafColReader::createDi if (reading_low_cardinality && std::same_as) { std::unique_ptr res; - visitColStrIndexType(dictionary->size(), [&](TCol *) { + visitColStrIndexType(dictionary->size(), [&](TCol *) + { res = std::make_unique>( col_descriptor.max_definition_level(), std::move(def_level_reader), diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index 42f131ff794..69e694a340f 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -27,7 +27,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int PARQUET_EXCEPTION; } @@ -142,7 +141,7 @@ std::unique_ptr createColReader( } } -} // anonymouse namespace +} // anonymous namespace ParquetRecordReader::ParquetRecordReader( Block header_, From 471dff6589abff5d05ab8a9bb267e198f377c536 Mon Sep 17 00:00:00 2001 From: copperybean Date: Sun, 25 Feb 2024 14:26:53 +0800 Subject: [PATCH 08/36] fix test Change-Id: Ia7dbf1d762f7f054a9aa677caaaff6bfe1a42c38 --- src/Core/SettingsChangesHistory.h | 1 + .../Formats/Impl/Parquet/ParquetDataBuffer.h | 13 +++++-------- .../Impl/Parquet/ParquetDataValuesReader.cpp | 2 +- .../Formats/Impl/Parquet/ParquetDataValuesReader.h | 4 ++-- .../Formats/Impl/Parquet/ParquetLeafColReader.cpp | 6 +++--- .../Formats/Impl/Parquet/ParquetRecordReader.cpp | 7 ++----- .../Formats/Impl/ParquetBlockInputFormat.cpp | 8 ++++++++ .../0_stateless/02998_native_parquet_reader.sh | 5 +++-- 8 files changed, 25 insertions(+), 21 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ece48620618..6fb8fb9358c 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -176,6 +176,7 @@ static std::map sett {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, + {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, }}, {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h index f21216d5b5d..5c37375fa0c 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h @@ -38,15 +38,13 @@ public: template void ALWAYS_INLINE readValue(TValue & dst) { - checkAvaible(sizeof(TValue)); - dst = *(reinterpret_cast(data)); - consume(sizeof(TValue)); + readBytes(&dst, sizeof(TValue)); } void ALWAYS_INLINE readBytes(void * dst, size_t bytes) { checkAvaible(bytes); - memcpy(dst, data, bytes); + std::copy(data, data + bytes, reinterpret_cast(dst)); consume(bytes); } @@ -68,13 +66,12 @@ public: 100000000 * spd, 1000000000 * spd}; - checkAvaible(sizeof(parquet::Int96)); - auto decoded = parquet::DecodeInt96Timestamp(*reinterpret_cast(data)); + parquet::Int96 tmp; + readValue(tmp); + auto decoded = parquet::DecodeInt96Timestamp(tmp); uint64_t scaled_nano = decoded.nanoseconds / pow10[datetime64_scale]; dst = static_cast(decoded.days_since_epoch * scaled_day[datetime64_scale] + scaled_nano); - - consume(sizeof(parquet::Int96)); } /** diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 4ebe3d6a636..6743086e9e6 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -306,7 +306,7 @@ void ParquetPlainValuesReader>::readBatch( }, /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) { - auto col_data_pos = column_data + nest_cursor; + auto * col_data_pos = column_data + nest_cursor; for (UInt32 i = 0; i < count; i++) { plain_data_buffer.readDateTime64(col_data_pos[i]); diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index 8bc381aa8d2..688de4f52eb 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -97,7 +97,7 @@ public: * @tparam ValueGetter A callback with signature: TValue(Int32 val) */ template - void setValues(TValue * column_data, UInt32 num_values, ValueGetter && val_getter); + void setValues(TValue * res_values, UInt32 num_values, ValueGetter && val_getter); /** * @brief Set the value by valid_index_steps generated in visitNullableBySteps. @@ -106,7 +106,7 @@ public: */ template void setValueBySteps( - TValue * column_data, + TValue * res_values, const std::vector & col_data_steps, ValueGetter && val_getter); diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index 17feea80b9f..52dfad7606a 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -113,7 +113,7 @@ ColumnPtr readDictPage>( const parquet::ColumnDescriptor & /* col_des */, const DataTypePtr & data_type) { - auto & datetime_type = assert_cast(*data_type); + const auto & datetime_type = assert_cast(*data_type); auto dict_col = ColumnDecimal::create(page.num_values(), datetime_type.getScale()); auto * col_data = dict_col->getData().data(); ParquetDataBuffer buffer(page.data(), page.size(), datetime_type.getScale()); @@ -282,7 +282,7 @@ void ParquetLeafColReader::degradeDictionary() dictionary = nullptr; return; } - assert(dictionary && column->size()); + assert(dictionary && !column->empty()); null_map = std::make_unique(reading_rows_num); auto col_existing = std::move(column); @@ -372,7 +372,7 @@ void ParquetLeafColReader::readPage() dict_page.encoding() != parquet::Encoding::PLAIN_DICTIONARY && dict_page.encoding() != parquet::Encoding::PLAIN)) { - throw new Exception( + throw Exception( ErrorCodes::NOT_IMPLEMENTED, "Unsupported dictionary page encoding {}", dict_page.encoding()); } LOG_DEBUG(log, "{} values in dictionary page of column {}", dict_page.num_values(), col_descriptor.name()); diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index 69e694a340f..9cde433b983 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -156,9 +156,6 @@ ParquetRecordReader::ParquetRecordReader( , row_groups_indices(std::move(row_groups_indices_)) , left_rows(getTotalRows(*file_reader->metadata())) { - // Only little endian system is supported currently - static_assert(std::endian::native == std::endian::little); - log = &Poco::Logger::get("ParquetRecordReader"); parquet_col_indice.reserve(header.columns()); @@ -230,9 +227,9 @@ void ParquetRecordReader::loadNextRowGroup() Int64 ParquetRecordReader::getTotalRows(const parquet::FileMetaData & meta_data) { Int64 res = 0; - for (size_t i = 0; i < row_groups_indices.size(); i++) + for (auto idx : row_groups_indices) { - res += meta_data.RowGroup(row_groups_indices[i])->num_rows(); + res += meta_data.RowGroup(idx)->num_rows(); } return res; } diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index e35d53dc4f4..2e849f09fda 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -484,6 +484,14 @@ void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_bat if (format_settings.parquet.use_native_reader) { +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wunreachable-code" + if constexpr (std::endian::native != std::endian::little) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "parquet native reader only supports little endian system currently"); +#pragma clang diagnostic pop + row_group_batch.native_record_reader = std::make_shared( getPort().getHeader(), std::move(properties), diff --git a/tests/queries/0_stateless/02998_native_parquet_reader.sh b/tests/queries/0_stateless/02998_native_parquet_reader.sh index 5c129e6c5ce..4e5169c4bf0 100755 --- a/tests/queries/0_stateless/02998_native_parquet_reader.sh +++ b/tests/queries/0_stateless/02998_native_parquet_reader.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -201,8 +202,8 @@ CH_SCHEMA="\ plain_encoding_str Nullable(String), \ mix_encoding_str Nullable(String), \ dict_encoding_str LowCardinality(Nullable(String)), \ - plain_encoding_dt64 Nullable(DateTime64(9)), \ - dict_encoding_dt64 Nullable(DateTime64(9)), \ + plain_encoding_dt64 Nullable(DateTime64(9, \\'UTC\\')), \ + dict_encoding_dt64 Nullable(DateTime64(9, \\'UTC\\')), \ plain_encoding_decimal128 Nullable(Decimal(38, 3))" QUERY="SELECT * from file('$PAR_PATH', 'Parquet', '$CH_SCHEMA')" From 47dfeaa487743d81c66bb280e8eeb8f31ef21507 Mon Sep 17 00:00:00 2001 From: copperybean Date: Sun, 12 May 2024 21:57:37 +0800 Subject: [PATCH 09/36] fix comments Change-Id: I2677dc20fc515bbbe91f54154fc4c081f164758e --- .../Formats/Impl/Parquet/ParquetDataBuffer.h | 9 +- .../Impl/Parquet/ParquetDataValuesReader.cpp | 18 +- .../Impl/Parquet/ParquetDataValuesReader.h | 13 +- .../Impl/Parquet/ParquetLeafColReader.cpp | 33 +- .../Impl/Parquet/ParquetRecordReader.cpp | 326 +++++++++++++----- .../Impl/Parquet/ParquetRecordReader.h | 6 +- .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 + .../02998_native_parquet_reader.sh | 4 +- .../native_parquet_reader.parquet} | Bin 9 files changed, 296 insertions(+), 115 deletions(-) rename tests/queries/0_stateless/{02998_native_parquet_reader.parquet => data_parquet/native_parquet_reader.parquet} (100%) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h index 5c37375fa0c..57df6f59f72 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataBuffer.h @@ -48,7 +48,7 @@ public: consume(bytes); } - void ALWAYS_INLINE readDateTime64(DateTime64 & dst) + void ALWAYS_INLINE readDateTime64FromInt96(DateTime64 & dst) { static const int max_scale_num = 9; static const UInt64 pow10[max_scale_num + 1] @@ -110,10 +110,7 @@ public: // refer to: RawBytesToDecimalBytes in reader_internal.cc, Decimal128::FromBigEndian in decimal.cc auto status = TArrowDecimal::FromBigEndian(getArrowData(), elem_bytes_num); - if (unlikely(!status.ok())) - { - throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Read parquet decimal failed: {}", status.status().ToString()); - } + assert(status.ok()); status.ValueUnsafe().ToBytes(reinterpret_cast(out)); consume(elem_bytes_num); } @@ -144,7 +141,7 @@ private: class LazyNullMap { public: - LazyNullMap(UInt64 size_) : size(size_), col_nullable(nullptr) {} + explicit LazyNullMap(UInt64 size_) : size(size_), col_nullable(nullptr) {} template requires std::is_integral_v diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 6743086e9e6..1f0c7105572 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -276,8 +276,7 @@ void ParquetPlainValuesReader::readBatch( auto idx = cursor; cursor += count; - // the type of offset_data is PaddedPODArray, which makes sure that the -1 index is available - for (auto val_offset = offset_data[idx - 1]; idx < cursor; idx++) + for (auto val_offset = chars_size_bak; idx < cursor; idx++) { offset_data[idx] = ++val_offset; } @@ -288,7 +287,7 @@ void ParquetPlainValuesReader::readBatch( template <> -void ParquetPlainValuesReader>::readBatch( +void ParquetPlainValuesReader, ParquetReaderTypes::TimestampInt96>::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) { auto cursor = col_ptr->size(); @@ -302,21 +301,21 @@ void ParquetPlainValuesReader>::readBatch( null_map, /* individual_visitor */ [&](size_t nest_cursor) { - plain_data_buffer.readDateTime64(column_data[nest_cursor]); + plain_data_buffer.readDateTime64FromInt96(column_data[nest_cursor]); }, /* repeated_visitor */ [&](size_t nest_cursor, UInt32 count) { auto * col_data_pos = column_data + nest_cursor; for (UInt32 i = 0; i < count; i++) { - plain_data_buffer.readDateTime64(col_data_pos[i]); + plain_data_buffer.readDateTime64FromInt96(col_data_pos[i]); } } ); } -template -void ParquetPlainValuesReader::readBatch( +template +void ParquetPlainValuesReader::readBatch( MutableColumnPtr & col_ptr, LazyNullMap & null_map, UInt32 num_values) { auto cursor = col_ptr->size(); @@ -542,11 +541,14 @@ void ParquetRleDictReader::readBatch( template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader; template class ParquetPlainValuesReader; +template class ParquetPlainValuesReader; template class ParquetPlainValuesReader; template class ParquetPlainValuesReader; template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader>; +template class ParquetPlainValuesReader>; template class ParquetPlainValuesReader; template class ParquetFixedLenPlainReader>; @@ -557,7 +559,9 @@ template class ParquetRleLCReader; template class ParquetRleLCReader; template class ParquetRleDictReader; +template class ParquetRleDictReader; template class ParquetRleDictReader; +template class ParquetRleDictReader; template class ParquetRleDictReader; template class ParquetRleDictReader; template class ParquetRleDictReader>; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index 688de4f52eb..0f916ff862d 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -24,7 +24,7 @@ public: /** * @brief Used when the bit_width is 0, so all elements have same value. */ - RleValuesReader(UInt32 total_size, Int32 val = 0) + explicit RleValuesReader(UInt32 total_size, Int32 val = 0) : bit_reader(nullptr), bit_width(0), cur_group_size(total_size), cur_value(val), cur_group_is_packed(false) {} @@ -72,7 +72,8 @@ public: * @tparam SteppedValidVisitor A callback with signature: * void(size_t cursor, const std::vector & valid_index_steps) * for n valid elements with null value interleaved in a BitPacked group, - * i-th item in valid_index_steps describes how many elements in column there are after (i-1)-th valid element. + * i-th item in valid_index_steps describes how many elements there are + * from i-th valid element (include) to (i+1)-th valid element (exclude). * * take following BitPacked group with 2 valid elements for example: * null valid null null valid null @@ -138,10 +139,16 @@ public: using ParquetDataValuesReaderPtr = std::unique_ptr; +enum class ParquetReaderTypes +{ + Normal, + TimestampInt96, +}; + /** * The definition level is RLE or BitPacked encoding, while data is read directly */ -template +template class ParquetPlainValuesReader : public ParquetDataValuesReader { public: diff --git a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp index 52dfad7606a..9e1cae9bb65 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetLeafColReader.cpp @@ -110,16 +110,24 @@ ColumnPtr readDictPage( template <> ColumnPtr readDictPage>( const parquet::DictionaryPage & page, - const parquet::ColumnDescriptor & /* col_des */, + const parquet::ColumnDescriptor & col_des, const DataTypePtr & data_type) { + const auto & datetime_type = assert_cast(*data_type); auto dict_col = ColumnDecimal::create(page.num_values(), datetime_type.getScale()); auto * col_data = dict_col->getData().data(); ParquetDataBuffer buffer(page.data(), page.size(), datetime_type.getScale()); - for (auto i = 0; i < page.num_values(); i++) + if (col_des.physical_type() == parquet::Type::INT64) { - buffer.readDateTime64(col_data[i]); + buffer.readBytes(dict_col->getData().data(), page.num_values() * sizeof(Int64)); + } + else + { + for (auto i = 0; i < page.num_values(); i++) + { + buffer.readDateTime64FromInt96(col_data[i]); + } } return dict_col; } @@ -190,8 +198,12 @@ std::unique_ptr createPlainReader( RleValuesReaderPtr def_level_reader, ParquetDataBuffer buffer) { - return std::make_unique>( - col_des.max_definition_level(), std::move(def_level_reader), std::move(buffer)); + if (std::is_same_v> && col_des.physical_type() == parquet::Type::INT96) + return std::make_unique>( + col_des.max_definition_level(), std::move(def_level_reader), std::move(buffer)); + else + return std::make_unique>( + col_des.max_definition_level(), std::move(def_level_reader), std::move(buffer)); } @@ -287,6 +299,7 @@ void ParquetLeafColReader::degradeDictionary() null_map = std::make_unique(reading_rows_num); auto col_existing = std::move(column); column = ColumnString::create(); + reserveColumnStrRows(column, reading_rows_num); ColumnString & col_dest = *static_cast(column.get()); const ColumnString & col_dict_str = *static_cast(dictionary.get()); @@ -294,8 +307,9 @@ void ParquetLeafColReader::degradeDictionary() visitColStrIndexType(dictionary->size(), [&](TColVec *) { const TColVec & col_src = *static_cast(col_existing.get()); - reserveColumnStrRows(column, reading_rows_num); + // It will be easier to create a ColumnLowCardinality and call convertToFullColumn() on it, + // while the performance loss is ignorable, the implementation can be updated next time. col_dest.getOffsets().resize(col_src.size()); for (size_t i = 0; i < col_src.size(); i++) { @@ -378,6 +392,11 @@ void ParquetLeafColReader::readPage() LOG_DEBUG(log, "{} values in dictionary page of column {}", dict_page.num_values(), col_descriptor.name()); dictionary = readDictPage(dict_page, col_descriptor, base_data_type); + if (unlikely(dictionary->size() < 2)) + { + // must not small than ColumnUnique::numSpecialValues() + dictionary->assumeMutable()->insertManyDefaults(2); + } if (std::is_same_v) { reading_low_cardinality = true; @@ -508,7 +527,9 @@ std::unique_ptr ParquetLeafColReader::createDi template class ParquetLeafColReader; +template class ParquetLeafColReader; template class ParquetLeafColReader; +template class ParquetLeafColReader; template class ParquetLeafColReader; template class ParquetLeafColReader; template class ParquetLeafColReader; diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index 9cde433b983..fddd8059925 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -36,8 +36,7 @@ namespace ErrorCodes try { (s); } \ catch (const ::parquet::ParquetException & e) \ { \ - auto msg = PreformattedMessage::create("Excepted when reading parquet: {}", e.what()); \ - throw Exception(std::move(msg), ErrorCodes::PARQUET_EXCEPTION); \ + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "Parquet exception: {}", e.what()); \ } \ } while (false) @@ -45,102 +44,252 @@ namespace { std::unique_ptr createFileReader( - std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file) + std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file, + std::shared_ptr metadata = nullptr) { std::unique_ptr res; - THROW_PARQUET_EXCEPTION(res = parquet::ParquetFileReader::Open(std::move(arrow_file))); + THROW_PARQUET_EXCEPTION(res = parquet::ParquetFileReader::Open( + std::move(arrow_file), + parquet::default_reader_properties(), + metadata)); return res; } -std::unique_ptr createColReader( - const parquet::ColumnDescriptor & col_descriptor, - DataTypePtr ch_type, - std::unique_ptr meta, - std::unique_ptr reader) +class ColReaderFactory { - if (col_descriptor.logical_type()->is_date() && parquet::Type::INT32 == col_descriptor.physical_type()) +public: + ColReaderFactory( + const parquet::ArrowReaderProperties & reader_properties_, + const parquet::ColumnDescriptor & col_descriptor_, + DataTypePtr ch_type_, + std::unique_ptr meta_, + std::unique_ptr page_reader_) + : reader_properties(reader_properties_) + , col_descriptor(col_descriptor_) + , ch_type(std::move(ch_type_)) + , meta(std::move(meta_)) + , page_reader(std::move(page_reader_)) {} + + std::unique_ptr makeReader(); + +private: + const parquet::ArrowReaderProperties & reader_properties; + const parquet::ColumnDescriptor & col_descriptor; + DataTypePtr ch_type; + std::unique_ptr meta; + std::unique_ptr page_reader; + + + UInt32 getScaleFromLogicalTimestamp(parquet::LogicalType::TimeUnit::unit tm_unit); + UInt32 getScaleFromArrowTimeUnit(arrow::TimeUnit::type tm_unit); + + std::unique_ptr fromInt32(); + std::unique_ptr fromInt64(); + std::unique_ptr fromByteArray(); + std::unique_ptr fromFLBA(); + + std::unique_ptr fromInt32INT(const parquet::IntLogicalType & int_type); + std::unique_ptr fromInt64INT(const parquet::IntLogicalType & int_type); + + template + auto makeLeafReader() { - return std::make_unique>( - col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); + return std::make_unique>( + col_descriptor, std::make_shared(), std::move(meta), std::move(page_reader)); } - else if (col_descriptor.logical_type()->is_decimal()) + + template + auto makeDecimalLeafReader() { - switch (col_descriptor.physical_type()) + auto data_type = std::make_shared>( + col_descriptor.type_precision(), col_descriptor.type_scale()); + return std::make_unique>>( + col_descriptor, std::move(data_type), std::move(meta), std::move(page_reader)); + } + + std::unique_ptr throwUnsupported(std::string msg = "") + { + throw Exception( + ErrorCodes::PARQUET_EXCEPTION, + "Unsupported logical type: {} and physical type: {} for field =={}=={}", + col_descriptor.logical_type()->ToString(), col_descriptor.physical_type(), col_descriptor.name(), msg); + } +}; + +UInt32 ColReaderFactory::getScaleFromLogicalTimestamp(parquet::LogicalType::TimeUnit::unit tm_unit) +{ + switch (tm_unit) + { + case parquet::LogicalType::TimeUnit::MILLIS: + return 3; + case parquet::LogicalType::TimeUnit::MICROS: + return 6; + case parquet::LogicalType::TimeUnit::NANOS: + return 9; + default: + throwUnsupported(PreformattedMessage::create(", invalid timestamp unit: {}", tm_unit)); + return 0; + } +} + +UInt32 ColReaderFactory::getScaleFromArrowTimeUnit(arrow::TimeUnit::type tm_unit) +{ + switch (tm_unit) + { + case arrow::TimeUnit::MILLI: + return 3; + case arrow::TimeUnit::MICRO: + return 6; + case arrow::TimeUnit::NANO: + return 9; + default: + throwUnsupported(PreformattedMessage::create(", invalid arrow time unit: {}", tm_unit)); + return 0; + } +} + +std::unique_ptr ColReaderFactory::fromInt32() +{ + switch (col_descriptor.logical_type()->type()) + { + case parquet::LogicalType::Type::INT: + return fromInt32INT(dynamic_cast(*col_descriptor.logical_type())); + case parquet::LogicalType::Type::NONE: + return makeLeafReader(); + case parquet::LogicalType::Type::DATE: + return makeLeafReader(); + case parquet::LogicalType::Type::DECIMAL: + return makeDecimalLeafReader(); + default: + return throwUnsupported(); + } +} + +std::unique_ptr ColReaderFactory::fromInt64() +{ + switch (col_descriptor.logical_type()->type()) + { + case parquet::LogicalType::Type::INT: + return fromInt64INT(dynamic_cast(*col_descriptor.logical_type())); + case parquet::LogicalType::Type::NONE: + return makeLeafReader(); + case parquet::LogicalType::Type::TIMESTAMP: { - case parquet::Type::INT32: - { - auto data_type = std::make_shared( - col_descriptor.type_precision(), col_descriptor.type_scale()); - return std::make_unique>>( - col_descriptor, data_type, std::move(meta), std::move(reader)); - } - case parquet::Type::INT64: - { - auto data_type = std::make_shared( - col_descriptor.type_precision(), col_descriptor.type_scale()); - return std::make_unique>>( - col_descriptor, data_type, std::move(meta), std::move(reader)); - } - case parquet::Type::FIXED_LEN_BYTE_ARRAY: - { - if (col_descriptor.type_length() <= static_cast(sizeof(Decimal128))) - { - auto data_type = std::make_shared( - col_descriptor.type_precision(), col_descriptor.type_scale()); - return std::make_unique>>( - col_descriptor, data_type, std::move(meta), std::move(reader)); - } - else - { - auto data_type = std::make_shared( - col_descriptor.type_precision(), col_descriptor.type_scale()); - return std::make_unique>>( - col_descriptor, data_type, std::move(meta), std::move(reader)); - } - } - default: - throw Exception( - ErrorCodes::PARQUET_EXCEPTION, - "Type not supported for decimal: {}", - col_descriptor.physical_type()); + const auto & tm_type = dynamic_cast(*col_descriptor.logical_type()); + auto read_type = std::make_shared(getScaleFromLogicalTimestamp(tm_type.time_unit())); + return std::make_unique>>( + col_descriptor, std::move(read_type), std::move(meta), std::move(page_reader)); } + case parquet::LogicalType::Type::DECIMAL: + return makeDecimalLeafReader(); + default: + return throwUnsupported(); } - else +} + +std::unique_ptr ColReaderFactory::fromByteArray() +{ + switch (col_descriptor.logical_type()->type()) { - switch (col_descriptor.physical_type()) - { - case parquet::Type::INT32: - return std::make_unique>( - col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); - case parquet::Type::INT64: - return std::make_unique>( - col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); - case parquet::Type::FLOAT: - return std::make_unique>( - col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); - case parquet::Type::INT96: - { - DataTypePtr read_type = ch_type; - if (!isDateTime64(ch_type)) - { - read_type = std::make_shared(ParquetRecordReader::default_datetime64_scale); - } - return std::make_unique>>( - col_descriptor, read_type, std::move(meta), std::move(reader)); - } - case parquet::Type::DOUBLE: - return std::make_unique>( - col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); - case parquet::Type::BYTE_ARRAY: - return std::make_unique>( - col_descriptor, std::make_shared(), std::move(meta), std::move(reader)); - default: - throw Exception( - ErrorCodes::PARQUET_EXCEPTION, "Type not supported: {}", col_descriptor.physical_type()); - } + case parquet::LogicalType::Type::STRING: + return makeLeafReader(); + default: + return throwUnsupported(); } } +std::unique_ptr ColReaderFactory::fromFLBA() +{ + switch (col_descriptor.logical_type()->type()) + { + case parquet::LogicalType::Type::DECIMAL: + { + if (col_descriptor.type_length() <= static_cast(sizeof(Decimal128))) + return makeDecimalLeafReader(); + else if (col_descriptor.type_length() <= static_cast(sizeof(Decimal256))) + return makeDecimalLeafReader(); + + return throwUnsupported(PreformattedMessage::create( + ", invalid type length: {}", col_descriptor.type_length())); + } + default: + return throwUnsupported(); + } +} + +std::unique_ptr ColReaderFactory::fromInt32INT(const parquet::IntLogicalType & int_type) +{ + switch (int_type.bit_width()) + { + case sizeof(Int32): + { + if (int_type.is_signed()) + return makeLeafReader(); + else + return makeLeafReader(); + } + default: + return throwUnsupported(PreformattedMessage::create(", bit width: {}", int_type.bit_width())); + } +} + +std::unique_ptr ColReaderFactory::fromInt64INT(const parquet::IntLogicalType & int_type) +{ + switch (int_type.bit_width()) + { + case sizeof(Int64): + { + if (int_type.is_signed()) + return makeLeafReader(); + else + return makeLeafReader(); + } + default: + return throwUnsupported(PreformattedMessage::create(", bit width: {}", int_type.bit_width())); + } +} + +// refer: GetArrowType method in schema_internal.cc of arrow +std::unique_ptr ColReaderFactory::makeReader() +{ + // this method should to be called only once for each instance + SCOPE_EXIT({ page_reader = nullptr; }); + assert(page_reader); + + switch (col_descriptor.physical_type()) + { + case parquet::Type::BOOLEAN: + break; + case parquet::Type::INT32: + return fromInt32(); + case parquet::Type::INT64: + return fromInt64(); + case parquet::Type::INT96: + { + DataTypePtr read_type = ch_type; + if (!isDateTime64(ch_type)) + { + auto scale = getScaleFromArrowTimeUnit(reader_properties.coerce_int96_timestamp_unit()); + read_type = std::make_shared(scale); + } + return std::make_unique>>( + col_descriptor, read_type, std::move(meta), std::move(page_reader)); + } + case parquet::Type::FLOAT: + return makeLeafReader(); + case parquet::Type::DOUBLE: + return makeLeafReader(); + case parquet::Type::BYTE_ARRAY: + return fromByteArray(); + case parquet::Type::FIXED_LEN_BYTE_ARRAY: + return fromFLBA(); + default: + break; + } + + return throwUnsupported(); +} + } // anonymous namespace ParquetRecordReader::ParquetRecordReader( @@ -148,8 +297,9 @@ ParquetRecordReader::ParquetRecordReader( parquet::ArrowReaderProperties reader_properties_, std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file, const FormatSettings & format_settings, - std::vector row_groups_indices_) - : file_reader(createFileReader(std::move(arrow_file))) + std::vector row_groups_indices_, + std::shared_ptr metadata) + : file_reader(createFileReader(std::move(arrow_file), std::move(metadata))) , reader_properties(reader_properties_) , header(std::move(header_)) , max_block_size(format_settings.parquet.max_block_size) @@ -210,15 +360,17 @@ void ParquetRecordReader::loadNextRowGroup() column_readers.clear(); for (size_t i = 0; i < parquet_col_indice.size(); i++) { - column_readers.emplace_back(createColReader( + ColReaderFactory factory( + reader_properties, *file_reader->metadata()->schema()->Column(parquet_col_indice[i]), header.getByPosition(i).type, cur_row_group_reader->metadata()->ColumnChunk(parquet_col_indice[i]), - cur_row_group_reader->GetColumnPageReader(parquet_col_indice[i]))); + cur_row_group_reader->GetColumnPageReader(parquet_col_indice[i])); + column_readers.emplace_back(factory.makeReader()); } auto duration = watch.elapsedNanoseconds() / 1e6; - LOG_DEBUG(log, "reading row group {} consumed {} ms", row_groups_indices[next_row_group_idx], duration); + LOG_DEBUG(log, "begin to read row group {} consumed {} ms", row_groups_indices[next_row_group_idx], duration); ++next_row_group_idx; cur_row_group_left_rows = cur_row_group_reader->metadata()->num_rows(); diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h index 4789be59ec8..2f728a586a0 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.h @@ -22,13 +22,11 @@ public: parquet::ArrowReaderProperties reader_properties_, std::shared_ptr<::arrow::io::RandomAccessFile> arrow_file, const FormatSettings & format_settings, - std::vector row_groups_indices_); + std::vector row_groups_indices_, + std::shared_ptr metadata = nullptr); Chunk readChunk(); - // follow the scale generated by spark - static constexpr UInt8 default_datetime64_scale = 9; - private: std::unique_ptr file_reader; parquet::ArrowReaderProperties reader_properties; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 2e849f09fda..7fc7b9c3cab 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -3,6 +3,7 @@ #if USE_PARQUET +#include #include #include #include @@ -623,6 +624,7 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un return; } + // TODO support defaults_for_omitted_fields feature when supporting nested columns auto num_rows = chunk.getNumRows(); res = get_pending_chunk(num_rows, std::move(chunk)); } diff --git a/tests/queries/0_stateless/02998_native_parquet_reader.sh b/tests/queries/0_stateless/02998_native_parquet_reader.sh index 4e5169c4bf0..d6369c4921b 100755 --- a/tests/queries/0_stateless/02998_native_parquet_reader.sh +++ b/tests/queries/0_stateless/02998_native_parquet_reader.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -PAR_PATH="$CURDIR"/02998_native_parquet_reader.parquet +PAR_PATH="$CURDIR"/data_parquet/native_parquet_reader.parquet # the content of parquet file can be generated by following codes # < Date: Fri, 17 May 2024 11:11:53 +0800 Subject: [PATCH 10/36] fix comments second time Change-Id: I4b75367233f99ef432cdff78f724195673755a83 --- src/Core/SettingsChangesHistory.h | 2 +- .../Formats/Impl/Parquet/ParquetDataValuesReader.cpp | 3 +++ .../Formats/Impl/Parquet/ParquetRecordReader.cpp | 12 ++++++++---- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 6fb8fb9358c..96ab7490c1f 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -91,6 +91,7 @@ static std::map sett {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, + {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, }}, @@ -176,7 +177,6 @@ static std::map sett {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, }}, {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 1f0c7105572..65f569ec264 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -29,6 +29,9 @@ void RleValuesReader::nextGroup() { cur_group_size *= 8; cur_packed_bit_values.resize(cur_group_size); + + // try to suppress clang tidy warnings by assertion + assert(bit_width < 64); bit_reader->GetBatch(bit_width, cur_packed_bit_values.data(), cur_group_size); } else diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index fddd8059925..0b797dd66ad 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -192,6 +192,7 @@ std::unique_ptr ColReaderFactory::fromByteArray() switch (col_descriptor.logical_type()->type()) { case parquet::LogicalType::Type::STRING: + case parquet::LogicalType::Type::NONE: return makeLeafReader(); default: return throwUnsupported(); @@ -204,10 +205,13 @@ std::unique_ptr ColReaderFactory::fromFLBA() { case parquet::LogicalType::Type::DECIMAL: { - if (col_descriptor.type_length() <= static_cast(sizeof(Decimal128))) - return makeDecimalLeafReader(); - else if (col_descriptor.type_length() <= static_cast(sizeof(Decimal256))) - return makeDecimalLeafReader(); + if (col_descriptor.type_length() > 0) + { + if (col_descriptor.type_length() <= static_cast(sizeof(Decimal128))) + return makeDecimalLeafReader(); + else if (col_descriptor.type_length() <= static_cast(sizeof(Decimal256))) + return makeDecimalLeafReader(); + } return throwUnsupported(PreformattedMessage::create( ", invalid type length: {}", col_descriptor.type_length())); From cc583185bdfe7f336af795d95cd97ce65cbef10b Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 17 May 2024 08:33:08 +0200 Subject: [PATCH 11/36] Add revision and make some formatting changes to other-functions page --- .../functions/other-functions.md | 437 ++++++++++++------ src/Functions/array/arrayUnion.cpp | 0 .../03155_function_array_clamp.sql | 11 + 3 files changed, 313 insertions(+), 135 deletions(-) create mode 100644 src/Functions/array/arrayUnion.cpp create mode 100755 tests/queries/0_stateless/03155_function_array_clamp.sql diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 11ee471d709..5b77f16027b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -6,11 +6,21 @@ sidebar_label: Other # Other Functions -## hostName() +## hostName Returns the name of the host on which this function was executed. If the function executes on a remote server (distributed processing), the remote server name is returned. If the function executes in the context of a distributed table, it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +**Syntax** + +```sql +hostName() +``` + +**Returned value** + +- Host name. [String](../data-types/string.md). + ## getMacro {#getMacro} Returns a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. @@ -27,9 +37,7 @@ getMacro(name); **Returned value** -- Value of the specified macro. - -Type: [String](../../sql-reference/data-types/string.md). +- Value of the specified macro.[String](../../sql-reference/data-types/string.md). **Example** @@ -82,9 +90,7 @@ This function is case-insensitive. **Returned value** -- String with the fully qualified domain name. - -Type: `String`. +- String with the fully qualified domain name. [String](../data-types/string.md). **Example** @@ -163,34 +169,58 @@ Result: └────────────────┴────────────────────────────┘ ``` -## visibleWidth(x) +## visibleWidth Calculates the approximate width when outputting values to the console in text format (tab-separated). -This function is used by the system to implement Pretty formats. +This function is used by the system to implement [Pretty formats](../formats.mdx). `NULL` is represented as a string corresponding to `NULL` in `Pretty` formats. +**Syntax** + +```sql +visibleWidth(x) +``` + +**Example** + +Query: + ```sql SELECT visibleWidth(NULL) ``` +Result: + ```text ┌─visibleWidth(NULL)─┐ │ 4 │ └────────────────────┘ ``` -## toTypeName(x) +## toTypeName Returns the type name of the passed argument. If `NULL` is passed, then the function returns type `Nullable(Nothing)`, which corresponds to ClickHouse's internal `NULL` representation. -## blockSize() {#blockSize} +**Syntax** + +```sql +toTypeName(x) +``` + +## blockSize {#blockSize} In ClickHouse, queries are processed in blocks (chunks). This function returns the size (row count) of the block the function is called on. +**Syntax** + +```sql +blockSize() +``` + ## byteSize Returns an estimation of uncompressed byte size of its arguments in memory. @@ -207,9 +237,7 @@ byteSize(argument [, ...]) **Returned value** -- Estimation of byte size of the arguments in memory. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- Estimation of byte size of the arguments in memory. [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** @@ -288,16 +316,28 @@ Result: └────────────────────────────┘ ``` -## materialize(x) +## materialize Turns a constant into a full column containing a single value. Full columns and constants are represented differently in memory. Functions usually execute different code for normal and constant arguments, although the result should typically be the same. This function can be used to debug this behavior. -## ignore(…) +**Syntax** + +```sql +materialize(x) +``` + +## ignore Accepts any arguments, including `NULL` and does nothing. Always returns 0. The argument is internally still evaluated. Useful e.g. for benchmarks. +**Syntax** + +```sql +ignore(…) +``` + ## sleep Used to introduce a delay or pause in the execution of a query. It is primarily used for testing and debugging purposes. @@ -392,27 +432,33 @@ The `sleepEachRow()` function is primarily used for testing and debugging purpos Like the [`sleep()` function](#sleep), it's important to use `sleepEachRow()` judiciously and only when necessary, as it can significantly impact the overall performance and responsiveness of your ClickHouse system, especially when dealing with large result sets. -## currentDatabase() +## currentDatabase Returns the name of the current database. Useful in table engine parameters of `CREATE TABLE` queries where you need to specify the database. -## currentUser() {#currentUser} +**Syntax** + +```sql +currentDatabase() +``` + +## currentUser {#currentUser} Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. +**Syntax** + ```sql -SELECT currentUser(); +currentUser() ``` Aliases: `user()`, `USER()`, `current_user()`. Aliases are case insensitive. **Returned values** -- The name of the current user. -- In distributed queries, the login of the user who initiated the query. - -Type: `String`. +- The name of the current user. [String](../data-types/string.md). +- In distributed queries, the login of the user who initiated the query. [String](../data-types/string.md). **Example** @@ -448,10 +494,8 @@ isConstant(x) **Returned values** -- `1` if `x` is constant. -- `0` if `x` is non-constant. - -Type: [UInt8](../../sql-reference/data-types/int-uint.md). +- `1` if `x` is constant. [UInt8](../../sql-reference/data-types/int-uint.md). +- `0` if `x` is non-constant. [UInt8](../../sql-reference/data-types/int-uint.md). **Examples** @@ -497,14 +541,26 @@ Result: └────────────────────┘ ``` -## isFinite(x) +## isFinite Returns 1 if the Float32 or Float64 argument not infinite and not a NaN, otherwise this function returns 0. -## isInfinite(x) +**Syntax** + +```sql +isFinite(x) +``` + +## isInfinite Returns 1 if the Float32 or Float64 argument is infinite, otherwise this function returns 0. Note that 0 is returned for a NaN. +**Syntax** + +```sql +isInfinite(x) +``` + ## ifNotFinite Checks whether a floating point value is finite. @@ -517,8 +573,8 @@ ifNotFinite(x,y) **Arguments** -- `x` — Value to check for infinity. Type: [Float\*](../../sql-reference/data-types/float.md). -- `y` — Fallback value. Type: [Float\*](../../sql-reference/data-types/float.md). +- `x` — Value to check for infinity. [Float\*](../../sql-reference/data-types/float.md). +- `y` — Fallback value. [Float\*](../../sql-reference/data-types/float.md). **Returned value** @@ -539,10 +595,16 @@ Result: You can get similar result by using the [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. -## isNaN(x) +## isNaN Returns 1 if the Float32 and Float64 argument is NaN, otherwise this function 0. +**Syntax** + +```sql +isNaN(x) +``` + ## hasColumnInTable Given the database name, the table name, and the column name as constant strings, returns 1 if the given column exists, otherwise 0. @@ -733,11 +795,19 @@ LIMIT 10 └────────────────┴─────────┘ ``` -## formatReadableDecimalSize(x) +## formatReadableDecimalSize Given a size (number of bytes), this function returns a readable, rounded size with suffix (KB, MB, etc.) as string. -Example: +**Syntax** + +```sql +formatReadableDecimalSize(x) +``` + +**Example** + +Query: ```sql SELECT @@ -745,6 +815,8 @@ SELECT formatReadableDecimalSize(filesize_bytes) AS filesize ``` +Result: + ```text ┌─filesize_bytes─┬─filesize───┐ │ 1 │ 1.00 B │ @@ -754,11 +826,20 @@ SELECT └────────────────┴────────────┘ ``` -## formatReadableSize(x) +## formatReadableSize Given a size (number of bytes), this function returns a readable, rounded size with suffix (KiB, MiB, etc.) as string. -Example: +**Syntax** + +```sql +formatReadableSize(x) +``` +Alias: `FORMAT_BYTES`. + +**Example** + +Query: ```sql SELECT @@ -766,7 +847,7 @@ SELECT formatReadableSize(filesize_bytes) AS filesize ``` -Alias: `FORMAT_BYTES`. +Result: ```text ┌─filesize_bytes─┬─filesize───┐ @@ -777,11 +858,19 @@ Alias: `FORMAT_BYTES`. └────────────────┴────────────┘ ``` -## formatReadableQuantity(x) +## formatReadableQuantity Given a number, this function returns a rounded number with suffix (thousand, million, billion, etc.) as string. -Example: +**Syntax** + +```sql +formatReadableQuantity(x) +``` + +**Example** + +Query: ```sql SELECT @@ -789,6 +878,8 @@ SELECT formatReadableQuantity(number) AS number_for_humans ``` +Result: + ```text ┌─────────number─┬─number_for_humans─┐ │ 1024 │ 1.02 thousand │ @@ -903,15 +994,27 @@ SELECT parseTimeDelta('1yr2mo') └──────────────────────────┘ ``` -## least(a, b) +## least Returns the smaller value of a and b. -## greatest(a, b) +**Syntax** + +```sql +least(a, b) +``` + +## greatest Returns the larger value of a and b. -## uptime() +**Syntax** + +```sql +greatest(a, b) +``` + +## uptime Returns the server’s uptime in seconds. If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. @@ -924,9 +1027,7 @@ uptime() **Returned value** -- Time value of seconds. - -Type: [UInt32](/docs/en/sql-reference/data-types/int-uint.md). +- Time value of seconds. [UInt32](/docs/en/sql-reference/data-types/int-uint.md). **Example** @@ -944,7 +1045,7 @@ Result: └────────┘ ``` -## version() +## version Returns the current version of ClickHouse as a string in the form of: @@ -971,7 +1072,7 @@ None. **Returned value** -Type: [String](../data-types/string) +- Current version of ClickHouse. [String](../data-types/string) **Implementation details** @@ -993,23 +1094,47 @@ SELECT version() └───────────┘ ``` -## buildId() +## buildId Returns the build ID generated by a compiler for the running ClickHouse server binary. If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. -## blockNumber() +**Syntax** + +```sql +buildId() +``` + +## blockNumber Returns the sequence number of the data block where the row is located. -## rowNumberInBlock() {#rowNumberInBlock} +**Syntax** + +```sql +blockNumber() +``` + +## rowNumberInBlock {#rowNumberInBlock} Returns the ordinal number of the row in the data block. Different data blocks are always recalculated. -## rowNumberInAllBlocks() +**Syntax** + +```sql +rowNumberInBlock() +``` + +## rowNumberInAllBlocks Returns the ordinal number of the row in the data block. This function only considers the affected data blocks. +**Syntax** + +```sql +rowNumberInAllBlocks() +``` + ## neighbor The window function that provides access to a row at a specified offset before or after the current row of a given column. @@ -1128,7 +1253,7 @@ Result: └────────────┴───────┴───────────┴────────────────┘ ``` -## runningDifference(x) {#runningDifference} +## runningDifference {#runningDifference} Calculates the difference between two consecutive row values in the data block. Returns 0 for the first row, and for subsequent rows the difference to the previous row. @@ -1143,7 +1268,15 @@ The result of the function depends on the affected data blocks and the order of The order of rows during calculation of `runningDifference()` can differ from the order of rows returned to the user. To prevent that you can create a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. -Example: +**Syntax** + +```sql +runningDifference(x) +``` + +**Example** + +Query: ```sql SELECT @@ -1162,6 +1295,8 @@ FROM ) ``` +Result: + ```text ┌─EventID─┬───────────EventTime─┬─delta─┐ │ 1106 │ 2016-11-24 00:00:04 │ 0 │ @@ -1174,6 +1309,8 @@ FROM Please note that the block size affects the result. The internal state of `runningDifference` state is reset for each new block. +Query: + ```sql SELECT number, @@ -1182,6 +1319,8 @@ FROM numbers(100000) WHERE diff != 1 ``` +Result: + ```text ┌─number─┬─diff─┐ │ 0 │ 0 │ @@ -1191,6 +1330,8 @@ WHERE diff != 1 └────────┴──────┘ ``` +Query: + ```sql set max_block_size=100000 -- default value is 65536! @@ -1201,6 +1342,8 @@ FROM numbers(100000) WHERE diff != 1 ``` +Result: + ```text ┌─number─┬─diff─┐ │ 0 │ 0 │ @@ -1238,9 +1381,7 @@ runningConcurrency(start, end) **Returned values** -- The number of concurrent events at each event start time. - -Type: [UInt32](../../sql-reference/data-types/int-uint.md) +- The number of concurrent events at each event start time. [UInt32](../../sql-reference/data-types/int-uint.md). **Example** @@ -1272,23 +1413,43 @@ Result: └────────────┴────────────────────────────────┘ ``` -## MACNumToString(num) +## MACNumToString Interprets a UInt64 number as a MAC address in big endian format. Returns the corresponding MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form) as string. -## MACStringToNum(s) +**Syntax** + +```sql +MACNumToString(num) +``` + +## MACStringToNum The inverse function of MACNumToString. If the MAC address has an invalid format, it returns 0. -## MACStringToOUI(s) +**Syntax** + +```sql +MACStringToNum(s) +``` + +## MACStringToOUI Given a MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form), returns the first three octets as a UInt64 number. If the MAC address has an invalid format, it returns 0. +**Syntax** + +```sql +MACStringToOUI(s) +``` + ## getSizeOfEnumType Returns the number of fields in [Enum](../../sql-reference/data-types/enum.md). An exception is thrown if the type is not `Enum`. +**Syntax** + ```sql getSizeOfEnumType(value) ``` @@ -1349,6 +1510,8 @@ Result: Returns the internal name of the data type that represents the value. +**Syntax** + ```sql toColumnTypeName(value) ``` @@ -1427,6 +1590,8 @@ Returns the default value for the given data type. Does not include default values for custom columns set by the user. +**Syntax** + ```sql defaultValueOfArgumentType(expression) ``` @@ -1625,29 +1790,31 @@ Result: Creates an array with a single value. -Used for the internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). +:::note +This function is used for the internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). +::: + +**Syntax** ```sql -SELECT replicate(x, arr); +replicate(x, arr) ``` -**Arguments:** +**Arguments** -- `arr` — An array. - `x` — The value to fill the result array with. +- `arr` — An array. [Array](../data-types/array.md). **Returned value** -An array of the lame length as `arr` filled with value `x`. - -Type: `Array`. +An array of the lame length as `arr` filled with value `x`. [Array](../data-types/array.md). **Example** Query: ```sql -SELECT replicate(1, ['a', 'b', 'c']) +SELECT replicate(1, ['a', 'b', 'c']); ``` Result: @@ -1658,6 +1825,36 @@ Result: └───────────────────────────────┘ ``` +## revision + +Returns the current ClickHouse [server revision](../../operations/system-tables/metrics#revision). + +**Syntax** + +```sql +revision() +``` + +**Returned value** + +- The current ClickHouse server revision. [UInt32](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT revision(); +``` + +Result: + +```response +┌─revision()─┐ +│ 54485 │ +└────────────┘ +``` + ## filesystemAvailable Returns the amount of free space in the filesystem hosting the database persistence. The returned value is always smaller than total free space ([filesystemFree](#filesystemfree)) because some space is reserved for the operating system. @@ -1670,9 +1867,7 @@ filesystemAvailable() **Returned value** -- The amount of remaining space available in bytes. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- The amount of remaining space available in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). **Example** @@ -1702,9 +1897,7 @@ filesystemFree() **Returned value** -- The amount of free space in bytes. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- The amount of free space in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). **Example** @@ -1734,9 +1927,7 @@ filesystemCapacity() **Returned value** -- Capacity of the filesystem in bytes. - -Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- Capacity of the filesystem in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). **Example** @@ -2100,7 +2291,7 @@ Result: └──────────────────────────────────────────────────┘ ``` -## catboostEvaluate(path_to_model, feature_1, feature_2, …, feature_n) +## catboostEvaluate :::note This function is not available in ClickHouse Cloud. @@ -2109,6 +2300,14 @@ This function is not available in ClickHouse Cloud. Evaluate an external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learning. Accepts a path to a catboost model and model arguments (features). Returns Float64. +**Syntax** + +```sql +catboostEvaluate(path_to_model, feature_1, feature_2, …, feature_n) +``` + +**Example** + ```sql SELECT feat1, ..., feat_n, catboostEvaluate('/path/to/model.bin', feat_1, ..., feat_n) AS prediction FROM data_table @@ -2145,10 +2344,16 @@ communicate using a HTTP interface. By default, port `9012` is used. A different See [Training and applying models](https://catboost.ai/docs/features/training.html#training) for how to train catboost models from a training data set. -## throwIf(x\[, message\[, error_code\]\]) +## throwIf Throw an exception if argument `x` is true. +**Syntax** + +```sql +throwIf(x\[, message\[, error_code\]\]) +``` + **Arguments** - `x` - the condition to check. @@ -2284,9 +2489,7 @@ countDigits(x) **Returned value** -Number of digits. - -Type: [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). +- Number of digits. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). :::note For `Decimal` values takes into account their scales: calculates result over underlying integer type which is `(value * scale)`. For example: `countDigits(42) = 2`, `countDigits(42.000) = 5`, `countDigits(0.04200) = 4`. I.e. you may check decimal overflow for `Decimal64` with `countDecimal(x) > 18`. It's a slow variant of [isDecimalOverflow](#is-decimal-overflow). @@ -2310,9 +2513,7 @@ Result: ## errorCodeToName -Returns the textual name of an error code. - -Type: [LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md). +- Returns the textual name of an error code. [LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md). **Syntax** @@ -2343,9 +2544,7 @@ tcpPort() **Returned value** -- The TCP port number. - -Type: [UInt16](../../sql-reference/data-types/int-uint.md). +- The TCP port number. [UInt16](../../sql-reference/data-types/int-uint.md). **Example** @@ -2381,9 +2580,7 @@ currentProfiles() **Returned value** -- List of the current user settings profiles. - -Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). +- List of the current user settings profiles. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## enabledProfiles @@ -2397,9 +2594,7 @@ enabledProfiles() **Returned value** -- List of the enabled settings profiles. - -Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). +- List of the enabled settings profiles. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## defaultProfiles @@ -2413,9 +2608,7 @@ defaultProfiles() **Returned value** -- List of the default settings profiles. - -Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). +- List of the default settings profiles. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## currentRoles @@ -2429,9 +2622,7 @@ currentRoles() **Returned value** -- A list of the current roles for the current user. - -Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). +- A list of the current roles for the current user. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## enabledRoles @@ -2445,9 +2636,7 @@ enabledRoles() **Returned value** -- List of the enabled roles for the current user. - -Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). +- List of the enabled roles for the current user. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## defaultRoles @@ -2461,9 +2650,7 @@ defaultRoles() **Returned value** -- List of the default roles for the current user. - -Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). +- List of the default roles for the current user. [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). ## getServerPort @@ -2492,9 +2679,7 @@ getServerPort(port_name) **Returned value** -- The number of the server port. - -Type: [UInt16](../../sql-reference/data-types/int-uint.md). +- The number of the server port. [UInt16](../../sql-reference/data-types/int-uint.md). **Example** @@ -2526,9 +2711,7 @@ queryID() **Returned value** -- The ID of the current query. - -Type: [String](../../sql-reference/data-types/string.md) +- The ID of the current query. [String](../../sql-reference/data-types/string.md). **Example** @@ -2562,9 +2745,7 @@ initialQueryID() **Returned value** -- The ID of the initial current query. - -Type: [String](../../sql-reference/data-types/string.md) +- The ID of the initial current query. [String](../../sql-reference/data-types/string.md). **Example** @@ -2597,9 +2778,7 @@ shardNum() **Returned value** -- Shard index or constant `0`. - -Type: [UInt32](../../sql-reference/data-types/int-uint.md). +- Shard index or constant `0`. [UInt32](../../sql-reference/data-types/int-uint.md). **Example** @@ -2639,9 +2818,7 @@ shardCount() **Returned value** -- Total number of shards or `0`. - -Type: [UInt32](../../sql-reference/data-types/int-uint.md). +- Total number of shards or `0`. [UInt32](../../sql-reference/data-types/int-uint.md). **See Also** @@ -2663,9 +2840,7 @@ getOSKernelVersion() **Returned value** -- The current OS kernel version. - -Type: [String](../../sql-reference/data-types/string.md). +- The current OS kernel version. [String](../../sql-reference/data-types/string.md). **Example** @@ -2699,9 +2874,7 @@ zookeeperSessionUptime() **Returned value** -- Uptime of the current ZooKeeper session in seconds. - -Type: [UInt32](../../sql-reference/data-types/int-uint.md). +- Uptime of the current ZooKeeper session in seconds. [UInt32](../../sql-reference/data-types/int-uint.md). **Example** @@ -2738,9 +2911,7 @@ All arguments must be constant. **Returned value** -- Randomly generated table structure. - -Type: [String](../../sql-reference/data-types/string.md). +- Randomly generated table structure. [String](../../sql-reference/data-types/string.md). **Examples** @@ -2807,9 +2978,7 @@ structureToCapnProtoSchema(structure) **Returned value** -- CapnProto schema - -Type: [String](../../sql-reference/data-types/string.md). +- CapnProto schema. [String](../../sql-reference/data-types/string.md). **Examples** @@ -2908,9 +3077,7 @@ structureToProtobufSchema(structure) **Returned value** -- Protobuf schema - -Type: [String](../../sql-reference/data-types/string.md). +- Protobuf schema. [String](../../sql-reference/data-types/string.md). **Examples** diff --git a/src/Functions/array/arrayUnion.cpp b/src/Functions/array/arrayUnion.cpp new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03155_function_array_clamp.sql b/tests/queries/0_stateless/03155_function_array_clamp.sql new file mode 100755 index 00000000000..4794dafda4b --- /dev/null +++ b/tests/queries/0_stateless/03155_function_array_clamp.sql @@ -0,0 +1,11 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, no-parallel, no-ordinary-database, long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# In previous versions this command took longer than ten minutes. Now it takes less than a second in release mode: + +python3 -c 'import sys; import struct; sys.stdout.buffer.write(b"".join(struct.pack(" Date: Fri, 17 May 2024 08:44:25 +0200 Subject: [PATCH 12/36] Remove files which shouldn't be on this branch --- src/Functions/array/arrayUnion.cpp | 0 .../0_stateless/03155_function_array_clamp.sql | 11 ----------- 2 files changed, 11 deletions(-) delete mode 100644 src/Functions/array/arrayUnion.cpp delete mode 100755 tests/queries/0_stateless/03155_function_array_clamp.sql diff --git a/src/Functions/array/arrayUnion.cpp b/src/Functions/array/arrayUnion.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/03155_function_array_clamp.sql b/tests/queries/0_stateless/03155_function_array_clamp.sql deleted file mode 100755 index 4794dafda4b..00000000000 --- a/tests/queries/0_stateless/03155_function_array_clamp.sql +++ /dev/null @@ -1,11 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-ordinary-database, long - -CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CUR_DIR"/../shell_config.sh - -# In previous versions this command took longer than ten minutes. Now it takes less than a second in release mode: - -python3 -c 'import sys; import struct; sys.stdout.buffer.write(b"".join(struct.pack(" Date: Fri, 17 May 2024 17:48:06 -0400 Subject: [PATCH 13/36] Use of the redefined context in process query pipline. --- .../Transforms/buildPushingToViewsChain.cpp | 5 +-- .../Transforms/buildPushingToViewsChain.h | 3 ++ ...te_view_with_sql_security_option.reference | 1 + ...84_create_view_with_sql_security_option.sh | 35 +++++++++++++++++++ 4 files changed, 42 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 5e8ecdca95e..cdcfad4442c 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -414,7 +414,8 @@ std::optional generateViewChain( out.getInputHeader(), view_id, nullptr, - std::move(runtime_stats)}); + std::move(runtime_stats), + insert_context}); if (type == QueryViewsLogElement::ViewType::MATERIALIZED) { @@ -590,7 +591,7 @@ Chain buildPushingToViewsChain( static QueryPipeline process(Block block, ViewRuntimeData & view, const ViewsData & views_data) { - const auto & context = views_data.context; + const auto & context = view.context; /// We create a table with the same name as original table and the same alias columns, /// but it will contain single block (that is INSERT-ed into main table). diff --git a/src/Processors/Transforms/buildPushingToViewsChain.h b/src/Processors/Transforms/buildPushingToViewsChain.h index 53aceeda1cc..a1feed91b60 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.h +++ b/src/Processors/Transforms/buildPushingToViewsChain.h @@ -33,6 +33,9 @@ struct ViewRuntimeData /// Info which is needed for query views log. std::unique_ptr runtime_stats; + /// An overridden context bounded to this view with the correct SQL security grants. + ContextPtr context; + void setException(std::exception_ptr e) { exception = e; diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index 9ba927fa201..931cf8ac19c 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -24,6 +24,7 @@ OK 2 OK OK +100 ===== TestGrants ===== OK OK diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index 9c9df120298..62b03b5d5ff 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -192,6 +192,41 @@ ${CLICKHOUSE_CLIENT} --user $user1 --query " ${CLICKHOUSE_CLIENT} --query "GRANT SET DEFINER ON $user2 TO $user1" +${CLICKHOUSE_CLIENT} --multiquery < Date: Sat, 18 May 2024 12:20:24 -0400 Subject: [PATCH 14/36] fix test --- .../0_stateless/02884_create_view_with_sql_security_option.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index 62b03b5d5ff..a9a306a9e27 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -197,18 +197,21 @@ CREATE TABLE $db.source ( a UInt64 ) +ENGINE = MergeTree ORDER BY a; CREATE TABLE $db.destination1 ( `a` UInt64 ) +ENGINE = MergeTree ORDER BY a; CREATE TABLE $db.destination2 ( `a` UInt64 ) +ENGINE = MergeTree ORDER BY a; CREATE MATERIALIZED VIEW $db.mv1 TO $db.destination1 From 3a79b1facc63aa9ae3a8deb986bd00cf51c14c1f Mon Sep 17 00:00:00 2001 From: pufit Date: Sat, 18 May 2024 17:15:01 -0400 Subject: [PATCH 15/36] fix test --- .../0_stateless/02884_create_view_with_sql_security_option.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index a9a306a9e27..f1da343da36 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -202,14 +202,14 @@ ORDER BY a; CREATE TABLE $db.destination1 ( - `a` UInt64 + a UInt64 ) ENGINE = MergeTree ORDER BY a; CREATE TABLE $db.destination2 ( - `a` UInt64 + a UInt64 ) ENGINE = MergeTree ORDER BY a; From bb0fcc929695701ccde2ca49298e50792636fa1c Mon Sep 17 00:00:00 2001 From: pufit Date: Sun, 19 May 2024 08:33:37 -0400 Subject: [PATCH 16/36] better tests --- ...te_view_with_sql_security_option.reference | 2 + ...84_create_view_with_sql_security_option.sh | 78 +++++++++---------- 2 files changed, 41 insertions(+), 39 deletions(-) diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference index 931cf8ac19c..0589fdeef04 100644 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.reference @@ -24,6 +24,8 @@ OK 2 OK OK +OK +100 100 ===== TestGrants ===== OK diff --git a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh index f1da343da36..f32aee44bee 100755 --- a/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh +++ b/tests/queries/0_stateless/02884_create_view_with_sql_security_option.sh @@ -159,6 +159,45 @@ ${CLICKHOUSE_CLIENT} --query "REVOKE SELECT ON $db.test_table FROM $user1" (( $(${CLICKHOUSE_CLIENT} --user $user2 --query "SELECT * FROM $db.test_mv_4" 2>&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" (( $(${CLICKHOUSE_CLIENT} --query "INSERT INTO $db.test_table VALUES ('foo'), ('bar');" 2>&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" +${CLICKHOUSE_CLIENT} --multiquery <&1 | grep -c "Not enough privileges") >= 1 )) && echo "OK" || echo "UNEXPECTED" +${CLICKHOUSE_CLIENT} --query "GRANT INSERT ON $db.source TO $user2" +${CLICKHOUSE_CLIENT} --user $user2 --query "INSERT INTO source SELECT * FROM generateRandom() LIMIT 100" + +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM destination1" +${CLICKHOUSE_CLIENT} --query "SELECT count() FROM destination2" echo "===== TestGrants =====" ${CLICKHOUSE_CLIENT} --query "GRANT CREATE ON *.* TO $user1" @@ -192,45 +231,6 @@ ${CLICKHOUSE_CLIENT} --user $user1 --query " ${CLICKHOUSE_CLIENT} --query "GRANT SET DEFINER ON $user2 TO $user1" -${CLICKHOUSE_CLIENT} --multiquery < Date: Sun, 19 May 2024 23:20:40 +0800 Subject: [PATCH 17/36] update comment of method visitNullableBySteps, try to suppress clang-18 tidy warnings Change-Id: I3119c44dc764caed0dc471f52ac5e634c75c7b50 --- .../Impl/Parquet/ParquetDataValuesReader.cpp | 14 +++++++++++--- .../Formats/Impl/Parquet/ParquetDataValuesReader.h | 13 +++++++------ 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp index 65f569ec264..b8e4db8700c 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.cpp @@ -14,6 +14,17 @@ namespace ErrorCodes extern const int PARQUET_EXCEPTION; } +RleValuesReader::RleValuesReader( + std::unique_ptr bit_reader_, Int32 bit_width_) + : bit_reader(std::move(bit_reader_)), bit_width(bit_width_) +{ + if (unlikely(bit_width >= 64)) + { + // e.g. in GetValue_ in bit_stream_utils.h, uint64 type is used to read bit values + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "unsupported bit width {}", bit_width); + } +} + void RleValuesReader::nextGroup() { // refer to: @@ -29,9 +40,6 @@ void RleValuesReader::nextGroup() { cur_group_size *= 8; cur_packed_bit_values.resize(cur_group_size); - - // try to suppress clang tidy warnings by assertion - assert(bit_width < 64); bit_reader->GetBatch(bit_width, cur_packed_bit_values.data(), cur_group_size); } else diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index 0f916ff862d..75adb55df7e 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -18,8 +18,7 @@ namespace DB class RleValuesReader { public: - RleValuesReader(std::unique_ptr bit_reader_, Int32 bit_width_) - : bit_reader(std::move(bit_reader_)), bit_width(bit_width_) {} + RleValuesReader(std::unique_ptr bit_reader_, Int32 bit_width_); /** * @brief Used when the bit_width is 0, so all elements have same value. @@ -71,12 +70,14 @@ public: * @tparam IndividualNullVisitor A callback with signature: void(size_t cursor), used to process null value * @tparam SteppedValidVisitor A callback with signature: * void(size_t cursor, const std::vector & valid_index_steps) - * for n valid elements with null value interleaved in a BitPacked group, + * valid_index_steps records the gap size between two valid elements, * i-th item in valid_index_steps describes how many elements there are * from i-th valid element (include) to (i+1)-th valid element (exclude). * - * take following BitPacked group with 2 valid elements for example: - * null valid null null valid null + * take following BitPacked group values for example, and assuming max_def_level is 1: + * [1, 0, 1, 1, 0, 1 ] + * null valid null null valid null + * the second line shows the corresponding validation state, * then the valid_index_steps has values [1, 3, 2]. * Please note that the the sum of valid_index_steps is same as elements number in this group. * @@ -117,7 +118,7 @@ private: std::vector cur_packed_bit_values; std::vector valid_index_steps; - Int32 bit_width; + const Int32 bit_width; UInt32 cur_group_size = 0; UInt32 cur_group_cursor = 0; From ad5f6f27dff104f6229819be27fba3732226603e Mon Sep 17 00:00:00 2001 From: copperybean Date: Mon, 20 May 2024 16:28:21 +0800 Subject: [PATCH 18/36] fix reader type, update comment Change-Id: Iefec91bca223eedaabe302b7891808c6d94eed9d --- .../Impl/Parquet/ParquetDataValuesReader.h | 1 + .../Impl/Parquet/ParquetRecordReader.cpp | 29 ++++++++++++++----- 2 files changed, 22 insertions(+), 8 deletions(-) diff --git a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h index 75adb55df7e..fbccb612b3c 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h +++ b/src/Processors/Formats/Impl/Parquet/ParquetDataValuesReader.h @@ -80,6 +80,7 @@ public: * the second line shows the corresponding validation state, * then the valid_index_steps has values [1, 3, 2]. * Please note that the the sum of valid_index_steps is same as elements number in this group. + * TODO the definition of valid_index_steps should be updated when supporting nested types * * @tparam RepeatedVisitor A callback with signature: void(bool is_valid, UInt32 cursor, UInt32 count) */ diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index 0b797dd66ad..69da40b47e6 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -27,6 +27,7 @@ namespace DB namespace ErrorCodes { + extern const int NOT_IMPLEMENTED; extern const int PARQUET_EXCEPTION; } @@ -225,7 +226,7 @@ std::unique_ptr ColReaderFactory::fromInt32INT(const parque { switch (int_type.bit_width()) { - case sizeof(Int32): + case 32: { if (int_type.is_signed()) return makeLeafReader(); @@ -241,7 +242,7 @@ std::unique_ptr ColReaderFactory::fromInt64INT(const parque { switch (int_type.bit_width()) { - case sizeof(Int64): + case 64: { if (int_type.is_signed()) return makeLeafReader(); @@ -312,16 +313,28 @@ ParquetRecordReader::ParquetRecordReader( { log = &Poco::Logger::get("ParquetRecordReader"); + std::unordered_map parquet_columns; + auto root = file_reader->metadata()->schema()->group_node(); + for (int i = 0; i < root->field_count(); ++i) + { + auto & node = root->field(i); + parquet_columns.emplace(node->name(), node); + } + parquet_col_indice.reserve(header.columns()); column_readers.reserve(header.columns()); for (const auto & col_with_name : header) { - auto idx = file_reader->metadata()->schema()->ColumnIndex(col_with_name.name); - if (idx < 0) - { - auto msg = PreformattedMessage::create("can not find column with name: {}", col_with_name.name); - throw Exception(std::move(msg), ErrorCodes::PARQUET_EXCEPTION); - } + auto it = parquet_columns.find(col_with_name.name); + if (it == parquet_columns.end()) + throw Exception(ErrorCodes::PARQUET_EXCEPTION, "no column with '{}' in parquet file", col_with_name.name); + + auto node = it->second; + if (!node->is_primitive()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "arrays and maps are not implemented in native parquet reader"); + + auto idx = file_reader->metadata()->schema()->ColumnIndex(*node); + chassert(idx >= 0); parquet_col_indice.push_back(idx); } if (reader_properties.pre_buffer()) From b253ca36084ec50e8d06dfe50cb3561cd915a602 Mon Sep 17 00:00:00 2001 From: copperybean Date: Mon, 20 May 2024 23:12:07 +0800 Subject: [PATCH 19/36] fix clang-tidy warnings Change-Id: Iff9f5f894e815b184ac35f61b4cac87908c612b5 --- contrib/arrow | 2 +- src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/contrib/arrow b/contrib/arrow index 8f36d71d185..5cfccd8ea65 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 8f36d71d18587f1f315ec832f424183cb6519cbb +Subproject commit 5cfccd8ea65f33d4517e7409815d761c7650b45d diff --git a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp index 69da40b47e6..a7e51f88b3c 100644 --- a/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp +++ b/src/Processors/Formats/Impl/Parquet/ParquetRecordReader.cpp @@ -314,10 +314,10 @@ ParquetRecordReader::ParquetRecordReader( log = &Poco::Logger::get("ParquetRecordReader"); std::unordered_map parquet_columns; - auto root = file_reader->metadata()->schema()->group_node(); + const auto * root = file_reader->metadata()->schema()->group_node(); for (int i = 0; i < root->field_count(); ++i) { - auto & node = root->field(i); + const auto & node = root->field(i); parquet_columns.emplace(node->name(), node); } @@ -329,7 +329,7 @@ ParquetRecordReader::ParquetRecordReader( if (it == parquet_columns.end()) throw Exception(ErrorCodes::PARQUET_EXCEPTION, "no column with '{}' in parquet file", col_with_name.name); - auto node = it->second; + const auto & node = it->second; if (!node->is_primitive()) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "arrays and maps are not implemented in native parquet reader"); From 311d6d6baa32ad0bdee1c58813c6d551aaeb53e0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 21 May 2024 09:38:36 +0000 Subject: [PATCH 20/36] Fix: 02124_insert_deduplication_token_multiple_blocks_replica --- .../02124_insert_deduplication_token_multiple_blocks_replica.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02124_insert_deduplication_token_multiple_blocks_replica.sh b/tests/queries/0_stateless/02124_insert_deduplication_token_multiple_blocks_replica.sh index 1c776263f78..0c95abb9867 100755 --- a/tests/queries/0_stateless/02124_insert_deduplication_token_multiple_blocks_replica.sh +++ b/tests/queries/0_stateless/02124_insert_deduplication_token_multiple_blocks_replica.sh @@ -9,6 +9,8 @@ INSERT_BLOCK_SETTINGS="max_insert_block_size=1&min_insert_block_size_rows=0&min_ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS block_dedup_token_replica SYNC" $CLICKHOUSE_CLIENT --query="CREATE TABLE block_dedup_token_replica (id Int32) ENGINE=ReplicatedMergeTree('/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{table}', '{replica}') ORDER BY id" +# Need to stop merges due to randomization of old_parts_lifetime setting, so all initial parts are guaranteed to exist when we check them +$CLICKHOUSE_CLIENT --query="SYSTEM STOP MERGES block_dedup_token_replica" $CLICKHOUSE_CLIENT --query="SELECT 'insert 2 blocks with dedup token, 1 row per block'" DEDUP_TOKEN='dedup1' From f1f8a35bab0e9dc46aa46faa4c3be7609b77a509 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 May 2024 15:03:16 +0000 Subject: [PATCH 21/36] Fix #64136 --- src/Interpreters/Cache/QueryCache.cpp | 26 ++++++++++++---- src/Interpreters/Cache/QueryCache.h | 3 +- src/Interpreters/executeQuery.cpp | 4 +-- .../02494_query_cache_use_database.reference | 2 ++ .../02494_query_cache_use_database.sql | 30 +++++++++++++++++++ 5 files changed, 56 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02494_query_cache_use_database.reference create mode 100644 tests/queries/0_stateless/02494_query_cache_use_database.sql diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index fafe50c170f..2fddbc0b044 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -177,6 +177,22 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast) return transformed_ast; } +IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database) +{ + ast = removeQueryCacheSettings(ast); + + /// Hash the AST, it must consider aliases (issue #56258) + constexpr bool ignore_aliases = false; + IAST::Hash ast_hash = ast->getTreeHash(ignore_aliases); + + /// Also hash the database specified via SQL `USE db`, otherwise identifiers in same query (AST) may mean different columns in different tables (issue #64136) + IAST::Hash cur_database_hash = CityHash_v1_0_2::CityHash128(current_database.data(), current_database.size()); + UInt64 low_combined = ast_hash.low64 ^ cur_database_hash.low64; + UInt64 high_combined = ast_hash.high64 ^ cur_database_hash.high64; + + return {low_combined, high_combined}; +} + String queryStringFromAST(ASTPtr ast) { WriteBufferFromOwnString buf; @@ -186,17 +202,15 @@ String queryStringFromAST(ASTPtr ast) } -/// Hashing of ASTs must consider aliases (issue #56258) -static constexpr bool ignore_aliases = false; - QueryCache::Key::Key( ASTPtr ast_, + String current_database, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, std::chrono::time_point expires_at_, bool is_compressed_) - : ast_hash(removeQueryCacheSettings(ast_)->getTreeHash(ignore_aliases)) + : ast_hash(calculateAstHash(ast_, current_database)) , header(header_) , user_id(user_id_) , current_user_roles(current_user_roles_) @@ -207,8 +221,8 @@ QueryCache::Key::Key( { } -QueryCache::Key::Key(ASTPtr ast_, std::optional user_id_, const std::vector & current_user_roles_) - : QueryCache::Key(ast_, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name +QueryCache::Key::Key(ASTPtr ast_, String current_database, std::optional user_id_, const std::vector & current_user_roles_) + : QueryCache::Key(ast_, current_database, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 814cad37f82..c234ea3d464 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -88,6 +88,7 @@ public: /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, + String current_database, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, @@ -95,7 +96,7 @@ public: bool is_compressed); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). - Key(ASTPtr ast_, std::optional user_id_, const std::vector & current_user_roles_); + Key(ASTPtr ast_, String current_database, std::optional user_id_, const std::vector & current_user_roles_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index f1f72a4ea4a..90e6406c792 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -1102,7 +1102,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getUserID(), context->getCurrentRoles()); + QueryCache::Key key(ast, context->getCurrentDatabase(), context->getUserID(), context->getCurrentRoles()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1225,7 +1225,7 @@ static std::tuple executeQueryImpl( && (!ast_contains_system_tables || system_table_handling == QueryCacheSystemTableHandling::Save)) { QueryCache::Key key( - ast, res.pipeline.getHeader(), + ast, context->getCurrentDatabase(), res.pipeline.getHeader(), context->getUserID(), context->getCurrentRoles(), settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), diff --git a/tests/queries/0_stateless/02494_query_cache_use_database.reference b/tests/queries/0_stateless/02494_query_cache_use_database.reference new file mode 100644 index 00000000000..1191247b6d9 --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_use_database.reference @@ -0,0 +1,2 @@ +1 +2 diff --git a/tests/queries/0_stateless/02494_query_cache_use_database.sql b/tests/queries/0_stateless/02494_query_cache_use_database.sql new file mode 100644 index 00000000000..df560f82ebb --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_use_database.sql @@ -0,0 +1,30 @@ +-- Tags: no-parallel, no-fasttest +-- Tag no-fasttest: Depends on OpenSSL +-- Tag no-parallel: Messes with internal cache + +-- Test for issue #64136 + +SYSTEM DROP QUERY CACHE; + +DROP DATABASE IF EXISTS db1; +DROP DATABASE IF EXISTS db2; + +CREATE DATABASE db1; +CREATE DATABASE db2; + +CREATE TABLE db1.tab(a UInt64, PRIMARY KEY a); +CREATE TABLE db2.tab(a UInt64, PRIMARY KEY a); + +INSERT INTO db1.tab values(1); +INSERT INTO db2.tab values(2); + +USE db1; +SELECT * FROM tab SETTINGS use_query_cache=1; + +USE db2; +SELECT * FROM tab SETTINGS use_query_cache=1; + +DROP DATABASE db1; +DROP DATABASE db2; + +SYSTEM DROP QUERY CACHE; From 3c4fb4f3b632ed4480e730536cb3fe976ca831d0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 May 2024 16:22:13 +0000 Subject: [PATCH 22/36] Incorporate review feedback --- src/Interpreters/Cache/QueryCache.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 2fddbc0b044..e30da7f233d 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -182,15 +182,14 @@ IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database) ast = removeQueryCacheSettings(ast); /// Hash the AST, it must consider aliases (issue #56258) - constexpr bool ignore_aliases = false; - IAST::Hash ast_hash = ast->getTreeHash(ignore_aliases); + SipHash hash; + ast->updateTreeHash(hash, /*ignore_aliases=*/ false); - /// Also hash the database specified via SQL `USE db`, otherwise identifiers in same query (AST) may mean different columns in different tables (issue #64136) - IAST::Hash cur_database_hash = CityHash_v1_0_2::CityHash128(current_database.data(), current_database.size()); - UInt64 low_combined = ast_hash.low64 ^ cur_database_hash.low64; - UInt64 high_combined = ast_hash.high64 ^ cur_database_hash.high64; + /// Also hash the database specified via SQL `USE db`, otherwise identifiers in same query (AST) may mean different columns in different + /// tables (issue #64136) + hash.update(current_database); - return {low_combined, high_combined}; + return getSipHash128AsPair(hash); } String queryStringFromAST(ASTPtr ast) From dc749325df1fa7f4d686beddd7551c30b881a0fc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 21 May 2024 17:31:13 +0000 Subject: [PATCH 23/36] Faaaaaaaaaster --- src/Interpreters/Cache/QueryCache.cpp | 4 ++-- src/Interpreters/Cache/QueryCache.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index e30da7f233d..4b10bfd3dcd 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -203,7 +203,7 @@ String queryStringFromAST(ASTPtr ast) QueryCache::Key::Key( ASTPtr ast_, - String current_database, + const String & current_database, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, @@ -220,7 +220,7 @@ QueryCache::Key::Key( { } -QueryCache::Key::Key(ASTPtr ast_, String current_database, std::optional user_id_, const std::vector & current_user_roles_) +QueryCache::Key::Key(ASTPtr ast_, const String & current_database, std::optional user_id_, const std::vector & current_user_roles_) : QueryCache::Key(ast_, current_database, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles { } diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index c234ea3d464..b5b6f477137 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -88,7 +88,7 @@ public: /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, - String current_database, + const String & current_database, Block header_, std::optional user_id_, const std::vector & current_user_roles_, bool is_shared_, @@ -96,7 +96,7 @@ public: bool is_compressed); /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). - Key(ASTPtr ast_, String current_database, std::optional user_id_, const std::vector & current_user_roles_); + Key(ASTPtr ast_, const String & current_database, std::optional user_id_, const std::vector & current_user_roles_); bool operator==(const Key & other) const; }; From 7ecfdbb3aaf4b7f4a68d6a332138dd90612e6120 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 22 May 2024 23:05:27 +0000 Subject: [PATCH 24/36] fix test_hdfsCluster_unset_skip_unavailable_shards --- tests/integration/test_storage_hdfs/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index bb72574c6e5..3c43918d8c0 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -895,7 +895,7 @@ def test_hdfsCluster_unset_skip_unavailable_shards(started_cluster): assert ( node1.query( - "select * from hdfsCluster('cluster_non_existent_port', 'hdfs://hdfs1:9000/skip_unavailable_shards', 'TSV', 'id UInt64, text String, number Float64')" + "select * from hdfsCluster('cluster_non_existent_port', 'hdfs://hdfs1:9000/unskip_unavailable_shards', 'TSV', 'id UInt64, text String, number Float64')" ) == data ) From c07c9d4c87efa2d4823526127bd52566773a2cd3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 22 May 2024 21:57:43 -0300 Subject: [PATCH 25/36] test for #45804 --- ...l_and_prewhere_condition_ver_column.reference | 2 ++ ...1_final_and_prewhere_condition_ver_column.sql | 16 ++++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.reference create mode 100644 tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql diff --git a/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.reference b/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql b/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql new file mode 100644 index 00000000000..78a58a979d1 --- /dev/null +++ b/tests/queries/0_stateless/00331_final_and_prewhere_condition_ver_column.sql @@ -0,0 +1,16 @@ +SET allow_experimental_analyzer = 1; + +-- https://github.com/ClickHouse/ClickHouse/issues/45804 + +CREATE TABLE myRMT( + key Int64, + someCol String, + ver DateTime +) ENGINE = ReplacingMergeTree(ver) +ORDER BY key as SELECT 1, 'test', '2020-01-01'; + +SELECT count(ver) FROM myRMT FINAL PREWHERE ver > '2000-01-01'; + +SELECT count() FROM myRMT FINAL PREWHERE ver > '2000-01-01'; + +DROP TABLE myRMT; From 9d63095db9445f4963da914ddbc819b0a57bc7e2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 16 Apr 2024 12:55:50 +0000 Subject: [PATCH 26/36] Revert "Revert "Speed up `splitByRegexp`"" This reverts commit 08e5c2ba4d9620551b0de5791876d35888d2c81a. --- src/Functions/splitByRegexp.cpp | 66 ++++++++++++++++++- tests/performance/function_tokens.xml | 2 + .../01866_split_by_regexp.reference | 12 ++++ .../0_stateless/01866_split_by_regexp.sql | 17 +++++ 4 files changed, 94 insertions(+), 3 deletions(-) diff --git a/src/Functions/splitByRegexp.cpp b/src/Functions/splitByRegexp.cpp index 32afb813a04..e28fe9c38bb 100644 --- a/src/Functions/splitByRegexp.cpp +++ b/src/Functions/splitByRegexp.cpp @@ -1,9 +1,11 @@ #include +#include +#include #include #include -#include #include #include +#include #include @@ -102,7 +104,7 @@ public: return false; } - pos += 1; + ++pos; token_end = pos; ++splits; } @@ -148,11 +150,69 @@ public: using FunctionSplitByRegexp = FunctionTokens; +/// Fallback splitByRegexp to splitByChar when its 1st argument is a trivial char for better performance +class SplitByRegexpOverloadResolver : public IFunctionOverloadResolver +{ +public: + static constexpr auto name = "splitByRegexp"; + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } + + explicit SplitByRegexpOverloadResolver(ContextPtr context_) + : context(context_) + , split_by_regexp(FunctionSplitByRegexp::create(context)) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return SplitByRegexpImpl::getNumberOfArguments(); } + bool isVariadic() const override { return SplitByRegexpImpl::isVariadic(); } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + if (patternIsTrivialChar(arguments)) + return FunctionFactory::instance().getImpl("splitByChar", context)->build(arguments); + else + return std::make_unique( + split_by_regexp, collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + return split_by_regexp->getReturnTypeImpl(arguments); + } + +private: + bool patternIsTrivialChar(const ColumnsWithTypeAndName & arguments) const + { + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + if (!col) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of first argument of function {}. " + "Must be constant string.", + arguments[0].column->getName(), + getName()); + + String pattern = col->getValue(); + if (pattern.size() == 1) + { + OptimizedRegularExpression re = Regexps::createRegexp(pattern); + + std::string required_substring; + bool is_trivial; + bool required_substring_is_prefix; + re.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + return is_trivial && required_substring == pattern; + } + return false; + } + + ContextPtr context; + FunctionPtr split_by_regexp; +}; } REGISTER_FUNCTION(SplitByRegexp) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/tests/performance/function_tokens.xml b/tests/performance/function_tokens.xml index 63b72f83df3..1ff56323d62 100644 --- a/tests/performance/function_tokens.xml +++ b/tests/performance/function_tokens.xml @@ -1,3 +1,5 @@ with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByChar(' ', materialize(s)) as w from numbers(1000000) + with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp(' ', materialize(s)) as w from numbers(1000000) + with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp('\s+', materialize(s)) as w from numbers(100000) diff --git a/tests/queries/0_stateless/01866_split_by_regexp.reference b/tests/queries/0_stateless/01866_split_by_regexp.reference index a3ae2f35a5f..62939940545 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.reference +++ b/tests/queries/0_stateless/01866_split_by_regexp.reference @@ -5,3 +5,15 @@ ['gbye','bug'] [''] [] +Test fallback of splitByRegexp to splitByChar if regexp is trivial +['a','b','c'] +['a','b','c'] +['','','','','',''] +['a^b^c'] +['a$b$c'] +['a)b)c'] +['a','b','c'] +['a','b','c'] +['a','b','c'] +['a|b|c'] +['a\\b\\c'] diff --git a/tests/queries/0_stateless/01866_split_by_regexp.sql b/tests/queries/0_stateless/01866_split_by_regexp.sql index e472fb68d94..570bd1ba5c0 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.sql +++ b/tests/queries/0_stateless/01866_split_by_regexp.sql @@ -3,3 +3,20 @@ select splitByRegexp('', 'abcde'); select splitByRegexp('<[^<>]*>', x) from (select arrayJoin(['

hello

world

', 'gbyebug']) x); select splitByRegexp('ab', ''); select splitByRegexp('', ''); + +SELECT 'Test fallback of splitByRegexp to splitByChar if regexp is trivial'; +select splitByRegexp(' ', 'a b c'); +select splitByRegexp('-', 'a-b-c'); +select splitByRegexp('.', 'a.b.c'); +select splitByRegexp('^', 'a^b^c'); +select splitByRegexp('$', 'a$b$c'); +select splitByRegexp('+', 'a+b+c'); -- { serverError CANNOT_COMPILE_REGEXP } +select splitByRegexp('?', 'a?b?c'); -- { serverError CANNOT_COMPILE_REGEXP } +select splitByRegexp('(', 'a(b(c'); -- { serverError CANNOT_COMPILE_REGEXP } +select splitByRegexp(')', 'a)b)c'); +select splitByRegexp('[', 'a[b[c'); -- { serverError CANNOT_COMPILE_REGEXP } +select splitByRegexp(']', 'a]b]c'); +select splitByRegexp('{', 'a{b{c'); +select splitByRegexp('}', 'a}b}c'); +select splitByRegexp('|', 'a|b|c'); +select splitByRegexp('\\', 'a\\b\\c'); From 299f0886bfda27e375be3edf9042af513cbf99c8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 23 May 2024 13:48:17 +0200 Subject: [PATCH 27/36] Followup for #63691 --- src/Processors/Transforms/SquashingChunksTransform.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/SquashingChunksTransform.cpp b/src/Processors/Transforms/SquashingChunksTransform.cpp index 267490dc89e..ed67dd508f3 100644 --- a/src/Processors/Transforms/SquashingChunksTransform.cpp +++ b/src/Processors/Transforms/SquashingChunksTransform.cpp @@ -71,7 +71,9 @@ Chunk SimpleSquashingChunksTransform::generate() if (squashed_chunk.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't generate chunk in SimpleSquashingChunksTransform"); - return std::move(squashed_chunk); + Chunk result_chunk; + result_chunk.swap(squashed_chunk); + return result_chunk; } bool SimpleSquashingChunksTransform::canGenerate() @@ -83,7 +85,10 @@ Chunk SimpleSquashingChunksTransform::getRemaining() { Block current_block = squashing.add({}); squashed_chunk.setColumns(current_block.getColumns(), current_block.rows()); - return std::move(squashed_chunk); + + Chunk result_chunk; + result_chunk.swap(squashed_chunk); + return result_chunk; } } From ce26c4f65746ec3058f1639f83b675feef4fda1c Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 23 May 2024 13:54:45 +0200 Subject: [PATCH 28/36] =?UTF-8?q?Review=20changes=20and=20replace=20?= =?UTF-8?q?=E2=80=A6=20with=20...?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../template-setting.md | 2 +- docs/changelogs/v20.7.1.4310-prestable.md | 2 +- docs/changelogs/v21.12.1.9017-prestable.md | 2 +- docs/changelogs/v21.3.3.14-lts.md | 2 +- docs/changelogs/v21.4.1.6422-prestable.md | 2 +- docs/changelogs/v21.4.2.10-prestable.md | 2 +- docs/changelogs/v22.6.1.1985-stable.md | 4 +- docs/changelogs/v22.7.1.2484-stable.md | 2 +- docs/changelogs/v22.8.13.20-lts.md | 2 +- docs/changelogs/v23.11.1.2711-stable.md | 2 +- docs/changelogs/v23.12.1.1368-stable.md | 2 +- docs/changelogs/v23.3.1.2823-lts.md | 2 +- docs/changelogs/v23.5.1.3174-stable.md | 2 +- docs/changelogs/v23.8.1.2992-lts.md | 2 +- docs/changelogs/v24.1.3.31-stable.md | 2 +- docs/changelogs/v24.2.1.2248-stable.md | 2 +- docs/changelogs/v24.3.1.2672-lts.md | 2 +- docs/en/development/style.md | 6 +- .../table-engines/integrations/hdfs.md | 2 +- .../engines/table-engines/integrations/s3.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/mergetree.md | 4 +- .../table-engines/special/external-data.md | 2 +- .../operations/settings/query-complexity.md | 4 +- docs/en/operations/settings/settings.md | 2 +- .../parametric-functions.md | 4 +- .../reference/quantiles.md | 2 +- .../data-types/aggregatefunction.md | 4 +- .../sql-reference/data-types/fixedstring.md | 4 +- .../nested-data-structures/index.md | 2 +- .../data-types/simpleaggregatefunction.md | 2 +- .../functions/arithmetic-functions.md | 54 ++++++++++++ .../functions/array-functions.md | 84 +++++++++---------- .../functions/date-time-functions.md | 2 +- .../sql-reference/functions/json-functions.md | 24 +++--- .../functions/other-functions.md | 62 +------------- .../functions/string-replace-functions.md | 2 +- .../functions/string-search-functions.md | 12 +-- .../functions/tuple-functions.md | 6 +- .../functions/tuple-map-functions.md | 4 +- .../sql-reference/functions/url-functions.md | 2 +- .../sql-reference/statements/alter/comment.md | 2 +- .../sql-reference/statements/alter/delete.md | 2 +- .../sql-reference/statements/alter/index.md | 2 +- .../sql-reference/statements/alter/update.md | 2 +- .../en/sql-reference/statements/alter/view.md | 6 +- .../sql-reference/statements/create/view.md | 2 +- .../sql-reference/statements/insert-into.md | 2 +- .../sql-reference/statements/select/limit.md | 4 +- .../statements/select/order-by.md | 2 +- docs/en/sql-reference/table-functions/file.md | 2 +- docs/en/sql-reference/table-functions/gcs.md | 2 +- docs/en/sql-reference/table-functions/hdfs.md | 2 +- docs/en/sql-reference/table-functions/s3.md | 2 +- docs/ru/development/style.md | 8 +- .../table-engines/integrations/hdfs.md | 2 +- .../engines/table-engines/integrations/s3.md | 2 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/mergetree.md | 4 +- .../table-engines/special/external-data.md | 2 +- docs/ru/faq/general/olap.md | 6 +- .../example-datasets/nyc-taxi.md | 2 +- docs/ru/index.md | 12 +-- .../operations/settings/query-complexity.md | 4 +- docs/ru/operations/settings/settings.md | 2 +- .../parametric-functions.md | 4 +- .../reference/quantiles.md | 2 +- .../data-types/aggregatefunction.md | 4 +- .../sql-reference/data-types/fixedstring.md | 4 +- .../nested-data-structures/nested.md | 2 +- docs/ru/sql-reference/data-types/tuple.md | 2 +- .../functions/array-functions.md | 40 ++++----- .../functions/date-time-functions.md | 2 +- .../sql-reference/functions/json-functions.md | 24 +++--- .../functions/other-functions.md | 2 +- .../functions/string-functions.md | 2 +- .../functions/string-search-functions.md | 18 ++-- .../functions/tuple-functions.md | 6 +- .../sql-reference/functions/url-functions.md | 2 +- .../sql-reference/statements/alter/comment.md | 2 +- .../sql-reference/statements/alter/delete.md | 2 +- .../sql-reference/statements/alter/index.md | 2 +- .../sql-reference/statements/alter/update.md | 2 +- .../ru/sql-reference/statements/alter/view.md | 4 +- .../sql-reference/statements/create/view.md | 2 +- .../sql-reference/statements/insert-into.md | 2 +- docs/ru/sql-reference/table-functions/file.md | 2 +- docs/ru/sql-reference/table-functions/s3.md | 2 +- docs/zh/changelog/index.md | 4 +- docs/zh/development/style.md | 8 +- .../table-engines/integrations/hdfs.md | 2 +- .../engines/table-engines/integrations/s3.md | 4 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/mergetree.md | 4 +- .../table-engines/special/external-data.md | 2 +- docs/zh/faq/general/olap.md | 6 +- .../example-datasets/nyc-taxi.md | 2 +- .../example-datasets/uk-price-paid.mdx | 2 +- .../sparse-primary-indexes.md | 2 +- docs/zh/index.md | 12 +-- .../operations/settings/query-complexity.md | 4 +- docs/zh/operations/settings/settings.md | 2 +- .../operations/system-tables/dictionaries.md | 2 +- .../parametric-functions.md | 4 +- .../reference/quantiles.md | 2 +- .../data-types/aggregatefunction.md | 2 +- .../sql-reference/data-types/domains/index.md | 4 +- .../sql-reference/data-types/fixedstring.md | 4 +- .../nested-data-structures/nested.md | 2 +- .../data-types/simpleaggregatefunction.md | 2 +- docs/zh/sql-reference/data-types/tuple.md | 2 +- .../functions/array-functions.md | 40 ++++----- .../functions/date-time-functions.md | 2 +- .../functions/higher-order-functions.md | 22 ++--- .../sql-reference/functions/in-functions.md | 4 +- .../sql-reference/functions/json-functions.md | 24 +++--- .../functions/other-functions.md | 2 +- .../functions/string-functions.md | 6 +- .../functions/string-search-functions.md | 18 ++-- .../sql-reference/functions/url-functions.md | 2 +- .../sql-reference/statements/alter/delete.md | 2 +- .../sql-reference/statements/alter/index.md | 2 +- .../sql-reference/statements/alter/update.md | 2 +- .../zh/sql-reference/statements/alter/view.md | 4 +- .../sql-reference/statements/create/view.md | 2 +- .../sql-reference/statements/insert-into.md | 2 +- .../sql-reference/statements/select/limit.md | 4 +- .../statements/select/order-by.md | 2 +- docs/zh/sql-reference/table-functions/file.md | 2 +- docs/zh/sql-reference/table-functions/hdfs.md | 2 +- docs/zh/sql-reference/table-functions/s3.md | 2 +- 131 files changed, 384 insertions(+), 384 deletions(-) diff --git a/docs/_description_templates/template-setting.md b/docs/_description_templates/template-setting.md index fc912aba3e1..f4525d872df 100644 --- a/docs/_description_templates/template-setting.md +++ b/docs/_description_templates/template-setting.md @@ -2,7 +2,7 @@ Description. -For the switch setting, use the typical phrase: “Enables or disables something …”. +For the switch setting, use the typical phrase: “Enables or disables something ...”. Possible values: diff --git a/docs/changelogs/v20.7.1.4310-prestable.md b/docs/changelogs/v20.7.1.4310-prestable.md index f47c7334228..aa1d993b263 100644 --- a/docs/changelogs/v20.7.1.4310-prestable.md +++ b/docs/changelogs/v20.7.1.4310-prestable.md @@ -166,4 +166,4 @@ * NO CL ENTRY: 'Revert "Abort on std::out_of_range in debug builds"'. [#12752](https://github.com/ClickHouse/ClickHouse/pull/12752) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * NO CL ENTRY: 'Bump protobuf from 3.12.2 to 3.12.4 in /docs/tools'. [#13102](https://github.com/ClickHouse/ClickHouse/pull/13102) ([dependabot-preview[bot]](https://github.com/apps/dependabot-preview)). * NO CL ENTRY: 'Merge [#12574](https://github.com/ClickHouse/ClickHouse/issues/12574)'. [#13158](https://github.com/ClickHouse/ClickHouse/pull/13158) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "Add QueryTimeMicroseconds, SelectQueryTimeMicroseconds and InsertQuer…"'. [#13303](https://github.com/ClickHouse/ClickHouse/pull/13303) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Add QueryTimeMicroseconds, SelectQueryTimeMicroseconds and InsertQuer..."'. [#13303](https://github.com/ClickHouse/ClickHouse/pull/13303) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v21.12.1.9017-prestable.md b/docs/changelogs/v21.12.1.9017-prestable.md index 88b8260e312..bd84873e67a 100644 --- a/docs/changelogs/v21.12.1.9017-prestable.md +++ b/docs/changelogs/v21.12.1.9017-prestable.md @@ -421,5 +421,5 @@ sidebar_label: 2022 * Fix possible crash in DataTypeAggregateFunction [#32287](https://github.com/ClickHouse/ClickHouse/pull/32287) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Update backport.py [#32323](https://github.com/ClickHouse/ClickHouse/pull/32323) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix graphite-bench build [#32351](https://github.com/ClickHouse/ClickHouse/pull/32351) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Revert "graphite: split tagged/plain rollup rules (for merges perfoma… [#32376](https://github.com/ClickHouse/ClickHouse/pull/32376) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Revert "graphite: split tagged/plain rollup rules (for merges perfoma... [#32376](https://github.com/ClickHouse/ClickHouse/pull/32376) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Another attempt to fix unit test Executor::RemoveTasksStress [#32390](https://github.com/ClickHouse/ClickHouse/pull/32390) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). diff --git a/docs/changelogs/v21.3.3.14-lts.md b/docs/changelogs/v21.3.3.14-lts.md index 57bde602f21..91d99deaa6b 100644 --- a/docs/changelogs/v21.3.3.14-lts.md +++ b/docs/changelogs/v21.3.3.14-lts.md @@ -18,4 +18,4 @@ sidebar_label: 2022 #### NOT FOR CHANGELOG / INSIGNIFICANT -* fix incorrect number of rows for Chunks with no columns in PartialSor… [#21761](https://github.com/ClickHouse/ClickHouse/pull/21761) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* fix incorrect number of rows for Chunks with no columns in PartialSor... [#21761](https://github.com/ClickHouse/ClickHouse/pull/21761) ([Alexander Kuzmenkov](https://github.com/akuzm)). diff --git a/docs/changelogs/v21.4.1.6422-prestable.md b/docs/changelogs/v21.4.1.6422-prestable.md index 2eadb0d4754..66937c3be15 100644 --- a/docs/changelogs/v21.4.1.6422-prestable.md +++ b/docs/changelogs/v21.4.1.6422-prestable.md @@ -223,7 +223,7 @@ sidebar_label: 2022 * Do not overlap zookeeper path for ReplicatedMergeTree in stateless *.sh tests [#21724](https://github.com/ClickHouse/ClickHouse/pull/21724) ([Azat Khuzhin](https://github.com/azat)). * make the fuzzer use sources from the CI [#21754](https://github.com/ClickHouse/ClickHouse/pull/21754) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Add one more variant to memcpy benchmark [#21759](https://github.com/ClickHouse/ClickHouse/pull/21759) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* fix incorrect number of rows for Chunks with no columns in PartialSor… [#21761](https://github.com/ClickHouse/ClickHouse/pull/21761) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* fix incorrect number of rows for Chunks with no columns in PartialSor... [#21761](https://github.com/ClickHouse/ClickHouse/pull/21761) ([Alexander Kuzmenkov](https://github.com/akuzm)). * docs(fix): typo [#21775](https://github.com/ClickHouse/ClickHouse/pull/21775) ([Ali Demirci](https://github.com/depyronick)). * DDLWorker.cpp: fixed exceeded amount of tries typo [#21807](https://github.com/ClickHouse/ClickHouse/pull/21807) ([Eldar Nasyrov](https://github.com/3ldar-nasyrov)). * fix integration MaterializeMySQL test [#21819](https://github.com/ClickHouse/ClickHouse/pull/21819) ([TCeason](https://github.com/TCeason)). diff --git a/docs/changelogs/v21.4.2.10-prestable.md b/docs/changelogs/v21.4.2.10-prestable.md index 3db17ddfcf3..b9bdbd80c0c 100644 --- a/docs/changelogs/v21.4.2.10-prestable.md +++ b/docs/changelogs/v21.4.2.10-prestable.md @@ -226,7 +226,7 @@ sidebar_label: 2022 * Do not overlap zookeeper path for ReplicatedMergeTree in stateless *.sh tests [#21724](https://github.com/ClickHouse/ClickHouse/pull/21724) ([Azat Khuzhin](https://github.com/azat)). * make the fuzzer use sources from the CI [#21754](https://github.com/ClickHouse/ClickHouse/pull/21754) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Add one more variant to memcpy benchmark [#21759](https://github.com/ClickHouse/ClickHouse/pull/21759) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* fix incorrect number of rows for Chunks with no columns in PartialSor… [#21761](https://github.com/ClickHouse/ClickHouse/pull/21761) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* fix incorrect number of rows for Chunks with no columns in PartialSor... [#21761](https://github.com/ClickHouse/ClickHouse/pull/21761) ([Alexander Kuzmenkov](https://github.com/akuzm)). * docs(fix): typo [#21775](https://github.com/ClickHouse/ClickHouse/pull/21775) ([Ali Demirci](https://github.com/depyronick)). * DDLWorker.cpp: fixed exceeded amount of tries typo [#21807](https://github.com/ClickHouse/ClickHouse/pull/21807) ([Eldar Nasyrov](https://github.com/3ldar-nasyrov)). * fix integration MaterializeMySQL test [#21819](https://github.com/ClickHouse/ClickHouse/pull/21819) ([TCeason](https://github.com/TCeason)). diff --git a/docs/changelogs/v22.6.1.1985-stable.md b/docs/changelogs/v22.6.1.1985-stable.md index c915d24fe00..7bd7038377a 100644 --- a/docs/changelogs/v22.6.1.1985-stable.md +++ b/docs/changelogs/v22.6.1.1985-stable.md @@ -160,7 +160,7 @@ sidebar_label: 2022 * fix toString error on DatatypeDate32. [#37775](https://github.com/ClickHouse/ClickHouse/pull/37775) ([LiuNeng](https://github.com/liuneng1994)). * The clickhouse-keeper setting `dead_session_check_period_ms` was transformed into microseconds (multiplied by 1000), which lead to dead sessions only being cleaned up after several minutes (instead of 500ms). [#37824](https://github.com/ClickHouse/ClickHouse/pull/37824) ([Michael Lex](https://github.com/mlex)). * Fix possible "No more packets are available" for distributed queries (in case of `async_socket_for_remote`/`use_hedged_requests` is disabled). [#37826](https://github.com/ClickHouse/ClickHouse/pull/37826) ([Azat Khuzhin](https://github.com/azat)). -* Do not drop the inner target table when executing `ALTER TABLE … MODIFY QUERY` in WindowView. [#37879](https://github.com/ClickHouse/ClickHouse/pull/37879) ([vxider](https://github.com/Vxider)). +* Do not drop the inner target table when executing `ALTER TABLE ... MODIFY QUERY` in WindowView. [#37879](https://github.com/ClickHouse/ClickHouse/pull/37879) ([vxider](https://github.com/Vxider)). * Fix directory ownership of coordination dir in clickhouse-keeper Docker image. Fixes [#37914](https://github.com/ClickHouse/ClickHouse/issues/37914). [#37915](https://github.com/ClickHouse/ClickHouse/pull/37915) ([James Maidment](https://github.com/jamesmaidment)). * Dictionaries fix custom query with update field and `{condition}`. Closes [#33746](https://github.com/ClickHouse/ClickHouse/issues/33746). [#37947](https://github.com/ClickHouse/ClickHouse/pull/37947) ([Maksim Kita](https://github.com/kitaisreal)). * Fix possible incorrect result of `SELECT ... WITH FILL` in the case when `ORDER BY` should be applied after `WITH FILL` result (e.g. for outer query). Incorrect result was caused by optimization for `ORDER BY` expressions ([#35623](https://github.com/ClickHouse/ClickHouse/issues/35623)). Closes [#37904](https://github.com/ClickHouse/ClickHouse/issues/37904). [#37959](https://github.com/ClickHouse/ClickHouse/pull/37959) ([Yakov Olkhovskiy](https://github.com/yakov-olkhovskiy)). @@ -180,7 +180,7 @@ sidebar_label: 2022 #### NO CL ENTRY * NO CL ENTRY: 'Revert "Fix mutations in tables with columns of type `Object`"'. [#37355](https://github.com/ClickHouse/ClickHouse/pull/37355) ([Alexander Tokmakov](https://github.com/tavplubix)). -* NO CL ENTRY: 'Revert "Remove height restrictions from the query div in play web tool, and m…"'. [#37501](https://github.com/ClickHouse/ClickHouse/pull/37501) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Remove height restrictions from the query div in play web tool, and m..."'. [#37501](https://github.com/ClickHouse/ClickHouse/pull/37501) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * NO CL ENTRY: 'Revert "Add support for preprocessing ZooKeeper operations in `clickhouse-keeper`"'. [#37534](https://github.com/ClickHouse/ClickHouse/pull/37534) ([Antonio Andelic](https://github.com/antonio2368)). * NO CL ENTRY: 'Revert "(only with zero-copy replication, non-production experimental feature not recommended to use) fix possible deadlock during fetching part"'. [#37545](https://github.com/ClickHouse/ClickHouse/pull/37545) ([Alexander Tokmakov](https://github.com/tavplubix)). * NO CL ENTRY: 'Revert "RFC: Fix converting types for UNION queries (may produce LOGICAL_ERROR)"'. [#37582](https://github.com/ClickHouse/ClickHouse/pull/37582) ([Dmitry Novik](https://github.com/novikd)). diff --git a/docs/changelogs/v22.7.1.2484-stable.md b/docs/changelogs/v22.7.1.2484-stable.md index 7464b0449ee..c4a76c66e0c 100644 --- a/docs/changelogs/v22.7.1.2484-stable.md +++ b/docs/changelogs/v22.7.1.2484-stable.md @@ -410,7 +410,7 @@ sidebar_label: 2022 * Add test for [#39132](https://github.com/ClickHouse/ClickHouse/issues/39132) [#39173](https://github.com/ClickHouse/ClickHouse/pull/39173) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Suppression for BC check (`Cannot parse string 'Hello' as UInt64`) [#39176](https://github.com/ClickHouse/ClickHouse/pull/39176) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix 01961_roaring_memory_tracking test [#39187](https://github.com/ClickHouse/ClickHouse/pull/39187) ([Dmitry Novik](https://github.com/novikd)). -* Cleanup: done during [#38719](https://github.com/ClickHouse/ClickHouse/issues/38719) (SortingStep: deduce way to sort based on … [#39191](https://github.com/ClickHouse/ClickHouse/pull/39191) ([Igor Nikonov](https://github.com/devcrafter)). +* Cleanup: done during [#38719](https://github.com/ClickHouse/ClickHouse/issues/38719) (SortingStep: deduce way to sort based on ... [#39191](https://github.com/ClickHouse/ClickHouse/pull/39191) ([Igor Nikonov](https://github.com/devcrafter)). * Fix exception in AsynchronousMetrics for s390x [#39193](https://github.com/ClickHouse/ClickHouse/pull/39193) ([Harry Lee](https://github.com/HarryLeeIBM)). * Optimize accesses to system.stack_trace (filter by name before sending signal) [#39212](https://github.com/ClickHouse/ClickHouse/pull/39212) ([Azat Khuzhin](https://github.com/azat)). * Enable warning "-Wdeprecated-dynamic-exception-spec" [#39213](https://github.com/ClickHouse/ClickHouse/pull/39213) ([Robert Schulze](https://github.com/rschu1ze)). diff --git a/docs/changelogs/v22.8.13.20-lts.md b/docs/changelogs/v22.8.13.20-lts.md index 0734f40bf3e..ad44fbfc5d6 100644 --- a/docs/changelogs/v22.8.13.20-lts.md +++ b/docs/changelogs/v22.8.13.20-lts.md @@ -20,4 +20,4 @@ sidebar_label: 2023 * Fix wrong approved_at, simplify conditions [#45302](https://github.com/ClickHouse/ClickHouse/pull/45302) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Get rid of artifactory in favor of r2 + ch-repos-manager [#45421](https://github.com/ClickHouse/ClickHouse/pull/45421) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Trim refs/tags/ from GITHUB_TAG in release workflow [#45636](https://github.com/ClickHouse/ClickHouse/pull/45636) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Merge pull request [#38262](https://github.com/ClickHouse/ClickHouse/issues/38262) from PolyProgrammist/fix-ordinary-system-un… [#45650](https://github.com/ClickHouse/ClickHouse/pull/45650) ([alesapin](https://github.com/alesapin)). +* Merge pull request [#38262](https://github.com/ClickHouse/ClickHouse/issues/38262) from PolyProgrammist/fix-ordinary-system-un... [#45650](https://github.com/ClickHouse/ClickHouse/pull/45650) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v23.11.1.2711-stable.md b/docs/changelogs/v23.11.1.2711-stable.md index e32dee41dc7..0bdee08f5c9 100644 --- a/docs/changelogs/v23.11.1.2711-stable.md +++ b/docs/changelogs/v23.11.1.2711-stable.md @@ -217,7 +217,7 @@ sidebar_label: 2023 * S3Queue minor fix [#56999](https://github.com/ClickHouse/ClickHouse/pull/56999) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix file path validation for DatabaseFileSystem [#57029](https://github.com/ClickHouse/ClickHouse/pull/57029) ([San](https://github.com/santrancisco)). * Fix `fuzzBits` with `ARRAY JOIN` [#57033](https://github.com/ClickHouse/ClickHouse/pull/57033) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix Nullptr dereference in partial merge join with joined_subquery_re… [#57048](https://github.com/ClickHouse/ClickHouse/pull/57048) ([vdimir](https://github.com/vdimir)). +* Fix Nullptr dereference in partial merge join with joined_subquery_re... [#57048](https://github.com/ClickHouse/ClickHouse/pull/57048) ([vdimir](https://github.com/vdimir)). * Fix race condition in RemoteSource [#57052](https://github.com/ClickHouse/ClickHouse/pull/57052) ([Raúl Marín](https://github.com/Algunenano)). * Implement `bitHammingDistance` for big integers [#57073](https://github.com/ClickHouse/ClickHouse/pull/57073) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * S3-style links bug fix [#57075](https://github.com/ClickHouse/ClickHouse/pull/57075) ([Yarik Briukhovetskyi](https://github.com/yariks5s)). diff --git a/docs/changelogs/v23.12.1.1368-stable.md b/docs/changelogs/v23.12.1.1368-stable.md index 1a322ae9c0f..cb8ba57100e 100644 --- a/docs/changelogs/v23.12.1.1368-stable.md +++ b/docs/changelogs/v23.12.1.1368-stable.md @@ -272,7 +272,7 @@ sidebar_label: 2023 * Bump Azure to v1.6.0 [#58052](https://github.com/ClickHouse/ClickHouse/pull/58052) ([Robert Schulze](https://github.com/rschu1ze)). * Correct values for randomization [#58058](https://github.com/ClickHouse/ClickHouse/pull/58058) ([Anton Popov](https://github.com/CurtizJ)). * Non post request should be readonly [#58060](https://github.com/ClickHouse/ClickHouse/pull/58060) ([San](https://github.com/santrancisco)). -* Revert "Merge pull request [#55710](https://github.com/ClickHouse/ClickHouse/issues/55710) from guoxiaolongzte/clickhouse-test… [#58066](https://github.com/ClickHouse/ClickHouse/pull/58066) ([Raúl Marín](https://github.com/Algunenano)). +* Revert "Merge pull request [#55710](https://github.com/ClickHouse/ClickHouse/issues/55710) from guoxiaolongzte/clickhouse-test... [#58066](https://github.com/ClickHouse/ClickHouse/pull/58066) ([Raúl Marín](https://github.com/Algunenano)). * fix typo in the test 02479 [#58072](https://github.com/ClickHouse/ClickHouse/pull/58072) ([Sema Checherinda](https://github.com/CheSema)). * Bump Azure to 1.7.2 [#58075](https://github.com/ClickHouse/ClickHouse/pull/58075) ([Robert Schulze](https://github.com/rschu1ze)). * Fix flaky test `02567_and_consistency` [#58076](https://github.com/ClickHouse/ClickHouse/pull/58076) ([Anton Popov](https://github.com/CurtizJ)). diff --git a/docs/changelogs/v23.3.1.2823-lts.md b/docs/changelogs/v23.3.1.2823-lts.md index 0c9be3601da..f81aba53ebe 100644 --- a/docs/changelogs/v23.3.1.2823-lts.md +++ b/docs/changelogs/v23.3.1.2823-lts.md @@ -520,7 +520,7 @@ sidebar_label: 2023 * Improve script for updating clickhouse-docs [#48135](https://github.com/ClickHouse/ClickHouse/pull/48135) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix stdlib compatibility issues [#48150](https://github.com/ClickHouse/ClickHouse/pull/48150) ([DimasKovas](https://github.com/DimasKovas)). * Make test test_disallow_concurrency less flaky [#48152](https://github.com/ClickHouse/ClickHouse/pull/48152) ([Vitaly Baranov](https://github.com/vitlibar)). -* Remove unused mockSystemDatabase from gtest_transform_query_for_exter… [#48162](https://github.com/ClickHouse/ClickHouse/pull/48162) ([Vladimir C](https://github.com/vdimir)). +* Remove unused mockSystemDatabase from gtest_transform_query_for_exter... [#48162](https://github.com/ClickHouse/ClickHouse/pull/48162) ([Vladimir C](https://github.com/vdimir)). * Update environmental-sensors.md [#48166](https://github.com/ClickHouse/ClickHouse/pull/48166) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Correctly handle NULL constants in logical optimizer for new analyzer [#48168](https://github.com/ClickHouse/ClickHouse/pull/48168) ([Antonio Andelic](https://github.com/antonio2368)). * Try making KeeperMap test more stable [#48170](https://github.com/ClickHouse/ClickHouse/pull/48170) ([Antonio Andelic](https://github.com/antonio2368)). diff --git a/docs/changelogs/v23.5.1.3174-stable.md b/docs/changelogs/v23.5.1.3174-stable.md index 2212eb6e893..4bdd4139afc 100644 --- a/docs/changelogs/v23.5.1.3174-stable.md +++ b/docs/changelogs/v23.5.1.3174-stable.md @@ -474,7 +474,7 @@ sidebar_label: 2023 * Fix flakiness of test_distributed_load_balancing test [#49921](https://github.com/ClickHouse/ClickHouse/pull/49921) ([Azat Khuzhin](https://github.com/azat)). * Add some logging [#49925](https://github.com/ClickHouse/ClickHouse/pull/49925) ([Kseniia Sumarokova](https://github.com/kssenii)). * Support hardlinking parts transactionally [#49931](https://github.com/ClickHouse/ClickHouse/pull/49931) ([Michael Kolupaev](https://github.com/al13n321)). -* Fix for analyzer: 02377_ optimize_sorting_by_input_stream_properties_e… [#49943](https://github.com/ClickHouse/ClickHouse/pull/49943) ([Igor Nikonov](https://github.com/devcrafter)). +* Fix for analyzer: 02377_ optimize_sorting_by_input_stream_properties_e... [#49943](https://github.com/ClickHouse/ClickHouse/pull/49943) ([Igor Nikonov](https://github.com/devcrafter)). * Follow up to [#49429](https://github.com/ClickHouse/ClickHouse/issues/49429) [#49964](https://github.com/ClickHouse/ClickHouse/pull/49964) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix flaky test_ssl_cert_authentication to use urllib3 [#49982](https://github.com/ClickHouse/ClickHouse/pull/49982) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * Fix woboq codebrowser build with -Wno-poison-system-directories [#49992](https://github.com/ClickHouse/ClickHouse/pull/49992) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v23.8.1.2992-lts.md b/docs/changelogs/v23.8.1.2992-lts.md index 7c224b19350..05385d9c52b 100644 --- a/docs/changelogs/v23.8.1.2992-lts.md +++ b/docs/changelogs/v23.8.1.2992-lts.md @@ -272,7 +272,7 @@ sidebar_label: 2023 * Add more checks into ThreadStatus ctor. [#42019](https://github.com/ClickHouse/ClickHouse/pull/42019) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Refactor Query Tree visitor [#46740](https://github.com/ClickHouse/ClickHouse/pull/46740) ([Dmitry Novik](https://github.com/novikd)). * Revert "Revert "Randomize JIT settings in tests"" [#48282](https://github.com/ClickHouse/ClickHouse/pull/48282) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix outdated cache configuration in s3 tests: s3_storage_policy_by_defau… [#48424](https://github.com/ClickHouse/ClickHouse/pull/48424) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix outdated cache configuration in s3 tests: s3_storage_policy_by_defau... [#48424](https://github.com/ClickHouse/ClickHouse/pull/48424) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix IN with decimal in analyzer [#48754](https://github.com/ClickHouse/ClickHouse/pull/48754) ([vdimir](https://github.com/vdimir)). * Some unclear change in StorageBuffer::reschedule() for something [#49723](https://github.com/ClickHouse/ClickHouse/pull/49723) ([DimasKovas](https://github.com/DimasKovas)). * MergeTree & SipHash checksum big-endian support [#50276](https://github.com/ClickHouse/ClickHouse/pull/50276) ([ltrk2](https://github.com/ltrk2)). diff --git a/docs/changelogs/v24.1.3.31-stable.md b/docs/changelogs/v24.1.3.31-stable.md index 046ca451fbc..e898fba5c87 100644 --- a/docs/changelogs/v24.1.3.31-stable.md +++ b/docs/changelogs/v24.1.3.31-stable.md @@ -13,7 +13,7 @@ sidebar_label: 2024 #### Bug Fix (user-visible misbehavior in an official stable release) -* Fix `ASTAlterCommand::formatImpl` in case of column specific settings… [#59445](https://github.com/ClickHouse/ClickHouse/pull/59445) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix `ASTAlterCommand::formatImpl` in case of column specific settings... [#59445](https://github.com/ClickHouse/ClickHouse/pull/59445) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). * Make MAX use the same rules as permutation for complex types [#59498](https://github.com/ClickHouse/ClickHouse/pull/59498) ([Raúl Marín](https://github.com/Algunenano)). * Fix corner case when passing `update_insert_deduplication_token_in_dependent_materialized_views` [#59544](https://github.com/ClickHouse/ClickHouse/pull/59544) ([Jordi Villar](https://github.com/jrdi)). * Fix incorrect result of arrayElement / map[] on empty value [#59594](https://github.com/ClickHouse/ClickHouse/pull/59594) ([Raúl Marín](https://github.com/Algunenano)). diff --git a/docs/changelogs/v24.2.1.2248-stable.md b/docs/changelogs/v24.2.1.2248-stable.md index 6113dd51ab1..02affe12c43 100644 --- a/docs/changelogs/v24.2.1.2248-stable.md +++ b/docs/changelogs/v24.2.1.2248-stable.md @@ -130,7 +130,7 @@ sidebar_label: 2024 * Fix translate() with FixedString input [#59356](https://github.com/ClickHouse/ClickHouse/pull/59356) ([Raúl Marín](https://github.com/Algunenano)). * Fix digest calculation in Keeper [#59439](https://github.com/ClickHouse/ClickHouse/pull/59439) ([Antonio Andelic](https://github.com/antonio2368)). * Fix stacktraces for binaries without debug symbols [#59444](https://github.com/ClickHouse/ClickHouse/pull/59444) ([Azat Khuzhin](https://github.com/azat)). -* Fix `ASTAlterCommand::formatImpl` in case of column specific settings… [#59445](https://github.com/ClickHouse/ClickHouse/pull/59445) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). +* Fix `ASTAlterCommand::formatImpl` in case of column specific settings... [#59445](https://github.com/ClickHouse/ClickHouse/pull/59445) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). * Fix `SELECT * FROM [...] ORDER BY ALL` with Analyzer [#59462](https://github.com/ClickHouse/ClickHouse/pull/59462) ([zhongyuankai](https://github.com/zhongyuankai)). * Fix possible uncaught exception during distributed query cancellation [#59487](https://github.com/ClickHouse/ClickHouse/pull/59487) ([Azat Khuzhin](https://github.com/azat)). * Make MAX use the same rules as permutation for complex types [#59498](https://github.com/ClickHouse/ClickHouse/pull/59498) ([Raúl Marín](https://github.com/Algunenano)). diff --git a/docs/changelogs/v24.3.1.2672-lts.md b/docs/changelogs/v24.3.1.2672-lts.md index e5d008680a8..006ab941203 100644 --- a/docs/changelogs/v24.3.1.2672-lts.md +++ b/docs/changelogs/v24.3.1.2672-lts.md @@ -526,7 +526,7 @@ sidebar_label: 2024 * No "please" [#61916](https://github.com/ClickHouse/ClickHouse/pull/61916) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Update version_date.tsv and changelogs after v23.12.6.19-stable [#61917](https://github.com/ClickHouse/ClickHouse/pull/61917) ([robot-clickhouse](https://github.com/robot-clickhouse)). * Update version_date.tsv and changelogs after v24.1.8.22-stable [#61918](https://github.com/ClickHouse/ClickHouse/pull/61918) ([robot-clickhouse](https://github.com/robot-clickhouse)). -* Fix flaky test_broken_projestions/test.py::test_broken_ignored_replic… [#61932](https://github.com/ClickHouse/ClickHouse/pull/61932) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test_broken_projestions/test.py::test_broken_ignored_replic... [#61932](https://github.com/ClickHouse/ClickHouse/pull/61932) ([Kseniia Sumarokova](https://github.com/kssenii)). * Check is Rust avaiable for build, if not, suggest a way to disable Rust support [#61938](https://github.com/ClickHouse/ClickHouse/pull/61938) ([Azat Khuzhin](https://github.com/azat)). * CI: new ci menu in PR body [#61948](https://github.com/ClickHouse/ClickHouse/pull/61948) ([Max K.](https://github.com/maxknv)). * Remove flaky test `01193_metadata_loading` [#61961](https://github.com/ClickHouse/ClickHouse/pull/61961) ([Nikita Taranov](https://github.com/nickitat)). diff --git a/docs/en/development/style.md b/docs/en/development/style.md index 77a550f2a0e..1444bc0e452 100644 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -57,7 +57,7 @@ memcpy(&buf[place_value], &x, sizeof(x)); for (size_t i = 0; i < rows; i += storage.index_granularity) ``` -**7.** Add spaces around binary operators (`+`, `-`, `*`, `/`, `%`, …) and the ternary operator `?:`. +**7.** Add spaces around binary operators (`+`, `-`, `*`, `/`, `%`, ...) and the ternary operator `?:`. ``` cpp UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); @@ -86,7 +86,7 @@ dst.ClickGoodEvent = click.GoodEvent; If necessary, the operator can be wrapped to the next line. In this case, the offset in front of it is increased. -**11.** Do not use a space to separate unary operators (`--`, `++`, `*`, `&`, …) from the argument. +**11.** Do not use a space to separate unary operators (`--`, `++`, `*`, `&`, ...) from the argument. **12.** Put a space after a comma, but not before it. The same rule goes for a semicolon inside a `for` expression. @@ -115,7 +115,7 @@ public: **16.** If the same `namespace` is used for the entire file, and there isn’t anything else significant, an offset is not necessary inside `namespace`. -**17.** If the block for an `if`, `for`, `while`, or other expression consists of a single `statement`, the curly brackets are optional. Place the `statement` on a separate line, instead. This rule is also valid for nested `if`, `for`, `while`, … +**17.** If the block for an `if`, `for`, `while`, or other expression consists of a single `statement`, the curly brackets are optional. Place the `statement` on a separate line, instead. This rule is also valid for nested `if`, `for`, `while`, ... But if the inner `statement` contains curly brackets or `else`, the external block should be written in curly brackets. diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index dbd1c270a4a..2749fa7e479 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -118,7 +118,7 @@ If the listing of files contains number ranges with leading zeros, use the const **Example** -Create table with files named `file000`, `file001`, … , `file999`: +Create table with files named `file000`, `file001`, ... , `file999`: ``` sql CREATE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV') diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index dfa06801d04..cb1da1c8e68 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -178,7 +178,7 @@ If the listing of files contains number ranges with leading zeros, use the const **Example with wildcards 1** -Create table with files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`: +Create table with files named `file-000.csv`, `file-001.csv`, ... , `file-999.csv`: ``` sql CREATE TABLE big_table (name String, value UInt32) diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index 23d98d4b20e..eda87fd06c1 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -71,7 +71,7 @@ WHERE table = 'visits' └───────────┴───────────────────┴────────┘ ``` -The `partition` column contains the names of the partitions. There are two partitions in this example: `201901` and `201902`. You can use this column value to specify the partition name in [ALTER … PARTITION](../../../sql-reference/statements/alter/partition.md) queries. +The `partition` column contains the names of the partitions. There are two partitions in this example: `201901` and `201902`. You can use this column value to specify the partition name in [ALTER ... PARTITION](../../../sql-reference/statements/alter/partition.md) queries. The `name` column contains the names of the partition data parts. You can use this column to specify the name of the part in the [ALTER ATTACH PART](../../../sql-reference/statements/alter/partition.md#alter_attach-partition) query. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7862eef69f8..a009c4a32f3 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -954,7 +954,7 @@ In the case of `MergeTree` tables, data is getting to disk in different ways: - As a result of an insert (`INSERT` query). - During background merges and [mutations](/docs/en/sql-reference/statements/alter/index.md#alter-mutations). - When downloading from another replica. -- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition). +- As a result of partition freezing [ALTER TABLE ... FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition). In all these cases except for mutations and partition freezing, a part is stored on a volume and a disk according to the given storage policy: @@ -966,7 +966,7 @@ Under the hood, mutations and partition freezing make use of [hard links](https: In the background, parts are moved between volumes on the basis of the amount of free space (`move_factor` parameter) according to the order the volumes are declared in the configuration file. Data is never transferred from the last one and into the first one. One may use system tables [system.part_log](/docs/en/operations/system-tables/part_log.md/#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](/docs/en/operations/system-tables/parts.md/#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs. -User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](/docs/en/sql-reference/statements/alter/partition.md/#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met. +User can force moving a part or a partition from one volume to another using the query [ALTER TABLE ... MOVE PART\|PARTITION ... TO VOLUME\|DISK ...](/docs/en/sql-reference/statements/alter/partition.md/#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met. Moving data does not interfere with data replication. Therefore, different storage policies can be specified for the same table on different replicas. diff --git a/docs/en/engines/table-engines/special/external-data.md b/docs/en/engines/table-engines/special/external-data.md index 7ea3f3e30d6..f6d6dae7eb6 100644 --- a/docs/en/engines/table-engines/special/external-data.md +++ b/docs/en/engines/table-engines/special/external-data.md @@ -29,7 +29,7 @@ Only a single table can be retrieved from stdin. The following parameters are optional: **–name**– Name of the table. If omitted, _data is used. **–format** – Data format in the file. If omitted, TabSeparated is used. -One of the following parameters is required:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, … +One of the following parameters is required:**–types** – A list of comma-separated column types. For example: `UInt64,String`. The columns will be named _1, _2, ... **–structure**– The table structure in the format`UserID UInt64`, `URL String`. Defines the column names and types. The files specified in ‘file’ will be parsed by the format specified in ‘format’, using the data types specified in ‘types’ or ‘structure’. The table will be uploaded to the server and accessible there as a temporary table with the name in ‘name’. diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index d86f18ff982..2a20e74e20f 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -303,7 +303,7 @@ What to do when the amount of data exceeds one of the limits: ‘throw’ or ‘ Limits the number of rows in the hash table that is used when joining tables. -This settings applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and the [Join](../../engines/table-engines/special/join.md) table engine. +This settings applies to [SELECT ... JOIN](../../sql-reference/statements/select/join.md#select-join) operations and the [Join](../../engines/table-engines/special/join.md) table engine. If a query contains multiple joins, ClickHouse checks this setting for every intermediate result. @@ -320,7 +320,7 @@ Default value: 0. Limits the size in bytes of the hash table used when joining tables. -This setting applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md). +This setting applies to [SELECT ... JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md). If the query contains joins, ClickHouse checks this setting for every intermediate result. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 91b544c6a82..2b5cd11819a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2248,7 +2248,7 @@ Default value: 0. ## count_distinct_implementation {#count_distinct_implementation} -Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md/#agg_function-count) construction. +Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT ...)](../../sql-reference/aggregate-functions/reference/count.md/#agg_function-count) construction. Possible values: diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 8981ac1f752..1dc89b8dcf9 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -82,7 +82,7 @@ FROM In this case, you should remember that you do not know the histogram bin borders. -## sequenceMatch(pattern)(timestamp, cond1, cond2, …) +## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) Checks whether the sequence contains an event chain that matches the pattern. @@ -172,7 +172,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM - [sequenceCount](#function-sequencecount) -## sequenceCount(pattern)(time, cond1, cond2, …) +## sequenceCount(pattern)(time, cond1, cond2, ...) Counts the number of event chains that matched the pattern. The function searches event chains that do not overlap. It starts to search for the next chain after the current chain is matched. diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiles.md b/docs/en/sql-reference/aggregate-functions/reference/quantiles.md index e2a5bc53e32..856d447ac13 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiles.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiles.md @@ -7,7 +7,7 @@ sidebar_position: 201 ## quantiles -Syntax: `quantiles(level1, level2, …)(x)` +Syntax: `quantiles(level1, level2, ...)(x)` All the quantile functions also have corresponding quantiles functions: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantileInterpolatedWeighted`, `quantilesTDigest`, `quantilesBFloat16`, `quantilesDD`. These functions calculate all the quantiles of the listed levels in one pass, and return an array of the resulting values. diff --git a/docs/en/sql-reference/data-types/aggregatefunction.md b/docs/en/sql-reference/data-types/aggregatefunction.md index 87511a505dc..37f0d0e50ae 100644 --- a/docs/en/sql-reference/data-types/aggregatefunction.md +++ b/docs/en/sql-reference/data-types/aggregatefunction.md @@ -6,9 +6,9 @@ sidebar_label: AggregateFunction # AggregateFunction -Aggregate functions can have an implementation-defined intermediate state that can be serialized to an `AggregateFunction(…)` data type and stored in a table, usually, by means of [a materialized view](../../sql-reference/statements/create/view.md). The common way to produce an aggregate function state is by calling the aggregate function with the `-State` suffix. To get the final result of aggregation in the future, you must use the same aggregate function with the `-Merge`suffix. +Aggregate functions can have an implementation-defined intermediate state that can be serialized to an `AggregateFunction(...)` data type and stored in a table, usually, by means of [a materialized view](../../sql-reference/statements/create/view.md). The common way to produce an aggregate function state is by calling the aggregate function with the `-State` suffix. To get the final result of aggregation in the future, you must use the same aggregate function with the `-Merge`suffix. -`AggregateFunction(name, types_of_arguments…)` — parametric data type. +`AggregateFunction(name, types_of_arguments...)` — parametric data type. **Parameters** diff --git a/docs/en/sql-reference/data-types/fixedstring.md b/docs/en/sql-reference/data-types/fixedstring.md index 0316df7fe34..0c021b28f74 100644 --- a/docs/en/sql-reference/data-types/fixedstring.md +++ b/docs/en/sql-reference/data-types/fixedstring.md @@ -21,8 +21,8 @@ The `FixedString` type is efficient when data has the length of precisely `N` by Examples of the values that can be efficiently stored in `FixedString`-typed columns: - The binary representation of IP addresses (`FixedString(16)` for IPv6). -- Language codes (ru_RU, en_US … ). -- Currency codes (USD, RUB … ). +- Language codes (ru_RU, en_US ... ). +- Currency codes (USD, RUB ... ). - Binary representation of hashes (`FixedString(16)` for MD5, `FixedString(32)` for SHA256). To store UUID values, use the [UUID](../../sql-reference/data-types/uuid.md) data type. diff --git a/docs/en/sql-reference/data-types/nested-data-structures/index.md b/docs/en/sql-reference/data-types/nested-data-structures/index.md index d118170cd39..579ee9bfa8b 100644 --- a/docs/en/sql-reference/data-types/nested-data-structures/index.md +++ b/docs/en/sql-reference/data-types/nested-data-structures/index.md @@ -6,7 +6,7 @@ sidebar_label: Nested(Name1 Type1, Name2 Type2, ...) # Nested -## Nested(name1 Type1, Name2 Type2, …) +## Nested(name1 Type1, Name2 Type2, ...) A nested data structure is like a table inside a cell. The parameters of a nested data structure – the column names and types – are specified the same way as in a [CREATE TABLE](../../../sql-reference/statements/create/table.md) query. Each table row can correspond to any number of rows in a nested data structure. diff --git a/docs/en/sql-reference/data-types/simpleaggregatefunction.md b/docs/en/sql-reference/data-types/simpleaggregatefunction.md index 39f8409c1e1..4fb74ac30e4 100644 --- a/docs/en/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/en/sql-reference/data-types/simpleaggregatefunction.md @@ -5,7 +5,7 @@ sidebar_label: SimpleAggregateFunction --- # SimpleAggregateFunction -`SimpleAggregateFunction(name, types_of_arguments…)` data type stores current value of the aggregate function, and does not store its full state as [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md) does. This optimization can be applied to functions for which the following property holds: the result of applying a function `f` to a row set `S1 UNION ALL S2` can be obtained by applying `f` to parts of the row set separately, and then again applying `f` to the results: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. This property guarantees that partial aggregation results are enough to compute the combined one, so we do not have to store and process any extra data. +`SimpleAggregateFunction(name, types_of_arguments...)` data type stores current value of the aggregate function, and does not store its full state as [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md) does. This optimization can be applied to functions for which the following property holds: the result of applying a function `f` to a row set `S1 UNION ALL S2` can be obtained by applying `f` to parts of the row set separately, and then again applying `f` to the results: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. This property guarantees that partial aggregation results are enough to compute the combined one, so we do not have to store and process any extra data. The common way to produce an aggregate function value is by calling the aggregate function with the [-SimpleState](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-simplestate) suffix. diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 6d95f3dc358..8b8527acfdf 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -140,6 +140,60 @@ Same as `intDiv` but returns zero when dividing by zero or when dividing a minim intDivOrZero(a, b) ``` +## isFinite + +Returns 1 if the Float32 or Float64 argument not infinite and not a NaN, otherwise this function returns 0. + +**Syntax** + +```sql +isFinite(x) +``` + +## isInfinite + +Returns 1 if the Float32 or Float64 argument is infinite, otherwise this function returns 0. Note that 0 is returned for a NaN. + +**Syntax** + +```sql +isInfinite(x) +``` + +## ifNotFinite + +Checks whether a floating point value is finite. + +**Syntax** + +```sql +ifNotFinite(x,y) +``` + +**Arguments** + +- `x` — Value to check for infinity. [Float\*](../../sql-reference/data-types/float.md). +- `y` — Fallback value. [Float\*](../../sql-reference/data-types/float.md). + +**Returned value** + +- `x` if `x` is finite. +- `y` if `x` is not finite. + +**Example** + +Query: + + SELECT 1/0 as infimum, ifNotFinite(infimum,42) + +Result: + + ┌─infimum─┬─ifNotFinite(divide(1, 0), 42)─┐ + │ inf │ 42 │ + └─────────┴───────────────────────────────┘ + +You can get similar result by using the [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. + ## modulo Calculates the remainder of the division of two values `a` by `b`. diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 87e733a4b0c..f929ea00b8b 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -561,7 +561,7 @@ Result: └─────────────┴─────────────┴────────────────┴─────────────────┘ ``` -## array(x1, …), operator \[x1, …\] +## array(x1, ...), operator \[x1, ...\] Creates an array from the function arguments. The arguments must be constants and have types that have the smallest common type. At least one argument must be passed, because otherwise it isn’t clear which type of array to create. That is, you can’t use this function to create an empty array (to do that, use the ‘emptyArray\*’ function described above). @@ -768,9 +768,9 @@ SELECT indexOf([1, 3, NULL, NULL], NULL) Elements set to `NULL` are handled as normal values. -## arrayCount(\[func,\] arr1, …) +## arrayCount(\[func,\] arr1, ...) -Returns the number of elements for which `func(arr1[i], …, arrN[i])` returns something other than 0. If `func` is not specified, it returns the number of non-zero elements in the array. +Returns the number of elements for which `func(arr1[i], ..., arrN[i])` returns something other than 0. If `func` is not specified, it returns the number of non-zero elements in the array. Note that the `arrayCount` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. @@ -847,7 +847,7 @@ SELECT countEqual([1, 2, NULL, NULL], NULL) ## arrayEnumerate(arr) -Returns the array \[1, 2, 3, …, length (arr) \] +Returns the array \[1, 2, 3, ..., length (arr) \] This function is normally used with ARRAY JOIN. It allows counting something just once for each array after applying ARRAY JOIN. Example: @@ -887,7 +887,7 @@ WHERE (CounterID = 160656) AND notEmpty(GoalsReached) This function can also be used in higher-order functions. For example, you can use it to get array indexes for elements that match a condition. -## arrayEnumerateUniq(arr, …) +## arrayEnumerateUniq(arr, ...) Returns an array the same size as the source array, indicating for each element what its position is among elements with the same value. For example: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. @@ -1206,7 +1206,7 @@ Result: └───────────────────┘ ``` -## arraySort(\[func,\] arr, …) {#sort} +## arraySort(\[func,\] arr, ...) {#sort} Sorts the elements of the `arr` array in ascending order. If the `func` function is specified, sorting order is determined by the result of the `func` function applied to the elements of the array. If `func` accepts multiple arguments, the `arraySort` function is passed several arrays that the arguments of `func` will correspond to. Detailed examples are shown at the end of `arraySort` description. @@ -1307,11 +1307,11 @@ SELECT arraySort((x, y) -> -y, [0, 1, 2], [1, 2, 3]) as res; To improve sorting efficiency, the [Schwartzian transform](https://en.wikipedia.org/wiki/Schwartzian_transform) is used. ::: -## arrayPartialSort(\[func,\] limit, arr, …) +## arrayPartialSort(\[func,\] limit, arr, ...) Same as `arraySort` with additional `limit` argument allowing partial sorting. Returns an array of the same size as the original array where elements in range `[1..limit]` are sorted in ascending order. Remaining elements `(limit..N]` shall contain elements in unspecified order. -## arrayReverseSort(\[func,\] arr, …) {#reverse-sort} +## arrayReverseSort(\[func,\] arr, ...) {#reverse-sort} Sorts the elements of the `arr` array in descending order. If the `func` function is specified, `arr` is sorted according to the result of the `func` function applied to the elements of the array, and then the sorted array is reversed. If `func` accepts multiple arguments, the `arrayReverseSort` function is passed several arrays that the arguments of `func` will correspond to. Detailed examples are shown at the end of `arrayReverseSort` description. @@ -1412,7 +1412,7 @@ SELECT arrayReverseSort((x, y) -> -y, [4, 3, 5], [1, 2, 3]) AS res; └─────────┘ ``` -## arrayPartialReverseSort(\[func,\] limit, arr, …) +## arrayPartialReverseSort(\[func,\] limit, arr, ...) Same as `arrayReverseSort` with additional `limit` argument allowing partial sorting. Returns an array of the same size as the original array where elements in range `[1..limit]` are sorted in descending order. Remaining elements `(limit..N]` shall contain elements in unspecified order. @@ -1535,7 +1535,7 @@ Result: [3,9,1,4,5,6,7,8,2,10] ``` -## arrayUniq(arr, …) +## arrayUniq(arr, ...) If one argument is passed, it counts the number of different elements in the array. If multiple arguments are passed, it counts the number of different tuples of elements at corresponding positions in multiple arrays. @@ -2079,9 +2079,9 @@ Result: └───────────────────────────────────────────────┘ ``` -## arrayMap(func, arr1, …) +## arrayMap(func, arr1, ...) -Returns an array obtained from the original arrays by application of `func(arr1[i], …, arrN[i])` for each element. Arrays `arr1` … `arrN` must have the same number of elements. +Returns an array obtained from the original arrays by application of `func(arr1[i], ..., arrN[i])` for each element. Arrays `arr1` ... `arrN` must have the same number of elements. Examples: @@ -2109,9 +2109,9 @@ SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res Note that the `arrayMap` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. -## arrayFilter(func, arr1, …) +## arrayFilter(func, arr1, ...) -Returns an array containing only the elements in `arr1` for which `func(arr1[i], …, arrN[i])` returns something other than 0. +Returns an array containing only the elements in `arr1` for which `func(arr1[i], ..., arrN[i])` returns something other than 0. Examples: @@ -2142,9 +2142,9 @@ SELECT Note that the `arrayFilter` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. -## arrayFill(func, arr1, …) +## arrayFill(func, arr1, ...) -Scan through `arr1` from the first element to the last element and replace `arr1[i]` by `arr1[i - 1]` if `func(arr1[i], …, arrN[i])` returns 0. The first element of `arr1` will not be replaced. +Scan through `arr1` from the first element to the last element and replace `arr1[i]` by `arr1[i - 1]` if `func(arr1[i], ..., arrN[i])` returns 0. The first element of `arr1` will not be replaced. Examples: @@ -2160,9 +2160,9 @@ SELECT arrayFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, Note that the `arrayFill` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. -## arrayReverseFill(func, arr1, …) +## arrayReverseFill(func, arr1, ...) -Scan through `arr1` from the last element to the first element and replace `arr1[i]` by `arr1[i + 1]` if `func(arr1[i], …, arrN[i])` returns 0. The last element of `arr1` will not be replaced. +Scan through `arr1` from the last element to the first element and replace `arr1[i]` by `arr1[i + 1]` if `func(arr1[i], ..., arrN[i])` returns 0. The last element of `arr1` will not be replaced. Examples: @@ -2178,9 +2178,9 @@ SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, Note that the `arrayReverseFill` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. -## arraySplit(func, arr1, …) +## arraySplit(func, arr1, ...) -Split `arr1` into multiple arrays. When `func(arr1[i], …, arrN[i])` returns something other than 0, the array will be split on the left hand side of the element. The array will not be split before the first element. +Split `arr1` into multiple arrays. When `func(arr1[i], ..., arrN[i])` returns something other than 0, the array will be split on the left hand side of the element. The array will not be split before the first element. Examples: @@ -2196,9 +2196,9 @@ SELECT arraySplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res Note that the `arraySplit` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. -## arrayReverseSplit(func, arr1, …) +## arrayReverseSplit(func, arr1, ...) -Split `arr1` into multiple arrays. When `func(arr1[i], …, arrN[i])` returns something other than 0, the array will be split on the right hand side of the element. The array will not be split after the last element. +Split `arr1` into multiple arrays. When `func(arr1[i], ..., arrN[i])` returns something other than 0, the array will be split on the right hand side of the element. The array will not be split after the last element. Examples: @@ -2214,30 +2214,30 @@ SELECT arrayReverseSplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res Note that the `arrayReverseSplit` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. -## arrayExists(\[func,\] arr1, …) +## arrayExists(\[func,\] arr1, ...) -Returns 1 if there is at least one element in `arr` for which `func(arr1[i], …, arrN[i])` returns something other than 0. Otherwise, it returns 0. +Returns 1 if there is at least one element in `arr` for which `func(arr1[i], ..., arrN[i])` returns something other than 0. Otherwise, it returns 0. Note that the `arrayExists` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. -## arrayAll(\[func,\] arr1, …) +## arrayAll(\[func,\] arr1, ...) -Returns 1 if `func(arr1[i], …, arrN[i])` returns something other than 0 for all the elements in arrays. Otherwise, it returns 0. +Returns 1 if `func(arr1[i], ..., arrN[i])` returns something other than 0 for all the elements in arrays. Otherwise, it returns 0. Note that the `arrayAll` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. -## arrayFirst(func, arr1, …) +## arrayFirst(func, arr1, ...) -Returns the first element in the `arr1` array for which `func(arr1[i], …, arrN[i])` returns something other than 0. +Returns the first element in the `arr1` array for which `func(arr1[i], ..., arrN[i])` returns something other than 0. ## arrayFirstOrNull -Returns the first element in the `arr1` array for which `func(arr1[i], …, arrN[i])` returns something other than 0, otherwise it returns `NULL`. +Returns the first element in the `arr1` array for which `func(arr1[i], ..., arrN[i])` returns something other than 0, otherwise it returns `NULL`. **Syntax** ```sql -arrayFirstOrNull(func, arr1, …) +arrayFirstOrNull(func, arr1, ...) ``` **Parameters** @@ -2292,20 +2292,20 @@ Result: \N ``` -## arrayLast(func, arr1, …) +## arrayLast(func, arr1, ...) -Returns the last element in the `arr1` array for which `func(arr1[i], …, arrN[i])` returns something other than 0. +Returns the last element in the `arr1` array for which `func(arr1[i], ..., arrN[i])` returns something other than 0. Note that the `arrayLast` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. ## arrayLastOrNull -Returns the last element in the `arr1` array for which `func(arr1[i], …, arrN[i])` returns something other than 0, otherwise returns `NULL`. +Returns the last element in the `arr1` array for which `func(arr1[i], ..., arrN[i])` returns something other than 0, otherwise returns `NULL`. **Syntax** ```sql -arrayLastOrNull(func, arr1, …) +arrayLastOrNull(func, arr1, ...) ``` **Parameters** @@ -2348,15 +2348,15 @@ Result: \N ``` -## arrayFirstIndex(func, arr1, …) +## arrayFirstIndex(func, arr1, ...) -Returns the index of the first element in the `arr1` array for which `func(arr1[i], …, arrN[i])` returns something other than 0. +Returns the index of the first element in the `arr1` array for which `func(arr1[i], ..., arrN[i])` returns something other than 0. Note that the `arrayFirstIndex` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. -## arrayLastIndex(func, arr1, …) +## arrayLastIndex(func, arr1, ...) -Returns the index of the last element in the `arr1` array for which `func(arr1[i], …, arrN[i])` returns something other than 0. +Returns the index of the last element in the `arr1` array for which `func(arr1[i], ..., arrN[i])` returns something other than 0. Note that the `arrayLastIndex` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You must pass a lambda function to it as the first argument, and it can’t be omitted. @@ -2580,9 +2580,9 @@ Result: └─────┘ ``` -## arrayCumSum(\[func,\] arr1, …) +## arrayCumSum(\[func,\] arr1, ...) -Returns an array of the partial (running) sums of the elements in the source array `arr1`. If `func` is specified, then the sum is computed from applying `func` to `arr1`, `arr2`, ..., `arrN`, i.e. `func(arr1[i], …, arrN[i])`. +Returns an array of the partial (running) sums of the elements in the source array `arr1`. If `func` is specified, then the sum is computed from applying `func` to `arr1`, `arr2`, ..., `arrN`, i.e. `func(arr1[i], ..., arrN[i])`. **Syntax** @@ -2614,9 +2614,9 @@ SELECT arrayCumSum([1, 1, 1, 1]) AS res Note that the `arrayCumSum` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. -## arrayCumSumNonNegative(\[func,\] arr1, …) +## arrayCumSumNonNegative(\[func,\] arr1, ...) -Same as `arrayCumSum`, returns an array of the partial (running) sums of the elements in the source array. If `func` is specified, then the sum is computed from applying `func` to `arr1`, `arr2`, ..., `arrN`, i.e. `func(arr1[i], …, arrN[i])`. Unlike `arrayCumSum`, if the current running sum is smaller than `0`, it is replaced by `0`. +Same as `arrayCumSum`, returns an array of the partial (running) sums of the elements in the source array. If `func` is specified, then the sum is computed from applying `func` to `arr1`, `arr2`, ..., `arrN`, i.e. `func(arr1[i], ..., arrN[i])`. Unlike `arrayCumSum`, if the current running sum is smaller than `0`, it is replaced by `0`. **Syntax** diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 843f22e5a6f..1a56691ffc0 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1499,7 +1499,7 @@ This function returns the week number for date or datetime. The two-argument for The following table describes how the mode argument works. -| Mode | First day of week | Range | Week 1 is the first week … | +| Mode | First day of week | Range | Week 1 is the first week ... | |------|-------------------|-------|-------------------------------| | 0 | Sunday | 0-53 | with a Sunday in this year | | 1 | Monday | 0-53 | with 4 or more days this year | diff --git a/docs/en/sql-reference/functions/json-functions.md b/docs/en/sql-reference/functions/json-functions.md index e920ab82988..ba72b3cc6ed 100644 --- a/docs/en/sql-reference/functions/json-functions.md +++ b/docs/en/sql-reference/functions/json-functions.md @@ -386,7 +386,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices_or_keys\]…) +## JSONHas(json\[, indices_or_keys\]...) If the value exists in the JSON document, `1` will be returned. @@ -419,7 +419,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices_or_keys\]…) +## JSONLength(json\[, indices_or_keys\]...) Return the length of a JSON array or a JSON object. @@ -432,7 +432,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices_or_keys\]…) +## JSONType(json\[, indices_or_keys\]...) Return the type of a JSON value. @@ -446,13 +446,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices_or_keys\]…) +## JSONExtractUInt(json\[, indices_or_keys\]...) -## JSONExtractInt(json\[, indices_or_keys\]…) +## JSONExtractInt(json\[, indices_or_keys\]...) -## JSONExtractFloat(json\[, indices_or_keys\]…) +## JSONExtractFloat(json\[, indices_or_keys\]...) -## JSONExtractBool(json\[, indices_or_keys\]…) +## JSONExtractBool(json\[, indices_or_keys\]...) Parses a JSON and extract a value. These functions are similar to `visitParam` functions. @@ -466,7 +466,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices_or_keys\]…) +## JSONExtractString(json\[, indices_or_keys\]...) Parses a JSON and extract a string. This function is similar to `visitParamExtractString` functions. @@ -484,7 +484,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices_or_keys…\], Return_type) +## JSONExtract(json\[, indices_or_keys...\], Return_type) Parses a JSON and extract a value of the given ClickHouse data type. @@ -506,7 +506,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) +## JSONExtractKeysAndValues(json\[, indices_or_keys...\], Value_type) Parses key-value pairs from a JSON where the values are of the given ClickHouse data type. @@ -554,7 +554,7 @@ text └────────────────────────────────────────────────────────────┘ ``` -## JSONExtractRaw(json\[, indices_or_keys\]…) +## JSONExtractRaw(json\[, indices_or_keys\]...) Returns a part of JSON as unparsed string. @@ -566,7 +566,7 @@ Example: SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]'; ``` -## JSONExtractArrayRaw(json\[, indices_or_keys…\]) +## JSONExtractArrayRaw(json\[, indices_or_keys...\]) Returns an array with elements of JSON array, each represented as unparsed string. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5b77f16027b..4501d1f43d3 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -172,7 +172,7 @@ Result: ## visibleWidth Calculates the approximate width when outputting values to the console in text format (tab-separated). -This function is used by the system to implement [Pretty formats](../formats.mdx). +This function is used by the system to implement [Pretty formats](../../interfaces/formats.md). `NULL` is represented as a string corresponding to `NULL` in `Pretty` formats. @@ -335,7 +335,7 @@ The argument is internally still evaluated. Useful e.g. for benchmarks. **Syntax** ```sql -ignore(…) +ignore(x) ``` ## sleep @@ -541,60 +541,6 @@ Result: └────────────────────┘ ``` -## isFinite - -Returns 1 if the Float32 or Float64 argument not infinite and not a NaN, otherwise this function returns 0. - -**Syntax** - -```sql -isFinite(x) -``` - -## isInfinite - -Returns 1 if the Float32 or Float64 argument is infinite, otherwise this function returns 0. Note that 0 is returned for a NaN. - -**Syntax** - -```sql -isInfinite(x) -``` - -## ifNotFinite - -Checks whether a floating point value is finite. - -**Syntax** - -```sql -ifNotFinite(x,y) -``` - -**Arguments** - -- `x` — Value to check for infinity. [Float\*](../../sql-reference/data-types/float.md). -- `y` — Fallback value. [Float\*](../../sql-reference/data-types/float.md). - -**Returned value** - -- `x` if `x` is finite. -- `y` if `x` is not finite. - -**Example** - -Query: - - SELECT 1/0 as infimum, ifNotFinite(infimum,42) - -Result: - - ┌─infimum─┬─ifNotFinite(divide(1, 0), 42)─┐ - │ inf │ 42 │ - └─────────┴───────────────────────────────┘ - -You can get similar result by using the [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. - ## isNaN Returns 1 if the Float32 and Float64 argument is NaN, otherwise this function 0. @@ -2303,7 +2249,7 @@ Accepts a path to a catboost model and model arguments (features). Returns Float **Syntax** ```sql -catboostEvaluate(path_to_model, feature_1, feature_2, …, feature_n) +catboostEvaluate(path_to_model, feature_1, feature_2, ..., feature_n) ``` **Example** @@ -2351,7 +2297,7 @@ Throw an exception if argument `x` is true. **Syntax** ```sql -throwIf(x\[, message\[, error_code\]\]) +throwIf(x[, message[, error_code]]) ``` **Arguments** diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 0b761b62006..0e183626555 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -139,7 +139,7 @@ Format the `pattern` string with the values (strings, integers, etc.) listed in **Syntax** ```sql -format(pattern, s0, s1, …) +format(pattern, s0, s1, ...) ``` **Example** diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 9738c19bf3c..a6eb4a4ceff 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -799,7 +799,7 @@ If you only want to search multiple substrings in a string, you can use function **Syntax** ```sql -multiMatchAny(haystack, \[pattern1, pattern2, …, patternn\]) +multiMatchAny(haystack, \[pattern1, pattern2, ..., patternn\]) ``` ## multiMatchAnyIndex @@ -809,7 +809,7 @@ Like `multiMatchAny` but returns any index that matches the haystack. **Syntax** ```sql -multiMatchAnyIndex(haystack, \[pattern1, pattern2, …, patternn\]) +multiMatchAnyIndex(haystack, \[pattern1, pattern2, ..., patternn\]) ``` ## multiMatchAllIndices @@ -819,7 +819,7 @@ Like `multiMatchAny` but returns the array of all indices that match the haystac **Syntax** ```sql -multiMatchAllIndices(haystack, \[pattern1, pattern2, …, patternn\]) +multiMatchAllIndices(haystack, \[pattern1, pattern2, ..., patternn\]) ``` ## multiFuzzyMatchAny @@ -833,7 +833,7 @@ Like `multiMatchAny` but returns 1 if any pattern matches the haystack within a **Syntax** ```sql -multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) +multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, ..., patternn\]) ``` ## multiFuzzyMatchAnyIndex @@ -843,7 +843,7 @@ Like `multiFuzzyMatchAny` but returns any index that matches the haystack within **Syntax** ```sql -multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) +multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, ..., patternn\]) ``` ## multiFuzzyMatchAllIndices @@ -853,7 +853,7 @@ Like `multiFuzzyMatchAny` but returns the array of all indices in any order that **Syntax** ```sql -multiFuzzyMatchAllIndices(haystack, distance, \[pattern1, pattern2, …, patternn\]) +multiFuzzyMatchAllIndices(haystack, distance, \[pattern1, pattern2, ..., patternn\]) ``` ## extract diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index 64b1732597f..c2219bb3f90 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -7,15 +7,15 @@ sidebar_label: Tuples ## tuple A function that allows grouping multiple columns. -For columns with the types T1, T2, …, it returns a Tuple(T1, T2, …) type tuple containing these columns. There is no cost to execute the function. +For columns with the types T1, T2, ..., it returns a Tuple(T1, T2, ...) type tuple containing these columns. There is no cost to execute the function. Tuples are normally used as intermediate values for an argument of IN operators, or for creating a list of formal parameters of lambda functions. Tuples can’t be written to a table. -The function implements the operator `(x, y, …)`. +The function implements the operator `(x, y, ...)`. **Syntax** ``` sql -tuple(x, y, …) +tuple(x, y, ...) ``` ## tupleElement diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 377283bc006..6386b4d5b1d 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -589,7 +589,7 @@ mapApply(func, map) **Returned value** -- Returns a map obtained from the original map by application of `func(map1[i], …, mapN[i])` for each element. +- Returns a map obtained from the original map by application of `func(map1[i], ..., mapN[i])` for each element. **Example** @@ -629,7 +629,7 @@ mapFilter(func, map) **Returned value** -- Returns a map containing only the elements in `map` for which `func(map1[i], …, mapN[i])` returns something other than 0. +- Returns a map containing only the elements in `map` for which `func(map1[i], ..., mapN[i])` returns something other than 0. **Example** diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index a0b0170721c..6da82e689a9 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -16,7 +16,7 @@ If the relevant part isn’t present in a URL, an empty string is returned. Extracts the protocol from a URL. -Examples of typical returned values: http, https, ftp, mailto, tel, magnet… +Examples of typical returned values: http, https, ftp, mailto, tel, magnet... ### domain diff --git a/docs/en/sql-reference/statements/alter/comment.md b/docs/en/sql-reference/statements/alter/comment.md index f6fb179d969..320828f0de9 100644 --- a/docs/en/sql-reference/statements/alter/comment.md +++ b/docs/en/sql-reference/statements/alter/comment.md @@ -4,7 +4,7 @@ sidebar_position: 51 sidebar_label: COMMENT --- -# ALTER TABLE … MODIFY COMMENT +# ALTER TABLE ... MODIFY COMMENT Adds, modifies, or removes comment to the table, regardless if it was set before or not. Comment change is reflected in both [system.tables](../../../operations/system-tables/tables.md) and `SHOW CREATE TABLE` query. diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index b6f45b67d52..af56bec7a11 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -4,7 +4,7 @@ sidebar_position: 39 sidebar_label: DELETE --- -# ALTER TABLE … DELETE Statement +# ALTER TABLE ... DELETE Statement ``` sql ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/index.md index 7961315c193..3cfb99cff83 100644 --- a/docs/en/sql-reference/statements/alter/index.md +++ b/docs/en/sql-reference/statements/alter/index.md @@ -42,7 +42,7 @@ These `ALTER` statements modify entities related to role-based access control: ## Mutations -`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](/docs/en/sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](/docs/en/sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts. +`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE ... DELETE](/docs/en/sql-reference/statements/alter/delete.md) and [ALTER TABLE ... UPDATE](/docs/en/sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts. For `*MergeTree` tables mutations execute by **rewriting whole data parts**. There is no atomicity - parts are substituted for mutated parts as soon as they are ready and a `SELECT` query that started executing during a mutation will see data from parts that have already been mutated along with data from parts that have not been mutated yet. diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index ab7d0ca7378..0b300e5849a 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -4,7 +4,7 @@ sidebar_position: 40 sidebar_label: UPDATE --- -# ALTER TABLE … UPDATE Statements +# ALTER TABLE ... UPDATE Statements ``` sql ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] [IN PARTITION partition_id] WHERE filter_expr diff --git a/docs/en/sql-reference/statements/alter/view.md b/docs/en/sql-reference/statements/alter/view.md index e063b27424e..83e8e9311b4 100644 --- a/docs/en/sql-reference/statements/alter/view.md +++ b/docs/en/sql-reference/statements/alter/view.md @@ -4,9 +4,9 @@ sidebar_position: 50 sidebar_label: VIEW --- -# ALTER TABLE … MODIFY QUERY Statement +# ALTER TABLE ... MODIFY QUERY Statement -You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement without interrupting ingestion process. +You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE ... MODIFY QUERY` statement without interrupting ingestion process. This command is created to change materialized view created with `TO [db.]name` clause. It does not change the structure of the underlying storage table and it does not change the columns' definition of the materialized view, because of this the application of this command is very limited for materialized views are created without `TO [db.]name` clause. @@ -198,6 +198,6 @@ SELECT * FROM mv; `ALTER LIVE VIEW ... REFRESH` statement refreshes a [Live view](../create/view.md#live-view). See [Force Live View Refresh](../create/view.md#live-view-alter-refresh). -## ALTER TABLE … MODIFY REFRESH Statement +## ALTER TABLE ... MODIFY REFRESH Statement `ALTER TABLE ... MODIFY REFRESH` statement changes refresh parameters of a [Refreshable Materialized View](../create/view.md#refreshable-materialized-view). See [Changing Refresh Parameters](../create/view.md#changing-refresh-parameters). diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 073a3c0d246..b526c94e508 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -306,7 +306,7 @@ CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTE Note that elements emitted by a late firing should be treated as updated results of a previous computation. Instead of firing at the end of windows, the window view will fire immediately when the late event arrives. Thus, it will result in multiple outputs for the same window. Users need to take these duplicated results into account or deduplicate them. -You can modify `SELECT` query that was specified in the window view by using `ALTER TABLE … MODIFY QUERY` statement. The data structure resulting in a new `SELECT` query should be the same as the original `SELECT` query when with or without `TO [db.]name` clause. Note that the data in the current window will be lost because the intermediate state cannot be reused. +You can modify `SELECT` query that was specified in the window view by using `ALTER TABLE ... MODIFY QUERY` statement. The data structure resulting in a new `SELECT` query should be the same as the original `SELECT` query when with or without `TO [db.]name` clause. Note that the data in the current window will be lost because the intermediate state cannot be reused. ### Monitoring New Windows diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index a76692cf291..f3dadabd25f 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -73,7 +73,7 @@ Data can be passed to the INSERT in any [format](../../interfaces/formats.md#for INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set ``` -For example, the following query format is identical to the basic version of INSERT … VALUES: +For example, the following query format is identical to the basic version of INSERT ... VALUES: ``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... diff --git a/docs/en/sql-reference/statements/select/limit.md b/docs/en/sql-reference/statements/select/limit.md index d61a5a44b58..58fdf988bf3 100644 --- a/docs/en/sql-reference/statements/select/limit.md +++ b/docs/en/sql-reference/statements/select/limit.md @@ -17,11 +17,11 @@ If there is no [ORDER BY](../../../sql-reference/statements/select/order-by.md) The number of rows in the result set can also depend on the [limit](../../../operations/settings/settings.md#limit) setting. ::: -## LIMIT … WITH TIES Modifier +## LIMIT ... WITH TIES Modifier When you set `WITH TIES` modifier for `LIMIT n[,m]` and specify `ORDER BY expr_list`, you will get in result first `n` or `n,m` rows and all rows with same `ORDER BY` fields values equal to row at position `n` for `LIMIT n` and `m` for `LIMIT n,m`. -This modifier also can be combined with [ORDER BY … WITH FILL modifier](../../../sql-reference/statements/select/order-by.md#orderby-with-fill). +This modifier also can be combined with [ORDER BY ... WITH FILL modifier](../../../sql-reference/statements/select/order-by.md#orderby-with-fill). For example, the following query diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index d6432a7b4f8..512a58d7cd9 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -283,7 +283,7 @@ In `MaterializedView`-engine tables the optimization works with views like `SELE ## ORDER BY Expr WITH FILL Modifier -This modifier also can be combined with [LIMIT … WITH TIES modifier](../../../sql-reference/statements/select/limit.md#limit-with-ties). +This modifier also can be combined with [LIMIT ... WITH TIES modifier](../../../sql-reference/statements/select/limit.md#limit-with-ties). `WITH FILL` modifier can be set after `ORDER BY expr` with optional `FROM expr`, `TO expr` and `STEP expr` parameters. All missed values of `expr` column will be filled sequentially and other columns will be filled as defaults. diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index 3a63811add6..f66178afbb2 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -169,7 +169,7 @@ If your listing of files contains number ranges with leading zeros, use the cons **Example** -Query the total number of rows in files named `file000`, `file001`, … , `file999`: +Query the total number of rows in files named `file000`, `file001`, ... , `file999`: ``` sql SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, value UInt32'); diff --git a/docs/en/sql-reference/table-functions/gcs.md b/docs/en/sql-reference/table-functions/gcs.md index 80077ecdb33..b891d88df31 100644 --- a/docs/en/sql-reference/table-functions/gcs.md +++ b/docs/en/sql-reference/table-functions/gcs.md @@ -130,7 +130,7 @@ FROM gcs('https://storage.googleapis.com/my-test-bucket-768/{some,another}_prefi If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: -Count the total amount of rows in files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`: +Count the total amount of rows in files named `file-000.csv`, `file-001.csv`, ... , `file-999.csv`: ``` sql SELECT count(*) diff --git a/docs/en/sql-reference/table-functions/hdfs.md b/docs/en/sql-reference/table-functions/hdfs.md index 92f904b8841..d65615e7588 100644 --- a/docs/en/sql-reference/table-functions/hdfs.md +++ b/docs/en/sql-reference/table-functions/hdfs.md @@ -85,7 +85,7 @@ If your listing of files contains number ranges with leading zeros, use the cons **Example** -Query the data from files named `file000`, `file001`, … , `file999`: +Query the data from files named `file000`, `file001`, ... , `file999`: ``` sql SELECT count(*) diff --git a/docs/en/sql-reference/table-functions/s3.md b/docs/en/sql-reference/table-functions/s3.md index 38d77a98749..cbef80371a3 100644 --- a/docs/en/sql-reference/table-functions/s3.md +++ b/docs/en/sql-reference/table-functions/s3.md @@ -137,7 +137,7 @@ FROM s3('https://clickhouse-public-datasets.s3.amazonaws.com/my-test-bucket-768/ If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. ::: -Count the total amount of rows in files named `file-000.csv`, `file-001.csv`, … , `file-999.csv`: +Count the total amount of rows in files named `file-000.csv`, `file-001.csv`, ... , `file-999.csv`: ``` sql SELECT count(*) diff --git a/docs/ru/development/style.md b/docs/ru/development/style.md index cd1297504af..08fa7a1e603 100644 --- a/docs/ru/development/style.md +++ b/docs/ru/development/style.md @@ -57,7 +57,7 @@ memcpy(&buf[place_value], &x, sizeof(x)); for (size_t i = 0; i < rows; i += storage.index_granularity) ``` -**7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, …), а также тернарного оператора `?:` ставятся пробелы. +**7.** Вокруг бинарных операторов (`+`, `-`, `*`, `/`, `%`, ...), а также тернарного оператора `?:` ставятся пробелы. ``` cpp UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); @@ -86,7 +86,7 @@ dst.ClickGoodEvent = click.GoodEvent; При необходимости, оператор может быть перенесён на новую строку. В этом случае, перед ним увеличивается отступ. -**11.** Унарные операторы `--`, `++`, `*`, `&`, … не отделяются от аргумента пробелом. +**11.** Унарные операторы `--`, `++`, `*`, `&`, ... не отделяются от аргумента пробелом. **12.** После запятой ставится пробел, а перед — нет. Аналогично для точки с запятой внутри выражения `for`. @@ -115,7 +115,7 @@ public: **16.** Если на весь файл один `namespace` и кроме него ничего существенного нет, то отступ внутри `namespace` не нужен. -**17.** Если блок для выражения `if`, `for`, `while`, … состоит из одного `statement`, то фигурные скобки не обязательны. Вместо этого поместите `statement` на отдельную строку. Это правило справедливо и для вложенных `if`, `for`, `while`, … +**17.** Если блок для выражения `if`, `for`, `while`, ... состоит из одного `statement`, то фигурные скобки не обязательны. Вместо этого поместите `statement` на отдельную строку. Это правило справедливо и для вложенных `if`, `for`, `while`, ... Если внутренний `statement` содержит фигурные скобки или `else`, то внешний блок следует писать в фигурных скобках. @@ -266,7 +266,7 @@ void executeQuery( Пример взят с ресурса http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/. -**7.** Нельзя писать мусорные комментарии (автор, дата создания…) в начале каждого файла. +**7.** Нельзя писать мусорные комментарии (автор, дата создания...) в начале каждого файла. **8.** Однострочные комментарии начинаются с трёх слешей: `///` , многострочные с `/**`. Такие комментарии считаются «документирующими». diff --git a/docs/ru/engines/table-engines/integrations/hdfs.md b/docs/ru/engines/table-engines/integrations/hdfs.md index 72087b56652..cf43eef73e3 100644 --- a/docs/ru/engines/table-engines/integrations/hdfs.md +++ b/docs/ru/engines/table-engines/integrations/hdfs.md @@ -103,7 +103,7 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = HDFS('hdfs **Example** -Создадим таблицу с именами `file000`, `file001`, … , `file999`: +Создадим таблицу с именами `file000`, `file001`, ... , `file999`: ``` sql CREATE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV') diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index 720aa589122..a1c69df4d0a 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -73,7 +73,7 @@ SELECT * FROM s3_engine_table LIMIT 2; **Пример подстановки 1** -Таблица содержит данные из файлов с именами `file-000.csv`, `file-001.csv`, … , `file-999.csv`: +Таблица содержит данные из файлов с именами `file-000.csv`, `file-001.csv`, ... , `file-999.csv`: ``` sql CREATE TABLE big_table (name String, value UInt32) diff --git a/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md index 46597c94370..c3203804211 100644 --- a/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -66,7 +66,7 @@ WHERE table = 'visits' └───────────┴───────────────────┴────────┘ ``` -Столбец `partition` содержит имена всех партиций таблицы. Таблица `visits` из нашего примера содержит две партиции: `201901` и `201902`. Используйте значения из этого столбца в запросах [ALTER … PARTITION](../../../sql-reference/statements/alter/partition.md). +Столбец `partition` содержит имена всех партиций таблицы. Таблица `visits` из нашего примера содержит две партиции: `201901` и `201902`. Используйте значения из этого столбца в запросах [ALTER ... PARTITION](../../../sql-reference/statements/alter/partition.md). Столбец `name` содержит названия кусков партиций. Значения из этого столбца можно использовать в запросах [ALTER ATTACH PART](../../../sql-reference/statements/alter/partition.md#alter_attach-partition). diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index faa492d4d85..49ba229b1d5 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -771,7 +771,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' - В результате вставки (запрос `INSERT`). - В фоновых операциях слияний и [мутаций](../../../sql-reference/statements/alter/index.md#mutations). - При скачивании данных с другой реплики. -- В результате заморозки партиций [ALTER TABLE … FREEZE PARTITION](../../../engines/table-engines/mergetree-family/mergetree.md#alter_freeze-partition). +- В результате заморозки партиций [ALTER TABLE ... FREEZE PARTITION](../../../engines/table-engines/mergetree-family/mergetree.md#alter_freeze-partition). Во всех случаях, кроме мутаций и заморозки партиций, при записи куска выбирается том и диск в соответствии с указанной конфигурацией хранилища: @@ -781,7 +781,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' Мутации и запросы заморозки партиций в реализации используют [жесткие ссылки](https://ru.wikipedia.org/wiki/%D0%96%D1%91%D1%81%D1%82%D0%BA%D0%B0%D1%8F_%D1%81%D1%81%D1%8B%D0%BB%D0%BA%D0%B0). Жесткие ссылки между различными дисками не поддерживаются, поэтому в случае таких операций куски размещаются на тех же дисках, что и исходные. В фоне куски перемещаются между томами на основе информации о занятом месте (настройка `move_factor`) по порядку, в котором указаны тома в конфигурации. Данные никогда не перемещаются с последнего тома и на первый том. Следить за фоновыми перемещениями можно с помощью системных таблиц [system.part_log](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-part-log) (поле `type = MOVE_PART`) и [system.parts](../../../engines/table-engines/mergetree-family/mergetree.md#system_tables-parts) (поля `path` и `disk`). Также подробная информация о перемещениях доступна в логах сервера. -С помощью запроса [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../engines/table-engines/mergetree-family/mergetree.md#alter_move-partition) пользователь может принудительно перенести кусок или партицию с одного раздела на другой. При этом учитываются все ограничения, указанные для фоновых операций. Запрос самостоятельно инициирует процесс перемещения не дожидаясь фоновых операций. В случае недостатка места или неудовлетворения ограничениям пользователь получит сообщение об ошибке. +С помощью запроса [ALTER TABLE ... MOVE PART\|PARTITION ... TO VOLUME\|DISK ...](../../../engines/table-engines/mergetree-family/mergetree.md#alter_move-partition) пользователь может принудительно перенести кусок или партицию с одного раздела на другой. При этом учитываются все ограничения, указанные для фоновых операций. Запрос самостоятельно инициирует процесс перемещения не дожидаясь фоновых операций. В случае недостатка места или неудовлетворения ограничениям пользователь получит сообщение об ошибке. Перемещения данных не взаимодействуют с репликацией данных, поэтому на разных репликах одной и той же таблицы могут быть указаны разные политики хранения. diff --git a/docs/ru/engines/table-engines/special/external-data.md b/docs/ru/engines/table-engines/special/external-data.md index 881566e5f34..3d9737096f5 100644 --- a/docs/ru/engines/table-engines/special/external-data.md +++ b/docs/ru/engines/table-engines/special/external-data.md @@ -31,7 +31,7 @@ ClickHouse позволяет отправить на сервер данные, - **--format** - формат данных в файле. Если не указано - используется TabSeparated. Должен быть указан один из следующих параметров: -- **--types** - список типов столбцов через запятую. Например, `UInt64,String`. Столбцы будут названы _1, _2, … +- **--types** - список типов столбцов через запятую. Например, `UInt64,String`. Столбцы будут названы _1, _2, ... - **--structure** - структура таблицы, в форме `UserID UInt64`, `URL String`. Определяет имена и типы столбцов. Файлы, указанные в file, будут разобраны форматом, указанным в format, с использованием типов данных, указанных в types или structure. Таблица будет загружена на сервер, и доступна там в качестве временной таблицы с именем name. diff --git a/docs/ru/faq/general/olap.md b/docs/ru/faq/general/olap.md index c9021f7c92e..bcfe9663381 100644 --- a/docs/ru/faq/general/olap.md +++ b/docs/ru/faq/general/olap.md @@ -9,13 +9,13 @@ sidebar_position: 100 [OLAP](https://ru.wikipedia.org/wiki/OLAP) (OnLine Analytical Processing) переводится как обработка данных в реальном времени. Это широкий термин, который можно рассмотреть с двух сторон: с технической и с точки зрения бизнеса. Для самого общего понимания можно просто прочитать его с конца: **Processing** - Обрабатываются некие исходные данные… + Обрабатываются некие исходные данные... **Analytical** -: … чтобы получить какие-то аналитические отчеты или новые знания… +: ... чтобы получить какие-то аналитические отчеты или новые знания... **OnLine** -: … в реальном времени, практически без задержек на обработку. +: ... в реальном времени, практически без задержек на обработку. ## OLAP с точки зрения бизнеса {#olap-from-the-business-perspective} diff --git a/docs/ru/getting-started/example-datasets/nyc-taxi.md b/docs/ru/getting-started/example-datasets/nyc-taxi.md index 12d0c18c3a1..a42033e7d41 100644 --- a/docs/ru/getting-started/example-datasets/nyc-taxi.md +++ b/docs/ru/getting-started/example-datasets/nyc-taxi.md @@ -196,7 +196,7 @@ real 75m56.214s (Импорт данных напрямую из Postgres также возможен с использованием `COPY ... TO PROGRAM`.) -К сожалению, все поля, связанные с погодой (precipitation…average_wind_speed) заполнены NULL. Из-за этого мы исключим их из финального набора данных. +К сожалению, все поля, связанные с погодой (precipitation...average_wind_speed) заполнены NULL. Из-за этого мы исключим их из финального набора данных. Для начала мы создадим таблицу на одном сервере. Позже мы сделаем таблицу распределенной. diff --git a/docs/ru/index.md b/docs/ru/index.md index 29f2bbe07fb..d551d492af5 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -15,7 +15,7 @@ ClickHouse — столбцовая система управления база | #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | | #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | | #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| #N | … | … | … | … | … | +| #N | ... | ... | ... | ... | ... | То есть, значения, относящиеся к одной строке, физически хранятся рядом. @@ -26,11 +26,11 @@ ClickHouse — столбцовая система управления база | Строка: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| -| WatchID: | 89354350662 | 90329509958 | 89953706054 | … | -| JavaEnable: | 1 | 0 | 1 | … | -| Title: | Investor Relations | Contact us | Mission | … | -| GoodEvent: | 1 | 1 | 1 | … | -| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | … | +| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... | +| JavaEnable: | 1 | 0 | 1 | ... | +| Title: | Investor Relations | Contact us | Mission | ... | +| GoodEvent: | 1 | 1 | 1 | ... | +| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | ... | В примерах изображён только порядок расположения данных. То есть значения из разных столбцов хранятся отдельно, а данные одного столбца — вместе. diff --git a/docs/ru/operations/settings/query-complexity.md b/docs/ru/operations/settings/query-complexity.md index d1d38a587c6..e82a5a008eb 100644 --- a/docs/ru/operations/settings/query-complexity.md +++ b/docs/ru/operations/settings/query-complexity.md @@ -260,7 +260,7 @@ FORMAT Null; Ограничивает количество строк в хэш-таблице, используемой при соединении таблиц. -Параметр применяется к операциям [SELECT… JOIN](../../sql-reference/statements/select/join.md#select-join) и к движку таблиц [Join](../../engines/table-engines/special/join.md). +Параметр применяется к операциям [SELECT... JOIN](../../sql-reference/statements/select/join.md#select-join) и к движку таблиц [Join](../../engines/table-engines/special/join.md). Если запрос содержит несколько `JOIN`, то ClickHouse проверяет значение настройки для каждого промежуточного результата. @@ -277,7 +277,7 @@ FORMAT Null; Ограничивает размер (в байтах) хэш-таблицы, используемой при объединении таблиц. -Параметр применяется к операциям [SELECT… JOIN](../../sql-reference/statements/select/join.md#select-join) и к движку таблиц [Join](../../engines/table-engines/special/join.md). +Параметр применяется к операциям [SELECT... JOIN](../../sql-reference/statements/select/join.md#select-join) и к движку таблиц [Join](../../engines/table-engines/special/join.md). Если запрос содержит несколько `JOIN`, то ClickHouse проверяет значение настройки для каждого промежуточного результата. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 2b3607dcf08..3a70a0bac12 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1859,7 +1859,7 @@ SELECT * FROM test_table ## count_distinct_implementation {#settings-count_distinct_implementation} -Задаёт, какая из функций `uniq*` используется при выполнении конструкции [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count). +Задаёт, какая из функций `uniq*` используется при выполнении конструкции [COUNT(DISTINCT ...)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count). Возможные значения: diff --git a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md index 6463f6bd95d..e6a61d9b381 100644 --- a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md @@ -82,7 +82,7 @@ FROM В этом случае необходимо помнить, что границы корзин гистограммы не известны. -## sequenceMatch(pattern)(timestamp, cond1, cond2, …) {#function-sequencematch} +## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) {#function-sequencematch} Проверяет, содержит ли последовательность событий цепочку, которая соответствует указанному шаблону. @@ -172,7 +172,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM - [sequenceCount](#function-sequencecount) -## sequenceCount(pattern)(time, cond1, cond2, …) {#function-sequencecount} +## sequenceCount(pattern)(time, cond1, cond2, ...) {#function-sequencecount} Вычисляет количество цепочек событий, соответствующих шаблону. Функция обнаруживает только непересекающиеся цепочки событий. Она начинает искать следующую цепочку только после того, как полностью совпала текущая цепочка событий. diff --git a/docs/ru/sql-reference/aggregate-functions/reference/quantiles.md b/docs/ru/sql-reference/aggregate-functions/reference/quantiles.md index fed0f8b328b..a0a430f7a68 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/quantiles.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/quantiles.md @@ -7,7 +7,7 @@ sidebar_position: 201 ## quantiles {#quantiles} -Синтаксис: `quantiles(level1, level2, …)(x)` +Синтаксис: `quantiles(level1, level2, ...)(x)` Все функции для вычисления квантилей имеют соответствующие функции для вычисления нескольких квантилей: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`, `quantilesBFloat16`. Эти функции вычисляют все квантили указанных уровней в один проход и возвращают массив с вычисленными значениями. diff --git a/docs/ru/sql-reference/data-types/aggregatefunction.md b/docs/ru/sql-reference/data-types/aggregatefunction.md index e42b467e4af..0481151c7e4 100644 --- a/docs/ru/sql-reference/data-types/aggregatefunction.md +++ b/docs/ru/sql-reference/data-types/aggregatefunction.md @@ -6,9 +6,9 @@ sidebar_label: AggregateFunction # AggregateFunction {#data-type-aggregatefunction} -Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(…), и быть записано в таблицу обычно посредством [материализованного представления](../../sql-reference/statements/create/view.md). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. +Агрегатные функции могут обладать определяемым реализацией промежуточным состоянием, которое может быть сериализовано в тип данных, соответствующий AggregateFunction(...), и быть записано в таблицу обычно посредством [материализованного представления](../../sql-reference/statements/create/view.md). Чтобы получить промежуточное состояние, обычно используются агрегатные функции с суффиксом `-State`. Чтобы в дальнейшем получить агрегированные данные необходимо использовать те же агрегатные функции с суффиксом `-Merge`. -`AggregateFunction(name, types_of_arguments…)` — параметрический тип данных. +`AggregateFunction(name, types_of_arguments...)` — параметрический тип данных. **Параметры** diff --git a/docs/ru/sql-reference/data-types/fixedstring.md b/docs/ru/sql-reference/data-types/fixedstring.md index d7a4e865903..56a5632f88d 100644 --- a/docs/ru/sql-reference/data-types/fixedstring.md +++ b/docs/ru/sql-reference/data-types/fixedstring.md @@ -21,8 +21,8 @@ sidebar_label: FixedString(N) Примеры значений, которые можно эффективно хранить в столбцах типа `FixedString`: - Двоичное представление IP-адреса (`FixedString(16)` для IPv6). -- Коды языков (ru_RU, en_US … ). -- Коды валют (USD, RUB … ). +- Коды языков (ru_RU, en_US ... ). +- Коды валют (USD, RUB ... ). - Двоичное представление хэшей (`FixedString(16)` для MD5, `FixedString(32)` для SHA256). Для хранения значений UUID используйте тип данных [UUID](uuid.md). diff --git a/docs/ru/sql-reference/data-types/nested-data-structures/nested.md b/docs/ru/sql-reference/data-types/nested-data-structures/nested.md index 4ec8333d563..8fd293a0415 100644 --- a/docs/ru/sql-reference/data-types/nested-data-structures/nested.md +++ b/docs/ru/sql-reference/data-types/nested-data-structures/nested.md @@ -3,7 +3,7 @@ slug: /ru/sql-reference/data-types/nested-data-structures/nested --- # Nested {#nested} -## Nested(Name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2} +## Nested(Name1 Type1, Name2 Type2, ...) {#nestedname1-type1-name2-type2} Вложенная структура данных - это как будто вложенная таблица. Параметры вложенной структуры данных - имена и типы столбцов, указываются так же, как у запроса CREATE. Каждой строке таблицы может соответствовать произвольное количество строк вложенной структуры данных. diff --git a/docs/ru/sql-reference/data-types/tuple.md b/docs/ru/sql-reference/data-types/tuple.md index 8953134d154..9d86c26c563 100644 --- a/docs/ru/sql-reference/data-types/tuple.md +++ b/docs/ru/sql-reference/data-types/tuple.md @@ -4,7 +4,7 @@ sidebar_position: 54 sidebar_label: Tuple(T1, T2, ...) --- -# Tuple(T1, T2, …) {#tuplet1-t2} +# Tuple(T1, T2, ...) {#tuplet1-t2} Кортеж из элементов любого [типа](index.md#data_types). Элементы кортежа могут быть одного или разных типов. diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 1f06bdf264a..825e3f06be2 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -161,7 +161,7 @@ SELECT range(5), range(1, 5), range(1, 5, 2); ``` -## array(x1, …), оператор \[x1, …\] {#arrayx1-operator-x1} +## array(x1, ...), оператор \[x1, ...\] {#arrayx1-operator-x1} Создаёт массив из аргументов функции. Аргументы должны быть константами и иметь типы, для которых есть наименьший общий тип. Должен быть передан хотя бы один аргумент, так как иначе непонятно, какого типа создавать массив. То есть, с помощью этой функции невозможно создать пустой массив (для этого используйте функции emptyArray\*, описанные выше). @@ -308,7 +308,7 @@ SELECT indexOf([1, 3, NULL, NULL], NULL) Элементы, равные `NULL`, обрабатываются как обычные значения. -## arrayCount(\[func,\] arr1, …) {#array-count} +## arrayCount(\[func,\] arr1, ...) {#array-count} Возвращает количество элементов массива `arr`, для которых функция `func` возвращает не 0. Если `func` не указана - возвращает количество ненулевых элементов массива. @@ -335,7 +335,7 @@ SELECT countEqual([1, 2, NULL, NULL], NULL) ## arrayEnumerate(arr) {#array_functions-arrayenumerate} -Возвращает массив \[1, 2, 3, …, length(arr)\] +Возвращает массив \[1, 2, 3, ..., length(arr)\] Эта функция обычно используется совместно с ARRAY JOIN. Она позволяет, после применения ARRAY JOIN, посчитать что-либо только один раз для каждого массива. Пример: @@ -375,7 +375,7 @@ WHERE (CounterID = 160656) AND notEmpty(GoalsReached) Также эта функция может быть использована в функциях высшего порядка. Например, с её помощью можно достать индексы массива для элементов, удовлетворяющих некоторому условию. -## arrayEnumerateUniq(arr, …) {#arrayenumerateuniqarr} +## arrayEnumerateUniq(arr, ...) {#arrayenumerateuniqarr} Возвращает массив, такого же размера, как исходный, где для каждого элемента указано, какой он по счету среди элементов с таким же значением. Например: arrayEnumerateUniq(\[10, 20, 10, 30\]) = \[1, 1, 2, 1\]. @@ -597,7 +597,7 @@ SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res; Элементы массива равные `NULL` обрабатываются как обычные значения. -## arraySort(\[func,\] arr, …) {#array_functions-sort} +## arraySort(\[func,\] arr, ...) {#array_functions-sort} Возвращает массив `arr`, отсортированный в восходящем порядке. Если задана функция `func`, то порядок сортировки определяется результатом применения этой функции на элементы массива `arr`. Если `func` принимает несколько аргументов, то в функцию `arraySort` нужно передавать несколько массивов, которые будут соответствовать аргументам функции `func`. Подробные примеры рассмотрены в конце описания `arraySort`. @@ -698,11 +698,11 @@ SELECT arraySort((x, y) -> -y, [0, 1, 2], [1, 2, 3]) as res; Для улучшения эффективности сортировки применяется [преобразование Шварца](https://ru.wikipedia.org/wiki/%D0%9F%D1%80%D0%B5%D0%BE%D0%B1%D1%80%D0%B0%D0%B7%D0%BE%D0%B2%D0%B0%D0%BD%D0%B8%D0%B5_%D0%A8%D0%B2%D0%B0%D1%80%D1%86%D0%B0). ::: -## arrayPartialSort(\[func,\] limit, arr, …) {#array_functions-sort} +## arrayPartialSort(\[func,\] limit, arr, ...) {#array_functions-sort} То же, что и `arraySort` с дополнительным аргументом `limit`, позволяющим частичную сортировку. Возвращает массив того же размера, как и исходный, в котором элементы `[1..limit]` отсортированы в возрастающем порядке. Остальные элементы `(limit..N]` остаются в неспецифицированном порядке. -## arrayReverseSort(\[func,\] arr, …) {#array_functions-reverse-sort} +## arrayReverseSort(\[func,\] arr, ...) {#array_functions-reverse-sort} Возвращает массив `arr`, отсортированный в нисходящем порядке. Если указана функция `func`, то массив `arr` сначала сортируется в порядке, который определяется функцией `func`, а затем отсортированный массив переворачивается. Если функция `func` принимает несколько аргументов, то в функцию `arrayReverseSort` необходимо передавать несколько массивов, которые будут соответствовать аргументам функции `func`. Подробные примеры рассмотрены в конце описания функции `arrayReverseSort`. @@ -803,11 +803,11 @@ SELECT arrayReverseSort((x, y) -> -y, [4, 3, 5], [1, 2, 3]) AS res; └─────────┘ ``` -## arrayPartialReverseSort(\[func,\] limit, arr, …) {#array_functions-sort} +## arrayPartialReverseSort(\[func,\] limit, arr, ...) {#array_functions-sort} То же, что и `arrayReverseSort` с дополнительным аргументом `limit`, позволяющим частичную сортировку. Возвращает массив того же размера, как и исходный, в котором элементы `[1..limit]` отсортированы в убывающем порядке. Остальные элементы `(limit..N]` остаются в неспецифицированном порядке. -## arrayUniq(arr, …) {#array-functions-arrayuniq} +## arrayUniq(arr, ...) {#array-functions-arrayuniq} Если передан один аргумент, считает количество разных элементов в массиве. Если передано несколько аргументов, считает количество разных кортежей из элементов на соответствующих позициях в нескольких массивах. @@ -1174,7 +1174,7 @@ SELECT arrayZip(['a', 'b', 'c'], [5, 2, 1]); └──────────────────────────────────────┘ ``` -## arrayMap(func, arr1, …) {#array-map} +## arrayMap(func, arr1, ...) {#array-map} Возвращает массив, полученный на основе результатов применения функции `func` к каждому элементу массива `arr`. @@ -1204,7 +1204,7 @@ SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res; Функция `arrayMap` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. -## arrayFilter(func, arr1, …) {#array-filter} +## arrayFilter(func, arr1, ...) {#array-filter} Возвращает массив, содержащий только те элементы массива `arr1`, для которых функция `func` возвращает не 0. @@ -1237,7 +1237,7 @@ SELECT Функция `arrayFilter` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. -## arrayFill(func, arr1, …) {#array-fill} +## arrayFill(func, arr1, ...) {#array-fill} Перебирает `arr1` от первого элемента к последнему и заменяет `arr1[i]` на `arr1[i - 1]`, если `func` вернула 0. Первый элемент `arr1` остаётся неизменным. @@ -1255,7 +1255,7 @@ SELECT arrayFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, Функция `arrayFill` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. -## arrayReverseFill(func, arr1, …) {#array-reverse-fill} +## arrayReverseFill(func, arr1, ...) {#array-reverse-fill} Перебирает `arr1` от последнего элемента к первому и заменяет `arr1[i]` на `arr1[i + 1]`, если `func` вернула 0. Последний элемент `arr1` остаётся неизменным. @@ -1273,7 +1273,7 @@ SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, Функция `arrayReverseFill` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. -## arraySplit(func, arr1, …) {#array-split} +## arraySplit(func, arr1, ...) {#array-split} Разделяет массив `arr1` на несколько. Если `func` возвращает не 0, то массив разделяется, а элемент помещается в левую часть. Массив не разбивается по первому элементу. @@ -1291,7 +1291,7 @@ SELECT arraySplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res Функция `arraySplit` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. -## arrayReverseSplit(func, arr1, …) {#array-reverse-split} +## arrayReverseSplit(func, arr1, ...) {#array-reverse-split} Разделяет массив `arr1` на несколько. Если `func` возвращает не 0, то массив разделяется, а элемент помещается в правую часть. Массив не разбивается по последнему элементу. @@ -1309,25 +1309,25 @@ SELECT arrayReverseSplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res Функция `arrayReverseSplit` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. -## arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1} +## arrayExists(\[func,\] arr1, ...) {#arrayexistsfunc-arr1} Возвращает 1, если существует хотя бы один элемент массива `arr`, для которого функция func возвращает не 0. Иначе возвращает 0. Функция `arrayExists` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) - в качестве первого аргумента ей можно передать лямбда-функцию. -## arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1} +## arrayAll(\[func,\] arr1, ...) {#arrayallfunc-arr1} Возвращает 1, если для всех элементов массива `arr`, функция `func` возвращает не 0. Иначе возвращает 0. Функция `arrayAll` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) - в качестве первого аргумента ей можно передать лямбда-функцию. -## arrayFirst(func, arr1, …) {#array-first} +## arrayFirst(func, arr1, ...) {#array-first} Возвращает первый элемент массива `arr1`, для которого функция func возвращает не 0. Функция `arrayFirst` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. -## arrayFirstIndex(func, arr1, …) {#array-first-index} +## arrayFirstIndex(func, arr1, ...) {#array-first-index} Возвращает индекс первого элемента массива `arr1`, для которого функция func возвращает не 0. @@ -1599,7 +1599,7 @@ SELECT arraySum(x -> x*x, [2, 3]) AS res; └─────┘ ``` -## arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1} +## arrayCumSum(\[func,\] arr1, ...) {#arraycumsumfunc-arr1} Возвращает массив из частичных сумм элементов исходного массива (сумма с накоплением). Если указана функция `func`, то значения элементов массива преобразуются этой функцией перед суммированием. diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 56ae4359bf1..bcc5f807c32 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -559,7 +559,7 @@ SELECT Описание режимов (mode): -| Mode | Первый день недели | Диапазон | Неделя 1 это первая неделя … | +| Mode | Первый день недели | Диапазон | Неделя 1 это первая неделя ... | | ----------- | -------- | -------- | ------------------ | |0|Воскресенье|0-53|с воскресеньем в этом году |1|Понедельник|0-53|с 4-мя или более днями в этом году diff --git a/docs/ru/sql-reference/functions/json-functions.md b/docs/ru/sql-reference/functions/json-functions.md index 123f40ce05d..18f625bf80f 100644 --- a/docs/ru/sql-reference/functions/json-functions.md +++ b/docs/ru/sql-reference/functions/json-functions.md @@ -88,7 +88,7 @@ SELECT isValidJSON('{"a": "hello", "b": [-100, 200.0, 300]}') = 1 SELECT isValidJSON('not a json') = 0 ``` -## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]...) {#jsonhasjson-indices-or-keys} Если значение существует в документе JSON, то возвращается `1`. @@ -121,7 +121,7 @@ SELECT JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' SELECT JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' ``` -## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]...) {#jsonlengthjson-indices-or-keys} Возвращает длину массива JSON или объекта JSON. @@ -134,7 +134,7 @@ SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 SELECT JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 ``` -## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]...) {#jsontypejson-indices-or-keys} Возвращает тип значения JSON. @@ -148,13 +148,13 @@ SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' SELECT JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' ``` -## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]...) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]...) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]...) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]...) {#jsonextractbooljson-indices-or-keys} Парсит JSON и извлекает значение. Эти функции аналогичны функциям `visitParam`. @@ -168,7 +168,7 @@ SELECT JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200 SELECT JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 ``` -## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]...) {#jsonextractstringjson-indices-or-keys} Парсит JSON и извлекает строку. Эта функция аналогична функции `visitParamExtractString`. @@ -186,7 +186,7 @@ SELECT JSONExtractString('{"abc":"\\u263"}', 'abc') = '' SELECT JSONExtractString('{"abc":"hello}', 'abc') = '' ``` -## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys...\], Return_type) {#jsonextractjson-indices-or-keys-return-type} Парсит JSON и извлекает значение с заданным типом данных. @@ -207,7 +207,7 @@ SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' ``` -## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys...\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} Разбор пар ключ-значение из JSON, где значение имеет тип данных ClickHouse. @@ -255,7 +255,7 @@ text └────────────────────────────────────────────────────────────┘ ``` -## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]...) {#jsonextractrawjson-indices-or-keys} Возвращает часть JSON в виде строки, содержащей неразобранную подстроку. @@ -267,7 +267,7 @@ text SELECT JSONExtractRaw('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = '[-100, 200.0, 300]'; ``` -## JSONExtractArrayRaw(json\[, indices_or_keys\]…) {#jsonextractarrayrawjson-indices-or-keys} +## JSONExtractArrayRaw(json\[, indices_or_keys\]...) {#jsonextractarrayrawjson-indices-or-keys} Возвращает массив из элементов JSON массива, каждый из которых представлен в виде строки с неразобранными подстроками из JSON. diff --git a/docs/ru/sql-reference/functions/other-functions.md b/docs/ru/sql-reference/functions/other-functions.md index 835aed934d5..f7637cfa3f7 100644 --- a/docs/ru/sql-reference/functions/other-functions.md +++ b/docs/ru/sql-reference/functions/other-functions.md @@ -286,7 +286,7 @@ SELECT byteSize(NULL, 1, 0.3, ''); Превращает константу в полноценный столбец, содержащий только одно значение. В ClickHouse полноценные столбцы и константы представлены в памяти по-разному. Функции по-разному работают для аргументов-констант и обычных аргументов (выполняется разный код), хотя результат почти всегда должен быть одинаковым. Эта функция предназначена для отладки такого поведения. -## ignore(…) {#ignore} +## ignore(...) {#ignore} Принимает любые аргументы, в т.ч. `NULL`, всегда возвращает 0. При этом, аргумент всё равно вычисляется. Это может использоваться для бенчмарков. diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index eeb5752c626..fc258f7b4cf 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -358,7 +358,7 @@ SELECT repeat('abc', 10); Разворачивает последовательность кодовых точек Unicode, при допущении, что строка содержит набор байтов, представляющий текст в кодировке UTF-8. Иначе — что-то делает (не кидает исключение). -## format(pattern, s0, s1, …) {#format} +## format(pattern, s0, s1, ...) {#format} Форматирует константный шаблон со строками, перечисленными в аргументах. `pattern` — упрощенная версия шаблона в языке Python. Шаблон содержит «заменяющие поля», которые окружены фигурными скобками `{}`. Всё, что не содержится в скобках, интерпретируется как обычный текст и просто копируется. Если нужно использовать символ фигурной скобки, можно экранировать двойной скобкой `{{ '{{' }}` или `{{ '}}' }}`. Имя полей могут быть числами (нумерация с нуля) или пустыми (тогда они интерпретируются как последовательные числа). diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index 4f9ae4428a4..53da9a6e791 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -311,19 +311,19 @@ Result: Смотрите `multiSearchAllPositions`. -## multiSearchFirstPosition(haystack, \[needle1, needle2, …, needlen\]) {#multisearchfirstpositionhaystack-needle1-needle2-needlen} +## multiSearchFirstPosition(haystack, \[needle1, needle2, ..., needlen\]) {#multisearchfirstpositionhaystack-needle1-needle2-needlen} Так же, как и `position`, только возвращает оффсет первого вхождения любого из needles. Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchFirstPositionCaseInsensitive, multiSearchFirstPositionUTF8, multiSearchFirstPositionCaseInsensitiveUTF8`. -## multiSearchFirstIndex(haystack, \[needle1, needle2, …, needlen\]) {#multisearchfirstindexhaystack-needle1-needle2-needlen} +## multiSearchFirstIndex(haystack, \[needle1, needle2, ..., needlen\]) {#multisearchfirstindexhaystack-needle1-needle2-needlen} Возвращает индекс `i` (нумерация с единицы) первой найденной строки needlei в строке `haystack` и 0 иначе. Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchFirstIndexCaseInsensitive, multiSearchFirstIndexUTF8, multiSearchFirstIndexCaseInsensitiveUTF8`. -## multiSearchAny(haystack, \[needle1, needle2, …, needlen\]) {#function-multisearchany} +## multiSearchAny(haystack, \[needle1, needle2, ..., needlen\]) {#function-multisearchany} Возвращает 1, если хотя бы одна подстрока needlei нашлась в строке `haystack` и 0 иначе. @@ -343,30 +343,30 @@ Result: Регулярное выражение работает со строкой как с набором байт. Регулярное выражение не может содержать нулевые байты. Для шаблонов на поиск подстроки в строке, лучше используйте LIKE или position, так как они работают существенно быстрее. -## multiMatchAny(haystack, \[pattern1, pattern2, …, patternn\]) {#multimatchanyhaystack-pattern1-pattern2-patternn} +## multiMatchAny(haystack, \[pattern1, pattern2, ..., patternn\]) {#multimatchanyhaystack-pattern1-pattern2-patternn} То же, что и `match`, но возвращает ноль, если ни одно регулярное выражение не подошло и один, если хотя бы одно. Используется библиотека [hyperscan](https://github.com/intel/hyperscan) для соответствия регулярных выражений. Для шаблонов на поиск многих подстрок в строке, лучше используйте `multiSearchAny`, так как она работает существенно быстрее. :::note Примечание Длина любой строки из `haystack` должна быть меньше 232 байт, иначе бросается исключение. Это ограничение связано с ограничением hyperscan API. ::: -## multiMatchAnyIndex(haystack, \[pattern1, pattern2, …, patternn\]) {#multimatchanyindexhaystack-pattern1-pattern2-patternn} +## multiMatchAnyIndex(haystack, \[pattern1, pattern2, ..., patternn\]) {#multimatchanyindexhaystack-pattern1-pattern2-patternn} То же, что и `multiMatchAny`, только возвращает любой индекс подходящего регулярного выражения. -## multiMatchAllIndices(haystack, \[pattern1, pattern2, …, patternn\]) {#multimatchallindiceshaystack-pattern1-pattern2-patternn} +## multiMatchAllIndices(haystack, \[pattern1, pattern2, ..., patternn\]) {#multimatchallindiceshaystack-pattern1-pattern2-patternn} То же, что и `multiMatchAny`, только возвращает массив всех индексов всех подходящих регулярных выражений в любом порядке. -## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} +## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, ..., patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} То же, что и `multiMatchAny`, но возвращает 1 если любой шаблон соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция основана на экспериментальной библиотеке [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) и может быть медленной для некоторых частных случаев. Производительность зависит от значения редакционного расстояния и используемых шаблонов, но всегда медленнее по сравнению с non-fuzzy вариантами. -## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} +## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, ..., patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} То же, что и `multiFuzzyMatchAny`, только возвращает любой индекс подходящего регулярного выражения в пределах константного редакционного расстояния. -## multiFuzzyMatchAllIndices(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchallindiceshaystack-distance-pattern1-pattern2-patternn} +## multiFuzzyMatchAllIndices(haystack, distance, \[pattern1, pattern2, ..., patternn\]) {#multifuzzymatchallindiceshaystack-distance-pattern1-pattern2-patternn} То же, что и `multiFuzzyMatchAny`, только возвращает массив всех индексов всех подходящих регулярных выражений в любом порядке в пределах константного редакционного расстояния. diff --git a/docs/ru/sql-reference/functions/tuple-functions.md b/docs/ru/sql-reference/functions/tuple-functions.md index c702e5d00b1..70ae44aa627 100644 --- a/docs/ru/sql-reference/functions/tuple-functions.md +++ b/docs/ru/sql-reference/functions/tuple-functions.md @@ -9,15 +9,15 @@ sidebar_label: Функции для работы с кортежами ## tuple {#tuple} Функция, позволяющая сгруппировать несколько столбцов. -Для столбцов, имеющих типы T1, T2, … возвращает кортеж типа Tuple(T1, T2, …), содержащий эти столбцы. Выполнение функции ничего не стоит. +Для столбцов, имеющих типы T1, T2, ... возвращает кортеж типа Tuple(T1, T2, ...), содержащий эти столбцы. Выполнение функции ничего не стоит. Кортежи обычно используются как промежуточное значение в качестве аргумента операторов IN, или для создания списка формальных параметров лямбда-функций. Кортежи не могут быть записаны в таблицу. -С помощью функции реализуется оператор `(x, y, …)`. +С помощью функции реализуется оператор `(x, y, ...)`. **Синтаксис** ``` sql -tuple(x, y, …) +tuple(x, y, ...) ``` ## tupleElement {#tupleelement} diff --git a/docs/ru/sql-reference/functions/url-functions.md b/docs/ru/sql-reference/functions/url-functions.md index 3c6e6151ef8..087891f4347 100644 --- a/docs/ru/sql-reference/functions/url-functions.md +++ b/docs/ru/sql-reference/functions/url-functions.md @@ -14,7 +14,7 @@ sidebar_label: "Функции для работы с URL" ### protocol {#protocol} -Возвращает протокол. Примеры: http, ftp, mailto, magnet… +Возвращает протокол. Примеры: http, ftp, mailto, magnet... ### domain {#domain} diff --git a/docs/ru/sql-reference/statements/alter/comment.md b/docs/ru/sql-reference/statements/alter/comment.md index 727af15d03e..f841c8540f3 100644 --- a/docs/ru/sql-reference/statements/alter/comment.md +++ b/docs/ru/sql-reference/statements/alter/comment.md @@ -4,7 +4,7 @@ sidebar_position: 51 sidebar_label: COMMENT --- -# ALTER TABLE … MODIFY COMMENT {#alter-modify-comment} +# ALTER TABLE ... MODIFY COMMENT {#alter-modify-comment} Добавляет, изменяет или удаляет комментарий к таблице, независимо от того, был ли он установлен раньше или нет. Изменение комментария отражается как в системной таблице [system.tables](../../../operations/system-tables/tables.md), так и в результате выполнения запроса `SHOW CREATE TABLE`. diff --git a/docs/ru/sql-reference/statements/alter/delete.md b/docs/ru/sql-reference/statements/alter/delete.md index dc968a17349..c91a79f5cdd 100644 --- a/docs/ru/sql-reference/statements/alter/delete.md +++ b/docs/ru/sql-reference/statements/alter/delete.md @@ -4,7 +4,7 @@ sidebar_position: 39 sidebar_label: DELETE --- -# ALTER TABLE … DELETE {#alter-mutations} +# ALTER TABLE ... DELETE {#alter-mutations} ``` sql ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr diff --git a/docs/ru/sql-reference/statements/alter/index.md b/docs/ru/sql-reference/statements/alter/index.md index 07f5ff0a298..e8b8af39e11 100644 --- a/docs/ru/sql-reference/statements/alter/index.md +++ b/docs/ru/sql-reference/statements/alter/index.md @@ -46,7 +46,7 @@ ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ### Мутации {#mutations} -Мутации - разновидность запроса ALTER, позволяющая изменять или удалять данные в таблице. В отличие от стандартных запросов [ALTER TABLE … DELETE](../../../sql-reference/statements/alter/delete.md) и [ALTER TABLE … UPDATE](../../../sql-reference/statements/alter/update.md), рассчитанных на точечное изменение данных, область применения мутаций - достаточно тяжёлые изменения, затрагивающие много строк в таблице. Поддержана для движков таблиц семейства [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md), в том числе для движков с репликацией. +Мутации - разновидность запроса ALTER, позволяющая изменять или удалять данные в таблице. В отличие от стандартных запросов [ALTER TABLE ... DELETE](../../../sql-reference/statements/alter/delete.md) и [ALTER TABLE ... UPDATE](../../../sql-reference/statements/alter/update.md), рассчитанных на точечное изменение данных, область применения мутаций - достаточно тяжёлые изменения, затрагивающие много строк в таблице. Поддержана для движков таблиц семейства [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md), в том числе для движков с репликацией. Конвертировать существующие таблицы для работы с мутациями не нужно. Но после применения первой мутации формат данных таблицы становится несовместимым с предыдущими версиями и откатиться на предыдущую версию уже не получится. diff --git a/docs/ru/sql-reference/statements/alter/update.md b/docs/ru/sql-reference/statements/alter/update.md index b2032ac77d1..01574a8a9b7 100644 --- a/docs/ru/sql-reference/statements/alter/update.md +++ b/docs/ru/sql-reference/statements/alter/update.md @@ -4,7 +4,7 @@ sidebar_position: 40 sidebar_label: UPDATE --- -# ALTER TABLE … UPDATE {#alter-table-update-statements} +# ALTER TABLE ... UPDATE {#alter-table-update-statements} ``` sql ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr diff --git a/docs/ru/sql-reference/statements/alter/view.md b/docs/ru/sql-reference/statements/alter/view.md index e6f6730ff99..53e295f6bbe 100644 --- a/docs/ru/sql-reference/statements/alter/view.md +++ b/docs/ru/sql-reference/statements/alter/view.md @@ -4,9 +4,9 @@ sidebar_position: 50 sidebar_label: VIEW --- -# Выражение ALTER TABLE … MODIFY QUERY {#alter-modify-query} +# Выражение ALTER TABLE ... MODIFY QUERY {#alter-modify-query} -Вы можете изменить запрос `SELECT`, который был задан при создании [материализованного представления](../create/view.md#materialized), с помощью запроса 'ALTER TABLE … MODIFY QUERY'. Используйте его если при создании материализованного представления не использовалась секция `TO [db.]name`. Настройка `allow_experimental_alter_materialized_view_structure` должна быть включена. +Вы можете изменить запрос `SELECT`, который был задан при создании [материализованного представления](../create/view.md#materialized), с помощью запроса 'ALTER TABLE ... MODIFY QUERY'. Используйте его если при создании материализованного представления не использовалась секция `TO [db.]name`. Настройка `allow_experimental_alter_materialized_view_structure` должна быть включена. Если при создании материализованного представления использовалась конструкция `TO [db.]name`, то для изменения отсоедините представление с помощью [DETACH](../detach.md), измените таблицу с помощью [ALTER TABLE](index.md), а затем снова присоедините запрос с помощью [ATTACH](../attach.md). diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md index 032bdc6e6d4..8fa30446bb3 100644 --- a/docs/ru/sql-reference/statements/create/view.md +++ b/docs/ru/sql-reference/statements/create/view.md @@ -60,7 +60,7 @@ AS SELECT ... Если указано `POPULATE`, то при создании представления в него будут добавлены данные, уже содержащиеся в исходной таблице, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Если `POPULATE` не указано, представление будет содержать только данные, добавленные в таблицу после создания представления. Использовать `POPULATE` не рекомендуется, так как в представление не попадут данные, добавляемые в таблицу во время создания представления. -Запрос `SELECT` может содержать `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Следует иметь ввиду, что соответствующие преобразования будут выполняться независимо, на каждый блок вставляемых данных. Например, при наличии `GROUP BY`, данные будут агрегироваться при вставке, но только в рамках одной пачки вставляемых данных. Далее, данные не будут доагрегированы. Исключение - использование ENGINE, производящего агрегацию данных самостоятельно, например, `SummingMergeTree`. +Запрос `SELECT` может содержать `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`... Следует иметь ввиду, что соответствующие преобразования будут выполняться независимо, на каждый блок вставляемых данных. Например, при наличии `GROUP BY`, данные будут агрегироваться при вставке, но только в рамках одной пачки вставляемых данных. Далее, данные не будут доагрегированы. Исключение - использование ENGINE, производящего агрегацию данных самостоятельно, например, `SummingMergeTree`. Выполнение запросов [ALTER](../../../sql-reference/statements/alter/view.md) над материализованными представлениями имеет свои особенности, поэтому эти запросы могут быть неудобными для использования. Если материализованное представление использует конструкцию `TO [db.]name`, то можно выполнить `DETACH` представления, `ALTER` для целевой таблицы и последующий `ATTACH` ранее отсоединенного (`DETACH`) представления. diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index 747e36b8809..309d4852b11 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -73,7 +73,7 @@ INSERT INTO insert_select_testtable VALUES (1, DEFAULT, 1) ; INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set ``` -Например, следующий формат запроса идентичен базовому варианту INSERT … VALUES: +Например, следующий формат запроса идентичен базовому варианту INSERT ... VALUES: ``` sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 5331cf00728..546a674d41a 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -116,7 +116,7 @@ SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UIn **Пример** -Запрос данных из файлов с именами `file000`, `file001`, … , `file999`: +Запрос данных из файлов с именами `file000`, `file001`, ... , `file999`: ``` sql SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, value UInt32'); diff --git a/docs/ru/sql-reference/table-functions/s3.md b/docs/ru/sql-reference/table-functions/s3.md index fe40cb0c507..2847a95bf19 100644 --- a/docs/ru/sql-reference/table-functions/s3.md +++ b/docs/ru/sql-reference/table-functions/s3.md @@ -108,7 +108,7 @@ FROM s3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefi Если список файлов содержит диапазоны чисел с ведущими нулями, используйте конструкцию с фигурными скобками для каждой цифры отдельно или используйте `?`. ::: -Подсчитаем общее количество строк в файлах с именами `file-000.csv`, `file-001.csv`, … , `file-999.csv`: +Подсчитаем общее количество строк в файлах с именами `file-000.csv`, `file-001.csv`, ... , `file-999.csv`: ``` sql SELECT count(*) diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index 7afcc07c6fb..c91d8bcf4d1 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -190,7 +190,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7" - 如果在获取系统数据时发生了zookeeper异常。副本,将其显示在单独的列中。 这实现了 [#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 原子删除destroy上的MergeTree数据部分。 [#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) - 支持分布式表的行级安全性。 [#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([伊万](https://github.com/abyss7)) -- Now we recognize suffix (like KB, KiB…) in settings values. [#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([米哈伊尔\*科罗托夫](https://github.com/millb)) +- Now we recognize suffix (like KB, KiB...) in settings values. [#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([米哈伊尔\*科罗托夫](https://github.com/millb)) - 在构建大型连接的结果时防止内存不足。 [#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) - 在交互模式下为建议添加群集名称 `clickhouse-client`. [#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries [#8820](https://github.com/ClickHouse/ClickHouse/pull/8820) ([伊万](https://github.com/abyss7)) @@ -523,7 +523,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7" - 现在后台在磁盘之间移动,运行它的seprate线程池。 [#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) - `SYSTEM RELOAD DICTIONARY` 现在同步执行。 [#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([维塔利\*巴拉诺夫](https://github.com/vitlibar)) - 堆栈跟踪现在显示物理地址(对象文件中的偏移量),而不是虚拟内存地址(加载对象文件的位置)。 这允许使用 `addr2line` 当二进制独立于位置并且ASLR处于活动状态时。 这修复 [#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 支持行级安全筛选器的新语法: `…
`. 修复 [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([伊万](https://github.com/abyss7)) +- 支持行级安全筛选器的新语法: `...
`. 修复 [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([伊万](https://github.com/abyss7)) - 现在 `cityHash` 功能可以与工作 `Decimal` 和 `UUID` 类型。 修复 [#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([米哈伊尔\*科罗托夫](https://github.com/millb)) - 从系统日志中删除了固定的索引粒度(它是1024),因为它在实现自适应粒度之后已经过时。 [#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 当ClickHouse在没有SSL的情况下编译时,启用MySQL兼容服务器。 [#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([尤里\*巴拉诺夫](https://github.com/yurriy)) diff --git a/docs/zh/development/style.md b/docs/zh/development/style.md index c0a08291e02..724b22ad461 100644 --- a/docs/zh/development/style.md +++ b/docs/zh/development/style.md @@ -53,7 +53,7 @@ memcpy(&buf[place_value], &x, sizeof(x)); for (size_t i = 0; i < rows; i += storage.index_granularity) ``` -**7.** 在二元运算符(`+`,`-`,`*`,`/`,`%`,…)和三元运算符 `?:` 周围添加空格。 +**7.** 在二元运算符(`+`,`-`,`*`,`/`,`%`,...)和三元运算符 `?:` 周围添加空格。 ``` cpp UInt16 year = (s[0] - '0') * 1000 + (s[1] - '0') * 100 + (s[2] - '0') * 10 + (s[3] - '0'); @@ -82,7 +82,7 @@ dst.ClickGoodEvent = click.GoodEvent; 如有必要,运算符可以包裹到下一行。 在这种情况下,它前面的偏移量增加。 -**11.** 不要使用空格来分开一元运算符 (`--`, `++`, `*`, `&`, …) 和参数。 +**11.** 不要使用空格来分开一元运算符 (`--`, `++`, `*`, `&`, ...) 和参数。 **12.** 在逗号后面加一个空格,而不是在之前。同样的规则也适合 `for` 循环中的分号。 @@ -111,7 +111,7 @@ public: **16.** 如果对整个文件使用相同的 `namespace`,并且没有其他重要的东西,则 `namespace` 中不需要偏移量。 -**17.** 在 `if`, `for`, `while` 中包裹的代码块中,若代码是一个单行的 `statement`,那么大括号是可选的。 可以将 `statement` 放到一行中。这个规则同样适用于嵌套的 `if`, `for`, `while`, … +**17.** 在 `if`, `for`, `while` 中包裹的代码块中,若代码是一个单行的 `statement`,那么大括号是可选的。 可以将 `statement` 放到一行中。这个规则同样适用于嵌套的 `if`, `for`, `while`, ... 但是如果内部 `statement` 包含大括号或 `else`,则外部块应该用大括号括起来。 @@ -262,7 +262,7 @@ void executeQuery( 这个示例来源于 http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/。 -**7.** 不要在每个文件的开头写入垃圾注释(作者,创建日期…)。 +**7.** 不要在每个文件的开头写入垃圾注释(作者,创建日期...)。 **8.** 单行注释用三个斜杆: `///` ,多行注释以 `/**`开始。 这些注释会当做文档。 diff --git a/docs/zh/engines/table-engines/integrations/hdfs.md b/docs/zh/engines/table-engines/integrations/hdfs.md index 55648afe407..be673b6ce92 100644 --- a/docs/zh/engines/table-engines/integrations/hdfs.md +++ b/docs/zh/engines/table-engines/integrations/hdfs.md @@ -103,7 +103,7 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = HDFS('hdfs **示例** -创建具有名为文件的表 `file000`, `file001`, … , `file999`: +创建具有名为文件的表 `file000`, `file001`, ... , `file999`: ``` sql CREARE TABLE big_table (name String, value UInt32) ENGINE = HDFS('hdfs://hdfs1:9000/big_dir/file{0..9}{0..9}{0..9}', 'CSV') diff --git a/docs/zh/engines/table-engines/integrations/s3.md b/docs/zh/engines/table-engines/integrations/s3.md index f2585decabf..f18814675c3 100644 --- a/docs/zh/engines/table-engines/integrations/s3.md +++ b/docs/zh/engines/table-engines/integrations/s3.md @@ -109,7 +109,7 @@ CREATE TABLE table_with_asterisk (name String, value UInt32) ENGINE = S3('https: **示例** -使用文件`file-000.csv`, `file-001.csv`, … , `file-999.csv`来创建表: +使用文件`file-000.csv`, `file-001.csv`, ... , `file-999.csv`来创建表: ``` sql CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV'); @@ -202,7 +202,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p !!! warning "Warning" 如果文件列表中包含有从0开头的数字范围,请对每个数字分别使用带括号的结构,或者使用`?`. -4. 从文件`file-000.csv`, `file-001.csv`, … , `file-999.csv`创建表: +4. 从文件`file-000.csv`, `file-001.csv`, ... , `file-999.csv`创建表: ``` sql CREATE TABLE big_table (name String, value UInt32) diff --git a/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md index 4fecf4e5669..e283a4c7510 100644 --- a/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/zh/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -59,7 +59,7 @@ WHERE table = 'visits' └───────────┴────────────────┴────────┘ ``` -`partition` 列存储分区的名称。此示例中有两个分区:`201901` 和 `201902`。在 [ALTER … PARTITION](#alter_manipulations-with-partitions) 语句中你可以使用该列值来指定分区名称。 +`partition` 列存储分区的名称。此示例中有两个分区:`201901` 和 `201902`。在 [ALTER ... PARTITION](#alter_manipulations-with-partitions) 语句中你可以使用该列值来指定分区名称。 `name` 列为分区中数据片段的名称。在 [ALTER ATTACH PART](#alter_attach-partition) 语句中你可以使用此列值中来指定片段名称。 diff --git a/docs/zh/engines/table-engines/mergetree-family/mergetree.md b/docs/zh/engines/table-engines/mergetree-family/mergetree.md index bfa69338657..67bd681269b 100644 --- a/docs/zh/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/zh/engines/table-engines/mergetree-family/mergetree.md @@ -702,7 +702,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' - 插入(`INSERT`查询) - 后台合并和[数据变异](../../../sql-reference/statements/alter.md#alter-mutations) - 从另一个副本下载 -- [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter.md#alter_freeze-partition) 冻结分区 +- [ALTER TABLE ... FREEZE PARTITION](../../../sql-reference/statements/alter.md#alter_freeze-partition) 冻结分区 除了数据变异和冻结分区以外的情况下,数据按照以下逻辑存储到卷或磁盘上: @@ -713,7 +713,7 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' 在后台,数据片段基于剩余空间(`move_factor`参数)根据卷在配置文件中定义的顺序进行转移。数据永远不会从最后一个移出也不会从第一个移入。可以通过系统表 [system.part_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (字段 `type = MOVE_PART`) 和 [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (字段 `path` 和 `disk`) 来监控后台的移动情况。具体细节可以通过服务器日志查看。 -用户可以通过 [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。 +用户可以通过 [ALTER TABLE ... MOVE PART\|PARTITION ... TO VOLUME\|DISK ...](../../../sql-reference/statements/alter.md#alter_move-partition) 强制移动一个数据片段或分区到另外一个卷,所有后台移动的限制都会被考虑在内。这个查询会自行启动,无需等待后台操作完成。如果没有足够的可用空间或任何必须条件没有被满足,用户会收到报错信息。 数据移动不会妨碍到数据复制。也就是说,同一张表的不同副本可以指定不同的存储策略。 diff --git a/docs/zh/engines/table-engines/special/external-data.md b/docs/zh/engines/table-engines/special/external-data.md index 688e25402ab..06c6331b4f3 100644 --- a/docs/zh/engines/table-engines/special/external-data.md +++ b/docs/zh/engines/table-engines/special/external-data.md @@ -26,7 +26,7 @@ ClickHouse 允许向服务器发送处理查询所需的数据以及 SELECT 查 以下的参数是可选的:**–name** – 表的名称,如果省略,则采用 _data。 **–format** – 文件中的数据格式。 如果省略,则使用 TabSeparated。 -以下的参数必选一个:**–types** – 逗号分隔列类型的列表。例如:`UInt64,String`。列将被命名为 _1,_2,… +以下的参数必选一个:**–types** – 逗号分隔列类型的列表。例如:`UInt64,String`。列将被命名为 _1,_2,... **–structure**– 表结构的格式 `UserID UInt64`,`URL String`。定义列的名字以及类型。 在 «file» 中指定的文件将由 «format» 中指定的格式解析,使用在 «types» 或 «structure» 中指定的数据类型。该表将被上传到服务器,并在作为名称为 «name»临时表。 diff --git a/docs/zh/faq/general/olap.md b/docs/zh/faq/general/olap.md index b014419578b..c4b36b138fa 100644 --- a/docs/zh/faq/general/olap.md +++ b/docs/zh/faq/general/olap.md @@ -10,13 +10,13 @@ sidebar_position: 100 [OLAP](https://en.wikipedia.org/wiki/Online_analytical_processing) stands for Online Analytical Processing. It is a broad term that can be looked at from two perspectives: technical and business. But at the very high level, you can just read these words backward: Processing -: Some source data is processed… +: Some source data is processed... Analytical -: …to produce some analytical reports and insights… +: ...to produce some analytical reports and insights... Online -: …in real-time. +: ...in real-time. ## OLAP from the Business Perspective {#olap-from-the-business-perspective} diff --git a/docs/zh/getting-started/example-datasets/nyc-taxi.md b/docs/zh/getting-started/example-datasets/nyc-taxi.md index 9c487140df3..ceeb6fbb9e0 100644 --- a/docs/zh/getting-started/example-datasets/nyc-taxi.md +++ b/docs/zh/getting-started/example-datasets/nyc-taxi.md @@ -196,7 +196,7 @@ real 75m56.214s (也可以直接使用`COPY ... TO PROGRAM`从Postgres中导入数据) -数据中所有与天气相关的字段(precipitation……average_wind_speed)都填充了NULL。 所以,我们将从最终数据集中删除它们 +数据中所有与天气相关的字段(precipitation...average_wind_speed)都填充了NULL。 所以,我们将从最终数据集中删除它们 首先,我们使用单台服务器创建表,后面我们将在多台节点上创建这些表。 diff --git a/docs/zh/getting-started/example-datasets/uk-price-paid.mdx b/docs/zh/getting-started/example-datasets/uk-price-paid.mdx index ecfdcddbbe2..7d4c299b919 100644 --- a/docs/zh/getting-started/example-datasets/uk-price-paid.mdx +++ b/docs/zh/getting-started/example-datasets/uk-price-paid.mdx @@ -212,7 +212,7 @@ ORDER BY year └──────┴─────────┴───────────────────────────────────────────────────────┘ ``` -2020 年房价出事了!但这并不令人意外…… +2020 年房价出事了!但这并不令人意外... ### 查询 3. 最昂贵的社区 {#most-expensive-neighborhoods} diff --git a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md index 758992e4084..975d5eb764c 100644 --- a/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md +++ b/docs/zh/guides/improving-query-performance/sparse-primary-indexes.md @@ -371,7 +371,7 @@ UserID.bin,URL.bin,和EventTime.bin是UserID :::note - 最后一个索引条目(上图中的“mark 1082”)存储了上图中颗粒1082的主键列的最大值。 -- 索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。例如,对于上图中的索引条目‘mark 0’,在我们的表中没有UserID为240.923且URL为“goal://metry=10000467796a411…”的行,相反,对于该表,有一个颗粒0,在该颗粒中,最小UserID值是240.923,最小URL值是“goal://metry=10000467796a411…”,这两个值来自不同的行。 +- 索引条目(索引标记)不是基于表中的特定行,而是基于颗粒。例如,对于上图中的索引条目‘mark 0’,在我们的表中没有UserID为240.923且URL为“goal://metry=10000467796a411...”的行,相反,对于该表,有一个颗粒0,在该颗粒中,最小UserID值是240.923,最小URL值是“goal://metry=10000467796a411...”,这两个值来自不同的行。 - 主索引文件完全加载到主内存中。如果文件大于可用的空闲内存空间,则ClickHouse将发生错误。 ::: diff --git a/docs/zh/index.md b/docs/zh/index.md index fab00dbcd1b..ec4b6dce1f8 100644 --- a/docs/zh/index.md +++ b/docs/zh/index.md @@ -16,7 +16,7 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) | #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | | #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | | #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| #N | … | … | … | … | … | +| #N | ... | ... | ... | ... | ... | 处于同一行中的数据总是被物理的存储在一起。 @@ -26,11 +26,11 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) | Row: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| -| WatchID: | 89354350662 | 90329509958 | 89953706054 | … | -| JavaEnable: | 1 | 0 | 1 | … | -| Title: | Investor Relations | Contact us | Mission | … | -| GoodEvent: | 1 | 1 | 1 | … | -| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | … | +| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... | +| JavaEnable: | 1 | 0 | 1 | ... | +| Title: | Investor Relations | Contact us | Mission | ... | +| GoodEvent: | 1 | 1 | 1 | ... | +| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | ... | 这些示例只显示了数据的排列顺序。来自不同列的值被单独存储,来自同一列的数据被存储在一起。 diff --git a/docs/zh/operations/settings/query-complexity.md b/docs/zh/operations/settings/query-complexity.md index 124d5fa5d1a..b1b5ca75018 100644 --- a/docs/zh/operations/settings/query-complexity.md +++ b/docs/zh/operations/settings/query-complexity.md @@ -196,7 +196,7 @@ Restrictions on the «maximum amount of something» can take the value 0, which Limits the number of rows in the hash table that is used when joining tables. -This settings applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and the [Join](../../engines/table-engines/special/join.md) table engine. +This settings applies to [SELECT ... JOIN](../../sql-reference/statements/select/join.md#select-join) operations and the [Join](../../engines/table-engines/special/join.md) table engine. If a query contains multiple joins, ClickHouse checks this setting for every intermediate result. @@ -213,7 +213,7 @@ Default value: 0. Limits the size in bytes of the hash table used when joining tables. -This settings applies to [SELECT … JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md). +This settings applies to [SELECT ... JOIN](../../sql-reference/statements/select/join.md#select-join) operations and [Join table engine](../../engines/table-engines/special/join.md). If the query contains joins, ClickHouse checks this setting for every intermediate result. diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index c3b4194ed44..5e59196f56c 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -1002,7 +1002,7 @@ ClickHouse生成异常 ## count_distinct_implementation {#settings-count_distinct_implementation} -指定其中的 `uniq*` 函数应用于执行 [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) 建筑。 +指定其中的 `uniq*` 函数应用于执行 [COUNT(DISTINCT ...)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) 建筑。 可能的值: diff --git a/docs/zh/operations/system-tables/dictionaries.md b/docs/zh/operations/system-tables/dictionaries.md index 0cf91e45e86..c7b1bdd04be 100644 --- a/docs/zh/operations/system-tables/dictionaries.md +++ b/docs/zh/operations/system-tables/dictionaries.md @@ -21,7 +21,7 @@ machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 - `FAILED_AND_RELOADING` — Could not load the dictionary as a result of an error and is loading now. - `origin` ([字符串](../../sql-reference/data-types/string.md)) — Path to the configuration file that describes the dictionary. - `type` ([字符串](../../sql-reference/data-types/string.md)) — Type of dictionary allocation. [在内存中存储字典](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md). -- `key` — [密钥类型](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key):数字键 ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) or Сomposite key ([字符串](../../sql-reference/data-types/string.md)) — form “(type 1, type 2, …, type n)”. +- `key` — [密钥类型](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-key):数字键 ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) or Сomposite key ([字符串](../../sql-reference/data-types/string.md)) — form “(type 1, type 2, ..., type n)”. - `attribute.names` ([阵列](../../sql-reference/data-types/array.md)([字符串](../../sql-reference/data-types/string.md))) — Array of [属性名称](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) 由字典提供。 - `attribute.types` ([阵列](../../sql-reference/data-types/array.md)([字符串](../../sql-reference/data-types/string.md))) — Corresponding array of [属性类型](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md#ext_dict_structure-attributes) 这是由字典提供。 - `bytes_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Amount of RAM allocated for the dictionary. diff --git a/docs/zh/sql-reference/aggregate-functions/parametric-functions.md b/docs/zh/sql-reference/aggregate-functions/parametric-functions.md index cb1dcc35f5c..27d3375aebb 100644 --- a/docs/zh/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/zh/sql-reference/aggregate-functions/parametric-functions.md @@ -80,7 +80,7 @@ FROM 在这种情况下,您应该记住您不知道直方图bin边界。 -## sequenceMatch(pattern)(timestamp, cond1, cond2, …) {#function-sequencematch} +## sequenceMatch(pattern)(timestamp, cond1, cond2, ...) {#function-sequencematch} 检查序列是否包含与模式匹配的事件链。 @@ -167,7 +167,7 @@ SELECT sequenceMatch('(?1)(?2)')(time, number = 1, number = 2, number = 4) FROM - [sequenceCount](#function-sequencecount) -## sequenceCount(pattern)(time, cond1, cond2, …) {#function-sequencecount} +## sequenceCount(pattern)(time, cond1, cond2, ...) {#function-sequencecount} 计算与模式匹配的事件链的数量。该函数搜索不重叠的事件链。当前链匹配后,它开始搜索下一个链。 diff --git a/docs/zh/sql-reference/aggregate-functions/reference/quantiles.md b/docs/zh/sql-reference/aggregate-functions/reference/quantiles.md index 4dce65af1ed..253eb9ef82d 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/quantiles.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/quantiles.md @@ -7,7 +7,7 @@ sidebar_position: 201 **语法** ``` sql -quantiles(level1, level2, …)(x) +quantiles(level1, level2, ...)(x) ``` 所有分位数函数(quantile)也有相应的分位数(quantiles)函数: `quantiles`, `quantilesDeterministic`, `quantilesTiming`, `quantilesTimingWeighted`, `quantilesExact`, `quantilesExactWeighted`, `quantilesTDigest`。 这些函数一次计算所列的级别的所有分位数, 并返回结果值的数组。 diff --git a/docs/zh/sql-reference/data-types/aggregatefunction.md b/docs/zh/sql-reference/data-types/aggregatefunction.md index e8f28b367a5..80648eb165b 100644 --- a/docs/zh/sql-reference/data-types/aggregatefunction.md +++ b/docs/zh/sql-reference/data-types/aggregatefunction.md @@ -1,7 +1,7 @@ --- slug: /zh/sql-reference/data-types/aggregatefunction --- -# AggregateFunction(name, types_of_arguments…) {#data-type-aggregatefunction} +# AggregateFunction(name, types_of_arguments...) {#data-type-aggregatefunction} 聚合函数的中间状态,可以通过聚合函数名称加`-State`后缀的形式得到它。与此同时,当您需要访问该类型的最终状态数据时,您需要以相同的聚合函数名加`-Merge`后缀的形式来得到最终状态数据。 diff --git a/docs/zh/sql-reference/data-types/domains/index.md b/docs/zh/sql-reference/data-types/domains/index.md index c123b10f6fe..9f12018732b 100644 --- a/docs/zh/sql-reference/data-types/domains/index.md +++ b/docs/zh/sql-reference/data-types/domains/index.md @@ -19,9 +19,9 @@ Domain类型是特定实现的类型,它总是与某个现存的基础类型 ### Domains的额外特性 {#domainsde-e-wai-te-xing} - 在执行SHOW CREATE TABLE 或 DESCRIBE TABLE时,其对应的列总是展示为Domain类型的名称 -- 在INSERT INTO domain_table(domain_column) VALUES(…)中输入数据总是以更人性化的格式进行输入 +- 在INSERT INTO domain_table(domain_column) VALUES(...)中输入数据总是以更人性化的格式进行输入 - 在SELECT domain_column FROM domain_table中数据总是以更人性化的格式输出 -- 在INSERT INTO domain_table FORMAT CSV …中,实现外部源数据以更人性化的格式载入 +- 在INSERT INTO domain_table FORMAT CSV ...中,实现外部源数据以更人性化的格式载入 ### Domains类型的限制 {#domainslei-xing-de-xian-zhi} diff --git a/docs/zh/sql-reference/data-types/fixedstring.md b/docs/zh/sql-reference/data-types/fixedstring.md index 633307938a9..d454e935fe7 100644 --- a/docs/zh/sql-reference/data-types/fixedstring.md +++ b/docs/zh/sql-reference/data-types/fixedstring.md @@ -18,8 +18,8 @@ slug: /zh/sql-reference/data-types/fixedstring 可以有效存储在`FixedString`类型的列中的值的示例: - 二进制表示的IP地址(IPv6使用`FixedString(16)`) -- 语言代码(ru_RU, en_US … ) -- 货币代码(USD, RUB … ) +- 语言代码(ru_RU, en_US ... ) +- 货币代码(USD, RUB ... ) - 二进制表示的哈希值(MD5使用`FixedString(16)`,SHA256使用`FixedString(32)`) 请使用[UUID](uuid.md)数据类型来存储UUID值,。 diff --git a/docs/zh/sql-reference/data-types/nested-data-structures/nested.md b/docs/zh/sql-reference/data-types/nested-data-structures/nested.md index 5ef8256b483..57b30de0881 100644 --- a/docs/zh/sql-reference/data-types/nested-data-structures/nested.md +++ b/docs/zh/sql-reference/data-types/nested-data-structures/nested.md @@ -1,7 +1,7 @@ --- slug: /zh/sql-reference/data-types/nested-data-structures/nested --- -# Nested(Name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2} +# Nested(Name1 Type1, Name2 Type2, ...) {#nestedname1-type1-name2-type2} 嵌套数据结构类似于嵌套表。嵌套数据结构的参数(列名和类型)与 CREATE 查询类似。每个表可以包含任意多行嵌套数据结构。 diff --git a/docs/zh/sql-reference/data-types/simpleaggregatefunction.md b/docs/zh/sql-reference/data-types/simpleaggregatefunction.md index 601cb602a78..fbaa76365ec 100644 --- a/docs/zh/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/zh/sql-reference/data-types/simpleaggregatefunction.md @@ -3,7 +3,7 @@ slug: /zh/sql-reference/data-types/simpleaggregatefunction --- # SimpleAggregateFunction {#data-type-simpleaggregatefunction} -`SimpleAggregateFunction(name, types_of_arguments…)` 数据类型存储聚合函数的当前值, 并不像 [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md) 那样存储其全部状态。这种优化可以应用于具有以下属性函数: 将函数 `f` 应用于行集合 `S1 UNION ALL S2` 的结果,可以通过将 `f` 分别应用于行集合的部分, 然后再将 `f` 应用于结果来获得: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`。 这个属性保证了部分聚合结果足以计算出合并的结果,所以我们不必存储和处理任何额外的数据。 +`SimpleAggregateFunction(name, types_of_arguments...)` 数据类型存储聚合函数的当前值, 并不像 [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md) 那样存储其全部状态。这种优化可以应用于具有以下属性函数: 将函数 `f` 应用于行集合 `S1 UNION ALL S2` 的结果,可以通过将 `f` 分别应用于行集合的部分, 然后再将 `f` 应用于结果来获得: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`。 这个属性保证了部分聚合结果足以计算出合并的结果,所以我们不必存储和处理任何额外的数据。 支持以下聚合函数: diff --git a/docs/zh/sql-reference/data-types/tuple.md b/docs/zh/sql-reference/data-types/tuple.md index 004c80ff916..38813701c70 100644 --- a/docs/zh/sql-reference/data-types/tuple.md +++ b/docs/zh/sql-reference/data-types/tuple.md @@ -1,7 +1,7 @@ --- slug: /zh/sql-reference/data-types/tuple --- -# Tuple(T1, T2, …) {#tuplet1-t2} +# Tuple(T1, T2, ...) {#tuplet1-t2} 元组,其中每个元素都有单独的 [类型](index.md#data_types)。 diff --git a/docs/zh/sql-reference/functions/array-functions.md b/docs/zh/sql-reference/functions/array-functions.md index d150b94b8af..69db34e4a36 100644 --- a/docs/zh/sql-reference/functions/array-functions.md +++ b/docs/zh/sql-reference/functions/array-functions.md @@ -152,7 +152,7 @@ SELECT range(5), range(1, 5), range(1, 5, 2), range(-1, 5, 2); └─────────────┴─────────────┴────────────────┴─────────────────┘ ``` -## array(x1, …), operator \[x1, …\] {#arrayx1-operator-x1} +## array(x1, ...), operator \[x1, ...\] {#arrayx1-operator-x1} 使用函数的参数作为数组元素创建一个数组。 参数必须是常量,并且具有最小公共类型的类型。必须至少传递一个参数,否则将不清楚要创建哪种类型的数组。也就是说,你不能使用这个函数来创建一个空数组(为此,使用上面描述的’emptyArray  \*’函数)。 @@ -337,7 +337,7 @@ SELECT indexOf([1, 3, NULL, NULL], NULL) 设置为«NULL»的元素将作为普通的元素值处理。 -## arrayCount(\[func,\] arr1, …) {#array-count} +## arrayCount(\[func,\] arr1, ...) {#array-count} `func`将arr数组作为参数,其返回结果为非零值的数量。如果未指定“func”,则返回数组中非零元素的数量。 @@ -363,7 +363,7 @@ SELECT countEqual([1, 2, NULL, NULL], NULL) ## arrayEnumerate(arr) {#array_functions-arrayenumerate} -返回 Array \[1, 2, 3, …, length (arr) \] +返回 Array \[1, 2, 3, ..., length (arr) \] 此功能通常与ARRAY JOIN一起使用。它允许在应用ARRAY JOIN后为每个数组计算一次。例如: @@ -403,7 +403,7 @@ WHERE (CounterID = 160656) AND notEmpty(GoalsReached) 此功能也可用于高阶函数。例如,您可以使用它来获取与条件匹配的元素的数组索引。 -## arrayEnumerateUniq(arr, …) {#arrayenumerateuniqarr} +## arrayEnumerateUniq(arr, ...) {#arrayenumerateuniqarr} 返回与源数组大小相同的数组,其中每个元素表示与其下标对应的源数组元素在源数组中出现的次数。 例如:arrayEnumerateUniq( \[10,20,10,30 \])=  \[1,1,2,1 \]。 @@ -621,7 +621,7 @@ SELECT arraySlice([1, 2, NULL, 4, 5], 2, 3) AS res 设置为«NULL»的数组元素作为普通的数组元素值处理。 -## arraySort(\[func,\] arr, …) {#array_functions-reverse-sort} +## arraySort(\[func,\] arr, ...) {#array_functions-reverse-sort} 以升序对`arr`数组的元素进行排序。如果指定了`func`函数,则排序顺序由`func`函数的调用结果决定。如果`func`接受多个参数,那么`arraySort`函数也将解析与`func`函数参数相同数量的数组参数。更详细的示例在`arraySort`的末尾。 @@ -721,7 +721,7 @@ SELECT arraySort((x, y) -> -y, [0, 1, 2], [1, 2, 3]) as res; !!! 注意 "注意" 为了提高排序效率, 使用了[施瓦茨变换](https://en.wikipedia.org/wiki/Schwartzian_transform)。 -## arrayReverseSort(\[func,\] arr, …) {#array_functions-reverse-sort} +## arrayReverseSort(\[func,\] arr, ...) {#array_functions-reverse-sort} 以降序对`arr`数组的元素进行排序。如果指定了`func`函数,则排序顺序由`func`函数的调用结果决定。如果`func`接受多个参数,那么`arrayReverseSort`函数也将解析与`func`函数参数相同数量的数组作为参数。更详细的示例在`arrayReverseSort`的末尾。 @@ -822,7 +822,7 @@ SELECT arrayReverseSort((x, y) -> -y, [4, 3, 5], [1, 2, 3]) AS res; └─────────┘ ``` -## arrayUniq(arr, …) {#arrayuniqarr} +## arrayUniq(arr, ...) {#arrayuniqarr} 如果传递一个参数,则计算数组中不同元素的数量。 如果传递了多个参数,则它计算多个数组中相应位置的不同元素元组的数量。 @@ -1221,7 +1221,7 @@ select arrayAUC([0.1, 0.4, 0.35, 0.8], [0, 0, 1, 1]); └───────────────────────────────────────────────┘ ``` -## arrayMap(func, arr1, …) {#array-map} +## arrayMap(func, arr1, ...) {#array-map} 将从 `func` 函数的原始应用中获得的数组返回给 `arr` 数组中的每个元素。 @@ -1251,7 +1251,7 @@ SELECT arrayMap((x, y) -> (x, y), [1, 2, 3], [4, 5, 6]) AS res 请注意,`arrayMap` 是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。 您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayFilter(func, arr1, …) {#array-filter} +## arrayFilter(func, arr1, ...) {#array-filter} 返回一个仅包含 `arr1` 中的元素的数组,其中 `func` 返回的值不是 0。 @@ -1284,7 +1284,7 @@ SELECT 请注意,`arrayFilter`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。 您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayFill(func, arr1, …) {#array-fill} +## arrayFill(func, arr1, ...) {#array-fill} 从第一个元素到最后一个元素扫描`arr1`,如果`func`返回0,则用`arr1[i - 1]`替换`arr1[i]`。`arr1`的第一个元素不会被替换。 @@ -1302,7 +1302,7 @@ SELECT arrayFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 6, 14, 请注意,`arrayFill` 是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。 您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayReverseFill(func, arr1, …) {#array-reverse-fill} +## arrayReverseFill(func, arr1, ...) {#array-reverse-fill} 从最后一个元素到第一个元素扫描`arr1`,如果`func`返回0,则用`arr1[i + 1]`替换`arr1[i]`。`arr1`的最后一个元素不会被替换。 @@ -1320,7 +1320,7 @@ SELECT arrayReverseFill(x -> not isNull(x), [1, null, 3, 11, 12, null, null, 5, 请注意,`arrayReverseFill`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。 您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arraySplit(func, arr1, …) {#array-split} +## arraySplit(func, arr1, ...) {#array-split} 将 `arr1` 拆分为多个数组。当 `func` 返回 0 以外的值时,数组将在元素的左侧拆分。数组不会在第一个元素之前被拆分。 @@ -1338,7 +1338,7 @@ SELECT arraySplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res 请注意,`arraySplit`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。 您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayReverseSplit(func, arr1, …) {#array-reverse-split} +## arrayReverseSplit(func, arr1, ...) {#array-reverse-split} 将 `arr1` 拆分为多个数组。当 `func` 返回 0 以外的值时,数组将在元素的右侧拆分。数组不会在最后一个元素之后被拆分。 @@ -1356,37 +1356,37 @@ SELECT arrayReverseSplit((x, y) -> y, [1, 2, 3, 4, 5], [1, 0, 0, 1, 0]) AS res 请注意,`arrayReverseSplit`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。 您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1} +## arrayExists(\[func,\] arr1, ...) {#arrayexistsfunc-arr1} 如果 `arr` 中至少有一个元素 `func` 返回 0 以外的值,则返回 1。否则,它返回 0。 请注意,`arrayExists`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。您可以将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1} +## arrayAll(\[func,\] arr1, ...) {#arrayallfunc-arr1} 如果 `func` 为 `arr` 中的所有元素返回 0 以外的值,则返回 1。否则,它返回 0。 请注意,`arrayAll`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。您可以将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayFirst(func, arr1, …) {#array-first} +## arrayFirst(func, arr1, ...) {#array-first} 返回 `arr1` 数组中 `func` 返回非 0 的值的第一个元素。 请注意,`arrayFirst`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayLast(func, arr1, …) {#array-last} +## arrayLast(func, arr1, ...) {#array-last} 返回 `arr1` 数组中的最后一个元素,其中 `func` 返回的值不是 0。 请注意,`arrayLast`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayFirstIndex(func, arr1, …) {#array-first-index} +## arrayFirstIndex(func, arr1, ...) {#array-first-index} 返回 `arr1` 数组中第一个元素的索引,其中 `func` 返回的值不是 0。 请注意,`arrayFirstIndex`是一个[高阶函数](../../sql-reference/functions/index.md#higher-order-functions)。您必须将 lambda 函数作为第一个参数传递给它,并且不能省略。 -## arrayLastIndex(func, arr1, …) {#array-last-index} +## arrayLastIndex(func, arr1, ...) {#array-last-index} 返回 `arr1` 数组中最后一个元素的索引,其中 `func` 返回的值不是 0。 @@ -1612,7 +1612,7 @@ SELECT arrayAvg(x -> (x * x), [2, 4]) AS res; └─────┘ ``` -## arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1} +## arrayCumSum(\[func,\] arr1, ...) {#arraycumsumfunc-arr1} 返回源数组中元素的部分和的数组(运行总和)。如果指定了 func 函数,则数组元素的值在求和之前由该函数转换。 diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index d6493ffe605..18b9f3495c0 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -443,7 +443,7 @@ SELECT toStartOfSecond(dt64, 'Asia/Istanbul'); `toISOWeek()`是一个兼容函数,等效于`toWeek(date,3)`。 下表描述了mode参数的工作方式。 -| Mode | First day of week | Range | Week 1 is the first week … | +| Mode | First day of week | Range | Week 1 is the first week ... | |------|-------------------|-------|-------------------------------| | 0 | Sunday | 0-53 | with a Sunday in this year | | 1 | Monday | 0-53 | with 4 or more days this year | diff --git a/docs/zh/sql-reference/functions/higher-order-functions.md b/docs/zh/sql-reference/functions/higher-order-functions.md index 929dc6f3ea7..0e08f88bba1 100644 --- a/docs/zh/sql-reference/functions/higher-order-functions.md +++ b/docs/zh/sql-reference/functions/higher-order-functions.md @@ -15,13 +15,13 @@ slug: /zh/sql-reference/functions/higher-order-functions 除了’arrayMap’和’arrayFilter’以外的所有其他函数,都可以省略第一个参数(lambda函数)。在这种情况下,默认返回数组元素本身。 -### arrayMap(func, arr1, …) {#higher_order_functions-array-map} +### arrayMap(func, arr1, ...) {#higher_order_functions-array-map} 将arr 将从’func’函数的原始应用程序获得的数组返回到’arr’数组中的每个元素。 返回从原始应用程序获得的数组 ‘func’ 函数中的每个元素 ‘arr’ 阵列。 -### arrayFilter(func, arr1, …) {#arrayfilterfunc-arr1} +### arrayFilter(func, arr1, ...) {#arrayfilterfunc-arr1} 返回一个仅包含以下元素的数组 ‘arr1’ 对于哪个 ‘func’ 返回0以外的内容。 @@ -48,31 +48,31 @@ SELECT │ [2] │ └─────┘ -### arrayCount(\[func,\] arr1, …) {#arraycountfunc-arr1} +### arrayCount(\[func,\] arr1, ...) {#arraycountfunc-arr1} 返回数组arr中非零元素的数量,如果指定了’func’,则通过’func’的返回值确定元素是否为非零元素。 -### arrayExists(\[func,\] arr1, …) {#arrayexistsfunc-arr1} +### arrayExists(\[func,\] arr1, ...) {#arrayexistsfunc-arr1} 返回数组’arr’中是否存在非零元素,如果指定了’func’,则使用’func’的返回值确定元素是否为非零元素。 -### arrayAll(\[func,\] arr1, …) {#arrayallfunc-arr1} +### arrayAll(\[func,\] arr1, ...) {#arrayallfunc-arr1} 返回数组’arr’中是否存在为零的元素,如果指定了’func’,则使用’func’的返回值确定元素是否为零元素。 -### arraySum(\[func,\] arr1, …) {#arraysumfunc-arr1} +### arraySum(\[func,\] arr1, ...) {#arraysumfunc-arr1} 计算arr数组的总和,如果指定了’func’,则通过’func’的返回值计算数组的总和。 -### arrayFirst(func, arr1, …) {#arrayfirstfunc-arr1} +### arrayFirst(func, arr1, ...) {#arrayfirstfunc-arr1} 返回数组中第一个匹配的元素,函数使用’func’匹配所有元素,直到找到第一个匹配的元素。 -### arrayFirstIndex(func, arr1, …) {#arrayfirstindexfunc-arr1} +### arrayFirstIndex(func, arr1, ...) {#arrayfirstindexfunc-arr1} 返回数组中第一个匹配的元素的下标索引,函数使用’func’匹配所有元素,直到找到第一个匹配的元素。 -### arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1} +### arrayCumSum(\[func,\] arr1, ...) {#arraycumsumfunc-arr1} 返回源数组部分数据的总和,如果指定了`func`函数,则使用`func`的返回值计算总和。 @@ -98,7 +98,7 @@ SELECT arrayCumSumNonNegative([1, 1, -4, 1]) AS res │ [1,2,0,1] │ └───────────┘ -### arraySort(\[func,\] arr1, …) {#arraysortfunc-arr1} +### arraySort(\[func,\] arr1, ...) {#arraysortfunc-arr1} 返回升序排序`arr1`的结果。如果指定了`func`函数,则排序顺序由`func`的结果决定。 @@ -124,7 +124,7 @@ SELECT arraySort([1, nan, 2, NULL, 3, nan, 4, NULL]) │ [1,2,3,4,nan,nan,NULL,NULL] │ └───────────────────────────────────────────────┘ -### arrayReverseSort(\[func,\] arr1, …) {#arrayreversesortfunc-arr1} +### arrayReverseSort(\[func,\] arr1, ...) {#arrayreversesortfunc-arr1} 返回降序排序`arr1`的结果。如果指定了`func`函数,则排序顺序由`func`的结果决定。 diff --git a/docs/zh/sql-reference/functions/in-functions.md b/docs/zh/sql-reference/functions/in-functions.md index 346e076310e..9858159a495 100644 --- a/docs/zh/sql-reference/functions/in-functions.md +++ b/docs/zh/sql-reference/functions/in-functions.md @@ -10,10 +10,10 @@ sidebar_label: IN 运算符 请参阅[IN 运算符](../../sql-reference/operators/in.md#select-in-operators)部分。 -## tuple(x, y, …), 运算符 (x, y, …) {#tuplex-y-operator-x-y} +## tuple(x, y, ...), 运算符 (x, y, ...) {#tuplex-y-operator-x-y} 函数用于对多个列进行分组。 -对于具有类型T1,T2,…的列,它返回包含这些列的元组(T1,T2,…)。 执行该函数没有任何成本。 +对于具有类型T1,T2,...的列,它返回包含这些列的元组(T1,T2,...)。 执行该函数没有任何成本。 元组通常用作IN运算符的中间参数值,或用于创建lambda函数的形参列表。 元组不能写入表。 ## tupleElement(tuple, n), 运算符 x.N {#tupleelementtuple-n-operator-x-n} diff --git a/docs/zh/sql-reference/functions/json-functions.md b/docs/zh/sql-reference/functions/json-functions.md index 52ec0ed1535..f07de564847 100644 --- a/docs/zh/sql-reference/functions/json-functions.md +++ b/docs/zh/sql-reference/functions/json-functions.md @@ -56,7 +56,7 @@ slug: /zh/sql-reference/functions/json-functions 以下函数基于[simdjson](https://github.com/lemire/simdjson),专为更复杂的JSON解析要求而设计。但上述假设2仍然适用。 -## JSONHas(json\[, indices_or_keys\]…) {#jsonhasjson-indices-or-keys} +## JSONHas(json\[, indices_or_keys\]...) {#jsonhasjson-indices-or-keys} 如果JSON中存在该值,则返回`1`。 @@ -83,7 +83,7 @@ slug: /zh/sql-reference/functions/json-functions select JSONExtractKey('{"a": "hello", "b": [-100, 200.0, 300]}', -2) = 'a' select JSONExtractString('{"a": "hello", "b": [-100, 200.0, 300]}', 1) = 'hello' -## JSONLength(json\[, indices_or_keys\]…) {#jsonlengthjson-indices-or-keys} +## JSONLength(json\[, indices_or_keys\]...) {#jsonlengthjson-indices-or-keys} 返回JSON数组或JSON对象的长度。 @@ -94,7 +94,7 @@ slug: /zh/sql-reference/functions/json-functions select JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 3 select JSONLength('{"a": "hello", "b": [-100, 200.0, 300]}') = 2 -## JSONType(json\[, indices_or_keys\]…) {#jsontypejson-indices-or-keys} +## JSONType(json\[, indices_or_keys\]...) {#jsontypejson-indices-or-keys} 返回JSON值的类型。 @@ -106,13 +106,13 @@ slug: /zh/sql-reference/functions/json-functions select JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'a') = 'String' select JSONType('{"a": "hello", "b": [-100, 200.0, 300]}', 'b') = 'Array' -## JSONExtractUInt(json\[, indices_or_keys\]…) {#jsonextractuintjson-indices-or-keys} +## JSONExtractUInt(json\[, indices_or_keys\]...) {#jsonextractuintjson-indices-or-keys} -## JSONExtractInt(json\[, indices_or_keys\]…) {#jsonextractintjson-indices-or-keys} +## JSONExtractInt(json\[, indices_or_keys\]...) {#jsonextractintjson-indices-or-keys} -## JSONExtractFloat(json\[, indices_or_keys\]…) {#jsonextractfloatjson-indices-or-keys} +## JSONExtractFloat(json\[, indices_or_keys\]...) {#jsonextractfloatjson-indices-or-keys} -## JSONExtractBool(json\[, indices_or_keys\]…) {#jsonextractbooljson-indices-or-keys} +## JSONExtractBool(json\[, indices_or_keys\]...) {#jsonextractbooljson-indices-or-keys} 解析JSON并提取值。这些函数类似于`visitParam*`函数。 @@ -124,7 +124,7 @@ slug: /zh/sql-reference/functions/json-functions select JSONExtractFloat('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', 2) = 200.0 select JSONExtractUInt('{"a": "hello", "b": [-100, 200.0, 300]}', 'b', -1) = 300 -## JSONExtractString(json\[, indices_or_keys\]…) {#jsonextractstringjson-indices-or-keys} +## JSONExtractString(json\[, indices_or_keys\]...) {#jsonextractstringjson-indices-or-keys} 解析JSON并提取字符串。此函数类似于`visitParamExtractString`函数。 @@ -140,11 +140,11 @@ slug: /zh/sql-reference/functions/json-functions select JSONExtractString('{"abc":"\\u263"}', 'abc') = '' select JSONExtractString('{"abc":"hello}', 'abc') = '' -## JSONExtract(json\[, indices_or_keys…\], Return_type) {#jsonextractjson-indices-or-keys-return-type} +## JSONExtract(json\[, indices_or_keys...\], Return_type) {#jsonextractjson-indices-or-keys-return-type} 解析JSON并提取给定ClickHouse数据类型的值。 -这是以前的`JSONExtract函数的变体。 这意味着`JSONExtract(…, ‘String’)`返回与`JSONExtractString()`返回完全相同。`JSONExtract(…, ‘Float64’)`返回于`JSONExtractFloat()\`返回完全相同。 +这是以前的`JSONExtract函数的变体。 这意味着`JSONExtract(..., ‘String’)`返回与`JSONExtractString()`返回完全相同。`JSONExtract(..., ‘Float64’)`返回于`JSONExtractFloat()\`返回完全相同。 示例: @@ -156,7 +156,7 @@ slug: /zh/sql-reference/functions/json-functions SELECT JSONExtract('{"day": "Thursday"}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Thursday' SELECT JSONExtract('{"day": 5}', 'day', 'Enum8(\'Sunday\' = 0, \'Monday\' = 1, \'Tuesday\' = 2, \'Wednesday\' = 3, \'Thursday\' = 4, \'Friday\' = 5, \'Saturday\' = 6)') = 'Friday' -## JSONExtractKeysAndValues(json\[, indices_or_keys…\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} +## JSONExtractKeysAndValues(json\[, indices_or_keys...\], Value_type) {#jsonextractkeysandvaluesjson-indices-or-keys-value-type} 从JSON中解析键值对,其中值是给定的ClickHouse数据类型。 @@ -164,7 +164,7 @@ slug: /zh/sql-reference/functions/json-functions SELECT JSONExtractKeysAndValues('{"x": {"a": 5, "b": 7, "c": 11}}', 'x', 'Int8') = [('a',5),('b',7),('c',11)]; -## JSONExtractRaw(json\[, indices_or_keys\]…) {#jsonextractrawjson-indices-or-keys} +## JSONExtractRaw(json\[, indices_or_keys\]...) {#jsonextractrawjson-indices-or-keys} 返回JSON的部分。 diff --git a/docs/zh/sql-reference/functions/other-functions.md b/docs/zh/sql-reference/functions/other-functions.md index 2eeaad63694..9c28ff867c5 100644 --- a/docs/zh/sql-reference/functions/other-functions.md +++ b/docs/zh/sql-reference/functions/other-functions.md @@ -90,7 +90,7 @@ SELECT 'some-file-name' AS a, basename(a) 将一个常量列变为一个非常量列。 在ClickHouse中,非常量列和常量列在内存中的表示方式不同。尽管函数对于常量列和非常量总是返回相同的结果,但它们的工作方式可能完全不同(执行不同的代码)。此函数用于调试这种行为。 -## ignore(…) {#ignore} +## ignore(...) {#ignore} 接受任何参数,包括`NULL`。始终返回0。 但是,函数的参数总是被计算的。该函数可以用于基准测试。 diff --git a/docs/zh/sql-reference/functions/string-functions.md b/docs/zh/sql-reference/functions/string-functions.md index d1914839d7c..c28735c7dc7 100644 --- a/docs/zh/sql-reference/functions/string-functions.md +++ b/docs/zh/sql-reference/functions/string-functions.md @@ -95,7 +95,7 @@ SELECT toValidUTF8('\x61\xF0\x80\x80\x80b') 以Unicode字符为单位反转UTF-8编码的字符串。如果字符串不是UTF-8编码,则可能获取到一个非预期的结果(不会抛出异常)。 -## format(pattern, s0, s1, …) {#formatpattern-s0-s1} +## format(pattern, s0, s1, ...) {#formatpattern-s0-s1} 使用常量字符串`pattern`格式化其他参数。`pattern`字符串中包含由大括号`{}`包围的«替换字段»。 未被包含在大括号中的任何内容都被视为文本内容,它将原样保留在返回值中。 如果你需要在文本内容中包含一个大括号字符,它可以通过加倍来转义:`{{ '{{' }}`和`{{ '{{' }} '}}' }}`。 字段名称可以是数字(从零开始)或空(然后将它们视为连续数字) @@ -113,11 +113,11 @@ SELECT format('{} {}', 'Hello', 'World') └───────────────────────────────────┘ ``` -## concat(s1, s2, …) {#concat-s1-s2} +## concat(s1, s2, ...) {#concat-s1-s2} 将参数中的多个字符串拼接,不带分隔符。 -## concatAssumeInjective(s1, s2, …) {#concatassumeinjectives1-s2} +## concatAssumeInjective(s1, s2, ...) {#concatassumeinjectives1-s2} 与[concat](#concat-s1-s2)相同,区别在于,你需要保证concat(s1, s2, s3) -\> s4是单射的,它将用于GROUP BY的优化。 diff --git a/docs/zh/sql-reference/functions/string-search-functions.md b/docs/zh/sql-reference/functions/string-search-functions.md index 972fd84e2a1..8ada76eeeda 100644 --- a/docs/zh/sql-reference/functions/string-search-functions.md +++ b/docs/zh/sql-reference/functions/string-search-functions.md @@ -204,7 +204,7 @@ SELECT multiSearchAllPositions('Hello, World!', ['hello', '!', 'world']); **语法** ```sql -multiSearchFirstPosition(haystack, [needle1, needle2, …, needleN]) +multiSearchFirstPosition(haystack, [needle1, needle2, ..., needleN]) ``` ## multiSearchFirstIndex @@ -216,7 +216,7 @@ multiSearchFirstPosition(haystack, [needle1, needle2, …, needleN]) **语法** ```sql -multiSearchFirstIndex(haystack, \[needle1, needle2, …, needlen\]) +multiSearchFirstIndex(haystack, \[needle1, needle2, ..., needlen\]) ``` ## multiSearchAny {#multisearchany} @@ -229,7 +229,7 @@ multiSearchFirstIndex(haystack, \[needle1, needle2, …, n **语法** ```sql -multiSearchAny(haystack, [needle1, needle2, …, needleN]) +multiSearchAny(haystack, [needle1, needle2, ..., needleN]) ``` ## match {#match} @@ -273,7 +273,7 @@ Hyperscan 通常容易受到正则表达式拒绝服务 (ReDoS) 攻击。有关 **语法** ```sql -multiMatchAny(haystack, \[pattern1, pattern2, …, patternn\]) +multiMatchAny(haystack, \[pattern1, pattern2, ..., patternn\]) ``` ## multiMatchAnyIndex @@ -283,7 +283,7 @@ multiMatchAny(haystack, \[pattern1, pattern2, …, pattern **语法** ```sql -multiMatchAnyIndex(haystack, \[pattern1, pattern2, …, patternn\]) +multiMatchAnyIndex(haystack, \[pattern1, pattern2, ..., patternn\]) ``` ## multiMatchAllIndices @@ -293,7 +293,7 @@ multiMatchAnyIndex(haystack, \[pattern1, pattern2, …, pa **语法** ```sql -multiMatchAllIndices(haystack, \[pattern1, pattern2, …, patternn\]) +multiMatchAllIndices(haystack, \[pattern1, pattern2, ..., patternn\]) ``` ## multiFuzzyMatchAny @@ -307,7 +307,7 @@ multiMatchAllIndices(haystack, \[pattern1, pattern2, …, **语法** ```sql -multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) +multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, ..., patternn\]) ``` ## multiFuzzyMatchAnyIndex @@ -317,7 +317,7 @@ multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern21, pattern2, …, patternn\]) +multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, ..., patternn\]) ``` ## multiFuzzyMatchAllIndices @@ -327,7 +327,7 @@ multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2 **语法** ```sql -multiFuzzyMatchAllIndices(haystack, distance, \[pattern1, pattern2, …, patternn\]) +multiFuzzyMatchAllIndices(haystack, distance, \[pattern1, pattern2, ..., patternn\]) ``` ## extract diff --git a/docs/zh/sql-reference/functions/url-functions.md b/docs/zh/sql-reference/functions/url-functions.md index 44880b6ca1a..e7a0354c0bf 100644 --- a/docs/zh/sql-reference/functions/url-functions.md +++ b/docs/zh/sql-reference/functions/url-functions.md @@ -11,7 +11,7 @@ slug: /zh/sql-reference/functions/url-functions ### 协议 {#protocol} -返回URL的协议。例如: http、ftp、mailto、magnet… +返回URL的协议。例如: http、ftp、mailto、magnet... ### 域 {#domain} diff --git a/docs/zh/sql-reference/statements/alter/delete.md b/docs/zh/sql-reference/statements/alter/delete.md index 5eb77c35a93..f0b41c4e214 100644 --- a/docs/zh/sql-reference/statements/alter/delete.md +++ b/docs/zh/sql-reference/statements/alter/delete.md @@ -4,7 +4,7 @@ sidebar_position: 39 sidebar_label: DELETE --- -# ALTER TABLE … DELETE 语句 {#alter-mutations} +# ALTER TABLE ... DELETE 语句 {#alter-mutations} ``` sql ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr diff --git a/docs/zh/sql-reference/statements/alter/index.md b/docs/zh/sql-reference/statements/alter/index.md index e173837a16c..2286dcccd13 100644 --- a/docs/zh/sql-reference/statements/alter/index.md +++ b/docs/zh/sql-reference/statements/alter/index.md @@ -38,7 +38,7 @@ sidebar_label: ALTER ## Mutations 突变 {#mutations} -用来操作表数据的ALTER查询是通过一种叫做“突变”的机制来实现的,最明显的是[ALTER TABLE … DELETE](../../../sql-reference/statements/alter/delete.md)和[ALTER TABLE … UPDATE](../../../sql-reference/statements/alter/update.md)。它们是异步的后台进程,类似于[MergeTree](../../../engines/table-engines/mergetree-family/index.md)表的合并,产生新的“突变”版本的部件。 +用来操作表数据的ALTER查询是通过一种叫做“突变”的机制来实现的,最明显的是[ALTER TABLE ... DELETE](../../../sql-reference/statements/alter/delete.md)和[ALTER TABLE ... UPDATE](../../../sql-reference/statements/alter/update.md)。它们是异步的后台进程,类似于[MergeTree](../../../engines/table-engines/mergetree-family/index.md)表的合并,产生新的“突变”版本的部件。 diff --git a/docs/zh/sql-reference/statements/alter/update.md b/docs/zh/sql-reference/statements/alter/update.md index 97b2b43d889..7cf37401dc5 100644 --- a/docs/zh/sql-reference/statements/alter/update.md +++ b/docs/zh/sql-reference/statements/alter/update.md @@ -4,7 +4,7 @@ sidebar_position: 40 sidebar_label: UPDATE --- -# ALTER TABLE … UPDATE 语句 {#alter-table-update-statements} +# ALTER TABLE ... UPDATE 语句 {#alter-table-update-statements} ``` sql ALTER TABLE [db.]table UPDATE column1 = expr1 [, ...] WHERE filter_expr diff --git a/docs/zh/sql-reference/statements/alter/view.md b/docs/zh/sql-reference/statements/alter/view.md index 34a612803c1..a19d918612a 100644 --- a/docs/zh/sql-reference/statements/alter/view.md +++ b/docs/zh/sql-reference/statements/alter/view.md @@ -4,9 +4,9 @@ sidebar_position: 50 sidebar_label: VIEW --- -# ALTER TABLE … MODIFY QUERY 语句 {#alter-modify-query} +# ALTER TABLE ... MODIFY QUERY 语句 {#alter-modify-query} -当使用`ALTER TABLE … MODIFY QUERY`语句创建一个[物化视图](../create/view.md#materialized)时,可以修改`SELECT`查询。当物化视图在没有 `TO [db.]name` 的情况下创建时使用它。必须启用 `allow_experimental_alter_materialized_view_structure`设置。 +当使用`ALTER TABLE ... MODIFY QUERY`语句创建一个[物化视图](../create/view.md#materialized)时,可以修改`SELECT`查询。当物化视图在没有 `TO [db.]name` 的情况下创建时使用它。必须启用 `allow_experimental_alter_materialized_view_structure`设置。 如果一个物化视图使用`TO [db.]name`,你必须先 [DETACH](../detach.mdx) 视图。用[ALTER TABLE](index.md)修改目标表,然后 [ATTACH](../attach.mdx)之前分离的(`DETACH`)视图。 diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index bce0994ecd2..49a1d66bdf1 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -55,7 +55,7 @@ ClickHouse 中的物化视图更像是插入触发器。 如果视图查询中 如果指定`POPULATE`,则在创建视图时将现有表数据插入到视图中,就像创建一个`CREATE TABLE ... AS SELECT ...`一样。 否则,查询仅包含创建视图后插入表中的数据。 我们**不建议**使用POPULATE,因为在创建视图期间插入表中的数据不会插入其中。 -`SELECT` 查询可以包含`DISTINCT`、`GROUP BY`、`ORDER BY`、`LIMIT`……请注意,相应的转换是在每个插入数据块上独立执行的。 例如,如果设置了`GROUP BY`,则在插入期间聚合数据,但仅在插入数据的单个数据包内。 数据不会被进一步聚合。 例外情况是使用独立执行数据聚合的`ENGINE`,例如`SummingMergeTree`。 +`SELECT` 查询可以包含`DISTINCT`、`GROUP BY`、`ORDER BY`、`LIMIT`...请注意,相应的转换是在每个插入数据块上独立执行的。 例如,如果设置了`GROUP BY`,则在插入期间聚合数据,但仅在插入数据的单个数据包内。 数据不会被进一步聚合。 例外情况是使用独立执行数据聚合的`ENGINE`,例如`SummingMergeTree`。 在物化视图上执行[ALTER](../../../sql-reference/statements/alter/index.md)查询有局限性,因此可能不方便。 如果物化视图使用构造`TO [db.]name`,你可以`DETACH`视图,为目标表运行`ALTER`,然后`ATTACH`先前分离的(`DETACH`)视图。 diff --git a/docs/zh/sql-reference/statements/insert-into.md b/docs/zh/sql-reference/statements/insert-into.md index f80c0a8a8ea..a08a78b6f1d 100644 --- a/docs/zh/sql-reference/statements/insert-into.md +++ b/docs/zh/sql-reference/statements/insert-into.md @@ -68,7 +68,7 @@ SELECT * FROM insert_select_testtable; INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set ``` -例如,下面的查询所使用的输入格式就与上面INSERT … VALUES的中使用的输入格式相同: +例如,下面的查询所使用的输入格式就与上面INSERT ... VALUES的中使用的输入格式相同: ``` sql INSERT INTO [TABLE] [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... diff --git a/docs/zh/sql-reference/statements/select/limit.md b/docs/zh/sql-reference/statements/select/limit.md index 2bbf2949707..795f3f4ecd1 100644 --- a/docs/zh/sql-reference/statements/select/limit.md +++ b/docs/zh/sql-reference/statements/select/limit.md @@ -13,11 +13,11 @@ sidebar_label: LIMIT 如果没有 [ORDER BY](../../../sql-reference/statements/select/order-by.md) 子句显式排序结果,结果的行选择可能是任意的和非确定性的。 -## LIMIT … WITH TIES 修饰符 {#limit-with-ties} +## LIMIT ... WITH TIES 修饰符 {#limit-with-ties} 如果为 `LIMIT n[,m]` 设置了 `WITH TIES` ,并且声明了 `ORDER BY expr_list`, 除了得到无修饰符的结果(正常情况下的 `limit n`, 前n行数据), 还会返回与第`n`行具有相同排序字段的行(即如果第n+1行的字段与第n行 拥有相同的排序字段,同样返回该结果. -此修饰符可以与: [ORDER BY … WITH FILL modifier](../../../sql-reference/statements/select/order-by.md#orderby-with-fill) 组合使用. +此修饰符可以与: [ORDER BY ... WITH FILL modifier](../../../sql-reference/statements/select/order-by.md#orderby-with-fill) 组合使用. 例如以下查询: diff --git a/docs/zh/sql-reference/statements/select/order-by.md b/docs/zh/sql-reference/statements/select/order-by.md index 3286fc9f9e7..2f2d9a4959c 100644 --- a/docs/zh/sql-reference/statements/select/order-by.md +++ b/docs/zh/sql-reference/statements/select/order-by.md @@ -89,7 +89,7 @@ SELECT a, b, c FROM t ORDER BY a, b, c ## ORDER BY Expr WITH FILL Modifier {#orderby-with-fill} -此修饰符可以与 [LIMIT … WITH TIES modifier](../../../sql-reference/statements/select/limit.md#limit-with-ties) 进行组合使用. +此修饰符可以与 [LIMIT ... WITH TIES modifier](../../../sql-reference/statements/select/limit.md#limit-with-ties) 进行组合使用. 可以在`ORDER BY expr`之后用可选的`FROM expr`,`TO expr`和`STEP expr`参数来设置`WITH FILL`修饰符。 所有`expr`列的缺失值将被顺序填充,而其他列将被填充为默认值。 diff --git a/docs/zh/sql-reference/table-functions/file.md b/docs/zh/sql-reference/table-functions/file.md index 28682255738..fa1ec12f7df 100644 --- a/docs/zh/sql-reference/table-functions/file.md +++ b/docs/zh/sql-reference/table-functions/file.md @@ -114,7 +114,7 @@ FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32') **示例** -从名为 `file000`, `file001`, … , `file999`的文件中查询数据: +从名为 `file000`, `file001`, ... , `file999`的文件中查询数据: ``` sql SELECT count(*) diff --git a/docs/zh/sql-reference/table-functions/hdfs.md b/docs/zh/sql-reference/table-functions/hdfs.md index b10b10ae2d2..f8320d8d0bb 100644 --- a/docs/zh/sql-reference/table-functions/hdfs.md +++ b/docs/zh/sql-reference/table-functions/hdfs.md @@ -84,7 +84,7 @@ FROM hdfs('hdfs://hdfs1:9000/{some,another}_dir/*', 'TSV', 'name String, value U **示例** -从名为 `file000`, `file001`, … , `file999`的文件中查询数据: +从名为 `file000`, `file001`, ... , `file999`的文件中查询数据: ``` sql SELECT count(*) diff --git a/docs/zh/sql-reference/table-functions/s3.md b/docs/zh/sql-reference/table-functions/s3.md index f7384a7526e..4f2c7299d95 100644 --- a/docs/zh/sql-reference/table-functions/s3.md +++ b/docs/zh/sql-reference/table-functions/s3.md @@ -99,7 +99,7 @@ FROM s3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_prefi !!! warning "Warning" 如果文件列表中包含有从零开头的数字范围,请对每个数字分别使用带括号的结构,或者使用`?`。 -计算名为 `file-000.csv`, `file-001.csv`, … , `file-999.csv` 文件的总行数: +计算名为 `file-000.csv`, `file-001.csv`, ... , `file-999.csv` 文件的总行数: ``` sql SELECT count(*) From 713764f62fa92db1fab04dcb426682b4859d6de1 Mon Sep 17 00:00:00 2001 From: Blargian Date: Thu, 23 May 2024 14:01:00 +0200 Subject: [PATCH 29/36] Add missing space before link --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 4501d1f43d3..829d46df9fa 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -37,7 +37,7 @@ getMacro(name); **Returned value** -- Value of the specified macro.[String](../../sql-reference/data-types/string.md). +- Value of the specified macro. [String](../../sql-reference/data-types/string.md). **Example** From 147516f1626f656da5fc4dcc0d9254202a8de860 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 16 Apr 2024 13:05:07 +0000 Subject: [PATCH 30/36] Fix AST fuzzer failure --- src/Functions/FunctionHelpers.cpp | 2 ++ src/Functions/splitByRegexp.cpp | 10 ++++------ .../0_stateless/01866_split_by_regexp.reference | 1 + tests/queries/0_stateless/01866_split_by_regexp.sql | 3 +++ 4 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionHelpers.cpp b/src/Functions/FunctionHelpers.cpp index d85bb0e7060..3b057779ffe 100644 --- a/src/Functions/FunctionHelpers.cpp +++ b/src/Functions/FunctionHelpers.cpp @@ -21,6 +21,8 @@ namespace ErrorCodes const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column) { + if (!column) + return {}; if (!isColumnConst(*column)) return {}; diff --git a/src/Functions/splitByRegexp.cpp b/src/Functions/splitByRegexp.cpp index e28fe9c38bb..042db97794d 100644 --- a/src/Functions/splitByRegexp.cpp +++ b/src/Functions/splitByRegexp.cpp @@ -164,6 +164,7 @@ public: String getName() const override { return name; } size_t getNumberOfArguments() const override { return SplitByRegexpImpl::getNumberOfArguments(); } bool isVariadic() const override { return SplitByRegexpImpl::isVariadic(); } + /// ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return SplitByRegexpImpl::getArgumentsThatAreAlwaysConstant(); } FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { @@ -182,14 +183,11 @@ public: private: bool patternIsTrivialChar(const ColumnsWithTypeAndName & arguments) const { + if (!arguments[0].column.get()) + return false; const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); if (!col) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of first argument of function {}. " - "Must be constant string.", - arguments[0].column->getName(), - getName()); + return false; String pattern = col->getValue(); if (pattern.size() == 1) diff --git a/tests/queries/0_stateless/01866_split_by_regexp.reference b/tests/queries/0_stateless/01866_split_by_regexp.reference index 62939940545..552d4d1f96a 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.reference +++ b/tests/queries/0_stateless/01866_split_by_regexp.reference @@ -17,3 +17,4 @@ Test fallback of splitByRegexp to splitByChar if regexp is trivial ['a','b','c'] ['a|b|c'] ['a\\b\\c'] +AST Fuzzer failure diff --git a/tests/queries/0_stateless/01866_split_by_regexp.sql b/tests/queries/0_stateless/01866_split_by_regexp.sql index 570bd1ba5c0..bc25d3e1093 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.sql +++ b/tests/queries/0_stateless/01866_split_by_regexp.sql @@ -20,3 +20,6 @@ select splitByRegexp('{', 'a{b{c'); select splitByRegexp('}', 'a}b}c'); select splitByRegexp('|', 'a|b|c'); select splitByRegexp('\\', 'a\\b\\c'); + +SELECT 'AST Fuzzer failure'; +SELECT splitByRegexp(materialize(1), NULL, 3) -- { serverError ILLEGAL_COLUMN } From 6e3a609907192d7cc378fb209d0e2431b8859eb0 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 23 May 2024 15:43:17 +0200 Subject: [PATCH 31/36] Fix formatting in ru/index.md --- docs/ru/index.md | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/ru/index.md b/docs/ru/index.md index d551d492af5..02be8912b94 100644 --- a/docs/ru/index.md +++ b/docs/ru/index.md @@ -12,10 +12,10 @@ ClickHouse — столбцовая система управления база | Строка | WatchID | JavaEnable | Title | GoodEvent | EventTime | |--------|-------------|------------|--------------------|-----------|---------------------| -| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| #N | ... | ... | ... | ... | ... | +| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| #N | ... | ... | ... | ... | ... | То есть, значения, относящиеся к одной строке, физически хранятся рядом. @@ -24,13 +24,13 @@ ClickHouse — столбцовая система управления база В столбцовых СУБД данные хранятся в таком порядке: -| Строка: | #0 | #1 | #2 | #N | +| Строка: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| -| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... | -| JavaEnable: | 1 | 0 | 1 | ... | -| Title: | Investor Relations | Contact us | Mission | ... | -| GoodEvent: | 1 | 1 | 1 | ... | -| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | ... | +| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... | +| JavaEnable: | 1 | 0 | 1 | ... | +| Title: | Investor Relations | Contact us | Mission | ... | +| GoodEvent: | 1 | 1 | 1 | ... | +| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | ... | В примерах изображён только порядок расположения данных. То есть значения из разных столбцов хранятся отдельно, а данные одного столбца — вместе. From e24253c097ed2f0325c9be77fc87ebbe8f086a5c Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 23 May 2024 15:45:26 +0200 Subject: [PATCH 32/36] Fix formatting in zh/index.md --- docs/zh/index.md | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/zh/index.md b/docs/zh/index.md index ec4b6dce1f8..c092f296722 100644 --- a/docs/zh/index.md +++ b/docs/zh/index.md @@ -13,10 +13,10 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) | Row | WatchID | JavaEnable | Title | GoodEvent | EventTime | |-----|-------------|------------|--------------------|-----------|---------------------| -| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| #N | ... | ... | ... | ... | ... | +| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | +| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | +| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | +| #N | ... | ... | ... | ... | ... | 处于同一行中的数据总是被物理的存储在一起。 @@ -24,13 +24,13 @@ ClickHouse是一个用于联机分析(OLAP)的列式数据库管理系统(DBMS) 在列式数据库系统中,数据按如下的顺序存储: -| Row: | #0 | #1 | #2 | #N | +| Row: | #0 | #1 | #2 | #N | |-------------|---------------------|---------------------|---------------------|-----| -| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... | -| JavaEnable: | 1 | 0 | 1 | ... | -| Title: | Investor Relations | Contact us | Mission | ... | -| GoodEvent: | 1 | 1 | 1 | ... | -| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | ... | +| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... | +| JavaEnable: | 1 | 0 | 1 | ... | +| Title: | Investor Relations | Contact us | Mission | ... | +| GoodEvent: | 1 | 1 | 1 | ... | +| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | ... | 这些示例只显示了数据的排列顺序。来自不同列的值被单独存储,来自同一列的数据被存储在一起。 From 508b0356543fc3a49e069166093147b3089ed29a Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Thu, 23 May 2024 14:08:48 +0000 Subject: [PATCH 33/36] Move is NaN from other-functions to arithmetic functions --- .../en/sql-reference/functions/arithmetic-functions.md | 10 ++++++++++ docs/en/sql-reference/functions/other-functions.md | 10 ---------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 8b8527acfdf..7b079152907 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -194,6 +194,16 @@ Result: You can get similar result by using the [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. +## isNaN + +Returns 1 if the Float32 and Float64 argument is NaN, otherwise this function 0. + +**Syntax** + +```sql +isNaN(x) +``` + ## modulo Calculates the remainder of the division of two values `a` by `b`. diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 79c0148d704..c16e8af1ef0 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -541,16 +541,6 @@ Result: └────────────────────┘ ``` -## isNaN - -Returns 1 if the Float32 and Float64 argument is NaN, otherwise this function 0. - -**Syntax** - -```sql -isNaN(x) -``` - ## hasColumnInTable Given the database name, the table name, and the column name as constant strings, returns 1 if the given column exists, otherwise 0. From 8d697123dac574e727101d241e4d16eae2bce8da Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 23 May 2024 16:36:24 +0200 Subject: [PATCH 34/36] CI: Cancel sync wf on new push --- .github/workflows/pull_request.yml | 3 +++ tests/ci/ci.py | 37 +++++++++++++++++++-------- tests/ci/ci_metadata.py | 41 +++++++++++++++++++++++++++--- tests/ci/env_helper.py | 1 + 4 files changed, 68 insertions(+), 14 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index f20e987db97..48b4a558580 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -33,6 +33,9 @@ jobs: clear-repository: true # to ensure correct digests fetch-depth: 0 # to get version filter: tree:0 + - name: Cancel Sync PR workflow + run: | + python3 "$GITHUB_WORKSPACE/tests/ci/ci.py" --cancel-previous-run - name: Labels check run: | cd "$GITHUB_WORKSPACE/tests/ci" diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 99555b06bbf..68db08fbe96 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1908,13 +1908,26 @@ def _get_ext_check_name(check_name: str) -> str: return check_name_with_group -def _cancel_pr_wf(s3: S3Helper, pr_number: int) -> None: - run_id = CiMetadata(s3, pr_number).fetch_meta().run_id - if not run_id: - print(f"ERROR: FIX IT: Run id has not been found PR [{pr_number}]!") +def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> None: + wf_data = CiMetadata(s3, pr_number).fetch_meta() + if not cancel_sync: + if not wf_data.run_id: + print(f"ERROR: FIX IT: Run id has not been found PR [{pr_number}]!") + else: + print( + f"Canceling PR workflow run_id: [{wf_data.run_id}], pr: [{pr_number}]" + ) + GitHub.cancel_wf(GITHUB_REPOSITORY, get_best_robot_token(), wf_data.run_id) else: - print(f"Canceling PR workflow run_id: [{run_id}], pr: [{pr_number}]") - GitHub.cancel_wf(GITHUB_REPOSITORY, get_best_robot_token(), run_id) + if not wf_data.sync_pr_run_id: + print("WARNING: Sync PR run id has not been found") + else: + print(f"Canceling sync PR workflow run_id: [{wf_data.sync_pr_run_id}]") + GitHub.cancel_wf( + "ClickHouse/clickhouse-private", + get_best_robot_token(), + wf_data.sync_pr_run_id, + ) def main() -> int: @@ -1947,7 +1960,7 @@ def main() -> int: if args.configure: if CI and pr_info.is_pr: # store meta on s3 (now we need it only for PRs) - meta = CiMetadata(s3, pr_info.number) + meta = CiMetadata(s3, pr_info.number, pr_info.head_ref) meta.run_id = int(GITHUB_RUN_ID) meta.push_meta() @@ -2245,10 +2258,12 @@ def main() -> int: ### CANCEL PREVIOUS WORKFLOW RUN elif args.cancel_previous_run: - assert ( - pr_info.is_merge_queue - ), "Currently it's supposed to be used in MQ wf to cancel running PR wf if any" - _cancel_pr_wf(s3, pr_info.merged_pr) + if pr_info.is_merge_queue: + _cancel_pr_wf(s3, pr_info.merged_pr) + elif pr_info.is_pr: + _cancel_pr_wf(s3, pr_info.number, cancel_sync=True) + else: + assert False, "BUG! Not supported scenario" ### print results _print_results(result, args.outfile, args.pretty) diff --git a/tests/ci/ci_metadata.py b/tests/ci/ci_metadata.py index 82d44cf1adc..a767d102811 100644 --- a/tests/ci/ci_metadata.py +++ b/tests/ci/ci_metadata.py @@ -4,9 +4,13 @@ from typing import Optional from env_helper import ( S3_BUILDS_BUCKET, TEMP_PATH, + GITHUB_UPSTREAM_REPOSITORY, + GITHUB_REPOSITORY, + S3_BUILDS_BUCKET_PUBLIC, ) from s3_helper import S3Helper from ci_utils import GHActions +from synchronizer_utils import SYNC_BRANCH_PREFIX # pylint: disable=too-many-lines @@ -22,13 +26,14 @@ class CiMetadata: _LOCAL_PATH = Path(TEMP_PATH) / "ci_meta" _FILE_SUFFIX = ".cimd" _FILENAME_RUN_ID = "run_id" + _FILE_SUFFIX + _FILENAME_SYNC_PR_RUN_ID = "sync_pr_run_id" + _FILE_SUFFIX def __init__( self, s3: S3Helper, pr_number: Optional[int] = None, - sha: Optional[str] = None, git_ref: Optional[str] = None, + sha: Optional[str] = None, ): assert pr_number or (sha and git_ref) @@ -37,12 +42,25 @@ class CiMetadata: self.git_ref = git_ref self.s3 = s3 self.run_id = 0 + self.upstream_pr_number = 0 + self.sync_pr_run_id = 0 if self.pr_number: self.s3_path = f"{self._S3_PREFIX}/PRs/{self.pr_number}/" else: self.s3_path = f"{self._S3_PREFIX}/{self.git_ref}/{self.sha}/" + # Process upstream StatusNames.SYNC: + # metadata path for upstream pr + self.s3_path_upstream = "" + if ( + self.git_ref + and self.git_ref.startswith(f"{SYNC_BRANCH_PREFIX}/pr/") + and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY + ): + self.upstream_pr_number = int(self.git_ref.split("/pr/", maxsplit=1)[1]) + self.s3_path_upstream = f"{self._S3_PREFIX}/PRs/{self.upstream_pr_number}/" + self._updated = False if not self._LOCAL_PATH.exists(): @@ -73,6 +91,8 @@ class CiMetadata: assert len(lines) == 1 if file_name.name == self._FILENAME_RUN_ID: self.run_id = int(lines[0]) + elif file_name.name == self._FILENAME_SYNC_PR_RUN_ID: + self.sync_pr_run_id = int(lines[0]) self._updated = True return self @@ -84,8 +104,15 @@ class CiMetadata: Uploads meta on s3 """ assert self.run_id + assert self.git_ref, "Push meta only with full info" + + if not self.upstream_pr_number: + log_title = f"Storing workflow metadata: PR [{self.pr_number}]" + else: + log_title = f"Storing workflow metadata: PR [{self.pr_number}], upstream PR [{self.upstream_pr_number}]" + GHActions.print_in_group( - f"Storing workflow metadata: PR [{self.pr_number}]", + log_title, [f"run_id: {self.run_id}"], ) @@ -96,9 +123,17 @@ class CiMetadata: _ = self.s3.upload_file( bucket=S3_BUILDS_BUCKET, file_path=local_file, - s3_path=self.s3_path + local_file.name, + s3_path=self.s3_path + self._FILENAME_RUN_ID, ) + if self.upstream_pr_number: + # store run id in upstream pr meta as well + _ = self.s3.upload_file( + bucket=S3_BUILDS_BUCKET_PUBLIC, + file_path=local_file, + s3_path=self.s3_path_upstream + self._FILENAME_SYNC_PR_RUN_ID, + ) + if __name__ == "__main__": # TEST: diff --git a/tests/ci/env_helper.py b/tests/ci/env_helper.py index 9b9652d5bd3..64614ffa611 100644 --- a/tests/ci/env_helper.py +++ b/tests/ci/env_helper.py @@ -31,6 +31,7 @@ IMAGES_PATH = os.getenv("IMAGES_PATH", TEMP_PATH) REPO_COPY = os.getenv("REPO_COPY", GITHUB_WORKSPACE) RUNNER_TEMP = os.getenv("RUNNER_TEMP", p.abspath(p.join(module_dir, "./tmp"))) S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") +S3_BUILDS_BUCKET_PUBLIC = "clickhouse-builds" S3_TEST_REPORTS_BUCKET = os.getenv("S3_TEST_REPORTS_BUCKET", "clickhouse-test-reports") S3_URL = os.getenv("S3_URL", "https://s3.amazonaws.com") S3_DOWNLOAD = os.getenv("S3_DOWNLOAD", S3_URL) From bd15e1311a949753a234cfed9571600af78eb906 Mon Sep 17 00:00:00 2001 From: Max K Date: Thu, 23 May 2024 22:35:21 +0200 Subject: [PATCH 35/36] CI: fix --- tests/ci/ci.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 68db08fbe96..4afd3f46f9d 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1917,7 +1917,7 @@ def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> No print( f"Canceling PR workflow run_id: [{wf_data.run_id}], pr: [{pr_number}]" ) - GitHub.cancel_wf(GITHUB_REPOSITORY, get_best_robot_token(), wf_data.run_id) + GitHub.cancel_wf(GITHUB_REPOSITORY, wf_data.run_id, get_best_robot_token()) else: if not wf_data.sync_pr_run_id: print("WARNING: Sync PR run id has not been found") @@ -1925,8 +1925,8 @@ def _cancel_pr_wf(s3: S3Helper, pr_number: int, cancel_sync: bool = False) -> No print(f"Canceling sync PR workflow run_id: [{wf_data.sync_pr_run_id}]") GitHub.cancel_wf( "ClickHouse/clickhouse-private", - get_best_robot_token(), wf_data.sync_pr_run_id, + get_best_robot_token(), ) From 5710b5852f9e067fbcd8809196c9c403a8de43dc Mon Sep 17 00:00:00 2001 From: Nataly Merezhuk Date: Thu, 23 May 2024 17:45:58 -0400 Subject: [PATCH 36/36] Adds note - file engine unavailable in ClickHouse Cloud. --- docs/en/engines/table-engines/special/file.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index fdf5242ba3b..0d422f64762 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -14,6 +14,10 @@ Usage scenarios: - Convert data from one format to another. - Updating data in ClickHouse via editing a file on a disk. +:::note +This engine is not currently available in ClickHouse Cloud, please [use the S3 table function instead](/docs/en/sql-reference/table-functions/s3.md). +::: + ## Usage in ClickHouse Server {#usage-in-clickhouse-server} ``` sql