[Enhancement] support group by compressed key (#61632)
Signed-off-by: stdpain <drfeng08@gmail.com>
This commit is contained in:
parent
89f5c11799
commit
5f6cdde3a0
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@
|
|||
|
||||
#include "exec/aggregate/aggregate_base_node.h"
|
||||
|
||||
#include "gutil/strings/substitute.h"
|
||||
#include "exec/aggregator.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
|
|
|
|||
|
|
@ -14,9 +14,7 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include <any>
|
||||
|
||||
#include "exec/aggregator.h"
|
||||
#include "exec/aggregator_fwd.h"
|
||||
#include "exec/exec_node.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include "exec/aggregator.h"
|
||||
#include "exec/aggregator_fwd.h"
|
||||
#include "exec/pipeline/source_operator.h"
|
||||
|
||||
namespace starrocks::pipeline {
|
||||
|
|
|
|||
|
|
@ -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)));
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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(
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -14,7 +14,6 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#include <new>
|
||||
#include <type_traits>
|
||||
|
||||
#include "column/column.h"
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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"
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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) \
|
||||
|
|
|
|||
|
|
@ -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_;
|
||||
|
|
|
|||
|
|
@ -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) {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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
|
||||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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);
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
|
|
|
|||
|
|
@ -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(");
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -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;
|
||||
Loading…
Reference in New Issue