[Enhancement] support group by compressed key (#61632)

Signed-off-by: stdpain <drfeng08@gmail.com>
This commit is contained in:
stdpain 2025-08-14 09:32:20 +08:00 committed by GitHub
parent 89f5c11799
commit 5f6cdde3a0
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
60 changed files with 2998 additions and 332 deletions

View File

@ -14,7 +14,6 @@
#include "connector/es_connector.h"
#include "common/logging.h"
#include "exec/es/es_predicate.h"
#include "exec/es/es_query_builder.h"
#include "exec/es/es_scan_reader.h"
@ -22,6 +21,7 @@
#include "exec/es/es_scroll_query.h"
#include "exec/exec_node.h"
#include "exprs/expr.h"
#include "service/backend_options.h"
#include "storage/chunk_helper.h"
namespace starrocks::connector {

View File

@ -51,6 +51,7 @@ set(EXEC_FILES
aggregator.cpp
sorted_streaming_aggregator.cpp
aggregate/agg_hash_variant.cpp
aggregate/compress_serializer.cpp
aggregate/aggregate_base_node.cpp
aggregate/aggregate_blocking_node.cpp
aggregate/distinct_blocking_node.cpp

View File

@ -0,0 +1,28 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
//
#pragma once
#include <cstdint>
namespace starrocks {
using AggDataPtr = uint8_t*;
using int128_t = __int128;
class SliceWithHash;
class HashOnSliceWithHash;
class EqualOnSliceWithHash;
} // namespace starrocks

View File

@ -14,27 +14,25 @@
#pragma once
#include <any>
#include <cstdint>
#include <limits>
#include <type_traits>
#include <utility>
#include "column/column.h"
#include "column/column_hash.h"
#include "column/column_helper.h"
#include "column/hash_set.h"
#include "column/type_traits.h"
#include "column/vectorized_fwd.h"
#include "common/compiler_util.h"
#include "exec/aggregate/agg_hash_set.h"
#include "exec/aggregate/agg_profile.h"
#include "exec/aggregate/compress_serializer.h"
#include "gutil/casts.h"
#include "gutil/strings/fastmem.h"
#include "runtime/mem_pool.h"
#include "util/fixed_hash_map.h"
#include "util/hash_util.hpp"
#include "util/phmap/phmap.h"
#include "util/phmap/phmap_dump.h"
namespace starrocks {
@ -245,9 +243,10 @@ struct AggHashMapWithOneNumberKeyWithNullable
DCHECK(!key_column->is_nullable());
const auto column = down_cast<const ColumnType*>(key_column);
size_t bucket_count = this->hash_map.bucket_count();
if (bucket_count < prefetch_threhold) {
if constexpr (is_no_prefetch_map<HashMap>) {
this->template compute_agg_noprefetch<Func, HTBuildOp>(column, agg_states,
std::forward<Func>(allocate_func), extra);
} else if (this->hash_map.bucket_count() < prefetch_threhold) {
this->template compute_agg_noprefetch<Func, HTBuildOp>(column, agg_states,
std::forward<Func>(allocate_func), extra);
} else {
@ -1091,4 +1090,151 @@ struct AggHashMapWithSerializedKeyFixedSize
int32_t _chunk_size;
};
template <typename HashMap>
struct AggHashMapWithCompressedKeyFixedSize
: public AggHashMapWithKey<HashMap, AggHashMapWithCompressedKeyFixedSize<HashMap>> {
using Self = AggHashMapWithCompressedKeyFixedSize<HashMap>;
using Base = AggHashMapWithKey<HashMap, AggHashMapWithCompressedKeyFixedSize<HashMap>>;
using KeyType = typename HashMap::key_type;
using Iterator = typename HashMap::iterator;
using FixedSizeSliceKey = typename HashMap::key_type;
using ResultVector = typename std::vector<FixedSizeSliceKey>;
template <class... Args>
AggHashMapWithCompressedKeyFixedSize(int chunk_size, Args&&... args)
: Base(chunk_size, std::forward<Args>(args)...),
mem_pool(std::make_unique<MemPool>()),
_chunk_size(chunk_size) {
fixed_keys.reserve(chunk_size);
}
AggDataPtr get_null_key_data() { return nullptr; }
void set_null_key_data(AggDataPtr data) {}
template <AllocFunc<Self> Func, typename HTBuildOp>
ALWAYS_NOINLINE void compute_agg_noprefetch(size_t chunk_size, const Columns& key_columns, MemPool* pool,
Func&& allocate_func, Buffer<AggDataPtr>* agg_states,
ExtraAggParam* extra) {
[[maybe_unused]] size_t hash_table_size = this->hash_map.size();
auto* __restrict not_founds = extra->not_founds;
// serialize
bitcompress_serialize(key_columns, bases, offsets, chunk_size, sizeof(FixedSizeSliceKey), fixed_keys.data());
for (size_t i = 0; i < chunk_size; ++i) {
if constexpr (HTBuildOp::process_limit) {
if (hash_table_size < extra->limits) {
_emplace_key(fixed_keys[i], (*agg_states)[i], allocate_func, [&] { hash_table_size++; });
} else {
_find_key((*agg_states)[i], (*not_founds)[i], fixed_keys[i]);
}
} else if constexpr (HTBuildOp::allocate) {
_emplace_key(fixed_keys[i], (*agg_states)[i], allocate_func,
FillNotFounds<HTBuildOp::fill_not_found>(not_founds, i));
} else if constexpr (HTBuildOp::fill_not_found) {
_find_key((*agg_states)[i], (*not_founds)[i], fixed_keys[i]);
}
}
}
template <AllocFunc<Self> Func, typename HTBuildOp>
ALWAYS_NOINLINE void compute_agg_prefetch(size_t chunk_size, const Columns& key_columns, MemPool* pool,
Func&& allocate_func, Buffer<AggDataPtr>* agg_states,
ExtraAggParam* extra) {
[[maybe_unused]] size_t hash_table_size = this->hash_map.size();
auto* __restrict not_founds = extra->not_founds;
// serialize
bitcompress_serialize(key_columns, bases, offsets, chunk_size, sizeof(FixedSizeSliceKey), fixed_keys.data());
hashs.reserve(chunk_size);
for (size_t i = 0; i < chunk_size; ++i) {
hashs[i] = this->hash_map.hash_function()(fixed_keys[i]);
}
size_t prefetch_index = AGG_HASH_MAP_DEFAULT_PREFETCH_DIST;
for (size_t i = 0; i < chunk_size; ++i) {
if (prefetch_index < chunk_size) {
this->hash_map.prefetch_hash(hashs[prefetch_index++]);
}
if constexpr (HTBuildOp::process_limit) {
if (hash_table_size < extra->limits) {
_emplace_key_with_hash(fixed_keys[i], hashs[i], (*agg_states)[i], allocate_func,
[&] { hash_table_size++; });
} else {
_find_key((*agg_states)[i], (*not_founds)[i], fixed_keys[i]);
}
} else if constexpr (HTBuildOp::allocate) {
_emplace_key_with_hash(fixed_keys[i], hashs[i], (*agg_states)[i], allocate_func,
FillNotFounds<HTBuildOp::fill_not_found>(not_founds, i));
} else if constexpr (HTBuildOp::fill_not_found) {
_find_key((*agg_states)[i], (*not_founds)[i], fixed_keys[i]);
}
}
}
template <AllocFunc<Self> Func, typename HTBuildOp>
void compute_agg_states(size_t chunk_size, const Columns& key_columns, MemPool* pool, Func&& allocate_func,
Buffer<AggDataPtr>* agg_states, ExtraAggParam* extra) {
auto* buffer = reinterpret_cast<uint8_t*>(fixed_keys.data());
memset(buffer, 0x0, sizeof(FixedSizeSliceKey) * chunk_size);
if constexpr (is_no_prefetch_map<HashMap>) {
this->template compute_agg_noprefetch<Func, HTBuildOp>(
chunk_size, key_columns, pool, std::forward<Func>(allocate_func), agg_states, extra);
} else if (this->hash_map.bucket_count() < prefetch_threhold) {
this->template compute_agg_noprefetch<Func, HTBuildOp>(
chunk_size, key_columns, pool, std::forward<Func>(allocate_func), agg_states, extra);
} else {
this->template compute_agg_prefetch<Func, HTBuildOp>(chunk_size, key_columns, pool,
std::forward<Func>(allocate_func), agg_states, extra);
}
}
template <AllocFunc<Self> Func, typename EmplaceCallBack>
ALWAYS_INLINE void _emplace_key(KeyType key, AggDataPtr& target_state, Func&& allocate_func,
EmplaceCallBack&& callback) {
auto iter = this->hash_map.lazy_emplace(key, [&](const auto& ctor) {
callback();
AggDataPtr pv = allocate_func(key);
ctor(key, pv);
});
target_state = iter->second;
}
template <AllocFunc<Self> Func, typename EmplaceCallBack>
ALWAYS_INLINE void _emplace_key_with_hash(KeyType key, size_t hash, AggDataPtr& target_state, Func&& allocate_func,
EmplaceCallBack&& callback) {
auto iter = this->hash_map.lazy_emplace_with_hash(key, hash, [&](const auto& ctor) {
callback();
AggDataPtr pv = allocate_func(key);
ctor(key, pv);
});
target_state = iter->second;
}
template <typename... Args>
ALWAYS_INLINE void _find_key(AggDataPtr& target_state, uint8_t& not_found, Args&&... args) {
if (auto iter = this->hash_map.find(std::forward<Args>(args)...); iter != this->hash_map.end()) {
target_state = iter->second;
} else {
not_found = 1;
}
}
void insert_keys_to_columns(ResultVector& keys, Columns& key_columns, int32_t chunk_size) {
bitcompress_deserialize(key_columns, bases, offsets, used_bits, chunk_size, sizeof(FixedSizeSliceKey),
keys.data());
}
static constexpr bool has_single_null_key = false;
std::vector<int> used_bits;
std::vector<int> offsets;
std::vector<std::any> bases;
std::vector<FixedSizeSliceKey> fixed_keys;
std::vector<size_t> hashs;
std::unique_ptr<MemPool> mem_pool;
ResultVector results;
int32_t _chunk_size;
};
} // namespace starrocks

View File

@ -14,19 +14,17 @@
#pragma once
#include <any>
#include "column/column_hash.h"
#include "column/column_helper.h"
#include "column/hash_set.h"
#include "column/type_traits.h"
#include "column/vectorized_fwd.h"
#include "exec/aggregate/agg_profile.h"
#include "gutil/casts.h"
#include "runtime/mem_pool.h"
#include "runtime/runtime_state.h"
#include "util/fixed_hash_map.h"
#include "util/hash_util.hpp"
#include "util/phmap/phmap.h"
#include "util/runtime_profile.h"
namespace starrocks {
@ -111,14 +109,6 @@ struct AggHashSet {
}
};
template <typename T>
struct no_prefetch_set : std::false_type {};
template <PhmapSeed seed>
struct no_prefetch_set<Int8AggHashSet<seed>> : std::true_type {};
template <class T>
constexpr bool is_no_prefetch_set = no_prefetch_set<T>::value;
// handle one number hash key
template <LogicalType logical_type, typename HashSet>
struct AggHashSetOfOneNumberKey : public AggHashSet<HashSet, AggHashSetOfOneNumberKey<logical_type, HashSet>> {
@ -147,12 +137,10 @@ struct AggHashSetOfOneNumberKey : public AggHashSet<HashSet, AggHashSetOfOneNumb
if constexpr (is_no_prefetch_set<HashSet>) {
this->template build_set_noprefetch<compute_and_allocate>(chunk_size, key_columns, pool, not_founds);
} else if (this->hash_set.bucket_count() < prefetch_threhold) {
this->template build_set_noprefetch<compute_and_allocate>(chunk_size, key_columns, pool, not_founds);
} else {
if (this->hash_set.bucket_count() < prefetch_threhold) {
this->template build_set_noprefetch<compute_and_allocate>(chunk_size, key_columns, pool, not_founds);
} else {
this->template build_set_prefetch<compute_and_allocate>(chunk_size, key_columns, pool, not_founds);
}
this->template build_set_prefetch<compute_and_allocate>(chunk_size, key_columns, pool, not_founds);
}
}
@ -754,10 +742,94 @@ struct AggHashSetOfSerializedKeyFixedSize : public AggHashSet<HashSet, AggHashSe
uint8_t* buffer;
ResultVector results;
Buffer<Slice> tmp_slices;
// std::vector<Slice> tmp_slices;
int32_t _chunk_size;
std::vector<size_t> hashes;
};
template <typename HashSet>
struct AggHashSetCompressedFixedSize : public AggHashSet<HashSet, AggHashSetCompressedFixedSize<HashSet>> {
using Base = AggHashSet<HashSet, AggHashSetCompressedFixedSize<HashSet>>;
using Iterator = typename HashSet::iterator;
using KeyType = typename HashSet::key_type;
using FixedSizeSliceKey = typename HashSet::key_type;
using ResultVector = typename std::vector<FixedSizeSliceKey>;
bool has_null_column = false;
static constexpr size_t max_fixed_size = sizeof(FixedSizeSliceKey);
template <class... Args>
AggHashSetCompressedFixedSize(int32_t chunk_size, Args&&... args)
: Base(chunk_size, std::forward<Args>(args)...), _chunk_size(chunk_size) {
fixed_keys.reserve(chunk_size);
}
// When compute_and_allocate=false:
// Elements queried in HashSet will be added to HashSet
// elements that cannot be queried are not processed,
// and are mainly used in the first stage of two-stage aggregation when aggr reduction is low
template <bool compute_and_allocate>
void build_set(size_t chunk_size, const Columns& key_columns, MemPool* pool, Filter* not_founds) {
if constexpr (!compute_and_allocate) {
DCHECK(not_founds);
not_founds->assign(chunk_size, 0);
}
auto* buffer = reinterpret_cast<uint8_t*>(fixed_keys.data());
memset(buffer, 0x0, sizeof(FixedSizeSliceKey) * chunk_size);
bitcompress_serialize(key_columns, bases, offsets, chunk_size, sizeof(FixedSizeSliceKey), fixed_keys.data());
if constexpr (is_no_prefetch_set<HashSet>) {
this->template build_set_noprefetch<compute_and_allocate>(chunk_size, pool, not_founds);
} else if (this->hash_set.bucket_count() < prefetch_threhold) {
this->template build_set_noprefetch<compute_and_allocate>(chunk_size, pool, not_founds);
} else {
this->template build_set_prefetch<compute_and_allocate>(chunk_size, pool, not_founds);
}
}
template <bool compute_and_allocate>
ALWAYS_NOINLINE void build_set_prefetch(size_t chunk_size, MemPool* pool, Filter* not_founds) {
auto* keys = reinterpret_cast<FixedSizeSliceKey*>(fixed_keys.data());
AGG_HASH_SET_PRECOMPUTE_HASH_VALS();
for (size_t i = 0; i < chunk_size; ++i) {
AGG_HASH_SET_PREFETCH_HASH_VAL();
if constexpr (compute_and_allocate) {
this->hash_set.emplace_with_hash(hashes[i], keys[i]);
} else {
(*not_founds)[i] = this->hash_set.find(keys[i], hashes[i]) == this->hash_set.end();
}
}
}
template <bool compute_and_allocate>
ALWAYS_NOINLINE void build_set_noprefetch(size_t chunk_size, MemPool* pool, Filter* not_founds) {
for (size_t i = 0; i < chunk_size; ++i) {
if constexpr (compute_and_allocate) {
this->hash_set.insert(fixed_keys[i]);
} else {
(*not_founds)[i] = !this->hash_set.contains(fixed_keys[i]);
}
}
}
void insert_keys_to_columns(ResultVector& keys, Columns& key_columns, int32_t chunk_size) {
bitcompress_deserialize(key_columns, bases, offsets, used_bits, chunk_size, sizeof(FixedSizeSliceKey),
keys.data());
}
static constexpr bool has_single_null_key = false;
bool has_null_key = false;
std::vector<int> used_bits;
std::vector<int> offsets;
std::vector<std::any> bases;
std::vector<FixedSizeSliceKey> fixed_keys;
std::vector<size_t> hashes;
ResultVector results;
int32_t _chunk_size;
};
} // namespace starrocks

View File

@ -15,11 +15,91 @@
#include "exec/aggregate/agg_hash_variant.h"
#include <tuple>
#include <type_traits>
#include <variant>
#include "runtime/runtime_state.h"
#include "util/phmap/phmap.h"
#define APPLY_FOR_AGG_VARIANT_ALL(M) \
M(phase1_uint8) \
M(phase1_int8) \
M(phase1_int16) \
M(phase1_int32) \
M(phase1_int64) \
M(phase1_int128) \
M(phase1_decimal32) \
M(phase1_decimal64) \
M(phase1_decimal128) \
M(phase1_decimal256) \
M(phase1_date) \
M(phase1_timestamp) \
M(phase1_string) \
M(phase1_slice) \
M(phase1_null_uint8) \
M(phase1_null_int8) \
M(phase1_null_int16) \
M(phase1_null_int32) \
M(phase1_null_int64) \
M(phase1_null_int128) \
M(phase1_null_decimal32) \
M(phase1_null_decimal64) \
M(phase1_null_decimal128) \
M(phase1_null_decimal256) \
M(phase1_null_date) \
M(phase1_null_timestamp) \
M(phase1_null_string) \
M(phase1_slice_two_level) \
M(phase1_int32_two_level) \
M(phase1_null_string_two_level) \
M(phase1_string_two_level) \
\
M(phase2_uint8) \
M(phase2_int8) \
M(phase2_int16) \
M(phase2_int32) \
M(phase2_int64) \
M(phase2_int128) \
M(phase2_decimal32) \
M(phase2_decimal64) \
M(phase2_decimal128) \
M(phase2_decimal256) \
M(phase2_date) \
M(phase2_timestamp) \
M(phase2_string) \
M(phase2_slice) \
M(phase2_null_uint8) \
M(phase2_null_int8) \
M(phase2_null_int16) \
M(phase2_null_int32) \
M(phase2_null_int64) \
M(phase2_null_int128) \
M(phase2_null_decimal32) \
M(phase2_null_decimal64) \
M(phase2_null_decimal128) \
M(phase2_null_decimal256) \
M(phase2_null_date) \
M(phase2_null_timestamp) \
M(phase2_null_string) \
M(phase2_slice_two_level) \
M(phase2_int32_two_level) \
M(phase2_null_string_two_level) \
M(phase2_string_two_level) \
\
M(phase1_slice_fx4) \
M(phase1_slice_fx8) \
M(phase1_slice_fx16) \
M(phase2_slice_fx4) \
M(phase2_slice_fx8) \
M(phase2_slice_fx16) \
M(phase1_slice_cx1) \
M(phase1_slice_cx4) \
M(phase1_slice_cx8) \
M(phase1_slice_cx16) \
M(phase2_slice_cx1) \
M(phase2_slice_cx4) \
M(phase2_slice_cx8) \
M(phase2_slice_cx16)
namespace starrocks {
namespace detail {
template <AggHashMapVariant::Type>
@ -65,6 +145,10 @@ DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_string_two_level, OneStringTwoLe
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_slice_fx4, SerializedKeyFixedSize4AggHashMap<PhmapSeed1>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_slice_fx8, SerializedKeyFixedSize8AggHashMap<PhmapSeed1>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_slice_fx16, SerializedKeyFixedSize16AggHashMap<PhmapSeed1>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_slice_cx1, CompressedFixedSize1AggHashMap<PhmapSeed1>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_slice_cx4, CompressedFixedSize4AggHashMap<PhmapSeed1>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_slice_cx8, CompressedFixedSize8AggHashMap<PhmapSeed1>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase1_slice_cx16, CompressedFixedSize16AggHashMap<PhmapSeed1>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_uint8, UInt8AggHashMapWithOneNumberKey<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_int8, Int8AggHashMapWithOneNumberKey<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_int16, Int16AggHashMapWithOneNumberKey<PhmapSeed2>);
@ -99,6 +183,10 @@ DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_string_two_level, OneStringTwoLe
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_slice_fx4, SerializedKeyFixedSize4AggHashMap<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_slice_fx8, SerializedKeyFixedSize8AggHashMap<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_slice_fx16, SerializedKeyFixedSize16AggHashMap<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_slice_cx1, CompressedFixedSize1AggHashMap<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_slice_cx4, CompressedFixedSize4AggHashMap<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_slice_cx8, CompressedFixedSize8AggHashMap<PhmapSeed2>);
DEFINE_MAP_TYPE(AggHashMapVariant::Type::phase2_slice_cx16, CompressedFixedSize16AggHashMap<PhmapSeed2>);
template <AggHashSetVariant::Type>
struct AggHashSetVariantTypeTraits;
@ -180,6 +268,15 @@ DEFINE_SET_TYPE(AggHashSetVariant::Type::phase2_slice_fx4, SerializedKeyAggHashS
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase2_slice_fx8, SerializedKeyAggHashSetFixedSize8<PhmapSeed2>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase2_slice_fx16, SerializedKeyAggHashSetFixedSize16<PhmapSeed2>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase1_slice_cx1, CompressedAggHashSetFixedSize1<PhmapSeed1>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase1_slice_cx4, CompressedAggHashSetFixedSize4<PhmapSeed1>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase1_slice_cx8, CompressedAggHashSetFixedSize8<PhmapSeed1>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase1_slice_cx16, CompressedAggHashSetFixedSize16<PhmapSeed1>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase2_slice_cx1, CompressedAggHashSetFixedSize1<PhmapSeed2>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase2_slice_cx4, CompressedAggHashSetFixedSize4<PhmapSeed2>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase2_slice_cx8, CompressedAggHashSetFixedSize8<PhmapSeed2>);
DEFINE_SET_TYPE(AggHashSetVariant::Type::phase2_slice_cx16, CompressedAggHashSetFixedSize16<PhmapSeed2>);
} // namespace detail
void AggHashMapVariant::init(RuntimeState* state, Type type, AggStatistics* agg_stat) {
_type = type;

View File

@ -17,93 +17,15 @@
#pragma once
#include <type_traits>
#include <utility>
#include <variant>
#include "column/hash_set.h"
#include "exec/aggregate/agg_hash_map.h"
#include "exec/aggregate/agg_hash_set.h"
#include "exec/aggregate/agg_profile.h"
#include "types/logical_type.h"
#include "util/phmap/phmap.h"
namespace starrocks {
enum AggrPhase { AggrPhase1, AggrPhase2 };
#define APPLY_FOR_AGG_VARIANT_ALL(M) \
M(phase1_uint8) \
M(phase1_int8) \
M(phase1_int16) \
M(phase1_int32) \
M(phase1_int64) \
M(phase1_int128) \
M(phase1_decimal32) \
M(phase1_decimal64) \
M(phase1_decimal128) \
M(phase1_decimal256) \
M(phase1_date) \
M(phase1_timestamp) \
M(phase1_string) \
M(phase1_slice) \
M(phase1_null_uint8) \
M(phase1_null_int8) \
M(phase1_null_int16) \
M(phase1_null_int32) \
M(phase1_null_int64) \
M(phase1_null_int128) \
M(phase1_null_decimal32) \
M(phase1_null_decimal64) \
M(phase1_null_decimal128) \
M(phase1_null_decimal256) \
M(phase1_null_date) \
M(phase1_null_timestamp) \
M(phase1_null_string) \
M(phase1_slice_two_level) \
M(phase1_int32_two_level) \
M(phase1_null_string_two_level) \
M(phase1_string_two_level) \
\
M(phase2_uint8) \
M(phase2_int8) \
M(phase2_int16) \
M(phase2_int32) \
M(phase2_int64) \
M(phase2_int128) \
M(phase2_decimal32) \
M(phase2_decimal64) \
M(phase2_decimal128) \
M(phase2_decimal256) \
M(phase2_date) \
M(phase2_timestamp) \
M(phase2_string) \
M(phase2_slice) \
M(phase2_null_uint8) \
M(phase2_null_int8) \
M(phase2_null_int16) \
M(phase2_null_int32) \
M(phase2_null_int64) \
M(phase2_null_int128) \
M(phase2_null_decimal32) \
M(phase2_null_decimal64) \
M(phase2_null_decimal128) \
M(phase2_null_decimal256) \
M(phase2_null_date) \
M(phase2_null_timestamp) \
M(phase2_null_string) \
M(phase2_slice_two_level) \
M(phase2_int32_two_level) \
M(phase2_null_string_two_level) \
M(phase2_string_two_level) \
\
M(phase1_slice_fx4) \
M(phase1_slice_fx8) \
M(phase1_slice_fx16) \
M(phase2_slice_fx4) \
M(phase2_slice_fx8) \
M(phase2_slice_fx16)
// Aggregate Hash maps
// no-nullable single key maps:
@ -187,6 +109,16 @@ using SerializedKeyFixedSize8AggHashMap = AggHashMapWithSerializedKeyFixedSize<F
template <PhmapSeed seed>
using SerializedKeyFixedSize16AggHashMap = AggHashMapWithSerializedKeyFixedSize<FixedSize16SliceAggHashMap<seed>>;
// fixed compress key
template <PhmapSeed seed>
using CompressedFixedSize1AggHashMap = AggHashMapWithCompressedKeyFixedSize<Int8AggHashMap<seed>>;
template <PhmapSeed seed>
using CompressedFixedSize4AggHashMap = AggHashMapWithCompressedKeyFixedSize<Int32AggHashMap<seed>>;
template <PhmapSeed seed>
using CompressedFixedSize8AggHashMap = AggHashMapWithCompressedKeyFixedSize<Int64AggHashMap<seed>>;
template <PhmapSeed seed>
using CompressedFixedSize16AggHashMap = AggHashMapWithCompressedKeyFixedSize<Int128AggHashMap<seed>>;
// Hash sets
//
template <PhmapSeed seed>
@ -270,6 +202,15 @@ using SerializedKeyAggHashSetFixedSize8 = AggHashSetOfSerializedKeyFixedSize<Fix
template <PhmapSeed seed>
using SerializedKeyAggHashSetFixedSize16 = AggHashSetOfSerializedKeyFixedSize<FixedSize16SliceAggHashSet<seed>>;
template <PhmapSeed seed>
using CompressedAggHashSetFixedSize1 = AggHashSetCompressedFixedSize<Int8AggHashSet<seed>>;
template <PhmapSeed seed>
using CompressedAggHashSetFixedSize4 = AggHashSetCompressedFixedSize<Int32AggHashSet<seed>>;
template <PhmapSeed seed>
using CompressedAggHashSetFixedSize8 = AggHashSetCompressedFixedSize<Int64AggHashSet<seed>>;
template <PhmapSeed seed>
using CompressedAggHashSetFixedSize16 = AggHashSetCompressedFixedSize<Int128AggHashSet<seed>>;
// aggregate key
template <class HashMapWithKey>
struct CombinedFixedSizeKey {
@ -294,6 +235,24 @@ static_assert(!is_combined_fixed_size_key<Int32TwoLevelAggHashSetOfOneNumberKey<
static_assert(is_combined_fixed_size_key<SerializedKeyAggHashSetFixedSize4<PhmapSeed1>>);
static_assert(!is_combined_fixed_size_key<Int32TwoLevelAggHashMapWithOneNumberKey<PhmapSeed1>>);
template <class HashMapWithKey>
struct CompressedFixedSizeKey {
static auto constexpr value = false;
};
template <typename HashMap>
struct CompressedFixedSizeKey<AggHashMapWithCompressedKeyFixedSize<HashMap>> {
static auto constexpr value = true;
};
template <typename HashSet>
struct CompressedFixedSizeKey<AggHashSetCompressedFixedSize<HashSet>> {
static auto constexpr value = true;
};
template <typename HashMapOrSetWithKey>
inline constexpr bool is_compressed_fixed_size_key = CompressedFixedSizeKey<HashMapOrSetWithKey>::value;
// 1) For different group by columns type, size, cardinality, volume, we should choose different
// hash functions and different hashmaps.
// When runtime, we will only have one hashmap.
@ -341,6 +300,10 @@ using AggHashMapWithKeyPtr = std::variant<
std::unique_ptr<SerializedKeyFixedSize4AggHashMap<PhmapSeed1>>,
std::unique_ptr<SerializedKeyFixedSize8AggHashMap<PhmapSeed1>>,
std::unique_ptr<SerializedKeyFixedSize16AggHashMap<PhmapSeed1>>,
std::unique_ptr<CompressedFixedSize1AggHashMap<PhmapSeed1>>,
std::unique_ptr<CompressedFixedSize4AggHashMap<PhmapSeed1>>,
std::unique_ptr<CompressedFixedSize8AggHashMap<PhmapSeed1>>,
std::unique_ptr<CompressedFixedSize16AggHashMap<PhmapSeed1>>,
std::unique_ptr<UInt8AggHashMapWithOneNumberKey<PhmapSeed2>>,
std::unique_ptr<Int8AggHashMapWithOneNumberKey<PhmapSeed2>>,
std::unique_ptr<Int16AggHashMapWithOneNumberKey<PhmapSeed2>>,
@ -373,7 +336,11 @@ using AggHashMapWithKeyPtr = std::variant<
std::unique_ptr<NullOneStringTwoLevelAggHashMap<PhmapSeed2>>,
std::unique_ptr<SerializedKeyFixedSize4AggHashMap<PhmapSeed2>>,
std::unique_ptr<SerializedKeyFixedSize8AggHashMap<PhmapSeed2>>,
std::unique_ptr<SerializedKeyFixedSize16AggHashMap<PhmapSeed2>>>;
std::unique_ptr<SerializedKeyFixedSize16AggHashMap<PhmapSeed2>>,
std::unique_ptr<CompressedFixedSize1AggHashMap<PhmapSeed2>>,
std::unique_ptr<CompressedFixedSize4AggHashMap<PhmapSeed2>>,
std::unique_ptr<CompressedFixedSize8AggHashMap<PhmapSeed2>>,
std::unique_ptr<CompressedFixedSize16AggHashMap<PhmapSeed2>>>;
using AggHashSetWithKeyPtr = std::variant<
std::unique_ptr<UInt8AggHashSetOfOneNumberKey<PhmapSeed1>>,
@ -441,7 +408,16 @@ using AggHashSetWithKeyPtr = std::variant<
std::unique_ptr<SerializedKeyAggHashSetFixedSize16<PhmapSeed1>>,
std::unique_ptr<SerializedKeyAggHashSetFixedSize4<PhmapSeed2>>,
std::unique_ptr<SerializedKeyAggHashSetFixedSize8<PhmapSeed2>>,
std::unique_ptr<SerializedKeyAggHashSetFixedSize16<PhmapSeed2>>>;
std::unique_ptr<SerializedKeyAggHashSetFixedSize16<PhmapSeed2>>,
std::unique_ptr<CompressedAggHashSetFixedSize1<PhmapSeed1>>,
std::unique_ptr<CompressedAggHashSetFixedSize4<PhmapSeed1>>,
std::unique_ptr<CompressedAggHashSetFixedSize8<PhmapSeed1>>,
std::unique_ptr<CompressedAggHashSetFixedSize16<PhmapSeed1>>,
std::unique_ptr<CompressedAggHashSetFixedSize1<PhmapSeed2>>,
std::unique_ptr<CompressedAggHashSetFixedSize4<PhmapSeed2>>,
std::unique_ptr<CompressedAggHashSetFixedSize8<PhmapSeed2>>,
std::unique_ptr<CompressedAggHashSetFixedSize16<PhmapSeed2>>>;
} // namespace detail
struct AggHashMapVariant {
enum class Type {
@ -481,6 +457,11 @@ struct AggHashMapVariant {
phase1_slice_fx8,
phase1_slice_fx16,
phase1_slice_cx1,
phase1_slice_cx4,
phase1_slice_cx8,
phase1_slice_cx16,
phase2_uint8,
phase2_int8,
phase2_int16,
@ -517,6 +498,10 @@ struct AggHashMapVariant {
phase2_slice_fx8,
phase2_slice_fx16,
phase2_slice_cx1,
phase2_slice_cx4,
phase2_slice_cx8,
phase2_slice_cx16,
};
detail::AggHashMapWithKeyPtr hash_map_with_key;
@ -630,6 +615,14 @@ struct AggHashSetVariant {
phase2_slice_fx8,
phase2_slice_fx16,
phase1_slice_cx1,
phase1_slice_cx4,
phase1_slice_cx8,
phase1_slice_cx16,
phase2_slice_cx1,
phase2_slice_cx4,
phase2_slice_cx8,
phase2_slice_cx16,
};
detail::AggHashSetWithKeyPtr hash_set_with_key;

View File

@ -14,7 +14,7 @@
#include "exec/aggregate/aggregate_base_node.h"
#include "gutil/strings/substitute.h"
#include "exec/aggregator.h"
namespace starrocks {

View File

@ -14,9 +14,7 @@
#pragma once
#include <any>
#include "exec/aggregator.h"
#include "exec/aggregator_fwd.h"
#include "exec/exec_node.h"
namespace starrocks {

View File

@ -16,13 +16,10 @@
#include <memory>
#include <type_traits>
#include <variant>
#include "exec/aggregator.h"
#include "exec/pipeline/aggregate/aggregate_blocking_sink_operator.h"
#include "exec/pipeline/aggregate/aggregate_blocking_source_operator.h"
#include "exec/pipeline/aggregate/aggregate_streaming_sink_operator.h"
#include "exec/pipeline/aggregate/aggregate_streaming_source_operator.h"
#include "exec/pipeline/aggregate/sorted_aggregate_streaming_sink_operator.h"
#include "exec/pipeline/aggregate/sorted_aggregate_streaming_source_operator.h"
#include "exec/pipeline/aggregate/spillable_aggregate_blocking_sink_operator.h"
@ -32,12 +29,8 @@
#include "exec/pipeline/chunk_accumulate_operator.h"
#include "exec/pipeline/exchange/local_exchange_source_operator.h"
#include "exec/pipeline/limit_operator.h"
#include "exec/pipeline/noop_sink_operator.h"
#include "exec/pipeline/operator.h"
#include "exec/pipeline/pipeline_builder.h"
#include "exec/pipeline/spill_process_operator.h"
#include "exec/sorted_streaming_aggregator.h"
#include "gutil/casts.h"
#include "runtime/current_thread.h"
#include "simd/simd.h"
@ -121,8 +114,7 @@ Status AggregateBlockingNode::open(RuntimeState* state) {
if (_aggregator->hash_map_variant().size() == 0) {
_aggregator->set_ht_eos();
}
_aggregator->hash_map_variant().visit(
[&](auto& hash_map_with_key) { _aggregator->it_hash() = _aggregator->_state_allocator.begin(); });
_aggregator->it_hash() = _aggregator->state_allocator().begin();
} else if (_aggregator->is_none_group_by_exprs()) {
// for aggregate no group by, if _num_input_rows is 0,
// In update phase, we directly return empty chunk.

View File

@ -204,7 +204,7 @@ Status AggregateStreamingNode::get_next(RuntimeState* state, ChunkPtr* chunk, bo
Status AggregateStreamingNode::_output_chunk_from_hash_map(ChunkPtr* chunk) {
if (!_aggregator->it_hash().has_value()) {
_aggregator->it_hash() = _aggregator->_state_allocator.begin();
_aggregator->it_hash() = _aggregator->state_allocator().begin();
COUNTER_SET(_aggregator->hash_table_size(), (int64_t)_aggregator->hash_map_variant().size());
}

View File

@ -0,0 +1,295 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include <any>
#include <optional>
#include "column/column_helper.h"
#include "column/column_visitor_adapter.h"
#include "column/decimalv3_column.h"
#include "column/nullable_column.h"
#include "common/status.h"
#include "exprs/literal.h"
#include "types/logical_type_infra.h"
#include "util/unaligned_access.h"
namespace starrocks {
template <size_t N>
struct int_type {};
template <>
struct int_type<1> {
using type = int8_t;
};
template <>
struct int_type<2> {
using type = int16_t;
};
template <>
struct int_type<4> {
using type = int32_t;
};
template <>
struct int_type<8> {
using type = int64_t;
};
template <>
struct int_type<16> {
using type = __int128;
};
template <class T>
int leading_zeros(T v) {
if (v == 0) return sizeof(T) * 8;
typename std::make_unsigned<T>::type uv = v;
return __builtin_clzll(static_cast<size_t>(uv)) - (sizeof(size_t) * 8 - sizeof(T) * 8);
}
template <>
int leading_zeros<int128_t>(int128_t v) {
uint64_t high = (uint64_t)(v >> 64);
uint64_t low = (uint64_t)v;
if (high != 0) {
return leading_zeros(high);
} else {
return 64 + leading_zeros(low);
}
}
template <class T>
int get_used_bits(T min, T max) {
using IntType = typename int_type<sizeof(T)>::type;
auto vmin = unaligned_load<IntType>(&min);
auto vmax = unaligned_load<IntType>(&max);
IntType delta = vmax - vmin;
return sizeof(T) * 8 - (leading_zeros<IntType>(delta));
}
std::optional<int> get_used_bits(LogicalType ltype, const VectorizedLiteral& begin, const VectorizedLiteral& end,
std::any& base) {
size_t used_bits = 0;
bool applied = scalar_type_dispatch(ltype, [&]<LogicalType Type>() {
if constexpr ((lt_is_integer<Type> || lt_is_decimal<Type> ||
lt_is_date<Type>)&&(sizeof(RunTimeCppType<Type>) <= 16)) {
RunTimeCppType<Type> cs_min = ColumnHelper::get_const_value<Type>(begin.value().get());
RunTimeCppType<Type> cs_max = ColumnHelper::get_const_value<Type>(end.value().get());
base = cs_min;
used_bits = get_used_bits(cs_min, cs_max);
return true;
}
return false;
});
if (applied) {
return used_bits;
}
return {};
}
template <class TSrc, class TDst>
void bitcompress_serialize(const TSrc* __restrict val, const uint8_t* __restrict nulls, TSrc base, size_t n, int offset,
TDst* __restrict dst) {
using UTSrc = typename std::make_unsigned<TSrc>::type;
if (nulls == nullptr) {
for (size_t i = 0; i < n; ++i) {
TDst v = UTSrc(val[i] - base);
dst[i] |= v << offset;
}
} else {
for (size_t i = 0; i < n; ++i) {
TDst v = UTSrc(val[i] - base) & ~(-static_cast<TSrc>(nulls[i]));
dst[i] |= TDst(nulls[i]) << offset;
dst[i] |= v << (offset + 1);
}
}
}
template <class Dst>
class CompressSerializer : public ColumnVisitorAdapter<CompressSerializer<Dst>> {
public:
using Base = ColumnVisitorAdapter<CompressSerializer<Dst>>;
CompressSerializer(Dst* dst, const std::any& base, int offset)
: Base(this), _dst(dst), _base(base), _offset(offset) {}
Status do_visit(const NullableColumn& column) {
_null_data = column.null_column_data().data();
return column.data_column()->accept(this);
}
template <typename Column, typename T>
void bit_compress(const Column& column) {
if constexpr (sizeof(T) == 1 || sizeof(T) == 2 || sizeof(T) == 4 || sizeof(T) == 8 || sizeof(T) == 16) {
using SrcType = typename int_type<sizeof(T)>::type;
const auto& container = column.get_data();
const auto& raw_data = container.data();
size_t n = container.size();
auto base = std::any_cast<T>(_base);
auto tbase = unaligned_load<SrcType>(&base);
bitcompress_serialize((SrcType*)raw_data, _null_data, tbase, n, _offset, _dst);
} else {
CHECK(false) << "unreachable";
}
}
template <typename T>
Status do_visit(const FixedLengthColumn<T>& column) {
bit_compress<FixedLengthColumn<T>, T>(column);
return Status::OK();
}
template <typename T>
Status do_visit(const DecimalV3Column<T>& column) {
bit_compress<DecimalV3Column<T>, T>(column);
return Status::OK();
}
template <typename T>
Status do_visit(const T& column) {
CHECK(false) << "unreachable";
return Status::NotSupported("unsupported type");
}
private:
Dst* _dst;
const std::any& _base;
int _offset;
const uint8_t* _null_data = nullptr;
};
template <class T>
T mask(T bits) {
return (T(1) << bits) - 1;
}
template <class TSrc, class TDst>
void bitcompress_deserialize(const TSrc* __restrict src, uint8_t* __restrict nulls, TDst base, int n, int used_bits,
int offset, TDst* __restrict dst) {
typename std::make_unsigned<TSrc>::type* usrc = (typename std::make_unsigned<TSrc>::type*)src;
const uint8_t mask1 = mask<uint8_t>(1);
const TSrc mask2 = mask<TSrc>(used_bits - offset - (nulls != nullptr));
if (nulls == nullptr) {
for (size_t i = 0; i < n; ++i) {
dst[i] = ((usrc[i] >> (offset)) & mask2) + base;
}
} else {
for (size_t i = 0; i < n; ++i) {
nulls[i] = (usrc[i] >> offset) & mask1;
dst[i] = ((usrc[i] >> (offset + 1)) & mask2) + base;
}
}
}
template <class Src>
class CompressDeserializer final : public ColumnVisitorMutableAdapter<CompressDeserializer<Src>> {
public:
using Base = ColumnVisitorMutableAdapter<CompressDeserializer<Src>>;
explicit CompressDeserializer(size_t num_rows, Src* src, const std::any& base, int offset, int used_bits)
: Base(this), _num_rows(num_rows), _src(src), _base(base), _offset(offset), _used_bits(used_bits) {}
Status do_visit(NullableColumn* column) {
// TODO: opt me
column->null_column_data().resize(_num_rows);
_null_data = column->null_column_data().data();
RETURN_IF_ERROR(column->data_column()->accept_mutable(this));
column->update_has_null();
return Status::OK();
}
template <typename Column, typename T>
void bit_decompress(Column* column) {
if constexpr (sizeof(T) == 1 || sizeof(T) == 2 || sizeof(T) == 4 || sizeof(T) == 8 || sizeof(T) == 16) {
using DstType = typename int_type<sizeof(T)>::type;
column->resize(_num_rows);
auto& container = column->get_data();
auto* raw_data = container.data();
auto base = std::any_cast<T>(_base);
auto tbase = unaligned_load<DstType>(&base);
bitcompress_deserialize(_src, _null_data, tbase, _num_rows, _used_bits, _offset, (DstType*)raw_data);
} else {
CHECK(false) << "unreachable";
}
}
template <typename T>
Status do_visit(FixedLengthColumn<T>* column) {
bit_decompress<FixedLengthColumn<T>, T>(column);
return Status::OK();
}
template <typename T>
Status do_visit(const DecimalV3Column<T>& column) {
bit_decompress<DecimalV3Column<T>, T>(column);
return Status::OK();
}
template <typename T>
Status do_visit(const T& column) {
DCHECK(false) << "unreachable";
return Status::NotSupported("unsupported type");
}
private:
size_t _num_rows;
const Src* _src;
const std::any& _base;
int _offset;
int _used_bits;
uint8_t* _null_data = nullptr;
};
void bitcompress_serialize(const Columns& columns, const std::vector<std::any>& bases, const std::vector<int>& offsets,
size_t num_rows, size_t fixed_key_size, void* buffer) {
for (size_t i = 0; i < columns.size(); ++i) {
if (fixed_key_size == 1) {
CompressSerializer<uint8_t> serializer((uint8_t*)buffer, bases[i], offsets[i]);
(void)columns[i]->accept(&serializer);
} else if (fixed_key_size == 4) {
CompressSerializer<int> serializer((int*)buffer, bases[i], offsets[i]);
(void)columns[i]->accept(&serializer);
} else if (fixed_key_size == 8) {
CompressSerializer<int64_t> serializer((int64_t*)buffer, bases[i], offsets[i]);
(void)columns[i]->accept(&serializer);
} else if (fixed_key_size == 16) {
CompressSerializer<int128_t> serializer((int128_t*)buffer, bases[i], offsets[i]);
(void)columns[i]->accept(&serializer);
} else {
DCHECK(false) << "unreachable path";
}
}
}
void bitcompress_deserialize(Columns& columns, const std::vector<std::any>& bases, const std::vector<int>& offsets,
const std::vector<int>& used_bits, size_t num_rows, size_t fixed_key_size, void* buffer) {
for (size_t i = 0; i < columns.size(); ++i) {
if (fixed_key_size == 1) {
CompressDeserializer<uint8_t> deserializer(num_rows, (uint8_t*)buffer, bases[i], offsets[i], used_bits[i]);
(void)columns[i]->accept_mutable(&deserializer);
} else if (fixed_key_size == 4) {
CompressDeserializer<int> deserializer(num_rows, (int*)buffer, bases[i], offsets[i], used_bits[i]);
(void)columns[i]->accept_mutable(&deserializer);
} else if (fixed_key_size == 8) {
CompressDeserializer<int64_t> deserializer(num_rows, (int64_t*)buffer, bases[i], offsets[i], used_bits[i]);
(void)columns[i]->accept_mutable(&deserializer);
} else if (fixed_key_size == 16) {
CompressDeserializer<int128_t> deserializer(num_rows, (int128_t*)buffer, bases[i], offsets[i],
used_bits[i]);
(void)columns[i]->accept_mutable(&deserializer);
} else {
DCHECK(false) << "unreachable path";
}
}
}
} // namespace starrocks

View File

@ -0,0 +1,48 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <any>
#include "column/column.h"
#include "types/logical_type.h"
namespace starrocks {
class VectorizedLiteral;
/**
* Calculates the number of bits used between a given range for a specified logical type.
*
* This function calculates the number of bits required for a given logical type and a specified range
* of start and end values. The result is an optional integer representing the calculated number of bits.
*
* If we input a column that does not support bit compress, we will return an empty optional.
*/
std::optional<int> get_used_bits(LogicalType ltype, const VectorizedLiteral& begin, const VectorizedLiteral& end,
std::any& base);
/**
* serialize column data into a bit-compressed format.
*/
void bitcompress_serialize(const Columns& columns, const std::vector<std::any>& bases, const std::vector<int>& offsets,
size_t num_rows, size_t fixed_key_size, void* buffer);
/**
* deserialize column data from a bit-compressed format.
*
*/
void bitcompress_deserialize(Columns& columns, const std::vector<std::any>& bases, const std::vector<int>& offsets,
const std::vector<int>& used_bits, size_t num_rows, size_t fixed_key_size, void* buffer);
} // namespace starrocks

View File

@ -16,28 +16,31 @@
#include <algorithm>
#include <memory>
#include <optional>
#include <type_traits>
#include <variant>
#include <utility>
#include "column/chunk.h"
#include "column/column_helper.h"
#include "column/vectorized_fwd.h"
#include "common/config.h"
#include "common/logging.h"
#include "common/status.h"
#include "exec/agg_runtime_filter_builder.h"
#include "exec/aggregate/agg_hash_variant.h"
#include "exec/aggregate/agg_profile.h"
#include "exec/exec_node.h"
#include "exec/limited_pipeline_chunk_buffer.h"
#include "exec/pipeline/operator.h"
#include "exec/spill/spiller.hpp"
#include "exprs/agg/agg_state_if.h"
#include "exprs/agg/agg_state_merge.h"
#include "exprs/agg/agg_state_union.h"
#include "exprs/agg/aggregate_factory.h"
#include "exprs/agg/aggregate_state_allocator.h"
#include "exprs/literal.h"
#include "gen_cpp/PlanNodes_types.h"
#include "runtime/current_thread.h"
#include "runtime/descriptors.h"
#include "runtime/memory/roaring_hook.h"
#include "types/logical_type.h"
#include "udf/java/utils.h"
#include "util/runtime_profile.h"
@ -52,6 +55,60 @@ static const std::string AGG_STATE_MERGE_SUFFIX = "_merge";
static const std::string AGG_STATE_IF_SUFFIX = "_if";
static const std::string FUNCTION_COUNT = "count";
template <class HashMapWithKey>
struct AllocateState {
AllocateState(Aggregator* aggregator_) : aggregator(aggregator_) {}
inline AggDataPtr operator()(const typename HashMapWithKey::KeyType& key);
inline AggDataPtr operator()(std::nullptr_t);
private:
Aggregator* aggregator;
};
template <class HashMapWithKey>
inline AggDataPtr AllocateState<HashMapWithKey>::operator()(const typename HashMapWithKey::KeyType& key) {
AggDataPtr agg_state = aggregator->_state_allocator.allocate();
*reinterpret_cast<typename HashMapWithKey::KeyType*>(agg_state) = key;
size_t created = 0;
size_t aggregate_function_sz = aggregator->_agg_fn_ctxs.size();
try {
for (int i = 0; i < aggregate_function_sz; i++) {
aggregator->_agg_functions[i]->create(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
created++;
}
return agg_state;
} catch (std::bad_alloc& e) {
for (size_t i = 0; i < created; ++i) {
aggregator->_agg_functions[i]->destroy(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
}
aggregator->_state_allocator.rollback();
throw;
}
}
template <class HashMapWithKey>
inline AggDataPtr AllocateState<HashMapWithKey>::operator()(std::nullptr_t) {
AggDataPtr agg_state = aggregator->_state_allocator.allocate_null_key_data();
size_t created = 0;
size_t aggregate_function_sz = aggregator->_agg_fn_ctxs.size();
try {
for (int i = 0; i < aggregate_function_sz; i++) {
aggregator->_agg_functions[i]->create(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
created++;
}
return agg_state;
} catch (std::bad_alloc& e) {
for (int i = 0; i < created; i++) {
aggregator->_agg_functions[i]->destroy(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
}
throw;
}
}
template <bool UseIntermediateAsOutput>
bool AggFunctionTypes::is_result_nullable() const {
if constexpr (UseIntermediateAsOutput) {
@ -143,6 +200,9 @@ AggregatorParamsPtr convert_to_aggregator_params(const TPlanNode& tnode) {
params->intermediate_aggr_exprs = tnode.agg_node.intermediate_aggr_exprs;
params->enable_pipeline_share_limit =
tnode.agg_node.__isset.enable_pipeline_share_limit ? tnode.agg_node.enable_pipeline_share_limit : false;
params->grouping_min_max =
tnode.agg_node.__isset.group_by_min_max ? tnode.agg_node.group_by_min_max : std::vector<TExpr>{};
break;
}
default:
@ -358,6 +418,16 @@ Status Aggregator::prepare(RuntimeState* state, ObjectPool* pool, RuntimeProfile
RETURN_IF_ERROR(Expr::create_expr_trees(_pool, _params->conjuncts, &_conjunct_ctxs, state, true));
RETURN_IF_ERROR(Expr::create_expr_trees(_pool, _params->grouping_exprs, &_group_by_expr_ctxs, state, true));
RETURN_IF_ERROR(Expr::create_expr_trees(_pool, _params->grouping_min_max, &_group_by_min_max, state, true));
_ranges.resize(_group_by_expr_ctxs.size());
if (_group_by_min_max.size() == _group_by_expr_ctxs.size() * 2) {
for (size_t i = 0; i < _group_by_expr_ctxs.size(); ++i) {
std::pair<VectorizedLiteral*, VectorizedLiteral*> range;
range.first = down_cast<VectorizedLiteral*>(_group_by_min_max[i * 2]->root());
range.second = down_cast<VectorizedLiteral*>(_group_by_min_max[i * 2 + 1]->root());
_ranges[i] = range;
}
}
// add profile attributes
if (!_params->sql_grouping_keys.empty()) {
@ -582,7 +652,7 @@ Status Aggregator::_create_aggregate_function(starrocks::RuntimeState* state, co
TypeDescriptor return_type = TypeDescriptor::from_thrift(fn.ret_type);
TypeDescriptor serde_type = TypeDescriptor::from_thrift(fn.aggregate_fn.intermediate_type);
DCHECK_LE(1, fn.arg_types.size());
TypeDescriptor arg_type = arg_types[0];
const TypeDescriptor& arg_type = arg_types[0];
auto* func = get_aggregate_function(func_name, return_type, arg_types, is_result_nullable, fn.binary_type,
state->func_version());
if (func == nullptr) {
@ -1287,19 +1357,54 @@ Status Aggregator::evaluate_agg_fn_exprs(Chunk* chunk, bool use_intermediate) {
return Status::OK();
}
bool is_group_columns_fixed_size(std::vector<ExprContext*>& group_by_expr_ctxs, std::vector<ColumnType>& group_by_types,
size_t* max_size, bool* has_null) {
bool could_apply_bitcompress_opt(
const std::vector<ColumnType>& group_by_types,
const std::vector<std::optional<std::pair<VectorizedLiteral*, VectorizedLiteral*>>>& ranges,
std::vector<std::any>& base, std::vector<int>& used_bytes, size_t* max_size, bool* has_null) {
size_t accumulated = 0;
for (size_t i = 0; i < group_by_types.size(); i++) {
size_t size = 0;
// 1 bytes for null flag.
if (group_by_types[i].is_nullable) {
*has_null = true;
size += 1;
}
if (group_by_types[i].result_type.is_complex_type()) {
return false;
}
LogicalType ltype = group_by_types[i].result_type.type;
size_t fixed_base_size = get_size_of_fixed_length_type(ltype);
if (fixed_base_size == 0) return false;
if (!ranges[i].has_value()) {
return false;
}
auto used_bits = get_used_bits(ltype, *ranges[i]->first, *ranges[i]->second, base[i]);
if (!used_bits.has_value()) {
return false;
}
size += used_bits.value();
accumulated += size;
used_bytes[i] = accumulated;
}
*max_size = accumulated;
return true;
}
bool is_group_columns_fixed_size(std::vector<ColumnType>& group_by_types, size_t* max_size, bool* has_null) {
size_t size = 0;
*has_null = false;
for (size_t i = 0; i < group_by_expr_ctxs.size(); i++) {
ExprContext* ctx = group_by_expr_ctxs[i];
for (size_t i = 0; i < group_by_types.size(); i++) {
// 1 bytes for null flag.
if (group_by_types[i].is_nullable) {
*has_null = true;
size += 1; // 1 bytes for null flag.
size += 1;
}
LogicalType ltype = ctx->root()->type().type;
if (ctx->root()->type().is_complex_type()) {
LogicalType ltype = group_by_types[i].result_type.type;
if (group_by_types[i].result_type.is_complex_type()) {
return false;
}
size_t byte_size = get_size_of_fixed_length_type(ltype);
@ -1311,20 +1416,30 @@ bool is_group_columns_fixed_size(std::vector<ExprContext*>& group_by_expr_ctxs,
}
template <typename HashVariantType>
void Aggregator::_init_agg_hash_variant(HashVariantType& hash_variant) {
typename HashVariantType::Type Aggregator::_get_hash_table_type() {
auto type = _aggr_phase == AggrPhase1 ? HashVariantType::Type::phase1_slice : HashVariantType::Type::phase2_slice;
if (_group_by_expr_ctxs.size() == 1) {
type = HashVariantResolver<HashVariantType>::instance().get_unary_type(
_aggr_phase, _group_by_types[0].result_type.type, _has_nullable_key);
if (_group_by_types.empty()) {
return type;
}
// using one key hash table
if (_group_by_types.size() == 1) {
bool nullable = _group_by_types[0].is_nullable;
LogicalType type = _group_by_types[0].result_type.type;
return HashVariantResolver<HashVariantType>::instance().get_unary_type(_aggr_phase, type, nullable);
}
return type;
}
template <typename HashVariantType>
typename HashVariantType::Type Aggregator::_try_to_apply_fixed_size_opt(typename HashVariantType::Type type,
bool* has_null, int* fixed_size) {
bool has_null_column = false;
int fixed_byte_size = 0;
// this optimization don't need to be limited to multi-column group by.
// single column like float/double/decimal/largeint could also be applied to.
if (type == HashVariantType::Type::phase1_slice || type == HashVariantType::Type::phase2_slice) {
size_t max_size = 0;
if (is_group_columns_fixed_size(_group_by_expr_ctxs, _group_by_types, &max_size, &has_null_column)) {
if (is_group_columns_fixed_size(_group_by_types, &max_size, &has_null_column)) {
// we need reserve a byte for serialization length for nullable columns
if (max_size < 4 || (!has_null_column && max_size == 4)) {
type = _aggr_phase == AggrPhase1 ? HashVariantType::Type::phase1_slice_fx4
@ -1341,6 +1456,102 @@ void Aggregator::_init_agg_hash_variant(HashVariantType& hash_variant) {
}
}
}
*has_null = has_null_column;
*fixed_size = fixed_byte_size;
return type;
}
template <typename HashVariantType>
typename HashVariantType::Type Aggregator::_try_to_apply_compressed_key_opt(typename HashVariantType::Type input_type,
CompressKeyContext* ctx) {
typename HashVariantType::Type type = input_type;
if (_group_by_types.empty()) {
return type;
}
for (size_t i = 0; i < _ranges.size(); ++i) {
if (!_ranges[i].has_value()) {
return type;
}
}
// check apply bit compress opt
{
bool has_null_column;
size_t new_max_bit_size = 0;
std::vector<int>& offsets = ctx->offsets;
std::vector<int>& used_bits = ctx->used_bits;
std::vector<std::any>& bases = ctx->bases;
size_t group_by_keys = _group_by_types.size();
used_bits.resize(group_by_keys);
offsets.resize(group_by_keys);
bases.resize(group_by_keys);
if (could_apply_bitcompress_opt(_group_by_types, _ranges, bases, used_bits, &new_max_bit_size,
&has_null_column)) {
if (new_max_bit_size <= 8 && _group_by_types.size() == 1) {
type = _aggr_phase == AggrPhase1 ? HashVariantType::Type::phase1_slice_cx1
: HashVariantType::Type::phase2_slice_cx1;
} else if (_group_by_types.size() > 1) {
if (new_max_bit_size <= 8) {
type = _aggr_phase == AggrPhase1 ? HashVariantType::Type::phase1_slice_cx1
: HashVariantType::Type::phase2_slice_cx1;
} else if (new_max_bit_size <= 4 * 8) {
type = _aggr_phase == AggrPhase1 ? HashVariantType::Type::phase1_slice_cx4
: HashVariantType::Type::phase2_slice_cx4;
} else if (new_max_bit_size <= 8 * 8) {
type = _aggr_phase == AggrPhase1 ? HashVariantType::Type::phase1_slice_cx8
: HashVariantType::Type::phase2_slice_cx8;
} else if (new_max_bit_size <= 16 * 8) {
type = _aggr_phase == AggrPhase1 ? HashVariantType::Type::phase1_slice_cx16
: HashVariantType::Type::phase2_slice_cx16;
}
}
}
offsets[0] = 0;
for (size_t i = 1; i < group_by_keys; ++i) {
offsets[i] = used_bits[i - 1];
}
}
return type;
}
template <typename HashVariantType>
void Aggregator::_build_hash_variant(HashVariantType& hash_variant, typename HashVariantType::Type type,
CompressKeyContext&& context) {
hash_variant.init(_state, type, _agg_stat);
hash_variant.visit([&](auto& variant) {
if constexpr (is_compressed_fixed_size_key<std::decay_t<decltype(*variant)>>) {
variant->offsets = std::move(context.offsets);
variant->used_bits = std::move(context.used_bits);
variant->bases = std::move(context.bases);
}
});
}
template <typename HashVariantType>
void Aggregator::_init_agg_hash_variant(HashVariantType& hash_variant) {
auto type = _get_hash_table_type<HashVariantType>();
CompressKeyContext compress_key_ctx;
bool apply_compress_key_opt = false;
typename HashVariantType::Type prev_type = type;
type = _try_to_apply_compressed_key_opt<HashVariantType>(type, &compress_key_ctx);
apply_compress_key_opt = prev_type != type;
if (apply_compress_key_opt) {
// build with compressed key
VLOG_ROW << "apply compressed key";
_build_hash_variant<HashVariantType>(hash_variant, type, std::move(compress_key_ctx));
return;
}
bool has_null_column = false;
int fixed_byte_size = 0;
if (_group_by_types.size() > 1) {
type = _try_to_apply_fixed_size_opt<HashVariantType>(type, &has_null_column, &fixed_byte_size);
}
VLOG_ROW << "hash type is "
<< static_cast<typename std::underlying_type<typename HashVariantType::Type>::type>(type);

View File

@ -19,40 +19,34 @@
#include <cstddef>
#include <cstdint>
#include <memory>
#include <mutex>
#include <new>
#include <queue>
#include <utility>
#include "column/chunk.h"
#include "column/column_helper.h"
#include "column/type_traits.h"
#include "column/vectorized_fwd.h"
#include "common/object_pool.h"
#include "common/statusor.h"
#include "exec/aggregate/agg_hash_variant.h"
#include "exec/aggregate/agg_profile.h"
#include "exec/chunk_buffer_memory_manager.h"
#include "exec/aggregator_fwd.h"
#include "exec/limited_pipeline_chunk_buffer.h"
#include "exec/pipeline/context_with_dependency.h"
#include "exec/pipeline/schedule/observer.h"
#include "exec/pipeline/spill_process_channel.h"
#include "exprs/agg/aggregate_factory.h"
#include "exprs/agg/aggregate.h"
#include "exprs/expr.h"
#include "gen_cpp/QueryPlanExtra_types.h"
#include "gutil/strings/substitute.h"
#include "runtime/current_thread.h"
#include "runtime/descriptors.h"
#include "runtime/mem_pool.h"
#include "runtime/memory/counting_allocator.h"
#include "runtime/runtime_state.h"
#include "runtime/types.h"
#include "util/defer_op.h"
namespace starrocks {
class RuntimeFilter;
class AggInRuntimeFilterMerger;
struct HashTableKeyAllocator;
class VectorizedLiteral;
struct RawHashTableIterator {
RawHashTableIterator(HashTableKeyAllocator* alloc_, size_t x_, int y_) : alloc(alloc_), x(x_), y(y_) {}
@ -117,19 +111,6 @@ inline uint8_t* RawHashTableIterator::value() {
return static_cast<uint8_t*>(alloc->vecs[x].first) + alloc->aggregate_key_size * y;
}
class Aggregator;
class SortedStreamingAggregator;
template <class HashMapWithKey>
struct AllocateState {
AllocateState(Aggregator* aggregator_) : aggregator(aggregator_) {}
inline AggDataPtr operator()(const typename HashMapWithKey::KeyType& key);
inline AggDataPtr operator()(std::nullptr_t);
private:
Aggregator* aggregator;
};
struct AggFunctionTypes {
TypeDescriptor result_type;
TypeDescriptor serde_type; // for serialize
@ -227,6 +208,7 @@ struct AggregatorParams {
std::vector<TExpr> grouping_exprs;
std::vector<TExpr> aggregate_functions;
std::vector<TExpr> intermediate_aggr_exprs;
std::vector<TExpr> grouping_min_max;
// Incremental MV
// Whether it's testing, use MemStateTable in testing, instead use IMTStateTable.
@ -255,12 +237,6 @@ AggregatorParamsPtr convert_to_aggregator_params(const TPlanNode& tnode);
// it contains common data struct and algorithm of aggregation
class Aggregator : public pipeline::ContextWithDependency {
public:
#ifdef NDEBUG
static constexpr size_t two_level_memory_threshold = 33554432; // 32M, L3 Cache
#else
static constexpr size_t two_level_memory_threshold = 64;
#endif
Aggregator(AggregatorParamsPtr params);
~Aggregator() noexcept override {
@ -414,7 +390,7 @@ public:
bool is_streaming_all_states() const { return _streaming_all_states; }
HashTableKeyAllocator _state_allocator;
HashTableKeyAllocator& state_allocator() { return _state_allocator; }
void attach_sink_observer(RuntimeState* state, pipeline::PipelineObserver* observer) {
_pip_observable.attach_sink_observer(state, observer);
@ -435,6 +411,8 @@ protected:
std::unique_ptr<MemPool> _mem_pool;
// used to count heap memory usage of agg states
std::unique_ptr<CountingAllocatorWithHook> _allocator;
HashTableKeyAllocator _state_allocator;
// The open phase still relies on the TFunction object for some initialization operations
std::vector<TFunction> _fns;
@ -501,6 +479,8 @@ protected:
// Exprs used to evaluate group by column
std::vector<ExprContext*> _group_by_expr_ctxs;
std::vector<ExprContext*> _group_by_min_max;
std::vector<std::optional<std::pair<VectorizedLiteral*, VectorizedLiteral*>>> _ranges;
Columns _group_by_columns;
std::vector<ColumnType> _group_by_types;
@ -598,6 +578,24 @@ protected:
// Choose different agg hash map/set by different group by column's count, type, nullable
template <typename HashVariantType>
void _init_agg_hash_variant(HashVariantType& hash_variant);
// get spec hash table/set type
template <typename HashVariantType>
typename HashVariantType::Type _get_hash_table_type();
template <typename HashVariantType>
typename HashVariantType::Type _try_to_apply_fixed_size_opt(typename HashVariantType::Type type,
bool* has_null_column, int* fixed_byte_size);
struct CompressKeyContext {
std::vector<int> offsets;
std::vector<int> used_bits;
std::vector<std::any> bases;
};
template <typename HashVariantType>
typename HashVariantType::Type _try_to_apply_compressed_key_opt(typename HashVariantType::Type input_type,
CompressKeyContext* ctx);
template <typename HashVariantType>
void _build_hash_variant(HashVariantType& hash_variant, typename HashVariantType::Type type,
CompressKeyContext&& context);
void _release_agg_memory();
@ -608,7 +606,7 @@ protected:
int64_t get_two_level_threahold() {
if (config::two_level_memory_threshold < 0) {
return two_level_memory_threshold;
return agg::two_level_memory_threshold;
}
return config::two_level_memory_threshold;
}
@ -617,50 +615,6 @@ protected:
friend struct AllocateState;
};
template <class HashMapWithKey>
inline AggDataPtr AllocateState<HashMapWithKey>::operator()(const typename HashMapWithKey::KeyType& key) {
AggDataPtr agg_state = aggregator->_state_allocator.allocate();
*reinterpret_cast<typename HashMapWithKey::KeyType*>(agg_state) = key;
size_t created = 0;
size_t aggregate_function_sz = aggregator->_agg_fn_ctxs.size();
try {
for (int i = 0; i < aggregate_function_sz; i++) {
aggregator->_agg_functions[i]->create(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
created++;
}
return agg_state;
} catch (std::bad_alloc& e) {
for (size_t i = 0; i < created; ++i) {
aggregator->_agg_functions[i]->destroy(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
}
aggregator->_state_allocator.rollback();
throw;
}
}
template <class HashMapWithKey>
inline AggDataPtr AllocateState<HashMapWithKey>::operator()(std::nullptr_t) {
AggDataPtr agg_state = aggregator->_state_allocator.allocate_null_key_data();
size_t created = 0;
size_t aggregate_function_sz = aggregator->_agg_fn_ctxs.size();
try {
for (int i = 0; i < aggregate_function_sz; i++) {
aggregator->_agg_functions[i]->create(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
created++;
}
return agg_state;
} catch (std::bad_alloc& e) {
for (int i = 0; i < created; i++) {
aggregator->_agg_functions[i]->destroy(aggregator->_agg_fn_ctxs[i],
agg_state + aggregator->_agg_states_offsets[i]);
}
throw;
}
}
inline bool LimitedMemAggState::has_limited(const Aggregator& aggregator) const {
return limited_memory_size > 0 && aggregator.memory_usage() >= limited_memory_size;
}
@ -702,11 +656,4 @@ private:
std::atomic<int64_t> _shared_limit_countdown;
};
using AggregatorFactory = AggregatorFactoryBase<Aggregator>;
using AggregatorFactoryPtr = std::shared_ptr<AggregatorFactory>;
using SortedStreamingAggregatorPtr = std::shared_ptr<SortedStreamingAggregator>;
using StreamingAggregatorFactory = AggregatorFactoryBase<SortedStreamingAggregator>;
using StreamingAggregatorFactoryPtr = std::shared_ptr<StreamingAggregatorFactory>;
} // namespace starrocks

View File

@ -0,0 +1,32 @@
#pragma once
#include <cstddef>
#include <memory>
namespace starrocks {
namespace agg {
#ifdef NDEBUG
constexpr size_t two_level_memory_threshold = 33554432; // 32M, L3 Cache
#else
constexpr size_t two_level_memory_threshold = 64;
#endif
} // namespace agg
class Aggregator;
class SortedStreamingAggregator;
using AggregatorPtr = std::shared_ptr<Aggregator>;
using SortedStreamingAggregatorPtr = std::shared_ptr<SortedStreamingAggregator>;
template <class HashMapWithKey>
struct AllocateState;
template <class T>
class AggregatorFactoryBase;
using AggregatorFactory = AggregatorFactoryBase<Aggregator>;
using AggregatorFactoryPtr = std::shared_ptr<AggregatorFactory>;
using StreamingAggregatorFactory = AggregatorFactoryBase<SortedStreamingAggregator>;
using StreamingAggregatorFactoryPtr = std::shared_ptr<StreamingAggregatorFactory>;
} // namespace starrocks

View File

@ -35,7 +35,6 @@
#pragma once
#include <functional>
#include <mutex>
#include <sstream>
#include <vector>
@ -48,10 +47,7 @@
#include "runtime/descriptors.h"
#include "runtime/mem_pool.h"
#include "runtime/query_statistics.h"
#include "service/backend_options.h"
#include "util/blocking_queue.hpp"
#include "util/runtime_profile.h"
#include "util/uid_util.h" // for print_id
namespace starrocks {

View File

@ -69,8 +69,7 @@ Status AggregateBlockingSinkOperator::set_finishing(RuntimeState* state) {
if (_aggregator->hash_map_variant().size() == 0) {
_aggregator->set_ht_eos();
}
_aggregator->hash_map_variant().visit(
[&](auto& hash_map_with_key) { _aggregator->it_hash() = _aggregator->_state_allocator.begin(); });
_aggregator->it_hash() = _aggregator->state_allocator().begin();
} else if (_aggregator->is_none_group_by_exprs()) {
// for aggregate no group by, if _num_input_rows is 0,

View File

@ -89,8 +89,7 @@ DEFINE_FAIL_POINT(force_reset_aggregator_after_agg_streaming_sink_finish);
Status AggregateStreamingSourceOperator::_output_chunk_from_hash_map(ChunkPtr* chunk, RuntimeState* state) {
if (!_aggregator->it_hash().has_value()) {
_aggregator->hash_map_variant().visit(
[&](auto& hash_map_with_key) { _aggregator->it_hash() = _aggregator->_state_allocator.begin(); });
_aggregator->it_hash() = _aggregator->state_allocator().begin();
COUNTER_SET(_aggregator->hash_table_size(), (int64_t)_aggregator->hash_map_variant().size());
}

View File

@ -14,7 +14,7 @@
#pragma once
#include "exec/aggregator.h"
#include "exec/aggregator_fwd.h"
#include "exec/pipeline/source_operator.h"
namespace starrocks::pipeline {

View File

@ -66,8 +66,7 @@ Status SpillableAggregateBlockingSinkOperator::set_finishing(RuntimeState* state
}
if (!_aggregator->spill_channel()->has_task()) {
if (_aggregator->hash_map_variant().size() > 0 || !_streaming_chunks.empty()) {
_aggregator->hash_map_variant().visit(
[&](auto& hash_map_with_key) { _aggregator->it_hash() = _aggregator->_state_allocator.begin(); });
_aggregator->it_hash() = _aggregator->state_allocator().begin();
_aggregator->spill_channel()->add_spill_task(_build_spill_task(state));
}
}
@ -270,8 +269,7 @@ Status SpillableAggregateBlockingSinkOperator::_try_to_spill_by_auto(RuntimeStat
Status SpillableAggregateBlockingSinkOperator::_spill_all_data(RuntimeState* state, bool should_spill_hash_table) {
RETURN_IF(_aggregator->hash_map_variant().size() == 0, Status::OK());
if (should_spill_hash_table) {
_aggregator->hash_map_variant().visit(
[&](auto& hash_map_with_key) { _aggregator->it_hash() = _aggregator->_state_allocator.begin(); });
_aggregator->it_hash() = _aggregator->state_allocator().begin();
}
CHECK(!_aggregator->spill_channel()->has_task());
RETURN_IF_ERROR(_aggregator->spill_aggregate_data(state, _build_spill_task(state, should_spill_hash_table)));

View File

@ -16,9 +16,8 @@
#include <utility>
#include "exec/aggregator.h"
#include "exec/aggregator_fwd.h"
#include "exec/pipeline/aggregate/aggregate_blocking_source_operator.h"
#include "exec/sorted_streaming_aggregator.h"
#include "runtime/runtime_state.h"
#include "storage/chunk_helper.h"

View File

@ -16,7 +16,7 @@
#include <utility>
#include "exec/aggregator.h"
#include "exec/aggregator_fwd.h"
#include "exec/pipeline/aggregate/aggregate_distinct_blocking_sink_operator.h"
#include "exec/pipeline/aggregate/aggregate_distinct_blocking_source_operator.h"
#include "exec/pipeline/operator.h"

View File

@ -49,9 +49,7 @@ Status SpillablePartitionWiseAggregateSinkOperator::set_finishing(RuntimeState*
}
if (!_agg_op->aggregator()->spill_channel()->has_task()) {
if (_agg_op->aggregator()->hash_map_variant().size() > 0 || !_streaming_chunks.empty()) {
_agg_op->aggregator()->hash_map_variant().visit([&](auto& hash_map_with_key) {
_agg_op->aggregator()->it_hash() = _agg_op->aggregator()->_state_allocator.begin();
});
_agg_op->aggregator()->it_hash() = _agg_op->aggregator()->state_allocator().begin();
_agg_op->aggregator()->spill_channel()->add_spill_task(_build_spill_task(state));
}
}
@ -279,9 +277,7 @@ ChunkPtr& SpillablePartitionWiseAggregateSinkOperator::_append_hash_column(Chunk
Status SpillablePartitionWiseAggregateSinkOperator::_spill_all_data(RuntimeState* state, bool should_spill_hash_table) {
RETURN_IF(_agg_op->aggregator()->hash_map_variant().size() == 0, Status::OK());
if (should_spill_hash_table) {
_agg_op->aggregator()->hash_map_variant().visit([&](auto& hash_map_with_key) {
_agg_op->aggregator()->it_hash() = _agg_op->aggregator()->_state_allocator.begin();
});
_agg_op->aggregator()->it_hash() = _agg_op->aggregator()->state_allocator().begin();
}
CHECK(!_agg_op->aggregator()->spill_channel()->has_task());
RETURN_IF_ERROR(

View File

@ -27,6 +27,7 @@
#include "runtime/mem_tracker.h"
#include "runtime/runtime_filter_cache.h"
#include "runtime/runtime_state.h"
#include "service/backend_options.h"
#include "util/failpoint/fail_point.h"
#include "util/runtime_profile.h"

View File

@ -16,17 +16,14 @@
#include <functional>
#include <memory>
#include <mutex>
#include <utility>
#include "column/vectorized_fwd.h"
#include "common/statusor.h"
#include "exec/spill/executor.h"
#include "exec/spill/spiller.h"
#include "runtime/runtime_state.h"
#include "util/blocking_queue.hpp"
#include "util/defer_op.h"
#include "util/runtime_profile.h"
namespace starrocks {
class SpillProcessChannel;
@ -73,7 +70,7 @@ using SpillProcessChannelFactoryPtr = std::shared_ptr<SpillProcessChannelFactory
// SpillProcessOperator
class SpillProcessChannel {
public:
SpillProcessChannel() {}
SpillProcessChannel() = default;
bool add_spill_task(SpillProcessTask&& task) {
DCHECK(!_is_finishing);

View File

@ -14,7 +14,6 @@
#pragma once
#include <limits>
#include <memory>
#include "column/vectorized_fwd.h"
@ -24,7 +23,7 @@
#include "exec/workgroup/work_group_fwd.h"
namespace starrocks {
class AggregatorParams;
struct AggregatorParams;
using AggregatorParamsPtr = std::shared_ptr<AggregatorParams>;
}; // namespace starrocks

View File

@ -757,8 +757,7 @@ Status PartitionedSpillerWriter::_compact_skew_chunks(size_t num_rows, std::vect
auto hash_set_sz = merger->hash_set_variant().size();
merger->convert_hash_set_to_chunk(hash_set_sz, &chunk_merged);
} else {
merger->hash_map_variant().visit(
[&](auto& hash_map_with_key) { merger->it_hash() = merger->_state_allocator.begin(); });
merger->it_hash() = merger->state_allocator().begin();
auto hash_map_sz = merger->hash_map_variant().size();
RETURN_IF_ERROR(merger->convert_hash_map_to_chunk(hash_map_sz, &chunk_merged, true));
}

View File

@ -20,6 +20,7 @@
#include "column/vectorized_fwd.h"
#include "common/status.h"
#include "exec/olap_scan_node.h"
#include "service/backend_options.h"
#include "storage/chunk_helper.h"
#include "storage/column_predicate_rewriter.h"
#include "storage/predicate_parser.h"

View File

@ -14,7 +14,6 @@
#pragma once
#include <new>
#include <type_traits>
#include "column/column.h"

View File

@ -14,10 +14,14 @@
#pragma once
#include "column/hash_set.h"
#include "common/config.h"
#include <type_traits>
#include "common/logging.h"
#include "exec/agg_hash_fwd.h"
#include "runtime/memory/allocator.h"
#include "runtime/memory/roaring_hook.h"
#include "util/hash_fwd.h"
#include "util/phmap/phmap_fwd_decl.h"
namespace starrocks {

View File

@ -16,18 +16,15 @@
#include <algorithm>
#include <cstring>
#include <limits>
#include <string>
#include <type_traits>
#include "column/array_column.h"
#include "column/binary_column.h"
#include "column/column_helper.h"
#include "column/fixed_length_column.h"
#include "column/hash_set.h"
#include "column/type_traits.h"
#include "column/vectorized_fwd.h"
#include "exec/aggregator.h"
#include "exec/aggregator_fwd.h"
#include "exprs/agg/aggregate.h"
#include "exprs/agg/aggregate_state_allocator.h"
#include "exprs/agg/sum.h"
@ -36,7 +33,6 @@
#include "glog/logging.h"
#include "gutil/casts.h"
#include "runtime/mem_pool.h"
#include "runtime/memory/counting_allocator.h"
#include "thrift/protocol/TJSONProtocol.h"
#include "util/phmap/phmap_dump.h"
#include "util/slice.h"
@ -110,7 +106,7 @@ struct AdaptiveSliceHashSet {
AdaptiveSliceHashSet() { set = std::make_shared<SliceHashSetWithAggStateAllocator>(); }
void try_convert_to_two_level(MemPool* mem_pool) {
if (distinct_size % 65536 == 0 && mem_pool->total_allocated_bytes() >= Aggregator::two_level_memory_threshold) {
if (distinct_size % 65536 == 0 && mem_pool->total_allocated_bytes() >= agg::two_level_memory_threshold) {
two_level_set = std::make_shared<SliceTwoLevelHashSetWithAggStateAllocator>();
two_level_set->reserve(set->capacity());
two_level_set->insert(set->begin(), set->end());

View File

@ -70,7 +70,7 @@ void MemPool::clear() {
current_chunk_idx_ = -1;
for (auto& chunk : chunks_) {
chunk.allocated_bytes = 0;
ASAN_POISON_MEMORY_REGION(chunk.chunk.data, chunk.chunk.size);
SR_ASAN_POISON_MEMORY_REGION(chunk.chunk.data, chunk.chunk.size);
}
total_allocated_bytes_ = 0;
DCHECK(check_integrity(false));
@ -139,7 +139,7 @@ bool MemPool::find_chunk(size_t min_size, bool check_limits) {
return false;
}
}
ASAN_POISON_MEMORY_REGION(chunk.data, chunk_size);
SR_ASAN_POISON_MEMORY_REGION(chunk.data, chunk_size);
// Put it before the first free chunk. If no free chunks, it goes at the end.
if (first_free_idx == static_cast<int>(chunks_.size())) {
chunks_.emplace_back(chunk);

View File

@ -37,18 +37,14 @@
#include <algorithm>
#include <cstddef>
#include <cstdio>
#include <new>
#include <string>
#include <vector>
#include "common/compiler_util.h"
#include "common/config.h"
#include "common/logging.h"
#include "common/status.h"
#include "gutil/dynamic_annotations.h"
#include "runtime/memory/mem_chunk.h"
#include "storage/olap_define.h"
#include "util/bit_util.h"
#include "util/posion.h"
namespace starrocks {
@ -208,7 +204,7 @@ private:
// Ensure the requested alignment is respected.
int64_t padding = aligned_allocated_bytes - info.allocated_bytes;
uint8_t* result = info.chunk.data + aligned_allocated_bytes;
ASAN_UNPOISON_MEMORY_REGION(result, size);
SR_ASAN_UNPOISON_MEMORY_REGION(result, size);
DCHECK_LE(info.allocated_bytes + size, info.chunk.size);
info.allocated_bytes += padding + size;
total_allocated_bytes_ += padding + size;
@ -226,7 +222,7 @@ private:
ChunkInfo& info = chunks_[current_chunk_idx_];
uint8_t* result = info.chunk.data + info.allocated_bytes;
ASAN_UNPOISON_MEMORY_REGION(result, size);
SR_ASAN_UNPOISON_MEMORY_REGION(result, size);
DCHECK_LE(info.allocated_bytes + size, info.chunk.size);
info.allocated_bytes += size;
total_allocated_bytes_ += size;

View File

@ -17,7 +17,7 @@
#include <cstdint>
#include <memory_resource>
#include "gutil/dynamic_annotations.h"
#include "util/posion.h"
namespace starrocks {
// memory resource provide for PMR
@ -36,11 +36,11 @@ public:
_stack_addr_end((uint8_t*)_stack_addr_start + sz),
_current_addr(_stack_addr_start),
_upstream(upstream) {
ASAN_POISON_MEMORY_REGION(_stack_addr_start, sz);
SR_ASAN_POISON_MEMORY_REGION(_stack_addr_start, sz);
}
~stack_memory_resource() override {
size_t sz = (uint8_t*)_stack_addr_end - (uint8_t*)_stack_addr_start;
ASAN_UNPOISON_MEMORY_REGION(_stack_addr_start, sz);
SR_ASAN_UNPOISON_MEMORY_REGION(_stack_addr_start, sz);
}
private:
@ -55,7 +55,7 @@ private:
if (res == nullptr) {
return _upstream->allocate(__bytes, __alignment);
}
ASAN_UNPOISON_MEMORY_REGION(res, __bytes);
SR_ASAN_UNPOISON_MEMORY_REGION(res, __bytes);
_current_addr = (uint8_t*)_current_addr + __bytes;
return res;
@ -64,7 +64,7 @@ private:
void do_deallocate(void* __p, size_t __bytes, size_t __alignment) override {
if (__p >= _stack_addr_start && __p <= _stack_addr_end) {
// nothing todo
ASAN_POISON_MEMORY_REGION(__p, __bytes);
SR_ASAN_POISON_MEMORY_REGION(__p, __bytes);
} else {
return _upstream->deallocate(__p, __bytes, __alignment);
}

View File

@ -27,13 +27,12 @@
#include "common/daemon.h"
#include "common/process_exit.h"
#include "common/status.h"
#include "exec/pipeline/query_context.h"
#include "fs/s3/poco_common.h"
#include "gutil/strings/join.h"
#include "runtime/exec_env.h"
#include "runtime/fragment_mgr.h"
#include "runtime/global_variables.h"
#include "runtime/jdbc_driver_manager.h"
#include "service/backend_options.h"
#include "service/brpc.h"
#include "service/service.h"
#include "service/service_be/arrow_flight_sql_service.h"

View File

@ -415,7 +415,7 @@ Status SegmentMetaCollecter::__collect_max_or_min(ColumnId cid, Column* column,
TypeInfoPtr type_info = get_type_info(delegate_type(type));
if constexpr (!is_max) { // min
Datum min;
if (!segment_zone_map_pb->has_null()) {
if (segment_zone_map_pb->has_not_null()) {
RETURN_IF_ERROR(datum_from_string(type_info.get(), &min, segment_zone_map_pb->min(), nullptr));
column->append_datum(min);
} else {

View File

@ -106,6 +106,14 @@ namespace starrocks {
M(VARBINARY) \
M(JSON)
#define APPLY_FOR_MIN_MAX_COMPRESSABLE_TYPE(M) \
APPLY_FOR_ALL_INT_TYPE(M) \
M(TYPE_DECIMAL32) \
M(TYPE_DECIMAL64) \
M(TYPE_DECIMAL128) \
M(TYPE_DATE) \
M(TYPE_DATETIME)
#define APPLY_FOR_BITSET_FILTER_SUPPORTED_TYPE(M) \
M(TYPE_BOOLEAN) \
M(TYPE_TINYINT) \

View File

@ -45,17 +45,17 @@ void faststring::GrowArray(size_t newcapacity) {
if (data_ != initial_data_) {
delete[] data_;
} else {
ASAN_POISON_MEMORY_REGION(initial_data_, arraysize(initial_data_));
SR_ASAN_POISON_MEMORY_REGION(initial_data_, arraysize(initial_data_));
}
data_ = newdata.release();
ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
SR_ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
}
void faststring::ShrinkToFitInternal() {
DCHECK_NE(data_, initial_data_);
if (len_ <= kInitialCapacity) {
ASAN_UNPOISON_MEMORY_REGION(initial_data_, len_);
SR_ASAN_UNPOISON_MEMORY_REGION(initial_data_, len_);
memcpy(initial_data_, &data_[0], len_);
delete[] data_;
data_ = initial_data_;

View File

@ -38,10 +38,9 @@
#include <cstring>
#include <string>
#include "gutil/dynamic_annotations.h"
#include "gutil/macros.h"
#include "gutil/port.h"
#include "gutil/strings/fastmem.h"
#include "util/posion.h"
#include "util/slice.h"
namespace starrocks {
@ -61,11 +60,11 @@ public:
data_ = new uint8_t[capacity];
capacity_ = capacity;
}
ASAN_POISON_MEMORY_REGION(data_, capacity_);
SR_ASAN_POISON_MEMORY_REGION(data_, capacity_);
}
~faststring() {
ASAN_UNPOISON_MEMORY_REGION(initial_data_, arraysize(initial_data_));
SR_ASAN_UNPOISON_MEMORY_REGION(initial_data_, arraysize(initial_data_));
if (data_ != initial_data_) {
delete[] data_;
}
@ -76,7 +75,7 @@ public:
// This does not free up any memory. The capacity of the string remains unchanged.
void clear() {
resize(0);
ASAN_POISON_MEMORY_REGION(data_, capacity_);
SR_ASAN_POISON_MEMORY_REGION(data_, capacity_);
}
// Resize the string to the given length.
@ -89,8 +88,8 @@ public:
reserve(newsize);
}
len_ = newsize;
ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
ASAN_UNPOISON_MEMORY_REGION(data_, len_);
SR_ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
SR_ASAN_UNPOISON_MEMORY_REGION(data_, len_);
}
// Return the buffer built so far and reset `this` to the initial status (size() == 0).
@ -105,7 +104,7 @@ public:
len_ = 0;
capacity_ = kInitialCapacity;
data_ = initial_data_;
ASAN_POISON_MEMORY_REGION(data_, capacity_);
SR_ASAN_POISON_MEMORY_REGION(data_, capacity_);
return result;
}
@ -123,7 +122,7 @@ public:
void append(const void* src_v, size_t count) {
const auto* src = reinterpret_cast<const uint8_t*>(src_v);
EnsureRoomForAppend(count);
ASAN_UNPOISON_MEMORY_REGION(data_ + len_, count);
SR_ASAN_UNPOISON_MEMORY_REGION(data_ + len_, count);
// appending short values is common enough that this
// actually helps, according to benchmarks. In theory
@ -147,7 +146,7 @@ public:
// Append the given character to this string.
void push_back(const char byte) {
EnsureRoomForAppend(1);
ASAN_UNPOISON_MEMORY_REGION(data_ + len_, 1);
SR_ASAN_UNPOISON_MEMORY_REGION(data_ + len_, 1);
data_[len_] = byte;
len_++;
}
@ -213,8 +212,8 @@ public:
if (data_ == initial_data_ && rhs.data_ == rhs.initial_data_) {
assert(capacity_ == kInitialCapacity);
assert(rhs.capacity_ == kInitialCapacity);
ASAN_UNPOISON_MEMORY_REGION(initial_data_, kInitialCapacity);
ASAN_UNPOISON_MEMORY_REGION(rhs.initial_data_, kInitialCapacity);
SR_ASAN_UNPOISON_MEMORY_REGION(initial_data_, kInitialCapacity);
SR_ASAN_UNPOISON_MEMORY_REGION(rhs.initial_data_, kInitialCapacity);
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wuninitialized"
memcpy(tmp_buff, initial_data_, kInitialCapacity);
@ -224,7 +223,7 @@ public:
std::swap(len_, rhs.len_);
} else if (data_ == initial_data_) {
assert(capacity_ == kInitialCapacity);
ASAN_UNPOISON_MEMORY_REGION(rhs.initial_data_, len_);
SR_ASAN_UNPOISON_MEMORY_REGION(rhs.initial_data_, len_);
strings::memcpy_inlined(rhs.initial_data_, initial_data_, len_);
std::swap(len_, rhs.len_);
data_ = rhs.data_;
@ -233,7 +232,7 @@ public:
rhs.capacity_ = kInitialCapacity;
} else if (rhs.data_ == rhs.initial_data_) {
assert(rhs.capacity_ == kInitialCapacity);
ASAN_UNPOISON_MEMORY_REGION(initial_data_, rhs.len_);
SR_ASAN_UNPOISON_MEMORY_REGION(initial_data_, rhs.len_);
strings::memcpy_inlined(initial_data_, rhs.initial_data_, rhs.len_);
std::swap(len_, rhs.len_);
rhs.data_ = data_;
@ -245,20 +244,20 @@ public:
std::swap(len_, rhs.len_);
std::swap(capacity_, rhs.capacity_);
}
ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
ASAN_POISON_MEMORY_REGION(rhs.data_ + rhs.len_, rhs.capacity_ - rhs.len_);
SR_ASAN_POISON_MEMORY_REGION(data_ + len_, capacity_ - len_);
SR_ASAN_POISON_MEMORY_REGION(rhs.data_ + rhs.len_, rhs.capacity_ - rhs.len_);
}
void advance(size_t count) {
EnsureRoomForAppend(count);
ASAN_UNPOISON_MEMORY_REGION(data_ + len_, count);
SR_ASAN_UNPOISON_MEMORY_REGION(data_ + len_, count);
len_ += count;
}
private:
faststring(const faststring&) = delete;
const faststring& operator=(const faststring&) = delete;
private:
// If necessary, expand the buffer to fit at least 'count' more bytes.
// If the array has to be grown, it is grown by at least 50%.
void EnsureRoomForAppend(size_t count) {

View File

@ -18,14 +18,12 @@
#include <cstddef>
#include <cstdint>
#include <cstring>
#include <functional>
#include <limits>
#include <optional>
#include <type_traits>
#include <utility>
#include "column/column_hash.h"
#include "glog/logging.h"
#include "util/hash_fwd.h"
namespace starrocks {
// FixedSizeHashMap
@ -119,17 +117,9 @@ public:
return lazy_emplace(key, f);
}
struct HashFunction {
size_t operator()(KeyType key) { return static_cast<size_t>(key); }
};
HashFunction hash_function() { return HashFunction(); }
size_t bucket_count() { return hash_table_size; }
size_t size() { return _size; }
size_t capacity() { return bucket_count(); }
size_t capacity() { return hash_table_size; }
size_t dump_bound() { return hash_table_size; }
@ -191,6 +181,8 @@ public:
_hash_table[static_cast<search_key_type>(key)] = 1;
}
void insert(KeyType key) { emplace(key); }
bool contains(KeyType key) { return _hash_table[static_cast<search_key_type>(key)]; }
size_t dump_bound() { return hash_table_size; }
@ -204,4 +196,21 @@ private:
uint8_t _hash_table[hash_table_size + 1];
};
template <typename T>
struct no_prefetch_set : std::false_type {};
template <typename KeyType, PhmapSeed seed>
struct no_prefetch_set<SmallFixedSizeHashSet<KeyType, seed>> : std::true_type {};
template <typename T>
struct no_prefetch_map : std::false_type {};
template <typename KeyType, typename ValueType, PhmapSeed seed>
struct no_prefetch_map<SmallFixedSizeHashMap<KeyType, ValueType, seed>> : std::true_type {};
template <class T>
constexpr bool is_no_prefetch_set = no_prefetch_set<T>::value;
template <class T>
constexpr bool is_no_prefetch_map = no_prefetch_map<T>::value;
} // namespace starrocks

View File

@ -2,7 +2,7 @@
#include <cstddef>
#include <cstdint>
#include "util/hash_util.hpp"
#include "util/hash_fwd.h"
#include "util/slice.h"
#include "util/unaligned_access.h"
#if defined(__aarch64__)
@ -45,8 +45,6 @@ public:
}
};
enum PhmapSeed { PhmapSeed1, PhmapSeed2 };
template <int n, PhmapSeed seed>
class phmap_mix_with_seed {
public:
@ -174,12 +172,6 @@ inline uint64_t crc_hash_64(const void* data, int32_t length, uint64_t hash) {
return phmap_mix<8>()(crc_hash_64_unmixed(data, length, hash));
}
struct CRC_HASH_SEEDS {
// TODO: 0x811C9DC5 is not prime number
static const uint32_t CRC_HASH_SEED1 = 0x811C9DC5;
static const uint32_t CRC_HASH_SEED2 = 0x811C9DD7;
};
class SliceHash {
public:
std::size_t operator()(const Slice& slice) const {

42
be/src/util/hash_fwd.h Normal file
View File

@ -0,0 +1,42 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <cstdint>
namespace starrocks {
enum PhmapSeed { PhmapSeed1, PhmapSeed2 };
struct CRC_HASH_SEEDS {
// TODO: 0x811C9DC5 is not prime number
static const uint32_t CRC_HASH_SEED1 = 0x811C9DC5;
static const uint32_t CRC_HASH_SEED2 = 0x811C9DD7;
};
template <class T, PhmapSeed seed>
class StdHashWithSeed;
template <PhmapSeed seed>
struct Hash128WithSeed;
template <PhmapSeed seed>
struct Hash256WithSeed;
template <PhmapSeed>
class SliceHashWithSeed;
template <class T>
class StdHash;
} // namespace starrocks

44
be/src/util/posion.h Normal file
View File

@ -0,0 +1,44 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
// Since gutil and absl have redefinitions, we rename the posion-related macros to avoid compiler warnings.
#if defined(__SANITIZE_ADDRESS__) || defined(ADDRESS_SANITIZER)
// Marks memory region [addr, addr+size) as unaddressable.
// This memory must be previously allocated by the user program. Accessing
// addresses in this region from instrumented code is forbidden until
// this region is unpoisoned. This function is not guaranteed to poison
// the whole region - it may poison only subregion of [addr, addr+size) due
// to ASan alignment restrictions.
// Method is NOT thread-safe in the sense that no two threads can
// (un)poison memory in the same memory region simultaneously.
extern "C" void __asan_poison_memory_region(void const volatile* addr, size_t size);
// Marks memory region [addr, addr+size) as addressable.
// This memory must be previously allocated by the user program. Accessing
// addresses in this region is allowed until this region is poisoned again.
// This function may unpoison a superregion of [addr, addr+size) due to
// ASan alignment restrictions.
// Method is NOT thread-safe in the sense that no two threads can
// (un)poison memory in the same memory region simultaneously.
extern "C" void __asan_unpoison_memory_region(void const volatile* addr, size_t size);
#define SR_ASAN_POISON_MEMORY_REGION(addr, size) __asan_poison_memory_region((addr), (size))
#define SR_ASAN_UNPOISON_MEMORY_REGION(addr, size) __asan_unpoison_memory_region((addr), (size))
#else
#define SR_ASAN_POISON_MEMORY_REGION(addr, size) ((void)(addr), (void)(size))
#define SR_ASAN_UNPOISON_MEMORY_REGION(addr, size) ((void)(addr), (void)(size))
#endif

View File

@ -52,7 +52,6 @@
#include "gutil/dynamic_annotations.h"
#include "gutil/once.h"
#include "gutil/strings/substitute.h"
#include "storage/olap_define.h"
#include "util/scoped_cleanup.h"
namespace starrocks {

View File

@ -1519,6 +1519,9 @@ public class OlapTable extends Table {
// check input partition has temporary partition
public boolean inputHasTempPartition(List<Long> partitionIds) {
if (partitionIds == null) {
return false;
}
for (Long pid : partitionIds) {
if (tempPartitions.getPartition(pid) != null) {
return true;

View File

@ -39,20 +39,25 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.starrocks.analysis.AggregateInfo;
import com.starrocks.analysis.DecimalLiteral;
import com.starrocks.analysis.DescriptorTable;
import com.starrocks.analysis.Expr;
import com.starrocks.analysis.FunctionCallExpr;
import com.starrocks.analysis.LiteralExpr;
import com.starrocks.analysis.SlotDescriptor;
import com.starrocks.analysis.SlotId;
import com.starrocks.analysis.SlotRef;
import com.starrocks.analysis.TupleId;
import com.starrocks.catalog.ScalarType;
import com.starrocks.catalog.Type;
import com.starrocks.common.AnalysisException;
import com.starrocks.common.FeConstants;
import com.starrocks.common.IdGenerator;
import com.starrocks.common.Pair;
import com.starrocks.qe.ConnectContext;
import com.starrocks.qe.SessionVariable;
import com.starrocks.sql.optimizer.operator.scalar.ColumnRefOperator;
import com.starrocks.sql.optimizer.operator.scalar.ConstantOperator;
import com.starrocks.sql.optimizer.statistics.ColumnStatistic;
import com.starrocks.thrift.TAggregationNode;
import com.starrocks.thrift.TExplainLevel;
@ -105,6 +110,8 @@ public class AggregationNode extends PlanNode implements RuntimeFilterBuildNode
private final List<RuntimeFilterDescription> buildRuntimeFilters = Lists.newArrayList();
private boolean withRuntimeFilters = false;
private List<Pair<ConstantOperator, ConstantOperator>> groupByMinMaxStats = Lists.newArrayList();
/**
* Create an agg node that is not an intermediate node.
* isIntermediate is true if it is a slave node in a 2-part agg plan.
@ -168,6 +175,10 @@ public class AggregationNode extends PlanNode implements RuntimeFilterBuildNode
this.usePerBucketOptimize = usePerBucketOptimize;
}
public void setGroupByMinMaxStats(List<Pair<ConstantOperator, ConstantOperator>> groupByMinMaxStats) {
this.groupByMinMaxStats = groupByMinMaxStats;
}
public void disablePhysicalPropertyOptimize() {
setUseSortAgg(false);
setUsePerBucketOptimize(false);
@ -249,6 +260,35 @@ public class AggregationNode extends PlanNode implements RuntimeFilterBuildNode
if (sqlGroupingKeysBuilder.length() > 0) {
msg.agg_node.setSql_grouping_keys(sqlGroupingKeysBuilder.toString());
}
List<Expr> minMaxStats = Lists.newArrayList();
if (groupByMinMaxStats.size() == groupingExprs.size()) {
for (int i = 0; i < groupingExprs.size(); i++) {
final Expr expr = groupingExprs.get(i);
String min = groupByMinMaxStats.get(i).first.getVarchar();
String max = groupByMinMaxStats.get(i).second.getVarchar();
try {
Type type = expr.getType();
LiteralExpr minExpr = LiteralExpr.create(min, type);
LiteralExpr maxExpr = LiteralExpr.create(max, type);
// cast decimal literal to matched precision type
if (minExpr instanceof DecimalLiteral) {
minExpr = (LiteralExpr) minExpr.uncheckedCastTo(type);
maxExpr = (LiteralExpr) maxExpr.uncheckedCastTo(type);
}
minMaxStats.add(minExpr);
minMaxStats.add(maxExpr);
} catch (AnalysisException e) {
break;
}
}
}
if (minMaxStats.size() == 2 * groupingExprs.size()) {
msg.agg_node.setGroup_by_min_max(Expr.treesToThrift(minMaxStats));
}
}
List<Expr> intermediateAggrExprs = aggInfo.getIntermediateAggrExprs();
@ -332,6 +372,14 @@ public class AggregationNode extends PlanNode implements RuntimeFilterBuildNode
output.append(detailPrefix).append("- ").append(rf.toExplainString(-1)).append("\n");
}
}
if (!aggInfo.getGroupingExprs().isEmpty() &&
groupByMinMaxStats.size() == aggInfo.getGroupingExprs().size()) {
output.append(detailPrefix).append("group by min-max stats:\n");
for (Pair<ConstantOperator, ConstantOperator> stat : groupByMinMaxStats) {
output.append(detailPrefix).append("- ").append(stat.first).append(":").append(stat.second)
.append("\n");
}
}
}
return output.toString();
@ -498,7 +546,8 @@ public class AggregationNode extends PlanNode implements RuntimeFilterBuildNode
ExecGroupSets execGroupSets) {
SessionVariable sv = ConnectContext.get().getSessionVariable();
// RF push down group by one column
if (limit > 0 && limit < sv.getAggInFilterLimit() && !aggInfo.getAggregateExprs().isEmpty() && !aggInfo.getGroupingExprs().isEmpty()) {
if (limit > 0 && limit < sv.getAggInFilterLimit() && !aggInfo.getAggregateExprs().isEmpty() &&
!aggInfo.getGroupingExprs().isEmpty()) {
Expr groupingExpr = aggInfo.getGroupingExprs().get(0);
pushDownUnaryInRuntimeFilter(generator, groupingExpr, descTbl, execGroupSets, 0);
}

View File

@ -316,6 +316,8 @@ public class SessionVariable implements Serializable, Writable, Cloneable {
public static final String JOIN_LATE_MATERIALIZATION = "join_late_materialization";
public static final String ENABLE_PARTITION_HASH_JOIN = "enable_partition_hash_join";
public static final String ENABLE_GROUP_BY_COMPRESSED_KEY = "enable_group_by_compressed_key";
public static final String ENABLE_PRUNE_COLUMN_AFTER_INDEX_FILTER =
"enable_prune_column_after_index_filter";
@ -1455,6 +1457,9 @@ public class SessionVariable implements Serializable, Writable, Cloneable {
@VariableMgr.VarAttr(name = ENABLE_PARTITION_HASH_JOIN)
private boolean enablePartitionHashJoin = true;
@VariableMgr.VarAttr(name = ENABLE_GROUP_BY_COMPRESSED_KEY)
private boolean enableGroupByCompressedKey = true;
@VariableMgr.VarAttr(name = ENABLE_PRUNE_COLUMN_AFTER_INDEX_FILTER, flag = VariableMgr.INVISIBLE)
private boolean enablePruneColumnAfterIndexFilter = true;
@ -3513,6 +3518,10 @@ public class SessionVariable implements Serializable, Writable, Cloneable {
return enablePartitionHashJoin;
}
public boolean enableGroupByCompressedKey() {
return enableGroupByCompressedKey;
}
public void disableTrimOnlyFilteredColumnsInScanStage() {
this.enableFilterUnusedColumnsInScanStage = false;
}

View File

@ -95,6 +95,7 @@ import com.starrocks.sql.optimizer.rule.transformation.pruner.RboTablePruneRule;
import com.starrocks.sql.optimizer.rule.transformation.pruner.UniquenessBasedTablePruneRule;
import com.starrocks.sql.optimizer.rule.tree.AddDecodeNodeForDictStringRule;
import com.starrocks.sql.optimizer.rule.tree.AddIndexOnlyPredicateRule;
import com.starrocks.sql.optimizer.rule.tree.ApplyMinMaxStatisticRule;
import com.starrocks.sql.optimizer.rule.tree.ApplyTuningGuideRule;
import com.starrocks.sql.optimizer.rule.tree.CloneDuplicateColRefRule;
import com.starrocks.sql.optimizer.rule.tree.DataCachePopulateRewriteRule;
@ -976,6 +977,7 @@ public class QueryOptimizer extends Optimizer {
result = new PhysicalDistributionAggOptRule().rewrite(result, rootTaskContext);
result = new AddDecodeNodeForDictStringRule().rewrite(result, rootTaskContext);
result = new LowCardinalityRewriteRule().rewrite(result, rootTaskContext);
result = new ApplyMinMaxStatisticRule().rewrite(result, rootTaskContext);
// Put before ScalarOperatorsReuseRule
result = new PruneSubfieldsForComplexType().rewrite(result, rootTaskContext);
result = new InlineCteProjectPruneRule().rewrite(result, rootTaskContext);

View File

@ -19,6 +19,7 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.starrocks.catalog.FunctionSet;
import com.starrocks.common.Pair;
import com.starrocks.qe.ConnectContext;
import com.starrocks.qe.SessionVariableConstants;
import com.starrocks.sql.optimizer.OptExpression;
@ -33,6 +34,7 @@ import com.starrocks.sql.optimizer.operator.OperatorVisitor;
import com.starrocks.sql.optimizer.operator.Projection;
import com.starrocks.sql.optimizer.operator.scalar.CallOperator;
import com.starrocks.sql.optimizer.operator.scalar.ColumnRefOperator;
import com.starrocks.sql.optimizer.operator.scalar.ConstantOperator;
import com.starrocks.sql.optimizer.operator.scalar.ScalarOperator;
import org.apache.commons.collections4.CollectionUtils;
@ -71,6 +73,8 @@ public class PhysicalHashAggregateOperator extends PhysicalOperator {
private boolean forcePreAggregation = false;
private List<Pair<ConstantOperator, ConstantOperator>> groupByMinMaxStatistic = Lists.newArrayList();
public PhysicalHashAggregateOperator(AggType type,
List<ColumnRefOperator> groupBys,
List<ColumnRefOperator> partitionByColumns,
@ -104,6 +108,7 @@ public class PhysicalHashAggregateOperator extends PhysicalOperator {
this.usePerBucketOptmize = aggregateOperator.usePerBucketOptmize;
this.withoutColocateRequirement = aggregateOperator.withoutColocateRequirement;
this.distinctColumnDataSkew = aggregateOperator.distinctColumnDataSkew;
this.groupByMinMaxStatistic = aggregateOperator.groupByMinMaxStatistic;
}
public List<ColumnRefOperator> getGroupBys() {
@ -195,6 +200,13 @@ public class PhysicalHashAggregateOperator extends PhysicalOperator {
this.distinctColumnDataSkew = distinctColumnDataSkew;
}
public void setGroupByMinMaxStatistic(List<Pair<ConstantOperator, ConstantOperator>> groupByMinMaxStatistic) {
this.groupByMinMaxStatistic = groupByMinMaxStatistic;
}
public List<Pair<ConstantOperator, ConstantOperator>> getGroupByMinMaxStatistic() {
return this.groupByMinMaxStatistic;
}
public DataSkewInfo getDistinctColumnDataSkew() {
return distinctColumnDataSkew;
}

View File

@ -49,7 +49,6 @@ import com.starrocks.sql.optimizer.statistics.StatsVersion;
import com.starrocks.statistic.StatisticUtils;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@ -236,11 +235,16 @@ public class RewriteSimpleAggToMetaScanRule extends TransformationRule {
LogicalOlapScanOperator scanOperator = input.inputAt(0).inputAt(0).getOp().cast();
OlapTable table = (OlapTable) scanOperator.getTable();
if (null == StatisticUtils.getTableLastUpdateTime(table)) {
LocalDateTime lastUpdateTime = StatisticUtils.getTableLastUpdateTime(table);
Long lastUpdateTimestamp = StatisticUtils.getTableLastUpdateTimestamp(table);
if (lastUpdateTime == null || lastUpdateTimestamp == null) {
return Optional.empty();
}
if (table.inputHasTempPartition(scanOperator.getSelectedPartitionId())) {
return Optional.empty();
}
LocalDateTime lastUpdateTime = StatisticUtils.getTableLastUpdateTime(table);
Map<ColumnRefOperator, ScalarOperator> constantMap = Maps.newHashMap();
Map<ColumnRefOperator, CallOperator> newAggCalls = Maps.newHashMap();
for (Map.Entry<ColumnRefOperator, CallOperator> entry : aggregationOperator.getAggregations().entrySet()) {
@ -262,7 +266,7 @@ public class RewriteSimpleAggToMetaScanRule extends TransformationRule {
Column c = scanOperator.getColRefToColumnMetaMap().get(ref);
Optional<IMinMaxStatsMgr.ColumnMinMax> minMax = IMinMaxStatsMgr.internalInstance()
.getStats(new ColumnIdentifier(table.getId(), c.getColumnId()),
new StatsVersion(-1, lastUpdateTime.toEpochSecond(ZoneOffset.UTC)));
new StatsVersion(-1, lastUpdateTimestamp));
if (minMax.isEmpty()) {
continue;
}

View File

@ -0,0 +1,117 @@
// Copyright 2021-present StarRocks, Inc. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package com.starrocks.sql.optimizer.rule.tree;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.starrocks.catalog.Column;
import com.starrocks.catalog.OlapTable;
import com.starrocks.common.Pair;
import com.starrocks.qe.ConnectContext;
import com.starrocks.sql.optimizer.OptExpression;
import com.starrocks.sql.optimizer.Utils;
import com.starrocks.sql.optimizer.base.ColumnIdentifier;
import com.starrocks.sql.optimizer.base.ColumnRefSet;
import com.starrocks.sql.optimizer.operator.OperatorType;
import com.starrocks.sql.optimizer.operator.physical.PhysicalHashAggregateOperator;
import com.starrocks.sql.optimizer.operator.physical.PhysicalOlapScanOperator;
import com.starrocks.sql.optimizer.operator.scalar.ColumnRefOperator;
import com.starrocks.sql.optimizer.operator.scalar.ConstantOperator;
import com.starrocks.sql.optimizer.statistics.ColumnDict;
import com.starrocks.sql.optimizer.statistics.IMinMaxStatsMgr;
import com.starrocks.sql.optimizer.statistics.StatsVersion;
import com.starrocks.sql.optimizer.task.TaskContext;
import com.starrocks.statistic.StatisticUtils;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
// This Rule should be after Low cardinality rewrite Rules
public class ApplyMinMaxStatisticRule implements TreeRewriteRule {
@Override
public OptExpression rewrite(OptExpression root, TaskContext taskContext) {
if (!ConnectContext.get().getSessionVariable().enableGroupByCompressedKey()) {
return root;
}
// collect aggregate operators
List<PhysicalHashAggregateOperator> aggLists = Lists.newArrayList();
Utils.extractOperator(root, aggLists, op -> OperatorType.PHYSICAL_HASH_AGG.equals(op.getOpType()));
// collect
ColumnRefSet groupByRefSets = new ColumnRefSet();
for (PhysicalHashAggregateOperator agg : aggLists) {
for (ColumnRefOperator groupBy : agg.getGroupBys()) {
groupByRefSets.union(groupBy.getId());
}
}
Map<Integer, Pair<ConstantOperator, ConstantOperator>> infos = Maps.newHashMap();
List<PhysicalOlapScanOperator> scanOperators = Utils.extractPhysicalOlapScanOperator(root);
for (PhysicalOlapScanOperator scanOperator : scanOperators) {
final Map<Integer, ColumnDict> globalDicts =
scanOperator.getGlobalDicts().stream().collect(Collectors.toMap(p -> p.first, p -> p.second));
OlapTable table = (OlapTable) scanOperator.getTable();
final Long lastUpdateTime = StatisticUtils.getTableLastUpdateTimestamp(table);
if (null == lastUpdateTime) {
continue;
}
if (table.inputHasTempPartition(scanOperator.getSelectedPartitionId())) {
continue;
}
for (ColumnRefOperator column : scanOperator.getColRefToColumnMetaMap().keySet()) {
if (groupByRefSets.contains(column.getId())) {
if (!column.getType().isNumericType() && !column.getType().isDate()) {
continue;
}
if (globalDicts.containsKey(column.getId())) {
final ConstantOperator min = ConstantOperator.createVarchar("0");
final ColumnDict columnDict = globalDicts.get(column.getId());
final ConstantOperator max = ConstantOperator.createVarchar("" + columnDict.getDictSize());
infos.put(column.getId(), new Pair<>(min, max));
continue;
}
Column c = table.getColumn(column.getName());
Optional<IMinMaxStatsMgr.ColumnMinMax> minMax = IMinMaxStatsMgr.internalInstance()
.getStats(new ColumnIdentifier(table.getId(), c.getColumnId()),
new StatsVersion(-1, lastUpdateTime));
if (minMax.isEmpty()) {
continue;
}
final ConstantOperator min = ConstantOperator.createVarchar(minMax.get().minValue());
final ConstantOperator max = ConstantOperator.createVarchar(minMax.get().maxValue());
infos.put(column.getId(), new Pair<>(min, max));
}
}
}
for (PhysicalHashAggregateOperator agg : aggLists) {
List<Pair<ConstantOperator, ConstantOperator>> groupByMinMaxStats = Lists.newArrayList();
for (ColumnRefOperator groupBy : agg.getGroupBys()) {
final Pair<ConstantOperator, ConstantOperator> minMaxStats = infos.get(groupBy.getId());
if (minMaxStats != null) {
groupByMinMaxStats.add(minMaxStats);
}
}
if (groupByMinMaxStats.size() == agg.getGroupBys().size()) {
agg.setGroupByMinMaxStatistic(groupByMinMaxStats);
}
}
return root;
}
}

View File

@ -26,6 +26,7 @@ import com.starrocks.catalog.Column;
import com.starrocks.catalog.Database;
import com.starrocks.catalog.InternalCatalog;
import com.starrocks.catalog.OlapTable;
import com.starrocks.catalog.PhysicalPartition;
import com.starrocks.catalog.Table;
import com.starrocks.common.Config;
import com.starrocks.common.Pair;
@ -131,11 +132,13 @@ public class ColumnMinMaxMgr implements IMinMaxStatsMgr, MemoryTrackable {
if (column == null || (!column.getType().isNumericType() && !column.getType().isDate())) {
return Optional.empty();
}
// We need the aggregated result, so this constraint is not satisfied.
if (column.isAggregated()) {
return Optional.empty();
}
long version = olapTable.getPartitions().stream()
.map(p -> p.getDefaultPhysicalPartition().getVisibleVersionTime())
.max(Long::compareTo).orElse(0L);
long version = olapTable.getPartitions().stream().flatMap(p -> p.getSubPartitions().stream()).map(
PhysicalPartition::getVisibleVersionTime).max(Long::compareTo).orElse(0L);
String catalogName = InternalCatalog.DEFAULT_INTERNAL_CATALOG_NAME;
String sql = "select min(" + column.getName() + ") as min, max(" + column.getName() + ") as max"
+ " from " + StatisticUtils.quoting(catalogName, db.getOriginName(), olapTable.getName())
@ -148,7 +151,11 @@ public class ColumnMinMaxMgr implements IMinMaxStatsMgr, MemoryTrackable {
return Optional.empty();
}
Preconditions.checkState(result.size() == 1);
return Optional.of(new CacheValue(toMinMax(result.get(0)), new StatsVersion(version, version)));
ColumnMinMax minMax = toMinMax(result.get(0));
if (minMax.minValue() == null || minMax.maxValue() == null) {
return Optional.empty();
}
return Optional.of(new CacheValue(minMax, new StatsVersion(version, version)));
} catch (Exception e) {
LOG.warn("get MinMax for column: {}, error: {}", key, e.getMessage(), e);
}
@ -161,8 +168,19 @@ public class ColumnMinMaxMgr implements IMinMaxStatsMgr, MemoryTrackable {
ByteBuf copied = Unpooled.copiedBuffer(buffer);
String jsonString = copied.toString(Charset.defaultCharset());
JsonElement obj = JsonParser.parseString(jsonString);
JsonArray data = obj.getAsJsonObject().get("data").getAsJsonArray();
return new ColumnMinMax(data.get(0).getAsString(), data.get(1).getAsString());
JsonElement dataElement = obj.getAsJsonObject().get("data");
if (dataElement == null || dataElement.isJsonNull() || !dataElement.isJsonArray()) {
throw new IllegalStateException("Invalid 'data' field");
}
JsonArray data = dataElement.getAsJsonArray();
JsonElement minElement = data.get(0);
JsonElement maxElement = data.get(1);
String min = minElement.isJsonNull() ? null : minElement.getAsString();
String max = maxElement.isJsonNull() ? null : maxElement.getAsString();
return new ColumnMinMax(min, max);
}
}

View File

@ -2343,6 +2343,7 @@ public class PlanFragmentBuilder {
aggregationNode.setStreamingPreaggregationMode(node.getNeededPreaggregationMode());
aggregationNode.setHasNullableGenerateChild();
aggregationNode.computeStatistics(optExpr.getStatistics());
aggregationNode.setGroupByMinMaxStats(node.getGroupByMinMaxStatistic());
if (node.isOnePhaseAgg() || node.isMergedLocalAgg() || node.getType().isDistinctGlobal()) {
// For ScanNode->LocalShuffle->AggNode, we needn't assign scan ranges per driver sequence.

View File

@ -34,6 +34,7 @@ import com.starrocks.catalog.KeysType;
import com.starrocks.catalog.LocalTablet;
import com.starrocks.catalog.OlapTable;
import com.starrocks.catalog.Partition;
import com.starrocks.catalog.PhysicalPartition;
import com.starrocks.catalog.PrimitiveType;
import com.starrocks.catalog.ScalarType;
import com.starrocks.catalog.StructField;
@ -222,8 +223,8 @@ public class StatisticUtils {
public static LocalDateTime getTableLastUpdateTime(Table table) {
if (table.isNativeTableOrMaterializedView()) {
long maxTime = table.getPartitions().stream().map(p -> p.getDefaultPhysicalPartition().getVisibleVersionTime())
.max(Long::compareTo).orElse(0L);
long maxTime = table.getPartitions().stream().flatMap(p -> p.getSubPartitions().stream()).map(
PhysicalPartition::getVisibleVersionTime).max(Long::compareTo).orElse(0L);
return LocalDateTime.ofInstant(Instant.ofEpochMilli(maxTime), Clock.systemDefaultZone().getZone());
} else {
try {
@ -235,6 +236,21 @@ public class StatisticUtils {
}
}
/**
* Retrieves the last update timestamp of the specified table in milliseconds.
*
* @param table The table object for which the last update time is to be retrieved.
* @return The timestamp in milliseconds since the Unix epoch, or null if the update time is not available.
*/
public static Long getTableLastUpdateTimestamp(Table table) {
LocalDateTime updateTime = getTableLastUpdateTime(table);
if (updateTime == null) {
return null;
}
Instant instant = updateTime.atZone(Clock.systemDefaultZone().getZone()).toInstant();
return instant.toEpochMilli();
}
public static Set<String> getUpdatedPartitionNames(Table table, LocalDateTime checkTime) {
// get updated partitions
Set<String> updatedPartitions = null;

View File

@ -21,13 +21,19 @@ import com.starrocks.planner.PlanFragment;
import com.starrocks.planner.ScanNode;
import com.starrocks.sql.analyzer.SemanticException;
import com.starrocks.sql.common.StarRocksPlannerException;
import com.starrocks.sql.optimizer.base.ColumnIdentifier;
import com.starrocks.sql.optimizer.statistics.IMinMaxStatsMgr;
import com.starrocks.sql.optimizer.statistics.StatsVersion;
import com.starrocks.system.BackendResourceStat;
import com.starrocks.thrift.TExplainLevel;
import com.starrocks.utframe.UtFrameUtils;
import mockit.Expectations;
import org.apache.commons.lang3.StringUtils;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.util.Optional;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertThrows;
@ -3030,4 +3036,26 @@ public class AggregateTest extends PlanTestBase {
" 0:OlapScanNode\n" +
" TABLE: tbl1");
}
@Test
public void testGroupByCompressedKey() throws Exception {
final IMinMaxStatsMgr minMaxStatsMgr = IMinMaxStatsMgr.internalInstance();
new Expectations(minMaxStatsMgr) {
{
minMaxStatsMgr.getStats((ColumnIdentifier) any, (StatsVersion) any);
result = Optional.of(new IMinMaxStatsMgr.ColumnMinMax("0", "1"));
minMaxStatsMgr.getStats((ColumnIdentifier) any, (StatsVersion) any);
result = Optional.of(new IMinMaxStatsMgr.ColumnMinMax("0", "1"));
minMaxStatsMgr.getStats((ColumnIdentifier) any, (StatsVersion) any);
result = Optional.of(new IMinMaxStatsMgr.ColumnMinMax("0", "1"));
}
};
String sql = "select distinct v1, v2, v3 from t0";
String plan = getVerboseExplain(sql);
assertContains(plan, "group by min-max stats:");
plan = getThriftPlan(sql);
assertContains(plan, "group_by_min_max:[TExpr(");
}
}

View File

@ -877,6 +877,9 @@ struct TAggregationNode {
29: optional bool enable_pipeline_share_limit = false
30: optional list<RuntimeFilter.TRuntimeFilterDescription> build_runtime_filters
31: optional list<Exprs.TExpr> group_by_min_max
}
struct TRepeatNode {

View File

@ -2326,6 +2326,23 @@ out.append("${{dictMgr.NO_DICT_STRING_COLUMNS.contains(cid)}}")
print("scirpt output:" + str(res))
tools.assert_true(str(res["result"][0][0]).strip() == "true", "column still could collect dictionary")
def wait_min_max_stat_ready(self, column_name, table_name):
"""
wait min max stats ready
"""
status = ""
count = 0
while True:
if count > 60:
tools.assert_true(False, "acquire min-max timeout for 60s")
sql = "explain verbose select distinct %s from %s" % (column_name, table_name)
res = self.execute_sql(sql, True)
if not res["status"]:
tools.assert_true(False, "acquire min-max error")
if str(res["result"]).find("min-max stats") > 0:
return ""
time.sleep(1)
def wait_submit_task_ready(self, task_name):
"""
wait submit task ready

View File

@ -0,0 +1,991 @@
-- name: test_agg_compressed_key
create table all_t0 (
c1 tinyint,
c2 smallint,
c3 int,
c4 bigint,
c5 largeint,
c6 date,
c7 datetime,
c8 string,
c9 string,
c10 char(100),
c11 float,
c12 double,
c13 tinyint NOT NULL,
c14 smallint NOT NULL,
c15 int NOT NULL,
c16 bigint NOT NULL,
c17 largeint NOT NULL,
c18 date NOT NULL,
c19 datetime NOT NULL,
c20 string NOT NULL,
c21 string NOT NULL,
c22 char(100) NOT NULL,
c23 float NOT NULL,
c24 double NOT NULL
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
-- result:
-- !result
insert into all_t0 SELECT x%200, x%200, x%200, x%200, x%200, x, x, x%200, x, x, x, x, x % 8, x % 8, x % 16, x %200, x%200, '2020-02-02', '2020-02-02', x%200, x, x, x, x FROM TABLE(generate_series(1, 30000)) as g(x);
-- result:
-- !result
insert into all_t0 values (null, null, null, null, null, null, null, null, null, null, null, null, -1,-2,-3,-4,-5, '2000-01-28', '2000-01-28', 'literal', 'literal', 'literal', -1, -1);
-- result:
-- !result
insert into all_t0 values (-1, -2, -3, null, null, null, null, null, null, null, null, null, -1,-2,-3,-4,-5, '2000-01-28', '2000-01-28', 'literal', 'literal', 'literal', -1, -1);
-- result:
-- !result
set pipeline_dop=2;
-- result:
-- !result
select distinct c1,c2,c3,c4,c5,c6,c7,c8 from all_t0 order by 1,2,3,4,5,6,7,8 limit 100,3;
-- result:
None 161 161 161 161 None None 161
None 162 162 162 162 None None 162
None 163 163 163 163 None None 163
-- !result
function: wait_global_dict_ready('c8', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c1', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c2', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c3', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c4', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c5', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c6', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c8', 'all_t0')
-- result:
-- !result
select distinct c9,c10,c11,c12,c13,c14,c15,c16 from all_t0 order by 1,2,3,4,5,6,7,8 limit 100,3;
-- result:
10087 10087 10087.0 10087.0 7 7 7 87
10088 10088 10088.0 10088.0 0 0 8 88
10089 10089 10089.0 10089.0 1 1 9 89
-- !result
function: wait_min_max_stat_ready('c11', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c12', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c13', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c14', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c15', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c16', 'all_t0')
-- result:
-- !result
select distinct c17,c18,c19,c20,c21,c22,c23,c24 from all_t0 order by 1,2,3,4,5,6,7,8 limit 100,3;
-- result:
0 2020-02-02 2020-02-02 00:00:00 0 27800 27800 27800.0 27800.0
0 2020-02-02 2020-02-02 00:00:00 0 2800 2800 2800.0 2800.0
0 2020-02-02 2020-02-02 00:00:00 0 28000 28000 28000.0 28000.0
-- !result
function: wait_global_dict_ready('c20', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c17', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c18', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c20', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c14', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c23', 'all_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c24', 'all_t0')
-- result:
-- !result
select c1, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3;
-- result:
None None
-1 -1
0 0
-- !result
select c1, c2, sum(c1) from all_t0 group by 1,2 order by 1,2,3 limit 3;
-- result:
None None None
None 128 None
None 129 None
-- !result
select c2, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
1 150
-- !result
select c3, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
1 150
-- !result
select c4, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2 300
-- !result
select c5, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2 300
-- !result
select c6, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2000-01-03 103
-- !result
select c7, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2000-01-03 00:00:00 103
-- !result
select c8, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
10 1500
-- !result
select c9, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
100 100
-- !result
select c13, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2 148800
-- !result
select c14, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2 148800
-- !result
select c14, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2 148800
-- !result
select c16, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2 300
-- !result
select c17, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
2 300
-- !result
select c18, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
-- !result
select c19, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
-- result:
-- !result
select c2, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
199 None
-- !result
select c3, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
199 None
-- !result
select c4, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
199 None
-- !result
select c5, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
199 None
-- !result
select c6, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
2002-12-31 31
-- !result
select c7, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
2002-12-31 00:00:00 31
-- !result
select c8, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
99 14850
-- !result
select c9, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
9999 None
-- !result
select c13, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
7 160800
-- !result
select c14, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
7 160800
-- !result
select c14, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
7 160800
-- !result
select c16, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
199 None
-- !result
select c17, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
199 None
-- !result
select c18, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
2020-02-02 1219200
-- !result
select c19, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
-- result:
2020-02-02 00:00:00 1219200
-- !result
select c3, c4, sum(c1) from all_t0 group by 1,2 order by 1, 2, 3 limit 30,1;
-- result:
28 28 4200
-- !result
select c3, c5, sum(c1) from all_t0 group by 1,2 order by 1, 2, 3 limit 30,1;
-- result:
28 28 4200
-- !result
select c3, c7, sum(c1) from all_t0 group by 1,2 order by 1, 2, 3 limit 30,1;
-- result:
2 2001-04-02 00:00:00 2
-- !result
select c1,c2,c3,c4,c5,c6,c8,sum(c1) from all_t0 group by 1,2,3,4,5,6,7 order by 1,2,3,4,5,6,7,8 limit 30, 1;
-- result:
None 129 129 129 129 2001-07-29 129 None
-- !result
select c1,c2,c3,c4,c5,c6,c8,c13,c14,c15,c16, sum(c1) from all_t0 group by 1,2,3,4,5,6,7,8,9,10,11 order by 1,2,3,4,5,6,7,8,9,10,11 limit 30, 1;
-- result:
None 129 129 129 129 2001-03-29 129 1 1 9 129 None
-- !result
select c1,c2,c3,c4,c5,c6,c8,c11,c12,c13,c14,c15,c16, sum(c1) from all_t0 group by 1,2,3,4,5,6,7,8,9,10,11,12,13 order by 1,2,3,4,5,6,7,8,9,10,11,12,13 limit 30,1;
-- result:
None 128 128 128 128 None 128 7128.0 7128.0 0 0 8 128 None
-- !result
select c1,c2,c3,c4,c5,c6,c8, sum(c1) from all_t0 where c10 > 0 group by 1,2,3,4,5,6,7 order by 1,2,3,4,5,6,7,8 limit 1;
-- result:
None 128 128 128 128 None 128 None
-- !result
create table all_decimal (
c1 decimal(4,2),
c2 decimal(10,2),
c3 decimal(27,9),
c4 decimal(38,5)
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
-- result:
-- !result
insert into all_decimal SELECT x%100, x%200, x%200, x%200 FROM TABLE(generate_series(1, 30000)) as g(x);
-- result:
-- !result
select distinct c1,c2,c3,c4 from all_decimal order by 1,2,3,4 limit 100,3;
-- result:
50.00 50.00 50.000000000 50.00000
50.00 150.00 150.000000000 150.00000
51.00 51.00 51.000000000 51.00000
-- !result
function: wait_min_max_stat_ready('c1', 'all_decimal')
-- result:
-- !result
function: wait_min_max_stat_ready('c2', 'all_decimal')
-- result:
-- !result
function: wait_min_max_stat_ready('c3', 'all_decimal')
-- result:
-- !result
function: wait_min_max_stat_ready('c4', 'all_decimal')
-- result:
-- !result
select c1, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
-- result:
0.00 0.00
-- !result
select c2, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
-- result:
0.00 0.00
-- !result
select c3, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
-- result:
0E-9 0.00
-- !result
select c4, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
-- result:
0.00000 0.00
-- !result
create table all_numbers_t0 (
c1 tinyint,
c2 smallint,
c3 int,
c4 bigint,
c5 largeint,
c13 tinyint NOT NULL,
c14 smallint NOT NULL,
c15 int NOT NULL,
c16 bigint NOT NULL,
c17 largeint NOT NULL
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
-- result:
-- !result
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (-128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728, -128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728);
-- result:
-- !result
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
-- result:
-- !result
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (null, null, null, null, null, 0, 0, 0, 0, 0);
-- result:
-- !result
insert into all_numbers_t0 SELECT x%128, x%200, x%200, x%200, x%200, x%128, x%200, x%200, x%200, x%200 FROM TABLE(generate_series(1, 30000)) as g(x);
-- result:
-- !result
select distinct c17,c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9,10 limit 30,1;
-- result:
1 1 1 1 97 1 1 1 1 97
-- !result
function: wait_min_max_stat_ready('c1', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c2', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c3', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c4', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c5', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c13', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c14', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c15', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c16', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c17', 'all_numbers_t0')
-- result:
-- !result
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c2 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c3 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c4 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c5 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c13 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c14 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c15 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c16 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c17 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c2,c1 from all_numbers_t0 order by 1,2 limit 30,1;
-- result:
1 97
-- !result
select distinct c3,c2,c1 from all_numbers_t0 order by 1,2,3 limit 30,1;
-- result:
1 1 97
-- !result
select distinct c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4 limit 30,1;
-- result:
1 1 1 97
-- !result
select distinct c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
-- result:
1 1 1 1 97
-- !result
select distinct c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6 limit 30,1;
-- result:
1 25 25 25 25 1
-- !result
select distinct c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7 limit 30,1;
-- result:
1 97 1 1 1 1 97
-- !result
select distinct c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8 limit 30,1;
-- result:
1 1 97 1 1 1 1 97
-- !result
select distinct c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9 limit 30,1;
-- result:
1 1 1 97 1 1 1 1 97
-- !result
select distinct c17,c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9,10 limit 30,1;
-- result:
1 1 1 1 97 1 1 1 1 97
-- !result
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727, 127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727);
-- result:
-- !result
select distinct c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
-- result:
1 1 1 1 97
-- !result
function: wait_min_max_stat_ready('c1', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c2', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c3', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c4', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c5', 'all_numbers_t0')
-- result:
-- !result
select distinct c17,c16,c15,c14,c13 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
-- result:
1 1 1 1 105
-- !result
function: wait_min_max_stat_ready('c13', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c14', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c15', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c16', 'all_numbers_t0')
-- result:
-- !result
function: wait_min_max_stat_ready('c17', 'all_numbers_t0')
-- result:
-- !result
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c2 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c3 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c4 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c5 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c13 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c14 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c15 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c16 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c17 from all_numbers_t0 order by 1 limit 30,1;
-- result:
29
-- !result
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
-- result:
28
-- !result
select distinct c2,c1 from all_numbers_t0 order by 1,2 limit 30,1;
-- result:
1 97
-- !result
select distinct c3,c2,c1 from all_numbers_t0 order by 1,2,3 limit 30,1;
-- result:
1 1 97
-- !result
select distinct c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4 limit 30,1;
-- result:
1 1 1 97
-- !result
select distinct c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
-- result:
1 1 1 1 97
-- !result
select distinct c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6 limit 30,1;
-- result:
1 25 25 25 25 1
-- !result
select distinct c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7 limit 30,1;
-- result:
1 97 1 1 1 1 97
-- !result
select distinct c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8 limit 30,1;
-- result:
1 1 97 1 1 1 1 97
-- !result
select distinct c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9 limit 30,1;
-- result:
1 1 1 97 1 1 1 1 97
-- !result
select distinct c17,c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9,10 limit 30,1;
-- result:
1 1 1 1 97 1 1 1 1 97
-- !result
select distinct c2,c1 from all_numbers_t0 where c2 = 7 order by 1,2 limit 1;
-- result:
7 7
-- !result
CREATE TABLE agged_table (
k1 int,
k2 int sum
)
AGGREGATE KEY(k1)
DISTRIBUTED BY HASH(k1)
properties (
"replication_num" = "1"
);
-- result:
-- !result
insert into agged_table values(1,1);
-- result:
-- !result
insert into agged_table values(1,2);
-- result:
-- !result
insert into agged_table values(1,3);
-- result:
-- !result
insert into agged_table values(1,4);
-- result:
-- !result
select distinct k2 from agged_table;
-- result:
10
-- !result
CREATE TABLE trand (
k1 int,
k2 int
) DUPLICATE KEY(k1)
properties (
"replication_num" = "1"
);
-- result:
-- !result
insert into trand values(1,1);
-- result:
-- !result
function: wait_min_max_stat_ready('k1', 'trand')
-- result:
-- !result
select k1 from trand group by k1;
-- result:
1
-- !result
insert into trand values(2,2);
-- result:
-- !result
function: wait_min_max_stat_ready('k1', 'trand')
-- result:
-- !result
select k1 from trand group by k1;
-- result:
1
2
-- !result
create table all_t1 (
c1 tinyint,
c2 tinyint,
c3 tinyint,
c4 tinyint,
c5 smallint,
c6 smallint,
c7 smallint,
c8 smallint,
c9 int,
c10 int,
c11 int,
c12 int,
c13 bigint,
c14 bigint,
c15 bigint,
c16 bigint,
c17 largeint,
c18 largeint,
c19 largeint,
c20 largeint,
c21 date,
c22 date,
c23 date,
c24 date
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
-- result:
-- !result
insert into all_t1 SELECT x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x FROM TABLE(generate_series(1, 300000)) as g(x);
-- result:
-- !result
select distinct c1, c2, c3, c4, c5, c6, c7, c8 from all_t1 order by 1,2,3,4,5,6,7,8 desc limit 1;
-- result:
None None None None None None None None
-- !result
function: wait_min_max_stat_ready('c1','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c2','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c3','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c4','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c5','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c6','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c7','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c8','all_t1')
-- result:
-- !result
select distinct c9, c10, c11, c12, c13, c14, c15, c16 from all_t1 order by 1,2,3,4,5,6,7,8 desc limit 1;
-- result:
1 1 1 1 1 1 1 1
-- !result
function: wait_min_max_stat_ready('c9','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c10','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c11','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c12','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c13','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c14','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c15','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c16','all_t1')
-- result:
-- !result
select distinct c17, c18, c19, c20, c21, c22, c23, c24 from all_t1 order by 1,2,3,4,5,6,7,8 desc limit 1;
-- result:
1 1 1 1 None None None None
-- !result
function: wait_min_max_stat_ready('c17','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c18','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c19','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c20','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c21','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c22','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c23','all_t1')
-- result:
-- !result
function: wait_min_max_stat_ready('c24','all_t1')
-- result:
-- !result
set group_concat_max_len=65535;
-- result:
-- !result
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 1' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 1 92fea2997ab6f5848123aad560a2372d
-- !result
WITH result AS (
SELECT c2, COUNT(*) AS cnt FROM all_t0 GROUP BY c2 ORDER BY c2 LIMIT 100
) SELECT 'Test Case 2' AS test_name, MD5(GROUP_CONCAT(CAST(c2 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 2 5e0364b0e8c29d819b0ddc37f5788777
-- !result
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_decimal GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 25' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 25 99fb9efb02cccfe487712d215c971085
-- !result
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 29' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 29 13318347ed379e157b0c9e9f4a46287a
-- !result
WITH result AS (
SELECT c1, c2, c3, COUNT(*) AS cnt FROM all_t0 GROUP BY c1, c2, c3 ORDER BY c1, c2, c3 LIMIT 100
) SELECT 'Test Case 39' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c2 AS STRING) || ':' || CAST(c3 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 39 7b5bec280b15d5daad3afb7639384c55
-- !result
WITH result AS (
SELECT c6, c7, c8, COUNT(*) AS cnt FROM all_t0 GROUP BY c6, c7, c8 ORDER BY c6, c7, c8 LIMIT 100
) SELECT 'Test Case 40' AS test_name, MD5(GROUP_CONCAT(CAST(c6 AS STRING) || ':' || CAST(c7 AS STRING) || ':' || c8 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 40 None
-- !result
WITH result AS (
SELECT c13, c14, c15, COUNT(*) AS cnt FROM all_t0 GROUP BY c13, c14, c15 ORDER BY c13, c14, c15 LIMIT 100
) SELECT 'Test Case 42' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(c14 AS STRING) || ':' || CAST(c15 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 42 58c87d5724326e8aa1e11ef3146af468
-- !result
WITH result AS (
SELECT c9, COUNT(*) AS cnt FROM all_t0 GROUP BY c9 ORDER BY c9 LIMIT 100
) SELECT 'Test Case 51' AS test_name, MD5(GROUP_CONCAT(c9 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 51 3fa440f2926d77a0ae8c62b069c8901b
-- !result
WITH result AS (
SELECT c9, COUNT(*) AS cnt FROM all_t1 GROUP BY c9 ORDER BY c9 LIMIT 1000
) SELECT 'Test Case 52' AS test_name, MD5(GROUP_CONCAT(CAST(c9 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 52 7c155d4a239a8364a02031c69ff0b650
-- !result
WITH result AS (
SELECT c18, COUNT(*) AS cnt FROM all_t0 GROUP BY c18 ORDER BY c18 LIMIT 100
) SELECT 'Test Case 53' AS test_name, MD5(GROUP_CONCAT(CAST(c18 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 53 5907f2825cc824723339da10e632159e
-- !result
WITH result AS (
SELECT c13, COUNT(*) AS cnt FROM all_t0 GROUP BY c13 ORDER BY c13 LIMIT 100
) SELECT 'Test Case 54' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 54 c0ac02a2a0a37d496a0385a1ed1ad64a
-- !result
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 GROUP BY c1 ORDER BY c1 NULLS FIRST LIMIT 100
) SELECT 'Test Case 55' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 55 92fea2997ab6f5848123aad560a2372d
-- !result
WITH result AS (
SELECT c5, COUNT(*) AS cnt FROM all_t0 GROUP BY c5 ORDER BY c5 NULLS FIRST LIMIT 100
) SELECT 'Test Case 56' AS test_name, MD5(GROUP_CONCAT(CAST(c5 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 56 2a2b8e9f52cf423ebd681b20084756e2
-- !result
WITH result AS (
SELECT c1, c5, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c1, c5 ORDER BY c1, c5 NULLS FIRST LIMIT 100
) SELECT 'Test Case 57' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c5 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 57 107788565111f8e9685f3cf7591d4002
-- !result
WITH result AS (
SELECT c1, c13, COUNT(*) AS cnt FROM all_t0 GROUP BY c1, c13 ORDER BY c1, c13 NULLS FIRST LIMIT 100
) SELECT 'Test Case 58' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c13 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 58 2acc50a5bae0f273bc364c528a16a9cd
-- !result
WITH result AS (
SELECT c1, c2, COUNT(*) AS cnt FROM all_t0 GROUP BY ROLLUP (c1, c2) ORDER BY 1,2,3 LIMIT 100
) SELECT 'Test Case 59' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c2 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 59 fd49e7e8a32b37b74fc432872b18c863
-- !result
WITH result AS (
SELECT c3, c6, COUNT(*) AS cnt FROM all_t0 GROUP BY CUBE(c3, c6) ORDER BY 1,2,3 LIMIT 100
) SELECT 'Test Case 60' AS test_name, MD5(GROUP_CONCAT(CAST(c3 AS STRING) || ':' || CAST(c6 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 60 None
-- !result
WITH result AS (
SELECT c13, c14, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY ROLLUP (c13, c14) ORDER BY 1,2,3 LIMIT 100
) SELECT 'Test Case 61' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(c14 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 61 a61b6bb35fcff5a0a66fe16f21141db6
-- !result
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 62' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 62 92fea2997ab6f5848123aad560a2372d
-- !result
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 WHERE c1 > 200 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 89' AS test_name, MD5('empty') AS result_hash FROM result LIMIT 1;
-- result:
-- !result
WITH result AS (
SELECT k1, COUNT(*) AS cnt FROM trand GROUP BY k1 ORDER BY k1
) SELECT 'Test Case 90' AS test_name, MD5(GROUP_CONCAT(CAST(k1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 90 dd11f593a8cf98c3ef244e5ed6ffcce9
-- !result
WITH result AS (
SELECT c8, COUNT(*) AS cnt FROM all_t0 GROUP BY c8 ORDER BY c8 LIMIT 100
) SELECT 'Test Case 93' AS test_name, MD5(GROUP_CONCAT(c8 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 93 d45117aabb68f78a61f7346316ed59a9
-- !result
WITH result AS (
SELECT c20, COUNT(*) AS cnt FROM all_t0 GROUP BY c20 ORDER BY c20 LIMIT 100
) SELECT 'Test Case 94' AS test_name, MD5(GROUP_CONCAT(c20 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 94 7d955eb3810b33ae70204a7dd2a3947c
-- !result
WITH result AS (
SELECT c1, c5, c9, c13, c17, COUNT(*) AS cnt FROM all_t1 GROUP BY c1, c5, c9, c13, c17 ORDER BY c1, c5, c9, c13, c17 LIMIT 1000
) SELECT 'Test Case 95' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c5 AS STRING) || ':' || CAST(c9 AS STRING) || ':' || CAST(c13 AS STRING) || ':' || CAST(c17 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 95 None
-- !result
WITH result AS (
SELECT c1, c2, c3, c4, c5, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c1, c2, c3, c4, c5 ORDER BY c1, c2, c3, c4, c5 LIMIT 100
) SELECT 'Test Case 96' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c2 AS STRING) || ':' || CAST(c3 AS STRING) || ':' || CAST(c4 AS STRING) || ':' || CAST(c5 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 96 e52c7896a7ad99c5f0267de784fbeb0f
-- !result
WITH result AS (
SELECT c13, c14, c15, c16, c17, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c13, c14, c15, c16, c17 ORDER BY c13, c14, c15, c16, c17 LIMIT 100
) SELECT 'Test Case 97' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(c14 AS STRING) || ':' || CAST(c15 AS STRING) || ':' || CAST(c16 AS STRING) || ':' || CAST(c17 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- result:
Test Case 97 4f9b50e5e27ed050425b72f98998db5c
-- !result

View File

@ -0,0 +1,465 @@
-- name: test_agg_compressed_key
create table all_t0 (
c1 tinyint,
c2 smallint,
c3 int,
c4 bigint,
c5 largeint,
c6 date,
c7 datetime,
c8 string,
c9 string,
c10 char(100),
c11 float,
c12 double,
c13 tinyint NOT NULL,
c14 smallint NOT NULL,
c15 int NOT NULL,
c16 bigint NOT NULL,
c17 largeint NOT NULL,
c18 date NOT NULL,
c19 datetime NOT NULL,
c20 string NOT NULL,
c21 string NOT NULL,
c22 char(100) NOT NULL,
c23 float NOT NULL,
c24 double NOT NULL
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
insert into all_t0 SELECT x%200, x%200, x%200, x%200, x%200, x, x, x%200, x, x, x, x, x % 8, x % 8, x % 16, x %200, x%200, '2020-02-02', '2020-02-02', x%200, x, x, x, x FROM TABLE(generate_series(1, 30000)) as g(x);
insert into all_t0 values (null, null, null, null, null, null, null, null, null, null, null, null, -1,-2,-3,-4,-5, '2000-01-28', '2000-01-28', 'literal', 'literal', 'literal', -1, -1);
insert into all_t0 values (-1, -2, -3, null, null, null, null, null, null, null, null, null, -1,-2,-3,-4,-5, '2000-01-28', '2000-01-28', 'literal', 'literal', 'literal', -1, -1);
set pipeline_dop=2;
select distinct c1,c2,c3,c4,c5,c6,c7,c8 from all_t0 order by 1,2,3,4,5,6,7,8 limit 100,3;
function: wait_global_dict_ready('c8', 'all_t0')
function: wait_min_max_stat_ready('c1', 'all_t0')
function: wait_min_max_stat_ready('c2', 'all_t0')
function: wait_min_max_stat_ready('c3', 'all_t0')
function: wait_min_max_stat_ready('c4', 'all_t0')
function: wait_min_max_stat_ready('c5', 'all_t0')
function: wait_min_max_stat_ready('c6', 'all_t0')
function: wait_min_max_stat_ready('c8', 'all_t0')
select distinct c9,c10,c11,c12,c13,c14,c15,c16 from all_t0 order by 1,2,3,4,5,6,7,8 limit 100,3;
function: wait_min_max_stat_ready('c11', 'all_t0')
function: wait_min_max_stat_ready('c12', 'all_t0')
function: wait_min_max_stat_ready('c13', 'all_t0')
function: wait_min_max_stat_ready('c14', 'all_t0')
function: wait_min_max_stat_ready('c15', 'all_t0')
function: wait_min_max_stat_ready('c16', 'all_t0')
select distinct c17,c18,c19,c20,c21,c22,c23,c24 from all_t0 order by 1,2,3,4,5,6,7,8 limit 100,3;
function: wait_global_dict_ready('c20', 'all_t0')
function: wait_min_max_stat_ready('c17', 'all_t0')
function: wait_min_max_stat_ready('c18', 'all_t0')
function: wait_min_max_stat_ready('c20', 'all_t0')
function: wait_min_max_stat_ready('c14', 'all_t0')
function: wait_min_max_stat_ready('c23', 'all_t0')
function: wait_min_max_stat_ready('c24', 'all_t0')
select c1, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3;
select c1, c2, sum(c1) from all_t0 group by 1,2 order by 1,2,3 limit 3;
-- any types
select c2, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c3, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c4, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c5, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c6, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c7, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c8, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c9, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c13, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c14, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c14, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c16, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c17, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c18, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c19, sum(c1) from all_t0 group by 1 order by 1, 2 limit 3, 1;
select c2, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c3, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c4, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c5, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c6, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c7, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c8, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c9, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c13, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c14, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c14, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c16, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c17, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c18, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c19, sum(c1) from all_t0 group by 1 order by 1 desc, 2 desc limit 1;
select c3, c4, sum(c1) from all_t0 group by 1,2 order by 1, 2, 3 limit 30,1;
select c3, c5, sum(c1) from all_t0 group by 1,2 order by 1, 2, 3 limit 30,1;
select c3, c7, sum(c1) from all_t0 group by 1,2 order by 1, 2, 3 limit 30,1;
select c1,c2,c3,c4,c5,c6,c8,sum(c1) from all_t0 group by 1,2,3,4,5,6,7 order by 1,2,3,4,5,6,7,8 limit 30, 1;
select c1,c2,c3,c4,c5,c6,c8,c13,c14,c15,c16, sum(c1) from all_t0 group by 1,2,3,4,5,6,7,8,9,10,11 order by 1,2,3,4,5,6,7,8,9,10,11 limit 30, 1;
select c1,c2,c3,c4,c5,c6,c8,c11,c12,c13,c14,c15,c16, sum(c1) from all_t0 group by 1,2,3,4,5,6,7,8,9,10,11,12,13 order by 1,2,3,4,5,6,7,8,9,10,11,12,13 limit 30,1;
-- projection
select c1,c2,c3,c4,c5,c6,c8, sum(c1) from all_t0 where c10 > 0 group by 1,2,3,4,5,6,7 order by 1,2,3,4,5,6,7,8 limit 1;
-- decimals
create table all_decimal (
c1 decimal(4,2),
c2 decimal(10,2),
c3 decimal(27,9),
c4 decimal(38,5)
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
insert into all_decimal SELECT x%100, x%200, x%200, x%200 FROM TABLE(generate_series(1, 30000)) as g(x);
select distinct c1,c2,c3,c4 from all_decimal order by 1,2,3,4 limit 100,3;
function: wait_min_max_stat_ready('c1', 'all_decimal')
function: wait_min_max_stat_ready('c2', 'all_decimal')
function: wait_min_max_stat_ready('c3', 'all_decimal')
function: wait_min_max_stat_ready('c4', 'all_decimal')
select c1, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
select c2, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
select c3, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
select c4, sum(c1) from all_decimal group by 1 order by 1, 2 limit 1;
-- int overflow
create table all_numbers_t0 (
c1 tinyint,
c2 smallint,
c3 int,
c4 bigint,
c5 largeint,
c13 tinyint NOT NULL,
c14 smallint NOT NULL,
c15 int NOT NULL,
c16 bigint NOT NULL,
c17 largeint NOT NULL
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (-128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728, -128, -32768, -2147483648, -9223372036854775808, -170141183460469231731687303715884105728);
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (null, null, null, null, null, 0, 0, 0, 0, 0);
insert into all_numbers_t0 SELECT x%128, x%200, x%200, x%200, x%200, x%128, x%200, x%200, x%200, x%200 FROM TABLE(generate_series(1, 30000)) as g(x);
select distinct c17,c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9,10 limit 30,1;
function: wait_min_max_stat_ready('c1', 'all_numbers_t0')
function: wait_min_max_stat_ready('c2', 'all_numbers_t0')
function: wait_min_max_stat_ready('c3', 'all_numbers_t0')
function: wait_min_max_stat_ready('c4', 'all_numbers_t0')
function: wait_min_max_stat_ready('c5', 'all_numbers_t0')
function: wait_min_max_stat_ready('c13', 'all_numbers_t0')
function: wait_min_max_stat_ready('c14', 'all_numbers_t0')
function: wait_min_max_stat_ready('c15', 'all_numbers_t0')
function: wait_min_max_stat_ready('c16', 'all_numbers_t0')
function: wait_min_max_stat_ready('c17', 'all_numbers_t0')
-- set enable_group_by_compressed_key = false;
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
select distinct c2 from all_numbers_t0 order by 1 limit 30,1;
select distinct c3 from all_numbers_t0 order by 1 limit 30,1;
select distinct c4 from all_numbers_t0 order by 1 limit 30,1;
select distinct c5 from all_numbers_t0 order by 1 limit 30,1;
select distinct c13 from all_numbers_t0 order by 1 limit 30,1;
select distinct c14 from all_numbers_t0 order by 1 limit 30,1;
select distinct c15 from all_numbers_t0 order by 1 limit 30,1;
select distinct c16 from all_numbers_t0 order by 1 limit 30,1;
select distinct c17 from all_numbers_t0 order by 1 limit 30,1;
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
select distinct c2,c1 from all_numbers_t0 order by 1,2 limit 30,1;
select distinct c3,c2,c1 from all_numbers_t0 order by 1,2,3 limit 30,1;
select distinct c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4 limit 30,1;
select distinct c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
select distinct c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6 limit 30,1;
select distinct c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7 limit 30,1;
select distinct c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8 limit 30,1;
select distinct c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9 limit 30,1;
select distinct c17,c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9,10 limit 30,1;
insert into all_numbers_t0 (c1, c2, c3, c4, c5, c13, c14, c15, c16, c17) values (127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727, 127, 32767, 2147483647, 9223372036854775807, 170141183460469231731687303715884105727);
select distinct c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
function: wait_min_max_stat_ready('c1', 'all_numbers_t0')
function: wait_min_max_stat_ready('c2', 'all_numbers_t0')
function: wait_min_max_stat_ready('c3', 'all_numbers_t0')
function: wait_min_max_stat_ready('c4', 'all_numbers_t0')
function: wait_min_max_stat_ready('c5', 'all_numbers_t0')
select distinct c17,c16,c15,c14,c13 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
function: wait_min_max_stat_ready('c13', 'all_numbers_t0')
function: wait_min_max_stat_ready('c14', 'all_numbers_t0')
function: wait_min_max_stat_ready('c15', 'all_numbers_t0')
function: wait_min_max_stat_ready('c16', 'all_numbers_t0')
function: wait_min_max_stat_ready('c17', 'all_numbers_t0')
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
select distinct c2 from all_numbers_t0 order by 1 limit 30,1;
select distinct c3 from all_numbers_t0 order by 1 limit 30,1;
select distinct c4 from all_numbers_t0 order by 1 limit 30,1;
select distinct c5 from all_numbers_t0 order by 1 limit 30,1;
select distinct c13 from all_numbers_t0 order by 1 limit 30,1;
select distinct c14 from all_numbers_t0 order by 1 limit 30,1;
select distinct c15 from all_numbers_t0 order by 1 limit 30,1;
select distinct c16 from all_numbers_t0 order by 1 limit 30,1;
select distinct c17 from all_numbers_t0 order by 1 limit 30,1;
select distinct c1 from all_numbers_t0 order by 1 limit 30,1;
select distinct c2,c1 from all_numbers_t0 order by 1,2 limit 30,1;
select distinct c3,c2,c1 from all_numbers_t0 order by 1,2,3 limit 30,1;
select distinct c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4 limit 30,1;
select distinct c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5 limit 30,1;
select distinct c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6 limit 30,1;
select distinct c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7 limit 30,1;
select distinct c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8 limit 30,1;
select distinct c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9 limit 30,1;
select distinct c17,c16,c15,c14,c13,c5,c4,c3,c2,c1 from all_numbers_t0 order by 1,2,3,4,5,6,7,8,9,10 limit 30,1;
select distinct c2,c1 from all_numbers_t0 where c2 = 7 order by 1,2 limit 1;
-- test aggregate table
CREATE TABLE agged_table (
k1 int,
k2 int sum
)
AGGREGATE KEY(k1)
DISTRIBUTED BY HASH(k1)
properties (
"replication_num" = "1"
);
insert into agged_table values(1,1);
insert into agged_table values(1,2);
insert into agged_table values(1,3);
insert into agged_table values(1,4);
select distinct k2 from agged_table;
-- test random table insert
CREATE TABLE trand (
k1 int,
k2 int
) DUPLICATE KEY(k1)
properties (
"replication_num" = "1"
);
insert into trand values(1,1);
function: wait_min_max_stat_ready('k1', 'trand')
select k1 from trand group by k1;
insert into trand values(2,2);
function: wait_min_max_stat_ready('k1', 'trand')
select k1 from trand group by k1;
create table all_t1 (
c1 tinyint,
c2 tinyint,
c3 tinyint,
c4 tinyint,
c5 smallint,
c6 smallint,
c7 smallint,
c8 smallint,
c9 int,
c10 int,
c11 int,
c12 int,
c13 bigint,
c14 bigint,
c15 bigint,
c16 bigint,
c17 largeint,
c18 largeint,
c19 largeint,
c20 largeint,
c21 date,
c22 date,
c23 date,
c24 date
) DUPLICATE KEY(c1) DISTRIBUTED BY RANDOM PROPERTIES('replication_num' = '1');
insert into all_t1 SELECT x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x,x FROM TABLE(generate_series(1, 300000)) as g(x);
select distinct c1, c2, c3, c4, c5, c6, c7, c8 from all_t1 order by 1,2,3,4,5,6,7,8 desc limit 1;
function: wait_min_max_stat_ready('c1','all_t1')
function: wait_min_max_stat_ready('c2','all_t1')
function: wait_min_max_stat_ready('c3','all_t1')
function: wait_min_max_stat_ready('c4','all_t1')
function: wait_min_max_stat_ready('c5','all_t1')
function: wait_min_max_stat_ready('c6','all_t1')
function: wait_min_max_stat_ready('c7','all_t1')
function: wait_min_max_stat_ready('c8','all_t1')
select distinct c9, c10, c11, c12, c13, c14, c15, c16 from all_t1 order by 1,2,3,4,5,6,7,8 desc limit 1;
function: wait_min_max_stat_ready('c9','all_t1')
function: wait_min_max_stat_ready('c10','all_t1')
function: wait_min_max_stat_ready('c11','all_t1')
function: wait_min_max_stat_ready('c12','all_t1')
function: wait_min_max_stat_ready('c13','all_t1')
function: wait_min_max_stat_ready('c14','all_t1')
function: wait_min_max_stat_ready('c15','all_t1')
function: wait_min_max_stat_ready('c16','all_t1')
select distinct c17, c18, c19, c20, c21, c22, c23, c24 from all_t1 order by 1,2,3,4,5,6,7,8 desc limit 1;
function: wait_min_max_stat_ready('c17','all_t1')
function: wait_min_max_stat_ready('c18','all_t1')
function: wait_min_max_stat_ready('c19','all_t1')
function: wait_min_max_stat_ready('c20','all_t1')
function: wait_min_max_stat_ready('c21','all_t1')
function: wait_min_max_stat_ready('c22','all_t1')
function: wait_min_max_stat_ready('c23','all_t1')
function: wait_min_max_stat_ready('c24','all_t1')
-- Test Case generated by AI
set group_concat_max_len=65535;
-- Test Case 1-24: GROUP BY on each column in all_t0
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 1' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
WITH result AS (
SELECT c2, COUNT(*) AS cnt FROM all_t0 GROUP BY c2 ORDER BY c2 LIMIT 100
) SELECT 'Test Case 2' AS test_name, MD5(GROUP_CONCAT(CAST(c2 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Repeat for c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18, c19, c20, c21, c22, c23, c24
-- Test Case 25-28: GROUP BY on each column in all_decimal
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_decimal GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 25' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Repeat for c2, c3, c4
-- Test Case 29-38: GROUP BY on each column in all_numbers_t0
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 29' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Repeat for c2, c3, c4, c5, c13, c14, c15, c16, c17
-- Test Case 39: TINYINT, SMALLINT, INT
WITH result AS (
SELECT c1, c2, c3, COUNT(*) AS cnt FROM all_t0 GROUP BY c1, c2, c3 ORDER BY c1, c2, c3 LIMIT 100
) SELECT 'Test Case 39' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c2 AS STRING) || ':' || CAST(c3 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 40: DATE, DATETIME, STRING
WITH result AS (
SELECT c6, c7, c8, COUNT(*) AS cnt FROM all_t0 GROUP BY c6, c7, c8 ORDER BY c6, c7, c8 LIMIT 100
) SELECT 'Test Case 40' AS test_name, MD5(GROUP_CONCAT(CAST(c6 AS STRING) || ':' || CAST(c7 AS STRING) || ':' || c8 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 41: FLOAT, DOUBLE, CHAR
-- unstable result
-- WITH result AS (
-- SELECT c11, c12, c22, COUNT(*) AS cnt FROM all_t0 GROUP BY c11, c12, c22 ORDER BY c11, c12, c22 LIMIT 100
-- ) SELECT 'Test Case 41' AS test_name, MD5(GROUP_CONCAT(CAST(c11 AS STRING) || ':' || CAST(c12 AS STRING) || ':' || c22 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 42: NOT NULL columns
WITH result AS (
SELECT c13, c14, c15, COUNT(*) AS cnt FROM all_t0 GROUP BY c13, c14, c15 ORDER BY c13, c14, c15 LIMIT 100
) SELECT 'Test Case 42' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(c14 AS STRING) || ':' || CAST(c15 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 51: High-cardinality STRING (all_t0)
WITH result AS (
SELECT c9, COUNT(*) AS cnt FROM all_t0 GROUP BY c9 ORDER BY c9 LIMIT 100
) SELECT 'Test Case 51' AS test_name, MD5(GROUP_CONCAT(c9 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 52: High-cardinality INT (all_t1)
WITH result AS (
SELECT c9, COUNT(*) AS cnt FROM all_t1 GROUP BY c9 ORDER BY c9 LIMIT 1000
) SELECT 'Test Case 52' AS test_name, MD5(GROUP_CONCAT(CAST(c9 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 53: Low-cardinality DATE NOT NULL (all_t0)
WITH result AS (
SELECT c18, COUNT(*) AS cnt FROM all_t0 GROUP BY c18 ORDER BY c18 LIMIT 100
) SELECT 'Test Case 53' AS test_name, MD5(GROUP_CONCAT(CAST(c18 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 54: Low-cardinality TINYINT NOT NULL (all_t0)
WITH result AS (
SELECT c13, COUNT(*) AS cnt FROM all_t0 GROUP BY c13 ORDER BY c13 LIMIT 100
) SELECT 'Test Case 54' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 55: Nullable TINYINT
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 GROUP BY c1 ORDER BY c1 NULLS FIRST LIMIT 100
) SELECT 'Test Case 55' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 56: Nullable LARGEINT
WITH result AS (
SELECT c5, COUNT(*) AS cnt FROM all_t0 GROUP BY c5 ORDER BY c5 NULLS FIRST LIMIT 100
) SELECT 'Test Case 56' AS test_name, MD5(GROUP_CONCAT(CAST(c5 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 57: Nullable columns in all_numbers_t0
WITH result AS (
SELECT c1, c5, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c1, c5 ORDER BY c1, c5 NULLS FIRST LIMIT 100
) SELECT 'Test Case 57' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c5 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 58: Mixed nullable and NOT NULL
WITH result AS (
SELECT c1, c13, COUNT(*) AS cnt FROM all_t0 GROUP BY c1, c13 ORDER BY c1, c13 NULLS FIRST LIMIT 100
) SELECT 'Test Case 58' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c13 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 59: ROLLUP on TINYINT, SMALLINT
WITH result AS (
SELECT c1, c2, COUNT(*) AS cnt FROM all_t0 GROUP BY ROLLUP (c1, c2) ORDER BY 1,2,3 LIMIT 100
) SELECT 'Test Case 59' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c2 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 60: CUBE on INT, DATE
WITH result AS (
SELECT c3, c6, COUNT(*) AS cnt FROM all_t0 GROUP BY CUBE(c3, c6) ORDER BY 1,2,3 LIMIT 100
) SELECT 'Test Case 60' AS test_name, MD5(GROUP_CONCAT(CAST(c3 AS STRING) || ':' || CAST(c6 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 61: ROLLUP on all_numbers_t0
WITH result AS (
SELECT c13, c14, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY ROLLUP (c13, c14) ORDER BY 1,2,3 LIMIT 100
) SELECT 'Test Case 61' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(c14 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 62-85: Repeat Test Cases 1-24 with compressed key disabled
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 62' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Repeat for c2, c3, ..., c24 (Test Cases 63-85)
-- Test Case 89: Empty groups
WITH result AS (
SELECT c1, COUNT(*) AS cnt FROM all_t0 WHERE c1 > 200 GROUP BY c1 ORDER BY c1 LIMIT 100
) SELECT 'Test Case 89' AS test_name, MD5('empty') AS result_hash FROM result LIMIT 1;
-- Test Case 90: Small dataset (trand)
WITH result AS (
SELECT k1, COUNT(*) AS cnt FROM trand GROUP BY k1 ORDER BY k1
) SELECT 'Test Case 90' AS test_name, MD5(GROUP_CONCAT(CAST(k1 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 91: Colocate single column
-- Test Case 92: Colocate multi-column
-- Test Case 93: STRING with dictionary
WITH result AS (
SELECT c8, COUNT(*) AS cnt FROM all_t0 GROUP BY c8 ORDER BY c8 LIMIT 100
) SELECT 'Test Case 93' AS test_name, MD5(GROUP_CONCAT(c8 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 94: STRING NOT NULL with dictionary
WITH result AS (
SELECT c20, COUNT(*) AS cnt FROM all_t0 GROUP BY c20 ORDER BY c20 LIMIT 100
) SELECT 'Test Case 94' AS test_name, MD5(GROUP_CONCAT(c20 || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 95: all_t1 multi-column
WITH result AS (
SELECT c1, c5, c9, c13, c17, COUNT(*) AS cnt FROM all_t1 GROUP BY c1, c5, c9, c13, c17 ORDER BY c1, c5, c9, c13, c17 LIMIT 1000
) SELECT 'Test Case 95' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c5 AS STRING) || ':' || CAST(c9 AS STRING) || ':' || CAST(c13 AS STRING) || ':' || CAST(c17 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 96: all_numbers_t0 multi-column
WITH result AS (
SELECT c1, c2, c3, c4, c5, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c1, c2, c3, c4, c5 ORDER BY c1, c2, c3, c4, c5 LIMIT 100
) SELECT 'Test Case 96' AS test_name, MD5(GROUP_CONCAT(CAST(c1 AS STRING) || ':' || CAST(c2 AS STRING) || ':' || CAST(c3 AS STRING) || ':' || CAST(c4 AS STRING) || ':' || CAST(c5 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;
-- Test Case 97-100: Additional combinations
WITH result AS (
SELECT c13, c14, c15, c16, c17, COUNT(*) AS cnt FROM all_numbers_t0 GROUP BY c13, c14, c15, c16, c17 ORDER BY c13, c14, c15, c16, c17 LIMIT 100
) SELECT 'Test Case 97' AS test_name, MD5(GROUP_CONCAT(CAST(c13 AS STRING) || ':' || CAST(c14 AS STRING) || ':' || CAST(c15 AS STRING) || ':' || CAST(c16 AS STRING) || ':' || CAST(c17 AS STRING) || ':' || CAST(cnt AS STRING))) AS result_hash FROM result;