Compare commits
1 Commits
main
...
cursor/add
| Author | SHA1 | Date |
|---|---|---|
|
|
8e2dd745db |
|
|
@ -16,17 +16,8 @@
|
|||
|
||||
#include "cache/disk_cache/local_disk_cache_engine.h"
|
||||
#include "common/status.h"
|
||||
|
||||
#ifdef WITH_STARCACHE
|
||||
#include "starcache/star_cache.h"
|
||||
#include "starcache/time_based_cache_adaptor.h"
|
||||
#else
|
||||
namespace starcache {
|
||||
class StarCache;
|
||||
class TimeBasedCacheAdaptor;
|
||||
struct CacheMetrics;
|
||||
} // namespace starcache
|
||||
#endif
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
|
|
@ -54,9 +45,7 @@ public:
|
|||
|
||||
Status update_inline_cache_count_limit(int32_t limit) override;
|
||||
|
||||
#ifdef WITH_STARCACHE
|
||||
const StarCacheMetrics starcache_metrics(int level) const;
|
||||
#endif
|
||||
|
||||
const DataCacheDiskMetrics cache_metrics() const override;
|
||||
|
||||
|
|
|
|||
|
|
@ -107,15 +107,6 @@ public:
|
|||
// Get the cache hit count.
|
||||
virtual size_t hit_count() const = 0;
|
||||
|
||||
// Get the insert count.
|
||||
virtual size_t insert_count() const = 0;
|
||||
|
||||
// Get the insert evict count.
|
||||
virtual size_t insert_evict_count() const = 0;
|
||||
|
||||
// Get the release evict count.
|
||||
virtual size_t release_evict_count() const = 0;
|
||||
|
||||
// Remove all cache entries that are not actively in use.
|
||||
virtual Status prune() = 0;
|
||||
};
|
||||
|
|
|
|||
|
|
@ -111,18 +111,6 @@ size_t LRUCacheEngine::hit_count() const {
|
|||
return _cache->get_hit_count();
|
||||
}
|
||||
|
||||
size_t LRUCacheEngine::insert_count() const {
|
||||
return _cache->get_insert_count();
|
||||
}
|
||||
|
||||
size_t LRUCacheEngine::insert_evict_count() const {
|
||||
return _cache->get_insert_evict_count();
|
||||
}
|
||||
|
||||
size_t LRUCacheEngine::release_evict_count() const {
|
||||
return _cache->get_release_evict_count();
|
||||
}
|
||||
|
||||
bool LRUCacheEngine::_check_write(size_t charge, const MemCacheWriteOptions& options) const {
|
||||
if (options.evict_probability >= 100) {
|
||||
return true;
|
||||
|
|
|
|||
|
|
@ -57,12 +57,6 @@ public:
|
|||
|
||||
size_t hit_count() const override;
|
||||
|
||||
size_t insert_count() const override;
|
||||
|
||||
size_t insert_evict_count() const override;
|
||||
|
||||
size_t release_evict_count() const override;
|
||||
|
||||
Status prune() override;
|
||||
|
||||
private:
|
||||
|
|
|
|||
|
|
@ -49,9 +49,6 @@ std::atomic<size_t> StoragePageCacheMetrics::released_page_handle_count{};
|
|||
|
||||
METRIC_DEFINE_UINT_GAUGE(page_cache_lookup_count, MetricUnit::OPERATIONS);
|
||||
METRIC_DEFINE_UINT_GAUGE(page_cache_hit_count, MetricUnit::OPERATIONS);
|
||||
METRIC_DEFINE_UINT_GAUGE(page_cache_insert_count, MetricUnit::OPERATIONS);
|
||||
METRIC_DEFINE_UINT_GAUGE(page_cache_insert_evict_count, MetricUnit::OPERATIONS);
|
||||
METRIC_DEFINE_UINT_GAUGE(page_cache_release_evict_count, MetricUnit::OPERATIONS);
|
||||
METRIC_DEFINE_UINT_GAUGE(page_cache_capacity, MetricUnit::BYTES);
|
||||
METRIC_DEFINE_UINT_GAUGE(page_cache_pinned_count, MetricUnit::BYTES);
|
||||
|
||||
|
|
@ -64,22 +61,6 @@ void StoragePageCache::init_metrics() {
|
|||
StarRocksMetrics::instance()->metrics()->register_hook(
|
||||
"page_cache_hit_count", [this]() { page_cache_hit_count.set_value(get_hit_count()); });
|
||||
|
||||
StarRocksMetrics::instance()->metrics()->register_metric("page_cache_insert_count", &page_cache_insert_count);
|
||||
StarRocksMetrics::instance()->metrics()->register_hook(
|
||||
"page_cache_insert_count", [this]() { page_cache_insert_count.set_value(get_insert_count()); });
|
||||
|
||||
StarRocksMetrics::instance()->metrics()->register_metric("page_cache_insert_evict_count",
|
||||
&page_cache_insert_evict_count);
|
||||
StarRocksMetrics::instance()->metrics()->register_hook("page_cache_insert_evict_count", [this]() {
|
||||
page_cache_insert_evict_count.set_value(get_insert_evict_count());
|
||||
});
|
||||
|
||||
StarRocksMetrics::instance()->metrics()->register_metric("page_cache_release_evict_count",
|
||||
&page_cache_release_evict_count);
|
||||
StarRocksMetrics::instance()->metrics()->register_hook("page_cache_release_evict_count", [this]() {
|
||||
page_cache_release_evict_count.set_value(get_release_evict_count());
|
||||
});
|
||||
|
||||
StarRocksMetrics::instance()->metrics()->register_metric("page_cache_capacity", &page_cache_capacity);
|
||||
StarRocksMetrics::instance()->metrics()->register_hook("page_cache_capacity",
|
||||
[this]() { page_cache_capacity.set_value(get_capacity()); });
|
||||
|
|
@ -110,18 +91,6 @@ uint64_t StoragePageCache::get_hit_count() const {
|
|||
return _cache->hit_count();
|
||||
}
|
||||
|
||||
uint64_t StoragePageCache::get_insert_count() const {
|
||||
return _cache->insert_count();
|
||||
}
|
||||
|
||||
uint64_t StoragePageCache::get_insert_evict_count() const {
|
||||
return _cache->insert_evict_count();
|
||||
}
|
||||
|
||||
uint64_t StoragePageCache::get_release_evict_count() const {
|
||||
return _cache->release_evict_count();
|
||||
}
|
||||
|
||||
bool StoragePageCache::adjust_capacity(int64_t delta, size_t min_capacity) {
|
||||
Status st = _cache->adjust_mem_quota(delta, min_capacity);
|
||||
if (!st.ok()) {
|
||||
|
|
|
|||
|
|
@ -103,12 +103,6 @@ public:
|
|||
|
||||
uint64_t get_hit_count() const;
|
||||
|
||||
uint64_t get_insert_count() const;
|
||||
|
||||
uint64_t get_insert_evict_count() const;
|
||||
|
||||
uint64_t get_release_evict_count() const;
|
||||
|
||||
bool adjust_capacity(int64_t delta, size_t min_capacity = 0);
|
||||
|
||||
void prune();
|
||||
|
|
|
|||
|
|
@ -412,7 +412,7 @@ CONF_Bool(enable_event_based_compaction_framework, "true");
|
|||
CONF_Bool(enable_size_tiered_compaction_strategy, "true");
|
||||
CONF_mBool(enable_pk_size_tiered_compaction_strategy, "true");
|
||||
// Enable parallel execution within tablet for primary key tables.
|
||||
CONF_mBool(enable_pk_parallel_execution, "true");
|
||||
CONF_mBool(enable_pk_parallel_execution, "false");
|
||||
// The minimum threshold of data size for enabling pk parallel execution.
|
||||
// Default is 300MB.
|
||||
CONF_mInt64(pk_parallel_execution_threshold_bytes, "314572800");
|
||||
|
|
|
|||
|
|
@ -340,7 +340,7 @@ void TEST_clear_configs();
|
|||
|
||||
template <>
|
||||
struct fmt::formatter<starrocks::config::MutableString> : formatter<std::string> {
|
||||
auto format(const starrocks::config::MutableString& s, format_context& ctx) const {
|
||||
auto format(const starrocks::config::MutableString& s, format_context& ctx) {
|
||||
return formatter<std::string>::format(s.value(), ctx);
|
||||
}
|
||||
};
|
||||
|
|
|
|||
|
|
@ -44,13 +44,6 @@
|
|||
#ifdef USE_STAROS
|
||||
#include "fslib/star_cache_handler.h"
|
||||
#endif
|
||||
#include <fmt/ranges.h>
|
||||
|
||||
#include <csignal>
|
||||
// Need POSIX signal APIs like sigaction/siginfo_t.
|
||||
// NOLINTNEXTLINE(modernize-deprecated-headers)
|
||||
#include <signal.h>
|
||||
|
||||
#include "fs/encrypt_file.h"
|
||||
#include "gutil/cpu.h"
|
||||
#include "jemalloc/jemalloc.h"
|
||||
|
|
@ -171,12 +164,6 @@ struct JemallocStats {
|
|||
};
|
||||
|
||||
static void retrieve_jemalloc_stats(JemallocStats* stats) {
|
||||
// On macOS, jemalloc may define je_mallctl as mallctl via macro in jemalloc.h
|
||||
#ifdef __APPLE__
|
||||
#ifndef je_mallctl
|
||||
#define je_mallctl mallctl
|
||||
#endif
|
||||
#endif
|
||||
uint64_t epoch = 1;
|
||||
size_t sz = sizeof(epoch);
|
||||
je_mallctl("epoch", &epoch, &sz, &epoch, sz);
|
||||
|
|
|
|||
|
|
@ -18,12 +18,6 @@
|
|||
#include <glog/logging.h>
|
||||
#include <glog/vlog_is_on.h>
|
||||
#include <jemalloc/jemalloc.h>
|
||||
#ifdef __APPLE__
|
||||
#include <mach/mach_init.h>
|
||||
#include <mach/mach_port.h>
|
||||
#include <mach/thread_act.h>
|
||||
#include <pthread.h>
|
||||
#endif
|
||||
|
||||
#include <cerrno>
|
||||
#include <cstdio>
|
||||
|
|
@ -134,12 +128,7 @@ static void dontdump_unused_pages() {
|
|||
static bool start_dump = false;
|
||||
struct timeval tv;
|
||||
gettimeofday(&tv, nullptr);
|
||||
// On macOS, pthread_t is an opaque pointer; convert to a numeric id for fmt
|
||||
#ifdef __APPLE__
|
||||
uint64_t tid = static_cast<uint64_t>(pthread_mach_thread_np(pthread_self()));
|
||||
#else
|
||||
pthread_t tid = pthread_self();
|
||||
#endif
|
||||
const uint32_t MAX_BUFFER_SIZE = 1024;
|
||||
char buffer[MAX_BUFFER_SIZE] = {};
|
||||
// memory_buffer allocate 500 bytes from stack
|
||||
|
|
@ -147,13 +136,7 @@ static void dontdump_unused_pages() {
|
|||
if (!start_dump) {
|
||||
int res = snprintf(buffer, MAX_BUFFER_SIZE, "arena.%d.purge", MALLCTL_ARENAS_ALL);
|
||||
buffer[res] = '\0';
|
||||
int ret =
|
||||
#ifdef __APPLE__
|
||||
mallctl
|
||||
#else
|
||||
je_mallctl
|
||||
#endif
|
||||
(buffer, nullptr, nullptr, nullptr, 0);
|
||||
int ret = je_mallctl(buffer, nullptr, nullptr, nullptr, 0);
|
||||
|
||||
if (ret != 0) {
|
||||
FMT_LOG("je_mallctl execute purge failed, errno:{}", ret);
|
||||
|
|
@ -163,13 +146,7 @@ static void dontdump_unused_pages() {
|
|||
|
||||
res = snprintf(buffer, MAX_BUFFER_SIZE, "arena.%d.dontdump", MALLCTL_ARENAS_ALL);
|
||||
buffer[res] = '\0';
|
||||
ret =
|
||||
#ifdef __APPLE__
|
||||
mallctl
|
||||
#else
|
||||
je_mallctl
|
||||
#endif
|
||||
(buffer, nullptr, nullptr, nullptr, 0);
|
||||
ret = je_mallctl(buffer, nullptr, nullptr, nullptr, 0);
|
||||
|
||||
if (ret != 0) {
|
||||
FMT_LOG("je_mallctl execute dontdump failed, errno:{}", ret);
|
||||
|
|
@ -222,10 +199,8 @@ bool init_glog(const char* basename, bool install_signal_handler) {
|
|||
FLAGS_logbuflevel = 0;
|
||||
// Buffer log messages for at most this many seconds.
|
||||
FLAGS_logbufsecs = 30;
|
||||
// Set roll num. Not available with Homebrew glog on macOS.
|
||||
#ifndef __APPLE__
|
||||
// Set roll num.
|
||||
FLAGS_log_filenum_quota = config::sys_log_roll_num;
|
||||
#endif
|
||||
|
||||
// Set log level.
|
||||
std::string loglevel = config::sys_log_level;
|
||||
|
|
@ -255,19 +230,13 @@ bool init_glog(const char* basename, bool install_signal_handler) {
|
|||
std::string sizeflag = "SIZE-MB-";
|
||||
bool ok = false;
|
||||
if (rollmode.compare("TIME-DAY") == 0) {
|
||||
#ifndef __APPLE__
|
||||
FLAGS_log_split_method = "day";
|
||||
#endif
|
||||
ok = true;
|
||||
} else if (rollmode.compare("TIME-HOUR") == 0) {
|
||||
#ifndef __APPLE__
|
||||
FLAGS_log_split_method = "hour";
|
||||
#endif
|
||||
ok = true;
|
||||
} else if (rollmode.substr(0, sizeflag.length()).compare(sizeflag) == 0) {
|
||||
#ifndef __APPLE__
|
||||
FLAGS_log_split_method = "size";
|
||||
#endif
|
||||
std::string sizestr = rollmode.substr(sizeflag.size(), rollmode.size() - sizeflag.size());
|
||||
if (sizestr.size() != 0) {
|
||||
char* end = nullptr;
|
||||
|
|
@ -309,10 +278,7 @@ bool init_glog(const char* basename, bool install_signal_handler) {
|
|||
if (config::dump_trace_info) {
|
||||
google::InstallFailureWriter(failure_writer);
|
||||
google::InstallFailureFunction((google::logging_fail_func_t)failure_function);
|
||||
#ifndef MACOS_DISABLE_GLOG_STACKTRACE
|
||||
// This symbol may be unavailable on macOS builds using system glog.
|
||||
google::InstallFailureHandlerAfterOutputLog(failure_handler_after_output_log);
|
||||
#endif
|
||||
}
|
||||
|
||||
logging_initialized = true;
|
||||
|
|
|
|||
|
|
@ -27,45 +27,21 @@
|
|||
namespace starrocks {
|
||||
// detail implements for allocator
|
||||
static int set_jemalloc_profiling(bool enable) {
|
||||
int ret =
|
||||
#ifdef __APPLE__
|
||||
mallctl
|
||||
#else
|
||||
je_mallctl
|
||||
#endif
|
||||
("prof.active", nullptr, nullptr, &enable, 1);
|
||||
ret |=
|
||||
#ifdef __APPLE__
|
||||
mallctl
|
||||
#else
|
||||
je_mallctl
|
||||
#endif
|
||||
("prof.thread_active_init", nullptr, nullptr, &enable, 1);
|
||||
int ret = je_mallctl("prof.active", nullptr, nullptr, &enable, 1);
|
||||
ret |= je_mallctl("prof.thread_active_init", nullptr, nullptr, &enable, 1);
|
||||
return ret;
|
||||
}
|
||||
|
||||
static int has_enable_heap_profile() {
|
||||
int value = 0;
|
||||
size_t size = sizeof(value);
|
||||
|
||||
#ifdef __APPLE__
|
||||
mallctl
|
||||
#else
|
||||
je_mallctl
|
||||
#endif
|
||||
("prof.active", &value, &size, nullptr, 0);
|
||||
je_mallctl("prof.active", &value, &size, nullptr, 0);
|
||||
return value;
|
||||
}
|
||||
|
||||
bool dump_snapshot(const std::string& filename) {
|
||||
const char* fname = filename.c_str();
|
||||
return (
|
||||
#ifdef __APPLE__
|
||||
mallctl
|
||||
#else
|
||||
je_mallctl
|
||||
#endif
|
||||
("prof.dump", nullptr, nullptr, &fname, sizeof(const char*))) == 0;
|
||||
return je_mallctl("prof.dump", nullptr, nullptr, &fname, sizeof(const char*)) == 0;
|
||||
}
|
||||
|
||||
// declare exec from script
|
||||
|
|
@ -109,4 +85,4 @@ std::string HeapProf::to_dot_format(const std::string& heapdump_filename) {
|
|||
return exec(fmt::format("{} --dot {} {}", jeprof, binary, heapdump_filename));
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
} // namespace starrocks
|
||||
|
|
@ -556,13 +556,7 @@ Status HashJoiner::_create_runtime_in_filters(RuntimeState* state) {
|
|||
SCOPED_TIMER(build_metrics().build_runtime_filter_timer);
|
||||
size_t ht_row_count = get_ht_row_count();
|
||||
|
||||
// Use FE session variable if set, otherwise fall back to BE config
|
||||
size_t max_conditions = config::max_pushdown_conditions_per_column;
|
||||
if (state->query_options().__isset.max_pushdown_conditions_per_column) {
|
||||
max_conditions = state->query_options().max_pushdown_conditions_per_column;
|
||||
}
|
||||
|
||||
if (ht_row_count > max_conditions) {
|
||||
if (ht_row_count > config::max_pushdown_conditions_per_column) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,110 @@
|
|||
// 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 "exec/pipeline/scan/olap_meta_scan_operator.h"
|
||||
|
||||
#include <utility>
|
||||
|
||||
#include "exec/olap_meta_scanner.h"
|
||||
#include "exec/pipeline/scan/olap_meta_chunk_source.h"
|
||||
#include "exec/pipeline/scan/olap_meta_scan_context.h"
|
||||
|
||||
namespace starrocks::pipeline {
|
||||
|
||||
OlapMetaScanOperatorFactory::OlapMetaScanOperatorFactory(int32_t id, ScanNode* meta_scan_node, size_t dop,
|
||||
std::shared_ptr<OlapMetaScanContextFactory> ctx_factory)
|
||||
: ScanOperatorFactory(id, meta_scan_node), _ctx_factory(std::move(ctx_factory)) {}
|
||||
|
||||
Status OlapMetaScanOperatorFactory::do_prepare(RuntimeState* state) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void OlapMetaScanOperatorFactory::do_close(RuntimeState* state) {}
|
||||
|
||||
OperatorPtr OlapMetaScanOperatorFactory::do_create(int32_t dop, int32_t driver_sequence) {
|
||||
return std::make_shared<OlapMetaScanOperator>(this, _id, driver_sequence, dop, _scan_node,
|
||||
_ctx_factory->get_or_create(driver_sequence));
|
||||
}
|
||||
|
||||
OlapMetaScanOperator::OlapMetaScanOperator(OperatorFactory* factory, int32_t id, int32_t driver_sequence, int32_t dop,
|
||||
ScanNode* meta_scan_node, OlapMetaScanContextPtr ctx)
|
||||
: ScanOperator(factory, id, driver_sequence, dop, meta_scan_node), _ctx(std::move(ctx)) {}
|
||||
|
||||
OlapMetaScanOperator::~OlapMetaScanOperator() = default;
|
||||
|
||||
bool OlapMetaScanOperator::has_output() const {
|
||||
if (!_ctx->is_prepare_finished()) {
|
||||
return false;
|
||||
}
|
||||
return ScanOperator::has_output();
|
||||
}
|
||||
|
||||
bool OlapMetaScanOperator::is_finished() const {
|
||||
if (!_ctx->is_prepare_finished()) {
|
||||
return false;
|
||||
}
|
||||
return ScanOperator::is_finished();
|
||||
}
|
||||
|
||||
Status OlapMetaScanOperator::do_prepare(RuntimeState* state) {
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
void OlapMetaScanOperator::do_close(RuntimeState* state) {}
|
||||
|
||||
ChunkSourcePtr OlapMetaScanOperator::create_chunk_source(MorselPtr morsel, int32_t chunk_source_index) {
|
||||
return std::make_shared<OlapMetaChunkSource>(this, _runtime_profile.get(), std::move(morsel), _ctx);
|
||||
}
|
||||
|
||||
ChunkPtr OlapMetaScanOperator::get_chunk_from_buffer() {
|
||||
ChunkPtr chunk = nullptr;
|
||||
if (_ctx->get_chunk_buffer().try_get(_driver_sequence, &chunk)) {
|
||||
return chunk;
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
size_t OlapMetaScanOperator::num_buffered_chunks() const {
|
||||
return _ctx->get_chunk_buffer().size(_driver_sequence);
|
||||
}
|
||||
|
||||
size_t OlapMetaScanOperator::buffer_size() const {
|
||||
return _ctx->get_chunk_buffer().limiter()->size();
|
||||
}
|
||||
|
||||
size_t OlapMetaScanOperator::buffer_capacity() const {
|
||||
return _ctx->get_chunk_buffer().limiter()->capacity();
|
||||
}
|
||||
|
||||
size_t OlapMetaScanOperator::buffer_memory_usage() const {
|
||||
return _ctx->get_chunk_buffer().memory_usage();
|
||||
}
|
||||
|
||||
size_t OlapMetaScanOperator::default_buffer_capacity() const {
|
||||
return _ctx->get_chunk_buffer().limiter()->default_capacity();
|
||||
}
|
||||
|
||||
ChunkBufferTokenPtr OlapMetaScanOperator::pin_chunk(int num_chunks) {
|
||||
return _ctx->get_chunk_buffer().limiter()->pin(num_chunks);
|
||||
}
|
||||
|
||||
bool OlapMetaScanOperator::is_buffer_full() const {
|
||||
return _ctx->get_chunk_buffer().limiter()->is_full();
|
||||
}
|
||||
|
||||
void OlapMetaScanOperator::set_buffer_finished() {
|
||||
_ctx->get_chunk_buffer().set_finished(_driver_sequence);
|
||||
}
|
||||
|
||||
} // namespace starrocks::pipeline
|
||||
|
|
@ -0,0 +1,73 @@
|
|||
// 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 "exec/olap_meta_scan_node.h"
|
||||
#include "exec/pipeline/pipeline_builder.h"
|
||||
#include "exec/pipeline/scan/balanced_chunk_buffer.h"
|
||||
#include "exec/pipeline/scan/olap_meta_scan_context.h"
|
||||
#include "exec/pipeline/scan/scan_operator.h"
|
||||
#include "gen_cpp/Types_types.h"
|
||||
|
||||
namespace starrocks::pipeline {
|
||||
|
||||
class OlapMetaScanOperatorFactory final : public ScanOperatorFactory {
|
||||
public:
|
||||
OlapMetaScanOperatorFactory(int32_t id, ScanNode* meta_scan_node, size_t dop,
|
||||
std::shared_ptr<OlapMetaScanContextFactory> ctx_factory);
|
||||
|
||||
~OlapMetaScanOperatorFactory() override = default;
|
||||
|
||||
bool with_morsels() const override { return true; }
|
||||
|
||||
Status do_prepare(RuntimeState* state) override;
|
||||
void do_close(RuntimeState* state) override;
|
||||
OperatorPtr do_create(int32_t dop, int32_t driver_sequence) override;
|
||||
|
||||
private:
|
||||
std::shared_ptr<OlapMetaScanContextFactory> _ctx_factory;
|
||||
};
|
||||
|
||||
class OlapMetaScanOperator final : public ScanOperator {
|
||||
public:
|
||||
OlapMetaScanOperator(OperatorFactory* factory, int32_t id, int32_t driver_sequence, int32_t dop,
|
||||
ScanNode* scan_node, OlapMetaScanContextPtr scan_ctx);
|
||||
|
||||
~OlapMetaScanOperator() override;
|
||||
|
||||
bool has_output() const override;
|
||||
bool is_finished() const override;
|
||||
|
||||
Status do_prepare(RuntimeState* state) override;
|
||||
void do_close(RuntimeState* state) override;
|
||||
ChunkSourcePtr create_chunk_source(MorselPtr morsel, int32_t chunk_source_index) override;
|
||||
|
||||
private:
|
||||
void attach_chunk_source(int32_t source_index) override {}
|
||||
void detach_chunk_source(int32_t source_index) override {}
|
||||
bool has_shared_chunk_source() const override { return false; }
|
||||
ChunkPtr get_chunk_from_buffer() override;
|
||||
size_t num_buffered_chunks() const override;
|
||||
size_t buffer_size() const override;
|
||||
size_t buffer_capacity() const override;
|
||||
size_t buffer_memory_usage() const override;
|
||||
size_t default_buffer_capacity() const override;
|
||||
ChunkBufferTokenPtr pin_chunk(int num_chunks) override;
|
||||
bool is_buffer_full() const override;
|
||||
void set_buffer_finished() override;
|
||||
|
||||
OlapMetaScanContextPtr _ctx;
|
||||
};
|
||||
} // namespace starrocks::pipeline
|
||||
|
|
@ -18,14 +18,6 @@
|
|||
#include <memory>
|
||||
#include <utility>
|
||||
|
||||
// On macOS, system headers may define a macro named current_task(),
|
||||
// which conflicts with the method name below. Undefine to avoid collisions.
|
||||
#ifdef __APPLE__
|
||||
#ifdef current_task
|
||||
#undef current_task
|
||||
#endif
|
||||
#endif
|
||||
|
||||
#include "column/vectorized_fwd.h"
|
||||
#include "common/statusor.h"
|
||||
#include "exec/spill/spiller.h"
|
||||
|
|
|
|||
|
|
@ -94,19 +94,17 @@ Status ScanNode::prepare(RuntimeState* state) {
|
|||
}
|
||||
|
||||
// Distribute morsels from a single queue to multiple queues
|
||||
static StatusOr<std::map<int, pipeline::MorselQueuePtr>> uniform_distribute_morsels(
|
||||
pipeline::MorselQueuePtr morsel_queue, int dop) {
|
||||
std::map<int, pipeline::MorselQueuePtr> queue_per_driver;
|
||||
static std::map<int, pipeline::MorselQueuePtr> uniform_distribute_morsels(pipeline::MorselQueuePtr morsel_queue,
|
||||
int dop) {
|
||||
std::map<int, pipeline::Morsels> morsels_per_driver;
|
||||
int driver_seq = 0;
|
||||
while (!morsel_queue->empty()) {
|
||||
auto maybe_morsel_status_or = morsel_queue->try_get();
|
||||
if (UNLIKELY(!maybe_morsel_status_or.ok())) {
|
||||
return maybe_morsel_status_or.status();
|
||||
}
|
||||
morsels_per_driver[driver_seq].push_back(std::move(maybe_morsel_status_or.value()));
|
||||
auto maybe_morsel = morsel_queue->try_get();
|
||||
DCHECK(maybe_morsel.ok());
|
||||
morsels_per_driver[driver_seq].push_back(std::move(maybe_morsel.value()));
|
||||
driver_seq = (driver_seq + 1) % dop;
|
||||
}
|
||||
std::map<int, pipeline::MorselQueuePtr> queue_per_driver;
|
||||
|
||||
auto morsel_queue_type = morsel_queue->type();
|
||||
DCHECK(morsel_queue_type == pipeline::MorselQueue::Type::FIXED ||
|
||||
|
|
@ -146,7 +144,7 @@ StatusOr<pipeline::MorselQueueFactoryPtr> ScanNode::convert_scan_range_to_morsel
|
|||
// If not so much morsels, try to assign morsel uniformly among operators to avoid data skew
|
||||
if (!always_shared_scan() && scan_dop > 1 && is_fixed_or_dynamic_morsel_queue &&
|
||||
morsel_queue->num_original_morsels() <= io_parallelism) {
|
||||
ASSIGN_OR_RETURN(auto morsel_queue_map, uniform_distribute_morsels(std::move(morsel_queue), scan_dop));
|
||||
auto morsel_queue_map = uniform_distribute_morsels(std::move(morsel_queue), scan_dop);
|
||||
return std::make_unique<pipeline::IndividualMorselQueueFactory>(std::move(morsel_queue_map),
|
||||
/*could_local_shuffle*/ true);
|
||||
} else {
|
||||
|
|
|
|||
|
|
@ -14,11 +14,7 @@
|
|||
|
||||
#pragma once
|
||||
|
||||
#ifdef __APPLE__
|
||||
#include <sys/mount.h>
|
||||
#else
|
||||
#include <sys/statfs.h>
|
||||
#endif
|
||||
|
||||
#include <atomic>
|
||||
#include <memory>
|
||||
|
|
@ -105,11 +101,7 @@ private:
|
|||
struct statfs stat1, stat2;
|
||||
statfs(path1.c_str(), &stat1);
|
||||
statfs(path2.c_str(), &stat2);
|
||||
#ifdef __APPLE__
|
||||
return stat1.f_fsid.val[0] == stat2.f_fsid.val[0] && stat1.f_fsid.val[1] == stat2.f_fsid.val[1];
|
||||
#else
|
||||
return stat1.f_fsid.__val[0] == stat2.f_fsid.__val[0] && stat1.f_fsid.__val[1] == stat2.f_fsid.__val[1];
|
||||
#endif
|
||||
}
|
||||
|
||||
std::vector<DirPtr> _dirs;
|
||||
|
|
@ -125,4 +117,4 @@ private:
|
|||
Status::RuntimeError(fmt::format("acquire size error: dir {} try acquire:{} usage:{} capacity:{}", dir->dir(), \
|
||||
acquire_size, dir->get_current_size(), dir->get_max_size()))
|
||||
|
||||
} // namespace starrocks::spill
|
||||
} // namespace starrocks::spill
|
||||
|
|
@ -325,6 +325,8 @@ void PartitionedSpillerWriter::_add_partition(SpilledPartitionPtr&& partition_pt
|
|||
}
|
||||
|
||||
void PartitionedSpillerWriter::_remove_partition(const SpilledPartition* partition) {
|
||||
auto affinity_group = partition->block_group->get_affinity_group();
|
||||
DCHECK(affinity_group != kDefaultBlockAffinityGroup);
|
||||
_id_to_partitions.erase(partition->partition_id);
|
||||
size_t level = partition->level;
|
||||
auto& partitions = _level_to_partitions[level];
|
||||
|
|
@ -332,12 +334,6 @@ void PartitionedSpillerWriter::_remove_partition(const SpilledPartition* partiti
|
|||
auto iter = std::find_if(partitions.begin(), partitions.end(),
|
||||
[partition](auto& val) { return val->partition_id == partition->partition_id; });
|
||||
_total_partition_num -= (iter != partitions.end());
|
||||
if (partition->block_group != nullptr) {
|
||||
auto affinity_group = partition->block_group->get_affinity_group();
|
||||
DCHECK(affinity_group != kDefaultBlockAffinityGroup);
|
||||
WARN_IF_ERROR(_spiller->block_manager()->release_affinity_group(affinity_group),
|
||||
fmt::format("release affinity group {} error", affinity_group));
|
||||
}
|
||||
partitions.erase(iter);
|
||||
if (partitions.empty()) {
|
||||
_level_to_partitions.erase(level);
|
||||
|
|
@ -345,6 +341,8 @@ void PartitionedSpillerWriter::_remove_partition(const SpilledPartition* partiti
|
|||
_min_level = level + 1;
|
||||
}
|
||||
}
|
||||
WARN_IF_ERROR(_spiller->block_manager()->release_affinity_group(affinity_group),
|
||||
fmt::format("release affinity group {} error", affinity_group));
|
||||
}
|
||||
|
||||
Status PartitionedSpillerWriter::_choose_partitions_to_flush(bool is_final_flush,
|
||||
|
|
|
|||
|
|
@ -46,9 +46,9 @@
|
|||
#include "column/column_helper.h"
|
||||
#include "column/map_column.h"
|
||||
#include "column/nullable_column.h"
|
||||
#include "common/config.h"
|
||||
#include "common/statusor.h"
|
||||
#include "common/tracer.h"
|
||||
#include "config.h"
|
||||
#include "exec/pipeline/query_context.h"
|
||||
#include "exec/tablet_sink_colocate_sender.h"
|
||||
#include "exprs/expr.h"
|
||||
|
|
|
|||
|
|
@ -19,10 +19,10 @@
|
|||
#include "column/chunk.h"
|
||||
#include "column/column_viewer.h"
|
||||
#include "column/nullable_column.h"
|
||||
#include "common/config.h"
|
||||
#include "common/statusor.h"
|
||||
#include "common/tracer.h"
|
||||
#include "common/utils.h"
|
||||
#include "config.h"
|
||||
#include "exec/tablet_sink.h"
|
||||
#include "exprs/expr_context.h"
|
||||
#include "gutil/strings/fastmem.h"
|
||||
|
|
|
|||
|
|
@ -16,12 +16,12 @@
|
|||
|
||||
#include <type_traits>
|
||||
|
||||
#include "column/array_column.h"
|
||||
#include "column/type_traits.h"
|
||||
#include "gutil/strings/fastmem.h"
|
||||
#include "types/logical_type.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
// Type traits from aggregate functions
|
||||
template <LogicalType lt, typename = guard::Guard>
|
||||
struct AggDataTypeTraits {};
|
||||
|
|
@ -41,7 +41,6 @@ struct AggDataTypeTraits<lt, FixedLengthLTGuard<lt>> {
|
|||
static RefType get_ref(const ValueType& value) { return value; }
|
||||
|
||||
static void update_max(ValueType& current, const RefType& input) { current = std::max<ValueType>(current, input); }
|
||||
|
||||
static void update_min(ValueType& current, const RefType& input) { current = std::min<ValueType>(current, input); }
|
||||
|
||||
static bool is_equal(const RefType& lhs, const RefType& rhs) { return lhs == rhs; }
|
||||
|
|
@ -57,9 +56,7 @@ struct AggDataTypeTraits<lt, ObjectFamilyLTGuard<lt>> {
|
|||
using RefType = RunTimeCppType<lt>;
|
||||
|
||||
static void assign_value(ValueType& value, RefType ref) { value = *ref; }
|
||||
|
||||
static void assign_value(ColumnType* column, size_t row, const RefType& ref) { *column->get_object(row) = *ref; }
|
||||
|
||||
static void assign_value(ColumnType* column, size_t row, const ValueType& ref) { *column->get_object(row) = ref; }
|
||||
|
||||
static void append_value(ColumnType* column, const ValueType& value) { column->append(&value); }
|
||||
|
|
@ -68,47 +65,12 @@ struct AggDataTypeTraits<lt, ObjectFamilyLTGuard<lt>> {
|
|||
static const RefType get_row_ref(const ColumnType& column, size_t row) { return column.get_object(row); }
|
||||
|
||||
static void update_max(ValueType& current, const RefType& input) { current = std::max<ValueType>(current, *input); }
|
||||
|
||||
static void update_min(ValueType& current, const RefType& input) { current = std::min<ValueType>(current, *input); }
|
||||
|
||||
static bool is_equal(const RefType& lhs, const RefType& rhs) { return *lhs == *rhs; }
|
||||
static bool equals(const ValueType& lhs, const RefType& rhs) { return lhs == *rhs; }
|
||||
};
|
||||
|
||||
// For pointer ref types
|
||||
template <LogicalType lt>
|
||||
struct AggDataTypeTraits<lt, ArrayGuard<lt>> {
|
||||
using CppType = RunTimeCppType<lt>;
|
||||
using ColumnType = RunTimeColumnType<lt>;
|
||||
using ValueType = typename ColumnType::MutablePtr;
|
||||
|
||||
struct RefType {
|
||||
const ColumnType* column;
|
||||
const size_t row;
|
||||
|
||||
RefType(const ColumnType* c, size_t r) : column(c), row(r) {}
|
||||
};
|
||||
|
||||
static void assign_value(ValueType& value, const RefType& ref) {
|
||||
value = ArrayColumn::static_pointer_cast(ref.column->clone_empty());
|
||||
value->append_datum(ref.column->get(ref.row).template get<CppType>());
|
||||
}
|
||||
|
||||
static void append_value(ColumnType* column, const ValueType& value) {
|
||||
column->append_datum(value->get(0).template get<CppType>());
|
||||
}
|
||||
|
||||
static RefType get_row_ref(const ColumnType& column, size_t row) { return RefType(&column, row); }
|
||||
|
||||
static bool is_equal(const ValueType& lhs, const ValueType& rhs) {
|
||||
return lhs->get(0).template get<CppType>() == rhs->get(0).template get<CppType>();
|
||||
}
|
||||
|
||||
static bool equals(const ValueType& lhs, const ValueType& rhs) {
|
||||
return lhs->get(0).template get<CppType>() == rhs->get(0).template get<CppType>();
|
||||
}
|
||||
};
|
||||
|
||||
template <LogicalType lt>
|
||||
struct AggDataTypeTraits<lt, StringLTGuard<lt>> {
|
||||
using ColumnType = RunTimeColumnType<lt>;
|
||||
|
|
@ -134,7 +96,6 @@ struct AggDataTypeTraits<lt, StringLTGuard<lt>> {
|
|||
memcpy(current.data(), input.data, input.size);
|
||||
}
|
||||
}
|
||||
|
||||
static void update_min(ValueType& current, const RefType& input) {
|
||||
if (Slice(current.data(), current.size()).compare(input) > 0) {
|
||||
current.resize(input.size);
|
||||
|
|
@ -149,4 +110,5 @@ template <LogicalType lt>
|
|||
using AggDataValueType = typename AggDataTypeTraits<lt>::ValueType;
|
||||
template <LogicalType lt>
|
||||
using AggDataRefType = typename AggDataTypeTraits<lt>::RefType;
|
||||
} // namespace starrocks
|
||||
|
||||
} // namespace starrocks
|
||||
|
|
@ -75,7 +75,7 @@ public:
|
|||
TYPE_BIGINT, TYPE_LARGEINT, TYPE_FLOAT, TYPE_DOUBLE,
|
||||
TYPE_VARCHAR, TYPE_CHAR, TYPE_DATE, TYPE_DATETIME,
|
||||
TYPE_DECIMALV2, TYPE_DECIMAL32, TYPE_DECIMAL64, TYPE_DECIMAL128,
|
||||
TYPE_DECIMAL256, TYPE_HLL, TYPE_OBJECT, TYPE_ARRAY};
|
||||
TYPE_DECIMAL256, TYPE_HLL, TYPE_OBJECT};
|
||||
return kTypes;
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ namespace starrocks {
|
|||
struct WindowDispatcher {
|
||||
template <LogicalType lt>
|
||||
void operator()(AggregateFuncResolver* resolver) {
|
||||
if constexpr (lt_is_aggregate<lt> || is_object_type(lt) || lt_is_array<lt>) {
|
||||
if constexpr (lt_is_aggregate<lt> || is_object_type(lt)) {
|
||||
resolver->add_aggregate_mapping_notnull<lt, lt>(
|
||||
"first_value", true, AggregateFactory::MakeFirstValueWindowFunction<lt, false>());
|
||||
// use first_value_in for first_value with ingnore nulls.
|
||||
|
|
|
|||
|
|
@ -13,7 +13,6 @@
|
|||
// limitations under the License.
|
||||
|
||||
#pragma once
|
||||
#include "column/array_column.h"
|
||||
#include "column/column_helper.h"
|
||||
#include "column/nullable_column.h"
|
||||
#include "column/vectorized_fwd.h"
|
||||
|
|
@ -102,10 +101,6 @@ struct ValueWindowStrategy<LT, JsonGuard<LT>> {
|
|||
/// The dst Object column hasn't been resized.
|
||||
static constexpr bool use_append = true;
|
||||
};
|
||||
template <LogicalType LT>
|
||||
struct ValueWindowStrategy<LT, ArrayGuard<LT>> {
|
||||
static constexpr bool use_append = true;
|
||||
};
|
||||
|
||||
template <LogicalType LT, typename State, typename T = RunTimeCppType<LT>>
|
||||
class ValueWindowFunction : public WindowFunction<State> {
|
||||
|
|
@ -129,7 +124,7 @@ public:
|
|||
|
||||
Column* data_column = nullable_column->mutable_data_column();
|
||||
auto* column = down_cast<InputColumnType*>(data_column);
|
||||
auto& value = AggregateFunctionStateHelper<State>::data(state).value;
|
||||
auto value = AggregateFunctionStateHelper<State>::data(state).value;
|
||||
for (size_t i = start; i < end; ++i) {
|
||||
AggDataTypeTraits<LT>::append_value(column, value);
|
||||
}
|
||||
|
|
@ -567,14 +562,8 @@ class LeadLagWindowFunction final : public ValueWindowFunction<LT, LeadLagState<
|
|||
if (default_column->is_nullable()) {
|
||||
this->data(state).default_is_null = true;
|
||||
} else {
|
||||
if constexpr (lt_is_array<LT>) {
|
||||
const auto* column = down_cast<const ArrayColumn*>(ColumnHelper::get_data_column(arg2));
|
||||
AggDataTypeTraits<LT>::assign_value(this->data(state).default_value,
|
||||
AggDataTypeTraits<LT>::get_row_ref(*column, 0));
|
||||
} else {
|
||||
auto value = ColumnHelper::get_const_value<LT>(arg2);
|
||||
AggDataTypeTraits<LT>::assign_value(this->data(state).default_value, value);
|
||||
}
|
||||
auto value = ColumnHelper::get_const_value<LT>(arg2);
|
||||
AggDataTypeTraits<LT>::assign_value(this->data(state).default_value, value);
|
||||
}
|
||||
|
||||
if constexpr (ignoreNulls) {
|
||||
|
|
@ -680,13 +669,7 @@ class LeadLagWindowFunction final : public ValueWindowFunction<LT, LeadLagState<
|
|||
if (this->data(state).default_is_null) {
|
||||
this->data(state).is_null = true;
|
||||
} else {
|
||||
if constexpr (lt_is_array<LT>) {
|
||||
AggDataTypeTraits<LT>::assign_value(
|
||||
this->data(state).value,
|
||||
AggDataTypeTraits<LT>::get_row_ref(*this->data(state).default_value, 0));
|
||||
} else {
|
||||
this->data(state).value = this->data(state).default_value;
|
||||
}
|
||||
this->data(state).value = this->data(state).default_value;
|
||||
}
|
||||
} else {
|
||||
const Column* data_column = ColumnHelper::get_data_column(columns[0]);
|
||||
|
|
@ -703,13 +686,7 @@ class LeadLagWindowFunction final : public ValueWindowFunction<LT, LeadLagState<
|
|||
this->data(state).is_null = true;
|
||||
} else {
|
||||
this->data(state).is_null = false;
|
||||
if constexpr (lt_is_array<LT>) {
|
||||
AggDataTypeTraits<LT>::assign_value(
|
||||
this->data(state).value,
|
||||
AggDataTypeTraits<LT>::get_row_ref(*this->data(state).default_value, 0));
|
||||
} else {
|
||||
this->data(state).value = this->data(state).default_value;
|
||||
}
|
||||
this->data(state).value = this->data(state).default_value;
|
||||
}
|
||||
return;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -176,8 +176,7 @@ StatusOr<ColumnPtr> CastStringToArray::evaluate_checked(ExprContext* context, Ch
|
|||
// return null if not valid array
|
||||
if (!is_valid_array(str, stack)) {
|
||||
if (_throw_exception_if_err) {
|
||||
return Status::InternalError(
|
||||
fmt::format("invalid array input: {}", std::string_view(str.get_data(), str.get_size())));
|
||||
return Status::InternalError(fmt::format("invalid array input: {}", str));
|
||||
} else {
|
||||
has_null = true;
|
||||
null_column->append(1);
|
||||
|
|
|
|||
|
|
@ -20,14 +20,10 @@
|
|||
#include "column/map_column.h"
|
||||
#include "column/struct_column.h"
|
||||
#include "column/type_traits.h"
|
||||
#ifndef MACOS_DISABLE_JAVA
|
||||
#include "exprs/agg/java_udaf_function.h"
|
||||
#endif
|
||||
#include "runtime/runtime_state.h"
|
||||
#include "types/logical_type_infra.h"
|
||||
#ifndef MACOS_DISABLE_JAVA
|
||||
#include "udf/java/java_udf.h"
|
||||
#endif
|
||||
#include "util/bloom_filter.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
|
@ -42,7 +38,7 @@ FunctionContext* FunctionContext::create_context(RuntimeState* state, MemPool* p
|
|||
ctx->_mem_pool = pool;
|
||||
ctx->_return_type = return_type;
|
||||
ctx->_arg_types = arg_types;
|
||||
#if !defined(MACOS_DISABLE_JAVA) && !defined(BUILD_FORMAT_LIB)
|
||||
#if !defined(BUILD_FORMAT_LIB)
|
||||
ctx->_jvm_udaf_ctxs = std::make_unique<JavaUDAFContext>();
|
||||
#endif
|
||||
return ctx;
|
||||
|
|
@ -58,7 +54,7 @@ FunctionContext* FunctionContext::create_context(RuntimeState* state, MemPool* p
|
|||
ctx->_mem_pool = pool;
|
||||
ctx->_return_type = return_type;
|
||||
ctx->_arg_types = arg_types;
|
||||
#if !defined(MACOS_DISABLE_JAVA) && !defined(BUILD_FORMAT_LIB)
|
||||
#if !defined(BUILD_FORMAT_LIB)
|
||||
ctx->_jvm_udaf_ctxs = std::make_unique<JavaUDAFContext>();
|
||||
#endif
|
||||
ctx->_is_distinct = is_distinct;
|
||||
|
|
@ -141,12 +137,10 @@ void* FunctionContext::get_function_state(FunctionStateScope scope) const {
|
|||
}
|
||||
|
||||
void FunctionContext::release_mems() {
|
||||
#ifndef MACOS_DISABLE_JAVA
|
||||
if (_jvm_udaf_ctxs != nullptr && _jvm_udaf_ctxs->states) {
|
||||
auto env = JVMFunctionHelper::getInstance().getEnv();
|
||||
_jvm_udaf_ctxs->states->clear(this, env);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
||||
void FunctionContext::set_error(const char* error_msg, const bool is_udf) {
|
||||
|
|
|
|||
|
|
@ -35,12 +35,6 @@ class RuntimeState;
|
|||
class Column;
|
||||
class Slice;
|
||||
struct JavaUDAFContext;
|
||||
#if defined(MACOS_DISABLE_JAVA)
|
||||
// On macOS build, Java is disabled. Provide an empty definition so that
|
||||
// std::unique_ptr<JavaUDAFContext> has a complete type and can be destroyed
|
||||
// without pulling in JNI headers.
|
||||
struct JavaUDAFContext {};
|
||||
#endif
|
||||
struct NgramBloomFilterState;
|
||||
|
||||
class FunctionContext {
|
||||
|
|
|
|||
|
|
@ -3897,39 +3897,6 @@ StatusOr<ColumnPtr> TimeFunctions::time_format(FunctionContext* context, const s
|
|||
return builder.build(ColumnHelper::is_all_const(columns));
|
||||
}
|
||||
|
||||
constexpr static const int64_t MAX_TIME = 3023999L;
|
||||
|
||||
static int64_t from_seconds_with_limit(int64_t time) {
|
||||
if (time > MAX_TIME) {
|
||||
return MAX_TIME;
|
||||
}
|
||||
if (time < -MAX_TIME) {
|
||||
return -MAX_TIME;
|
||||
}
|
||||
return time;
|
||||
}
|
||||
|
||||
StatusOr<ColumnPtr> TimeFunctions::sec_to_time(FunctionContext* context, const starrocks::Columns& columns) {
|
||||
const auto& bigint_column = columns[0];
|
||||
|
||||
RETURN_IF_COLUMNS_ONLY_NULL(columns);
|
||||
|
||||
auto bigint_viewer = ColumnViewer<TYPE_BIGINT>(bigint_column);
|
||||
const size_t size = bigint_column->size();
|
||||
auto builder = ColumnBuilder<TYPE_TIME>(size);
|
||||
|
||||
for (size_t i = 0; i < size; ++i) {
|
||||
if (bigint_viewer.is_null(i)) {
|
||||
builder.append_null();
|
||||
continue;
|
||||
}
|
||||
auto time = static_cast<double>(from_seconds_with_limit(bigint_viewer.value(i)));
|
||||
builder.append(time);
|
||||
}
|
||||
|
||||
return builder.build(ColumnHelper::is_all_const(columns));
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
|
||||
#include "gen_cpp/opcode/TimeFunctions.inc"
|
||||
|
|
|
|||
|
|
@ -766,14 +766,6 @@ public:
|
|||
*/
|
||||
DEFINE_VECTORIZED_FN(time_to_sec);
|
||||
|
||||
/**
|
||||
* return time
|
||||
* @param: [int]
|
||||
* @paramType columns: [BinaryColumn]
|
||||
* @return Int64Column
|
||||
*/
|
||||
DEFINE_VECTORIZED_FN(sec_to_time);
|
||||
|
||||
/**
|
||||
* Returns the date of the first specified DOW (day of week) that occurs after the input date.
|
||||
* @param: [timestamp, dow]
|
||||
|
|
|
|||
|
|
@ -41,7 +41,7 @@ static constexpr size_t kDefaultPageHeaderSize = 16 * 1024;
|
|||
// 16MB is borrowed from Arrow
|
||||
static constexpr size_t kMaxPageHeaderSize = 16 * 1024 * 1024;
|
||||
|
||||
PageReader::PageReader(io::SeekableInputStream* stream, size_t start_offset, size_t length, size_t num_values,
|
||||
PageReader::PageReader(io::SeekableInputStream* stream, uint64_t start_offset, uint64_t length, uint64_t num_values,
|
||||
const ColumnReaderOptions& opts, const tparquet::CompressionCodec::type codec)
|
||||
: _stream(stream),
|
||||
_finish_offset(start_offset + length),
|
||||
|
|
|
|||
|
|
@ -266,8 +266,7 @@ StatusOr<::parquet::Compression::type> ParquetFileWriter::_convert_compression_t
|
|||
|
||||
// Check if arrow supports indicated compression type
|
||||
if (!::parquet::IsCodecSupported(converted_type)) {
|
||||
return Status::NotSupported(
|
||||
fmt::format("not supported compression codec {}", static_cast<int>(converted_type)));
|
||||
return Status::NotSupported(fmt::format("not supported compression codec {}", converted_type));
|
||||
}
|
||||
|
||||
return converted_type;
|
||||
|
|
|
|||
|
|
@ -28,11 +28,6 @@
|
|||
#include <mach/clock.h>
|
||||
#include <mach/mach.h>
|
||||
#include <mach/mach_time.h>
|
||||
// macOS system headers may define a function-like macro named current_task(),
|
||||
// which conflicts with StarRocks member function names. Undefine it here.
|
||||
#ifdef current_task
|
||||
#undef current_task
|
||||
#endif
|
||||
|
||||
#include "gutil/once.h"
|
||||
#endif // #if defined(__APPLE__)
|
||||
|
|
|
|||
|
|
@ -149,10 +149,6 @@ public:
|
|||
operator int32_t() const { return static_cast<int32_t>(operator int64_t()); }
|
||||
|
||||
operator size_t() const { return static_cast<size_t>(operator int64_t()); }
|
||||
#if defined(__APPLE__)
|
||||
// Disambiguate C-style casts to uint64_t on libc++
|
||||
operator uint64_t() const { return static_cast<uint64_t>(operator int64_t()); }
|
||||
#endif
|
||||
|
||||
operator float() const { return (float)operator double(); }
|
||||
|
||||
|
|
|
|||
|
|
@ -44,13 +44,6 @@ template <>
|
|||
struct unsigned_type<int128_t> {
|
||||
using type = uint128_t;
|
||||
};
|
||||
// On macOS with libc++, std::make_unsigned cannot be specialized for user types.
|
||||
// Provide an explicit mapping for int256_t to itself, which is sufficient for
|
||||
// decimal operations that only require absolute value and arithmetic.
|
||||
template <>
|
||||
struct unsigned_type<int256_t> {
|
||||
using type = int256_t;
|
||||
};
|
||||
|
||||
template <typename T, bool check_overflow>
|
||||
class DecimalV3Arithmetics {
|
||||
|
|
@ -302,8 +295,7 @@ public:
|
|||
|
||||
if constexpr (rule == ROUND_HALF_UP || rule == ROUND_HALF_EVEN) {
|
||||
//TODO(by satanson): ROUND_HALF_UP is different from ROUND_HALF_EVEN
|
||||
auto abs_remainder = (remainder >= 0) ? remainder : -remainder;
|
||||
need_round = abs_remainder >= (divisor >> 1);
|
||||
need_round = std::abs(remainder) >= (divisor >> 1);
|
||||
} else if constexpr (rule == ROUND_FLOOR) {
|
||||
need_round = remainder > 0 && quotient > 0;
|
||||
} else if constexpr (rule == ROUND_CEILING) {
|
||||
|
|
|
|||
|
|
@ -748,30 +748,13 @@ Status LakeTabletsChannel::_create_delta_writers(const PTabletWriterOpenRequest&
|
|||
|
||||
std::vector<int64_t> tablet_ids;
|
||||
tablet_ids.reserve(params.tablets_size());
|
||||
bool multi_stmt = _is_multi_statements_txn(params);
|
||||
for (const PTabletWithPartition& tablet : params.tablets()) {
|
||||
BundleWritableFileContext* bundle_writable_file_context = nullptr;
|
||||
// Do NOT enable bundle write for a multi-statements transaction.
|
||||
// Rationale:
|
||||
// A multi-statements txn may invoke multiple open/add_chunk cycles and flush segments in batches.
|
||||
// If some early segments are appended into a bundle file while later segments are flushed as
|
||||
// standalone segment files (because a subsequent writer/open does not attach to the previous
|
||||
// bundle context), the final Rowset metadata will contain a mixed set: a subset of segments with
|
||||
// bundle offsets recorded and the remaining without any offsets. Downstream rowset load logic
|
||||
// assumes a 1:1 correspondence between the 'segments' list and 'bundle_file_offsets' when the
|
||||
// latter is present. A partial list therefore triggers size mismatch errors when loading.
|
||||
// Mitigation Strategy:
|
||||
// Disable bundling entirely for multi-statements txns so every segment is materialized as an
|
||||
// independent file, guaranteeing consistent metadata and eliminating offset mismatch risk.
|
||||
// NOTE: If future optimization desires bundling + multi-stmt, it must introduce an atomic merge
|
||||
// mechanism ensuring offsets array completeness before publish.
|
||||
if (!multi_stmt && _is_data_file_bundle_enabled(params)) {
|
||||
if (_is_data_file_bundle_enabled(params)) {
|
||||
if (_bundle_wfile_ctx_by_partition.count(tablet.partition_id()) == 0) {
|
||||
_bundle_wfile_ctx_by_partition[tablet.partition_id()] = std::make_unique<BundleWritableFileContext>();
|
||||
}
|
||||
bundle_writable_file_context = _bundle_wfile_ctx_by_partition[tablet.partition_id()].get();
|
||||
} else if (multi_stmt && _is_data_file_bundle_enabled(params)) {
|
||||
VLOG(1) << "disable bundle write for multi statements txn partition=" << tablet.partition_id();
|
||||
}
|
||||
if (_delta_writers.count(tablet.tablet_id()) != 0) {
|
||||
// already created for the tablet, usually in incremental open case
|
||||
|
|
@ -795,7 +778,7 @@ Status LakeTabletsChannel::_create_delta_writers(const PTabletWriterOpenRequest&
|
|||
.set_profile(_profile)
|
||||
.set_bundle_writable_file_context(bundle_writable_file_context)
|
||||
.set_global_dicts(&_global_dicts)
|
||||
.set_is_multi_statements_txn(multi_stmt)
|
||||
.set_is_multi_statements_txn(_is_multi_statements_txn(params))
|
||||
.build());
|
||||
_delta_writers.emplace(tablet.tablet_id(), std::move(writer));
|
||||
tablet_ids.emplace_back(tablet.tablet_id());
|
||||
|
|
|
|||
|
|
@ -713,6 +713,29 @@ void RuntimeFilterMerger::_send_total_runtime_filter(int rf_version, int32_t fil
|
|||
status->isSent = true;
|
||||
}
|
||||
|
||||
struct RuntimeFilterWorkerEvent {
|
||||
public:
|
||||
RuntimeFilterWorkerEvent() = default;
|
||||
|
||||
EventType type;
|
||||
|
||||
TUniqueId query_id;
|
||||
|
||||
/// For OPEN_QUERY.
|
||||
TQueryOptions query_options;
|
||||
TRuntimeFilterParams create_rf_merger_request;
|
||||
bool is_opened_by_pipeline;
|
||||
|
||||
/// For SEND_PART_RF.
|
||||
std::vector<TNetworkAddress> transmit_addrs;
|
||||
std::vector<TRuntimeFilterDestination> destinations;
|
||||
int transmit_timeout_ms;
|
||||
int64_t transmit_via_http_min_size = 64L * 1024 * 1024;
|
||||
|
||||
/// For SEND_PART_RF, RECEIVE_PART_RF, and RECEIVE_TOTAL_RF.
|
||||
PTransmitRuntimeFilterParams transmit_rf_request;
|
||||
};
|
||||
|
||||
static_assert(std::is_move_assignable<RuntimeFilterWorkerEvent>::value);
|
||||
|
||||
RuntimeFilterWorker::RuntimeFilterWorker(ExecEnv* env) : _exec_env(env), _thread([this] { execute(); }) {
|
||||
|
|
|
|||
|
|
@ -187,22 +187,7 @@ inline std::string EventTypeToString(EventType type) {
|
|||
// - receive total RF and send it to RuntimeFilterPort
|
||||
// - send partitioned RF(for hash join node)
|
||||
// - close a query(delete runtime filter merger)
|
||||
struct RuntimeFilterWorkerEvent {
|
||||
RuntimeFilterWorkerEvent() = default;
|
||||
EventType type;
|
||||
TUniqueId query_id;
|
||||
// For OPEN_QUERY.
|
||||
TQueryOptions query_options;
|
||||
TRuntimeFilterParams create_rf_merger_request;
|
||||
bool is_opened_by_pipeline;
|
||||
// For SEND_PART_RF.
|
||||
std::vector<TNetworkAddress> transmit_addrs;
|
||||
std::vector<TRuntimeFilterDestination> destinations;
|
||||
int transmit_timeout_ms;
|
||||
int64_t transmit_via_http_min_size = 64L * 1024 * 1024;
|
||||
// For SEND_PART_RF, RECEIVE_PART_RF, and RECEIVE_TOTAL_RF.
|
||||
PTransmitRuntimeFilterParams transmit_rf_request;
|
||||
};
|
||||
struct RuntimeFilterWorkerEvent;
|
||||
|
||||
struct RuntimeFilterWorkerMetrics {
|
||||
void update_event_nums(EventType event_type, int64_t delta) { event_nums[event_type] += delta; }
|
||||
|
|
|
|||
|
|
@ -707,9 +707,6 @@ void MetaFileBuilder::add_rowset(const RowsetMetadataPB& rowset_pb, const std::m
|
|||
_pending_rowset_data.dels.insert(_pending_rowset_data.dels.end(), dels.begin(), dels.end());
|
||||
_pending_rowset_data.del_encryption_metas.insert(_pending_rowset_data.del_encryption_metas.end(),
|
||||
del_encryption_metas.begin(), del_encryption_metas.end());
|
||||
|
||||
// Track cumulative number of segments already added when batch applying multiple opwrites.
|
||||
_pending_rowset_data.assigned_segment_id += rowset_pb.segments_size();
|
||||
}
|
||||
|
||||
void MetaFileBuilder::set_final_rowset() {
|
||||
|
|
|
|||
|
|
@ -72,9 +72,6 @@ public:
|
|||
void set_recover_flag(RecoverFlag flag) { _recover_flag = flag; }
|
||||
RecoverFlag recover_flag() const { return _recover_flag; }
|
||||
|
||||
// Number of segments already assigned (accumulated) in current pending batch rowset build.
|
||||
uint32_t assigned_segment_id() const { return _pending_rowset_data.assigned_segment_id; }
|
||||
|
||||
void finalize_sstable_meta(const PersistentIndexSstableMetaPB& sstable_meta);
|
||||
|
||||
void remove_compacted_sst(const TxnLogPB_OpCompaction& op_compaction);
|
||||
|
|
@ -108,7 +105,6 @@ private:
|
|||
std::vector<FileMetaPB> orphan_files;
|
||||
std::vector<std::string> dels;
|
||||
std::vector<std::string> del_encryption_metas;
|
||||
uint32_t assigned_segment_id = 0;
|
||||
};
|
||||
|
||||
Tablet _tablet;
|
||||
|
|
|
|||
|
|
@ -171,18 +171,7 @@ Status TabletReader::open(const TabletReaderParams& read_params) {
|
|||
split_morsel_queue->set_tablet_schema(_tablet_schema);
|
||||
|
||||
while (true) {
|
||||
auto split_status_or = split_morsel_queue->try_get();
|
||||
if (UNLIKELY(!split_status_or.ok())) {
|
||||
LOG(WARNING) << "failed to get split morsel: " << split_status_or.status()
|
||||
<< ", query_id: " << print_id(read_params.runtime_state->query_id())
|
||||
<< ", tablet_id: " << tablet_shared_ptr->tablet_id();
|
||||
// clear split tasks, and fallback to non-split mode
|
||||
_split_tasks.clear();
|
||||
_need_split = false;
|
||||
return init_collector(read_params);
|
||||
}
|
||||
|
||||
auto split = std::move(split_status_or.value());
|
||||
auto split = split_morsel_queue->try_get().value();
|
||||
if (split != nullptr) {
|
||||
auto ctx = std::make_unique<pipeline::LakeSplitContext>();
|
||||
|
||||
|
|
|
|||
|
|
@ -225,6 +225,8 @@ public:
|
|||
&_tablet, _index_entry, &_builder,
|
||||
_base_version, true));
|
||||
}
|
||||
_metadata->set_next_rowset_id(_metadata->next_rowset_id() +
|
||||
std::max(1, op_write.rowset().segments_size()));
|
||||
}
|
||||
}
|
||||
_builder.set_final_rowset();
|
||||
|
|
|
|||
|
|
@ -261,64 +261,50 @@ Status UpdateManager::publish_primary_key_tablet(const TxnLogPB_OpWrite& op_writ
|
|||
};
|
||||
state.init(params);
|
||||
// Init delvec state.
|
||||
// Map from rssid (rowset id + segment offset) to the list of deleted rowids collected during this publish.
|
||||
PrimaryIndex::DeletesMap new_deletes;
|
||||
// Global segment id offset assigned by builder when batch applying multiple op_write in a single publish.
|
||||
uint32_t assigned_global_segments = batch_apply ? builder->assigned_segment_id() : 0;
|
||||
// Number of segments in the incoming rowset of this op_write.
|
||||
uint32_t local_segments = op_write.rowset().segments_size();
|
||||
for (uint32_t local_id = 0; local_id < local_segments; ++local_id) {
|
||||
uint32_t global_segment_id = assigned_global_segments + local_id;
|
||||
new_deletes[rowset_id + global_segment_id] = {};
|
||||
for (uint32_t segment_id = 0; segment_id < op_write.rowset().segments_size(); segment_id++) {
|
||||
new_deletes[rowset_id + segment_id] = {};
|
||||
}
|
||||
// The rssid for delete files equals `rowset_id + op_offset`. Since delete currently happens after upsert,
|
||||
// we use the max segment id as the `op_offset` for rebuild. This is a simplification until mixed
|
||||
// upsert+delete order in a single transaction is supported.
|
||||
// TODO: Support the actual interleaving order of upsert and delete within one transaction.
|
||||
// Rssid of delete files is equal to `rowset_id + op_offset`, and delete is always after upsert now,
|
||||
// so we use max segment id as `op_offset`.
|
||||
// TODO : support real order of mix upsert and delete in one transaction.
|
||||
const uint32_t del_rebuild_rssid = rowset_id + std::max(op_write.rowset().segments_size(), 1) - 1;
|
||||
// 2. Handle segment one by one to save memory usage.
|
||||
for (uint32_t local_id = 0; local_id < local_segments; ++local_id) {
|
||||
uint32_t global_segment_id = assigned_global_segments + local_id;
|
||||
// Load update state of the current segment, resolving conflicts but without taking index lock.
|
||||
RETURN_IF_ERROR(
|
||||
state.load_segment(local_id, params, base_version, true /*reslove conflict*/, false /*no need lock*/));
|
||||
for (uint32_t segment_id = 0; segment_id < op_write.rowset().segments_size(); segment_id++) {
|
||||
RETURN_IF_ERROR(state.load_segment(segment_id, params, base_version, true /*reslove conflict*/,
|
||||
false /*no need lock*/));
|
||||
_update_state_cache.update_object_size(state_entry, state.memory_usage());
|
||||
// 2.1 For partial update, rewrite the segment file to generate replace segments and orphan files.
|
||||
RETURN_IF_ERROR(state.rewrite_segment(local_id, txn_id, params, &replace_segments, &orphan_files));
|
||||
rssid_fileinfo_container.add_rssid_to_file(op_write.rowset(), metadata->next_rowset_id(), local_id,
|
||||
// 2.1 rewrite segment file if it is partial update
|
||||
RETURN_IF_ERROR(state.rewrite_segment(segment_id, txn_id, params, &replace_segments, &orphan_files));
|
||||
rssid_fileinfo_container.add_rssid_to_file(op_write.rowset(), metadata->next_rowset_id(), segment_id,
|
||||
replace_segments);
|
||||
VLOG(2) << strings::Substitute(
|
||||
"[publish_pk_tablet][segment_loop] tablet:$0 txn:$1 assigned:$2 local_id:$3 global_id:$4 "
|
||||
"segments_local:$5",
|
||||
tablet->id(), txn_id, assigned_global_segments, local_id, global_segment_id, local_segments);
|
||||
// If a merge condition is configured, only update rows that satisfy the condition.
|
||||
// handle merge condition, skip update row which's merge condition column value is smaller than current row
|
||||
int32_t condition_column = _get_condition_column(op_write, *tablet_schema);
|
||||
// 2.2 Update primary index and collect delete information caused by key replacement.
|
||||
// 2.2 update primary index, and generate delete info.
|
||||
TRACE_COUNTER_SCOPE_LATENCY_US("update_index_latency_us");
|
||||
DCHECK(state.upserts(local_id) != nullptr);
|
||||
DCHECK(state.upserts(segment_id) != nullptr);
|
||||
if (condition_column < 0) {
|
||||
RETURN_IF_ERROR(
|
||||
_do_update(rowset_id, global_segment_id, state.upserts(local_id), index, &new_deletes,
|
||||
_do_update(rowset_id, segment_id, state.upserts(segment_id), index, &new_deletes,
|
||||
op_write.ssts_size() > 0 &&
|
||||
use_cloud_native_pk_index(*metadata) /* read pk index only when ingest sst */));
|
||||
} else {
|
||||
RETURN_IF_ERROR(_do_update_with_condition(params, rowset_id, global_segment_id, condition_column,
|
||||
state.upserts(local_id)->pk_column, index, &new_deletes));
|
||||
RETURN_IF_ERROR(_do_update_with_condition(params, rowset_id, segment_id, condition_column,
|
||||
state.upserts(segment_id)->pk_column, index, &new_deletes));
|
||||
}
|
||||
// 2.3 Apply deletes generated by auto-increment conflict handling (if any).
|
||||
if (state.auto_increment_deletes(local_id) != nullptr) {
|
||||
// 2.3 handle auto increment deletes
|
||||
if (state.auto_increment_deletes(segment_id) != nullptr) {
|
||||
RETURN_IF_ERROR(
|
||||
index.erase(metadata, *state.auto_increment_deletes(local_id), &new_deletes, del_rebuild_rssid));
|
||||
index.erase(metadata, *state.auto_increment_deletes(segment_id), &new_deletes, del_rebuild_rssid));
|
||||
}
|
||||
// Refresh memory accounting after index/state changes.
|
||||
_index_cache.update_object_size(index_entry, index.memory_usage());
|
||||
state.release_segment(local_id);
|
||||
state.release_segment(segment_id);
|
||||
_update_state_cache.update_object_size(state_entry, state.memory_usage());
|
||||
if (op_write.ssts_size() > 0 && condition_column < 0 && use_cloud_native_pk_index(*metadata)) {
|
||||
// TODO support condition column with sst ingestion.
|
||||
// rowset_id + segment_id is the rssid of this segment
|
||||
RETURN_IF_ERROR(index.ingest_sst(op_write.ssts(local_id), rowset_id + global_segment_id,
|
||||
metadata->version(), DelvecPagePB() /* empty */, nullptr));
|
||||
RETURN_IF_ERROR(index.ingest_sst(op_write.ssts(segment_id), rowset_id + segment_id, metadata->version(),
|
||||
DelvecPagePB() /* empty */, nullptr));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -341,9 +327,7 @@ Status UpdateManager::publish_primary_key_tablet(const TxnLogPB_OpWrite& op_writ
|
|||
std::map<uint32_t, size_t> segment_id_to_add_dels;
|
||||
for (auto& new_delete : new_deletes) {
|
||||
uint32_t rssid = new_delete.first;
|
||||
uint32_t assigned_segment_id = batch_apply ? builder->assigned_segment_id() : 0;
|
||||
if (rssid >= rowset_id + assigned_segment_id &&
|
||||
rssid < rowset_id + assigned_segment_id + op_write.rowset().segments_size()) {
|
||||
if (rssid >= rowset_id && rssid < rowset_id + op_write.rowset().segments_size()) {
|
||||
// it's newly added rowset's segment, do not have latest delvec yet
|
||||
new_del_vecs[idx].first = rssid;
|
||||
new_del_vecs[idx].second = std::make_shared<DelVector>();
|
||||
|
|
|
|||
|
|
@ -66,6 +66,7 @@ TableBuilder::TableBuilder(const Options& options, WritableFile* file) : rep_(ne
|
|||
}
|
||||
|
||||
TableBuilder::~TableBuilder() {
|
||||
assert(rep_->closed); // Catch errors where caller forgot to call Finish()
|
||||
delete rep_->filter_block;
|
||||
delete rep_;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -170,11 +170,6 @@ public:
|
|||
|
||||
explicit operator long() const { return static_cast<long>(static_cast<uint64_t>(low)); }
|
||||
|
||||
#if defined(__APPLE__)
|
||||
/// macOS/libc++: provide explicit conversion to 64-bit signed
|
||||
explicit operator long long() const { return static_cast<long long>(static_cast<uint64_t>(low)); }
|
||||
#endif
|
||||
|
||||
explicit operator int128_t() const { return static_cast<__int128>(low); }
|
||||
|
||||
/// Convert to signed char
|
||||
|
|
@ -186,12 +181,6 @@ public:
|
|||
/// Convert to size_t
|
||||
explicit operator size_t() const { return static_cast<size_t>(low); }
|
||||
|
||||
#if defined(__APPLE__)
|
||||
/// macOS/libc++: provide explicit conversion to 64-bit unsigned
|
||||
/// Note: uint64_t is a typedef of unsigned long long on macOS, so only define one
|
||||
explicit operator unsigned long long() const { return static_cast<unsigned long long>(static_cast<uint64_t>(low)); }
|
||||
#endif
|
||||
|
||||
// =============================================================================
|
||||
// Unary Operators
|
||||
// =============================================================================
|
||||
|
|
@ -924,8 +913,6 @@ int256_t parse_int256(const std::string& str);
|
|||
namespace std {
|
||||
|
||||
/// Specialization of make_signed for int256_t
|
||||
// On libc++ (macOS), user specializations of standard type traits are disallowed.
|
||||
#if !defined(__APPLE__)
|
||||
template <>
|
||||
struct make_signed<starrocks::int256_t> {
|
||||
using type = starrocks::int256_t;
|
||||
|
|
@ -936,7 +923,6 @@ template <>
|
|||
struct make_unsigned<starrocks::int256_t> {
|
||||
using type = starrocks::int256_t;
|
||||
};
|
||||
#endif // !__APPLE__
|
||||
|
||||
/// Hash function specialization for int256_t
|
||||
template <>
|
||||
|
|
@ -975,9 +961,8 @@ inline std::ostream& operator<<(std::ostream& os, const starrocks::int256_t& val
|
|||
/// Formatter specialization for int256_t (fmt library)
|
||||
template <>
|
||||
struct fmt::formatter<starrocks::int256_t> : formatter<std::string> {
|
||||
constexpr auto parse(fmt::format_parse_context& ctx) { return formatter<std::string>::parse(ctx); }
|
||||
template <typename FormatContext>
|
||||
auto format(const starrocks::int256_t& value, FormatContext& ctx) const {
|
||||
auto format(const starrocks::int256_t& value, FormatContext& ctx) {
|
||||
return formatter<std::string>::format(value.to_string(), ctx);
|
||||
}
|
||||
};
|
||||
|
|
|
|||
|
|
@ -359,7 +359,7 @@ public:
|
|||
DISALLOW_COPY(Promise);
|
||||
|
||||
~Promise() {
|
||||
if (static_cast<bool>(_state) && _state.use_count() > 1) _state->break_promise();
|
||||
if (static_cast<bool>(_state) && !_state.unique()) _state->break_promise();
|
||||
}
|
||||
|
||||
// Assignment
|
||||
|
|
@ -407,7 +407,7 @@ public:
|
|||
DISALLOW_COPY(Promise);
|
||||
|
||||
~Promise() {
|
||||
if (static_cast<bool>(_state) && _state.use_count() > 1) _state->break_promise();
|
||||
if (static_cast<bool>(_state) && !_state.unique()) _state->break_promise();
|
||||
}
|
||||
|
||||
// Assignment
|
||||
|
|
@ -451,7 +451,7 @@ public:
|
|||
DISALLOW_COPY(Promise);
|
||||
|
||||
~Promise() {
|
||||
if (static_cast<bool>(_state) && _state.use_count() > 1) _state->break_promise();
|
||||
if (static_cast<bool>(_state) && !_state.unique()) _state->break_promise();
|
||||
}
|
||||
|
||||
// Assignment
|
||||
|
|
@ -484,4 +484,4 @@ inline void swap(Promise<R>& x, Promise<R>& y) noexcept {
|
|||
x.swap(y);
|
||||
}
|
||||
|
||||
} // namespace starrocks::bthreads
|
||||
} // namespace starrocks::bthreads
|
||||
|
|
@ -24,22 +24,6 @@
|
|||
#include <thread>
|
||||
#include <vector>
|
||||
|
||||
#ifdef __APPLE__
|
||||
#include <pthread.h>
|
||||
#include <stdint.h>
|
||||
// macOS does not provide sched_getcpu; provide a lightweight fallback
|
||||
// that hashes the thread id into [0, hw_concurrency).
|
||||
static inline unsigned starrocks_sched_getcpu_fallback() {
|
||||
uint64_t tid = 0;
|
||||
(void)pthread_threadid_np(nullptr, &tid);
|
||||
unsigned n = (unsigned)std::max(1u, (unsigned)std::thread::hardware_concurrency());
|
||||
return (unsigned)(tid % n);
|
||||
}
|
||||
#define sched_getcpu starrocks_sched_getcpu_fallback
|
||||
#else
|
||||
#include <sched.h>
|
||||
#endif
|
||||
|
||||
#include "common/compiler_util.h"
|
||||
#include "gutil/macros.h"
|
||||
#include "util/cpu_info.h"
|
||||
|
|
|
|||
|
|
@ -203,21 +203,6 @@ uint64_t LRUCache::get_hit_count() const {
|
|||
return _hit_count;
|
||||
}
|
||||
|
||||
uint64_t LRUCache::get_insert_count() const {
|
||||
std::lock_guard l(_mutex);
|
||||
return _insert_count;
|
||||
}
|
||||
|
||||
uint64_t LRUCache::get_insert_evict_count() const {
|
||||
std::lock_guard l(_mutex);
|
||||
return _insert_evict_count;
|
||||
}
|
||||
|
||||
uint64_t LRUCache::get_release_evict_count() const {
|
||||
std::lock_guard l(_mutex);
|
||||
return _release_evict_count;
|
||||
}
|
||||
|
||||
size_t LRUCache::get_usage() const {
|
||||
std::lock_guard l(_mutex);
|
||||
return _usage;
|
||||
|
|
@ -265,8 +250,6 @@ void LRUCache::release(Cache::Handle* handle) {
|
|||
_unref(e);
|
||||
_usage -= e->charge;
|
||||
last_ref = true;
|
||||
// Track evictions caused by release
|
||||
++_release_evict_count;
|
||||
} else {
|
||||
// put it to LRU free list
|
||||
_lru_append(&_lru, e);
|
||||
|
|
@ -331,19 +314,9 @@ Cache::Handle* LRUCache::insert(const CacheKey& key, uint32_t hash, void* value,
|
|||
{
|
||||
std::lock_guard l(_mutex);
|
||||
|
||||
// Track insert count
|
||||
++_insert_count;
|
||||
|
||||
// Free the space following strict LRU policy until enough space
|
||||
// is freed or the lru list is empty
|
||||
size_t evicted_count_before = last_ref_list.size();
|
||||
_evict_from_lru(kv_mem_size, &last_ref_list);
|
||||
size_t evicted_count_after = last_ref_list.size();
|
||||
|
||||
// Track evictions caused by insert
|
||||
if (evicted_count_after > evicted_count_before) {
|
||||
_insert_evict_count += (evicted_count_after - evicted_count_before);
|
||||
}
|
||||
|
||||
// insert into the cache
|
||||
// note that the cache might get larger than its capacity if not enough
|
||||
|
|
@ -523,18 +496,6 @@ size_t ShardedLRUCache::get_hit_count() const {
|
|||
return _get_stat(&LRUCache::get_hit_count);
|
||||
}
|
||||
|
||||
size_t ShardedLRUCache::get_insert_count() const {
|
||||
return _get_stat(&LRUCache::get_insert_count);
|
||||
}
|
||||
|
||||
size_t ShardedLRUCache::get_insert_evict_count() const {
|
||||
return _get_stat(&LRUCache::get_insert_evict_count);
|
||||
}
|
||||
|
||||
size_t ShardedLRUCache::get_release_evict_count() const {
|
||||
return _get_stat(&LRUCache::get_release_evict_count);
|
||||
}
|
||||
|
||||
void ShardedLRUCache::get_cache_status(rapidjson::Document* document) {
|
||||
size_t shard_count = sizeof(_shards) / sizeof(LRUCache);
|
||||
|
||||
|
|
|
|||
|
|
@ -183,9 +183,6 @@ public:
|
|||
virtual size_t get_memory_usage() const = 0;
|
||||
virtual size_t get_lookup_count() const = 0;
|
||||
virtual size_t get_hit_count() const = 0;
|
||||
virtual size_t get_insert_count() const = 0;
|
||||
virtual size_t get_insert_evict_count() const = 0;
|
||||
virtual size_t get_release_evict_count() const = 0;
|
||||
|
||||
// Decrease or increase cache capacity.
|
||||
virtual bool adjust_capacity(int64_t delta, size_t min_capacity = 0) = 0;
|
||||
|
|
@ -281,9 +278,6 @@ public:
|
|||
|
||||
uint64_t get_lookup_count() const;
|
||||
uint64_t get_hit_count() const;
|
||||
uint64_t get_insert_count() const;
|
||||
uint64_t get_insert_evict_count() const;
|
||||
uint64_t get_release_evict_count() const;
|
||||
size_t get_usage() const;
|
||||
size_t get_capacity() const;
|
||||
static size_t key_handle_size(const CacheKey& key) { return sizeof(LRUHandle) - 1 + key.size(); }
|
||||
|
|
@ -311,9 +305,6 @@ private:
|
|||
|
||||
uint64_t _lookup_count{0};
|
||||
uint64_t _hit_count{0};
|
||||
uint64_t _insert_count{0};
|
||||
uint64_t _insert_evict_count{0};
|
||||
uint64_t _release_evict_count{0};
|
||||
};
|
||||
|
||||
static const int kNumShardBits = 5;
|
||||
|
|
@ -337,11 +328,8 @@ public:
|
|||
void set_capacity(size_t capacity) override;
|
||||
size_t get_memory_usage() const override;
|
||||
size_t get_capacity() const override;
|
||||
size_t get_lookup_count() const override;
|
||||
size_t get_hit_count() const override;
|
||||
size_t get_insert_count() const override;
|
||||
size_t get_insert_evict_count() const override;
|
||||
size_t get_release_evict_count() const override;
|
||||
uint64_t get_lookup_count() const override;
|
||||
uint64_t get_hit_count() const override;
|
||||
bool adjust_capacity(int64_t delta, size_t min_capacity = 0) override;
|
||||
|
||||
private:
|
||||
|
|
|
|||
|
|
@ -41,7 +41,6 @@
|
|||
#include <cstring>
|
||||
#include <iostream>
|
||||
#include <map>
|
||||
#include <ostream>
|
||||
#include <string>
|
||||
#include <string_view>
|
||||
#include <vector>
|
||||
|
|
|
|||
|
|
@ -278,7 +278,6 @@ void StarRocksMetrics::initialize(const std::vector<std::string>& paths, bool in
|
|||
_system_metrics.install(&_metrics, disk_devices, network_interfaces);
|
||||
}
|
||||
|
||||
#ifndef MACOS_DISABLE_JAVA
|
||||
if (init_jvm_metrics) {
|
||||
auto status = _jvm_metrics.init();
|
||||
if (!status.ok()) {
|
||||
|
|
@ -287,7 +286,6 @@ void StarRocksMetrics::initialize(const std::vector<std::string>& paths, bool in
|
|||
}
|
||||
_jvm_metrics.install(&_metrics);
|
||||
}
|
||||
#endif
|
||||
}
|
||||
|
||||
void StarRocksMetrics::_update() {
|
||||
|
|
|
|||
|
|
@ -40,9 +40,7 @@
|
|||
#include <vector>
|
||||
|
||||
#include "exec/pipeline/pipeline_metrics.h"
|
||||
#ifndef MACOS_DISABLE_JAVA
|
||||
#include "util/jvm_metrics.h"
|
||||
#endif
|
||||
#include "util/metrics.h"
|
||||
#include "util/system_metrics.h"
|
||||
#include "util/table_metrics.h"
|
||||
|
|
@ -439,9 +437,7 @@ private:
|
|||
|
||||
MetricRegistry _metrics;
|
||||
SystemMetrics _system_metrics;
|
||||
#ifndef MACOS_DISABLE_JAVA
|
||||
JVMMetrics _jvm_metrics;
|
||||
#endif
|
||||
TableMetricsManager _table_metrics_mgr;
|
||||
};
|
||||
|
||||
|
|
|
|||
|
|
@ -35,11 +35,7 @@
|
|||
#pragma once
|
||||
|
||||
#include <pthread.h>
|
||||
#ifdef __APPLE__
|
||||
#include <sys/syscall.h>
|
||||
#else
|
||||
#include <syscall.h>
|
||||
#endif
|
||||
|
||||
#include <atomic>
|
||||
#include <thread>
|
||||
|
|
|
|||
|
|
@ -4727,79 +4727,4 @@ TEST_F(TimeFunctionsTest, hourFromUnixTime) {
|
|||
}
|
||||
}
|
||||
|
||||
// Tests for sec_to_time function
|
||||
TEST_F(TimeFunctionsTest, secToTimeTest) {
|
||||
{
|
||||
auto int_value = ColumnHelper::create_column(TypeDescriptor(TYPE_BIGINT), false);
|
||||
|
||||
int_value->append_datum(0L);
|
||||
int_value->append_datum(1L);
|
||||
int_value->append_datum(60L);
|
||||
int_value->append_datum(3600L);
|
||||
int_value->append_datum(36000L);
|
||||
int_value->append_datum(86399L);
|
||||
int_value->append_datum(3024000L);
|
||||
int_value->append_datum(4000000L);
|
||||
|
||||
Columns columns;
|
||||
columns.emplace_back(int_value);
|
||||
|
||||
ColumnPtr result = TimeFunctions::sec_to_time(_utils->get_fn_ctx(), columns).value();
|
||||
auto v = ColumnHelper::cast_to<TYPE_TIME>(result);
|
||||
|
||||
EXPECT_EQ(8, result->size());
|
||||
EXPECT_EQ(0, v->get_data()[0]);
|
||||
EXPECT_EQ(1, v->get_data()[1]);
|
||||
EXPECT_EQ(60, v->get_data()[2]);
|
||||
EXPECT_EQ(3600, v->get_data()[3]);
|
||||
EXPECT_EQ(36000, v->get_data()[4]);
|
||||
EXPECT_EQ(86399, v->get_data()[5]);
|
||||
EXPECT_EQ(3023999, v->get_data()[6]);
|
||||
EXPECT_EQ(3023999, v->get_data()[7]);
|
||||
}
|
||||
{
|
||||
auto int_value = ColumnHelper::create_column(TypeDescriptor(TYPE_BIGINT), false);
|
||||
|
||||
int_value->append_datum(-0L);
|
||||
int_value->append_datum(-1L);
|
||||
int_value->append_datum(-60L);
|
||||
int_value->append_datum(-3600L);
|
||||
int_value->append_datum(-36000L);
|
||||
int_value->append_datum(-86399L);
|
||||
int_value->append_datum(-3024000L);
|
||||
int_value->append_datum(-4000000L);
|
||||
|
||||
Columns columns;
|
||||
columns.emplace_back(int_value);
|
||||
|
||||
ColumnPtr result = TimeFunctions::sec_to_time(_utils->get_fn_ctx(), columns).value();
|
||||
auto v = ColumnHelper::cast_to<TYPE_TIME>(result);
|
||||
|
||||
EXPECT_EQ(8, result->size());
|
||||
EXPECT_EQ(0, v->get_data()[0]);
|
||||
EXPECT_EQ(-1, v->get_data()[1]);
|
||||
EXPECT_EQ(-60, v->get_data()[2]);
|
||||
EXPECT_EQ(-3600, v->get_data()[3]);
|
||||
EXPECT_EQ(-36000, v->get_data()[4]);
|
||||
EXPECT_EQ(-86399, v->get_data()[5]);
|
||||
EXPECT_EQ(-3023999, v->get_data()[6]);
|
||||
EXPECT_EQ(-3023999, v->get_data()[7]);
|
||||
}
|
||||
|
||||
{
|
||||
// Create null column
|
||||
auto null_value = ColumnHelper::create_column(TypeDescriptor(TYPE_BIGINT), true);
|
||||
|
||||
(void)null_value->append_nulls(1);
|
||||
|
||||
Columns columns;
|
||||
columns.emplace_back(null_value);
|
||||
|
||||
ColumnPtr result = TimeFunctions::sec_to_time(_utils->get_fn_ctx(), columns).value();
|
||||
|
||||
EXPECT_EQ(1, result->size());
|
||||
ASSERT_TRUE(result->is_nullable());
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
} // namespace starrocks
|
||||
|
|
|
|||
|
|
@ -99,15 +99,15 @@ public:
|
|||
}
|
||||
|
||||
bool check(const ChunkPtr& chunk) {
|
||||
std::map<std::string, std::pair<int, int>> tmp_chunk;
|
||||
std::map<int, std::pair<int, int>> tmp_chunk;
|
||||
for (int i = 0; i < chunk->num_rows(); i++) {
|
||||
if (tmp_chunk.count(chunk->columns()[0]->get(i).get_slice().to_string()) > 0) {
|
||||
if (tmp_chunk.count(chunk->columns()[0]->get(i).get_int32()) > 0) {
|
||||
// duplicate pk
|
||||
LOG(ERROR) << "duplicate pk: " << chunk->columns()[0]->get(i).get_slice().to_string();
|
||||
LOG(ERROR) << "duplicate pk: " << chunk->columns()[0]->get(i).get_int32();
|
||||
return false;
|
||||
}
|
||||
tmp_chunk[chunk->columns()[0]->get(i).get_slice().to_string()] = {chunk->columns()[1]->get(i).get_int32(),
|
||||
chunk->columns()[2]->get(i).get_int32()};
|
||||
tmp_chunk[chunk->columns()[0]->get(i).get_int32()] = {chunk->columns()[1]->get(i).get_int32(),
|
||||
chunk->columns()[2]->get(i).get_int32()};
|
||||
}
|
||||
if (tmp_chunk.size() != _replayer_index.size()) {
|
||||
LOG(ERROR) << "inconsistency row number, actual : " << tmp_chunk.size()
|
||||
|
|
@ -146,24 +146,24 @@ public:
|
|||
if (log.op == ReplayerOP::UPSERT) {
|
||||
// Upsert
|
||||
for (int i = 0; i < chunk->num_rows(); i++) {
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_slice().to_string()] = {
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_int32()] = {
|
||||
chunk->columns()[1]->get(i).get_int32(), chunk->columns()[2]->get(i).get_int32()};
|
||||
}
|
||||
} else if (log.op == ReplayerOP::ERASE) {
|
||||
// Delete
|
||||
for (int i = 0; i < chunk->num_rows(); i++) {
|
||||
_replayer_index.erase(chunk->columns()[0]->get(i).get_slice().to_string());
|
||||
_replayer_index.erase(chunk->columns()[0]->get(i).get_int32());
|
||||
}
|
||||
} else if (log.op == ReplayerOP::PARTIAL_UPSERT || log.op == ReplayerOP::PARTIAL_UPDATE) {
|
||||
// Partial update
|
||||
for (int i = 0; i < chunk->num_rows(); i++) {
|
||||
auto iter = _replayer_index.find(chunk->columns()[0]->get(i).get_slice().to_string());
|
||||
auto iter = _replayer_index.find(chunk->columns()[0]->get(i).get_int32());
|
||||
if (iter != _replayer_index.end()) {
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_slice().to_string()] = {
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_int32()] = {
|
||||
chunk->columns()[1]->get(i).get_int32(), iter->second.second};
|
||||
} else if (log.op == ReplayerOP::PARTIAL_UPSERT) {
|
||||
// insert new record with default val
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_slice().to_string()] = {
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_int32()] = {
|
||||
chunk->columns()[1]->get(i).get_int32(), 0};
|
||||
} else {
|
||||
// do nothing
|
||||
|
|
@ -186,11 +186,11 @@ public:
|
|||
return false;
|
||||
};
|
||||
for (int i = 0; i < chunk->num_rows(); i++) {
|
||||
auto iter = _replayer_index.find(chunk->columns()[0]->get(i).get_slice().to_string());
|
||||
auto iter = _replayer_index.find(chunk->columns()[0]->get(i).get_int32());
|
||||
if (iter == _replayer_index.end() || is_condition_meet_fn(iter->second, i)) {
|
||||
// update if condition meet or not found
|
||||
// insert new record
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_slice().to_string()] = {
|
||||
_replayer_index[chunk->columns()[0]->get(i).get_int32()] = {
|
||||
chunk->columns()[1]->get(i).get_int32(), chunk->columns()[2]->get(i).get_int32()};
|
||||
}
|
||||
}
|
||||
|
|
@ -206,7 +206,7 @@ private:
|
|||
// logs for replay.
|
||||
std::vector<ReplayEntry> _redo_logs;
|
||||
// c0 -> <c1, c2>
|
||||
std::map<std::string, std::pair<int, int>> _replayer_index;
|
||||
std::map<int, std::pair<int, int>> _replayer_index;
|
||||
};
|
||||
|
||||
class LakePrimaryKeyConsistencyTest : public TestBase, testing::WithParamInterface<PrimaryKeyParam> {
|
||||
|
|
@ -216,8 +216,8 @@ public:
|
|||
_tablet_metadata->set_enable_persistent_index(true);
|
||||
_tablet_metadata->set_persistent_index_type(GetParam().persistent_index_type);
|
||||
|
||||
_slots.emplace_back(0, "c0", TypeDescriptor{LogicalType::TYPE_VARCHAR});
|
||||
_partial_slots.emplace_back(0, "c0", TypeDescriptor{LogicalType::TYPE_VARCHAR});
|
||||
_slots.emplace_back(0, "c0", TypeDescriptor{LogicalType::TYPE_INT});
|
||||
_partial_slots.emplace_back(0, "c0", TypeDescriptor{LogicalType::TYPE_INT});
|
||||
_slots.emplace_back(1, "c1", TypeDescriptor{LogicalType::TYPE_INT});
|
||||
_partial_slots.emplace_back(1, "c1", TypeDescriptor{LogicalType::TYPE_INT});
|
||||
_slots.emplace_back(2, "c2", TypeDescriptor{LogicalType::TYPE_INT});
|
||||
|
|
@ -269,8 +269,6 @@ public:
|
|||
config::enable_pindex_minor_compaction = false;
|
||||
_old_enable_pk_strict_memcheck = config::enable_pk_strict_memcheck;
|
||||
config::enable_pk_strict_memcheck = false;
|
||||
_old_pk_parallel_execution_threshold_bytes = config::pk_parallel_execution_threshold_bytes;
|
||||
config::pk_parallel_execution_threshold_bytes = 1;
|
||||
}
|
||||
|
||||
void TearDown() override {
|
||||
|
|
@ -278,7 +276,6 @@ public:
|
|||
config::l0_max_mem_usage = _old_l0_size;
|
||||
config::write_buffer_size = _old_memtable_size;
|
||||
config::enable_pk_strict_memcheck = _old_enable_pk_strict_memcheck;
|
||||
config::pk_parallel_execution_threshold_bytes = _old_pk_parallel_execution_threshold_bytes;
|
||||
}
|
||||
|
||||
std::shared_ptr<TabletMetadataPB> generate_tablet_metadata(KeysType keys_type) {
|
||||
|
|
@ -290,7 +287,7 @@ public:
|
|||
//
|
||||
// | column | type | KEY | NULL |
|
||||
// +--------+------+-----+------+
|
||||
// | c0 | STRING | YES | NO |
|
||||
// | c0 | INT | YES | NO |
|
||||
// | c1 | INT | NO | NO |
|
||||
// | c2 | INT | NO | NO |
|
||||
auto schema = metadata->mutable_schema();
|
||||
|
|
@ -302,10 +299,9 @@ public:
|
|||
{
|
||||
c0->set_unique_id(next_id());
|
||||
c0->set_name("c0");
|
||||
c0->set_type("VARCHAR");
|
||||
c0->set_type("INT");
|
||||
c0->set_is_key(true);
|
||||
c0->set_is_nullable(false);
|
||||
c0->set_length(3200);
|
||||
}
|
||||
auto c1 = schema->add_column();
|
||||
{
|
||||
|
|
@ -331,22 +327,14 @@ public:
|
|||
std::pair<ChunkPtr, std::vector<uint32_t>> gen_upsert_data(bool is_upsert) {
|
||||
const size_t chunk_size = (size_t)_random_generator->random_n();
|
||||
std::vector<std::vector<int>> cols(3);
|
||||
std::vector<std::string> key_col_str;
|
||||
std::vector<Slice> key_col;
|
||||
std::vector<uint8_t> v3(chunk_size, is_upsert ? TOpType::UPSERT : TOpType::DELETE);
|
||||
_random_generator->random_cols(chunk_size, &cols);
|
||||
for (size_t i = 0; i < chunk_size; i++) {
|
||||
key_col_str.emplace_back(std::to_string(cols[0][i]));
|
||||
}
|
||||
for (const auto& s : key_col_str) {
|
||||
key_col.emplace_back(Slice(s));
|
||||
}
|
||||
|
||||
auto c0 = BinaryColumn::create();
|
||||
auto c0 = Int32Column::create();
|
||||
auto c1 = Int32Column::create();
|
||||
auto c2 = Int32Column::create();
|
||||
auto c3 = Int8Column::create();
|
||||
c0->append_strings(key_col.data(), key_col.size());
|
||||
c0->append_numbers(cols[0].data(), cols[0].size() * sizeof(int));
|
||||
c1->append_numbers(cols[1].data(), cols[1].size() * sizeof(int));
|
||||
c2->append_numbers(cols[2].data(), cols[2].size() * sizeof(int));
|
||||
c3->append_numbers(v3.data(), v3.size() * sizeof(uint8_t));
|
||||
|
|
@ -362,20 +350,12 @@ public:
|
|||
std::pair<ChunkPtr, std::vector<uint32_t>> gen_partial_update_data() {
|
||||
const size_t chunk_size = (size_t)_random_generator->random_n();
|
||||
std::vector<std::vector<int>> cols(2);
|
||||
std::vector<std::string> key_col_str;
|
||||
std::vector<Slice> key_col;
|
||||
std::vector<uint8_t> v3(chunk_size, TOpType::UPSERT);
|
||||
_random_generator->random_cols(chunk_size, &cols);
|
||||
for (size_t i = 0; i < chunk_size; i++) {
|
||||
key_col_str.emplace_back(std::to_string(cols[0][i]));
|
||||
}
|
||||
for (const auto& s : key_col_str) {
|
||||
key_col.emplace_back(Slice(s));
|
||||
}
|
||||
|
||||
auto c0 = BinaryColumn::create();
|
||||
auto c0 = Int32Column::create();
|
||||
auto c1 = Int32Column::create();
|
||||
c0->append_strings(key_col.data(), key_col.size());
|
||||
c0->append_numbers(cols[0].data(), cols[0].size() * sizeof(int));
|
||||
c1->append_numbers(cols[1].data(), cols[1].size() * sizeof(int));
|
||||
auto indexes = std::vector<uint32_t>(chunk_size);
|
||||
for (uint32_t i = 0; i < chunk_size; i++) {
|
||||
|
|
@ -384,29 +364,6 @@ public:
|
|||
return {std::make_shared<Chunk>(Columns{std::move(c0), std::move(c1)}, _slot_cid_map), std::move(indexes)};
|
||||
}
|
||||
|
||||
// 5% chance to force index memtable flush
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> random_force_index_mem_flush() {
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> force_flush_guard;
|
||||
uint32_t r = _random_generator->random() % 100;
|
||||
if (r < 5) {
|
||||
// 5% chance to force index memtable flush
|
||||
force_flush_guard = std::make_unique<ConfigResetGuard<int64_t>>(&config::l0_max_mem_usage, 1);
|
||||
}
|
||||
return force_flush_guard;
|
||||
}
|
||||
|
||||
// 20% chance to enable pk parallel execution
|
||||
std::unique_ptr<ConfigResetGuard<bool>> random_pk_parallel_execution() {
|
||||
std::unique_ptr<ConfigResetGuard<bool>> pk_parallel_execution_guard;
|
||||
uint32_t r = _random_generator->random() % 100;
|
||||
if (r < 20) {
|
||||
// 20% chance to enable pk parallel execution
|
||||
pk_parallel_execution_guard =
|
||||
std::make_unique<ConfigResetGuard<bool>>(&config::enable_pk_parallel_execution, true);
|
||||
}
|
||||
return pk_parallel_execution_guard;
|
||||
}
|
||||
|
||||
ChunkPtr read(int64_t tablet_id, int64_t version) {
|
||||
ASSIGN_OR_ABORT(auto metadata, _tablet_mgr->get_tablet_metadata(tablet_id, version));
|
||||
auto reader = std::make_shared<TabletReader>(_tablet_mgr.get(), metadata, *_schema);
|
||||
|
|
@ -426,8 +383,6 @@ public:
|
|||
}
|
||||
|
||||
Status upsert_op() {
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> force_index_mem_flush_guard = random_force_index_mem_flush();
|
||||
std::unique_ptr<ConfigResetGuard<bool>> pk_parallel_execution_guard = random_pk_parallel_execution();
|
||||
auto txn_id = next_id();
|
||||
ASSIGN_OR_ABORT(auto delta_writer, DeltaWriterBuilder()
|
||||
.set_tablet_manager(_tablet_mgr.get())
|
||||
|
|
@ -463,8 +418,6 @@ public:
|
|||
}
|
||||
|
||||
Status partial_update_op(PartialUpdateMode mode) {
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> force_index_mem_flush_guard = random_force_index_mem_flush();
|
||||
std::unique_ptr<ConfigResetGuard<bool>> pk_parallel_execution_guard = random_pk_parallel_execution();
|
||||
auto txn_id = next_id();
|
||||
ASSIGN_OR_ABORT(auto delta_writer, DeltaWriterBuilder()
|
||||
.set_tablet_manager(_tablet_mgr.get())
|
||||
|
|
@ -500,8 +453,6 @@ public:
|
|||
}
|
||||
|
||||
Status condition_update() {
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> force_index_mem_flush_guard = random_force_index_mem_flush();
|
||||
std::unique_ptr<ConfigResetGuard<bool>> pk_parallel_execution_guard = random_pk_parallel_execution();
|
||||
auto txn_id = next_id();
|
||||
// c2 as merge_condition
|
||||
std::string merge_condition = "c2";
|
||||
|
|
@ -533,8 +484,6 @@ public:
|
|||
}
|
||||
|
||||
Status upsert_with_batch_pub_op() {
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> force_index_mem_flush_guard = random_force_index_mem_flush();
|
||||
std::unique_ptr<ConfigResetGuard<bool>> pk_parallel_execution_guard = random_pk_parallel_execution();
|
||||
size_t batch_cnt = std::max(_random_generator->random() % MaxBatchCnt, (size_t)1);
|
||||
std::vector<int64_t> txn_ids;
|
||||
for (int i = 0; i < batch_cnt; i++) {
|
||||
|
|
@ -576,8 +525,6 @@ public:
|
|||
}
|
||||
|
||||
Status delete_op() {
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> force_index_mem_flush_guard = random_force_index_mem_flush();
|
||||
std::unique_ptr<ConfigResetGuard<bool>> pk_parallel_execution_guard = random_pk_parallel_execution();
|
||||
auto chunk_index = gen_upsert_data(false);
|
||||
auto txn_id = next_id();
|
||||
ASSIGN_OR_ABORT(auto delta_writer, DeltaWriterBuilder()
|
||||
|
|
@ -603,8 +550,6 @@ public:
|
|||
}
|
||||
|
||||
Status compact_op() {
|
||||
std::unique_ptr<ConfigResetGuard<int64_t>> force_index_mem_flush_guard = random_force_index_mem_flush();
|
||||
std::unique_ptr<ConfigResetGuard<bool>> pk_parallel_execution_guard = random_pk_parallel_execution();
|
||||
auto txn_id = next_id();
|
||||
auto task_context = std::make_unique<CompactionTaskContext>(txn_id, _tablet_metadata->id(), _version, false,
|
||||
false, nullptr);
|
||||
|
|
@ -713,7 +658,6 @@ protected:
|
|||
int64_t _old_l0_size = 0;
|
||||
int64_t _old_memtable_size = 0;
|
||||
bool _old_enable_pk_strict_memcheck = false;
|
||||
int64_t _old_pk_parallel_execution_threshold_bytes = 0;
|
||||
};
|
||||
|
||||
TEST_P(LakePrimaryKeyConsistencyTest, test_local_pk_consistency) {
|
||||
|
|
|
|||
|
|
@ -204,17 +204,6 @@ add_definitions(-DMACOS_DISABLE_GEO)
|
|||
add_definitions(-DMACOS_DISABLE_S3)
|
||||
add_definitions(-DMACOS_DISABLE_MYSQL)
|
||||
add_definitions(-DMACOS_DISABLE_CLUCENE)
|
||||
add_definitions(-DMACOS_DISABLE_STARCACHE)
|
||||
# Disable spill feature on macOS
|
||||
add_definitions(-DMACOS_DISABLE_SPILL)
|
||||
# Disable Elasticsearch connector on macOS
|
||||
add_definitions(-DMACOS_DISABLE_ES)
|
||||
|
||||
# Force-include macOS shim headers across all translation units to avoid
|
||||
# touching core source files.
|
||||
add_compile_options(-include ${CMAKE_CURRENT_LIST_DIR}/shims/include/starrocks_macos_fmt_shims.h)
|
||||
# Ensure starcache/staros code paths are disabled by making the macros undefined
|
||||
add_compile_options(-UWITH_STARCACHE -UUSE_STAROS)
|
||||
|
||||
message(STATUS "macOS: Disabled problematic components for ARM64 compatibility")
|
||||
|
||||
|
|
@ -424,17 +413,10 @@ set_target_properties(zlib PROPERTIES
|
|||
IMPORTED_LOCATION ${ZLIB_LIB}
|
||||
INTERFACE_INCLUDE_DIRECTORIES /opt/homebrew/include)
|
||||
|
||||
# vectorscan (hyperscan-compatible) from thirdparty
|
||||
# Vectorscan installs libhs.a and libhs_runtime.a (hyperscan ABI)
|
||||
# vectorscan (hyperscan replacement) from thirdparty
|
||||
add_library(vectorscan STATIC IMPORTED GLOBAL)
|
||||
set_target_properties(vectorscan PROPERTIES
|
||||
IMPORTED_LOCATION ${THIRDPARTY_DIR}/installed/lib/libhs.a
|
||||
INTERFACE_INCLUDE_DIRECTORIES ${THIRDPARTY_DIR}/installed/include)
|
||||
|
||||
# hs_runtime from vectorscan
|
||||
add_library(hs_runtime STATIC IMPORTED GLOBAL)
|
||||
set_target_properties(hs_runtime PROPERTIES
|
||||
IMPORTED_LOCATION ${THIRDPARTY_DIR}/installed/lib/libhs_runtime.a
|
||||
IMPORTED_LOCATION ${THIRDPARTY_DIR}/installed/lib/libvectorscan.a
|
||||
INTERFACE_INCLUDE_DIRECTORIES ${THIRDPARTY_DIR}/installed/include)
|
||||
|
||||
# roaring bitmap from homebrew (dynamic library)
|
||||
|
|
@ -534,8 +516,7 @@ set(CMAKE_SKIP_RPATH TRUE)
|
|||
set(CMAKE_EXPORT_COMPILE_COMMANDS ON)
|
||||
|
||||
# Base compiler flags for ARM64
|
||||
# Enable CRC32 intrinsics on ARM64
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -march=armv8-a+crc")
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -march=armv8-a")
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -std=gnu++20")
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -stdlib=libc++")
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -fPIC")
|
||||
|
|
@ -556,12 +537,6 @@ set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-vla-cxx-extension")
|
|||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-#warnings")
|
||||
# Disable deprecated declarations warning (e.g., std::iterator deprecated in C++17)
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-deprecated-declarations")
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-macro-redefined")
|
||||
# Enable removed std::unary_function/binary_function in libc++ for legacy code
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -D_LIBCPP_ENABLE_CXX17_REMOVED_UNARY_BINARY_FUNCTION")
|
||||
|
||||
# Suppress all warnings on macOS build as requested
|
||||
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -Wno-everything")
|
||||
|
||||
# macOS SDK and deployment target
|
||||
set(CMAKE_OSX_DEPLOYMENT_TARGET "15.0")
|
||||
|
|
@ -584,7 +559,7 @@ endif()
|
|||
|
||||
# Apply common flags
|
||||
set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${CXX_COMMON_FLAGS}")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${CXX_COMMON_FLAGS} -Wno-everything")
|
||||
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${CXX_COMMON_FLAGS}")
|
||||
|
||||
# ==============================================================================
|
||||
# LINK LIBRARIES CONFIGURATION
|
||||
|
|
@ -611,7 +586,6 @@ list(APPEND STARROCKS_LINK_LIBS
|
|||
snappy
|
||||
zlib
|
||||
vectorscan
|
||||
hs_runtime
|
||||
roaring
|
||||
arrow
|
||||
parquet
|
||||
|
|
@ -663,7 +637,6 @@ message(STATUS "Apple frameworks configured for brpc support")
|
|||
# SOURCE FILES CONFIGURATION
|
||||
# ==============================================================================
|
||||
# Include all source directories
|
||||
include_directories(BEFORE ${CMAKE_CURRENT_LIST_DIR}/shims/include)
|
||||
include_directories(${SRC_DIR})
|
||||
include_directories(${GENSRC_DIR})
|
||||
include_directories(${BASE_DIR}/build)
|
||||
|
|
@ -729,18 +702,6 @@ list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*python_.*\\.cpp$")
|
|||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/fs/hdfs/.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*hdfs.*\\.cpp$")
|
||||
|
||||
# Exclude Elasticsearch (ES) related files on macOS
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/exec/es/.*\\.cpp$")
|
||||
|
||||
# Exclude Avro format and scanner sources (no Avro C++ on macOS)
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/formats/avro/.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/exec/file_scanner/avro_.*\\.cpp$")
|
||||
|
||||
# Exclude StarCache-related sources (unsupported on macOS build)
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/cache/disk_cache/starcache_engine\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/http/action/datacache_action\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/exec/schema_scanner/schema_be_datacache_metrics_scanner\\.cpp$")
|
||||
|
||||
# Completely exclude S3 related files
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/fs/s3/.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*s3.*\\.cpp$")
|
||||
|
|
@ -774,7 +735,6 @@ list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/data_consumer\\.cpp$")
|
|||
# Exclude problematic components
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/inverted_plugin_factory\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*clucene.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/exprs/gin_functions\\.cpp$")
|
||||
|
||||
# Exclude geo components that need special parsing files
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/geo/.*\\.cpp$")
|
||||
|
|
@ -784,7 +744,6 @@ list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*spillable.*\\.cpp$")
|
|||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*spill.*operator.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*spill.*factory.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*spill_process.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/exec/pipeline/spill_process_channel\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/spill/.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*SpillableHashJoin.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*SpillableNLJoin.*\\.cpp$")
|
||||
|
|
@ -820,9 +779,6 @@ list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*connector.*orc.*\\.cpp$")
|
|||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/http/action/jit_.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/http/action/.*jit.*\\.cpp$")
|
||||
|
||||
# Exclude JVM/JNI dependent metrics on macOS
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*/util/jvm_metrics\\.cpp$")
|
||||
|
||||
# Exclude Arrow Flight components
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*arrow_flight.*\\.cpp$")
|
||||
list(FILTER STARROCKS_SOURCES EXCLUDE REGEX ".*flight.*\\.cpp$")
|
||||
|
|
@ -917,4 +873,4 @@ message(STATUS "To build:")
|
|||
message(STATUS " ninja starrocks_be")
|
||||
message(STATUS " ninja install")
|
||||
message(STATUS "========================================")
|
||||
message(STATUS "")
|
||||
message(STATUS "")
|
||||
|
|
@ -205,9 +205,6 @@ if [[ $SKIP_THIRDPARTY -eq 0 ]]; then
|
|||
"librocksdb.a"
|
||||
"libvelocypack.a"
|
||||
"libbitshuffle.a"
|
||||
# vectorscan (hyperscan ABI)
|
||||
"libhs.a"
|
||||
"libhs_runtime.a"
|
||||
)
|
||||
|
||||
missing_libs=()
|
||||
|
|
@ -433,4 +430,4 @@ if [[ $DO_INSTALL -eq 1 ]]; then
|
|||
echo " ./starrocks_be --help"
|
||||
fi
|
||||
|
||||
log_success "StarRocks BE build completed successfully!"
|
||||
log_success "StarRocks BE build completed successfully!"
|
||||
|
|
@ -204,29 +204,6 @@ BITSHUFFLE_VERSION="0.5.1"
|
|||
VECTORSCAN_VERSION="5.4.12"
|
||||
VELOCYPACK_VERSION="XYZ1.0"
|
||||
|
||||
# datasketches-cpp
|
||||
DATASKETCHES_VERSION="4.0.0"
|
||||
DATASKETCHES_DOWNLOAD="https://github.com/apache/datasketches-cpp/archive/refs/tags/${DATASKETCHES_VERSION}.tar.gz"
|
||||
DATASKETCHES_NAME="datasketches-cpp-${DATASKETCHES_VERSION}.tar.gz"
|
||||
DATASKETCHES_SOURCE="datasketches-cpp-${DATASKETCHES_VERSION}"
|
||||
|
||||
# RYU (build from source; pinned commit snapshot)
|
||||
RYU_DOWNLOAD="https://github.com/ulfjack/ryu/archive/aa31ca9361d21b1a00ee054aac49c87d07e74abc.zip"
|
||||
RYU_NAME="ryu-aa31ca9361d21b1a00ee054aac49c87d07e74abc.zip"
|
||||
RYU_SOURCE="ryu-aa31ca9361d21b1a00ee054aac49c87d07e74abc"
|
||||
|
||||
# icu
|
||||
ICU_DOWNLOAD="https://github.com/unicode-org/icu/releases/download/release-76-1/icu4c-76_1-src.zip"
|
||||
ICU_NAME="icu4c-76_1-src.zip"
|
||||
ICU_SOURCE="icu"
|
||||
ICU_MD5SUM="f5f5c827d94af8445766c7023aca7f6b"
|
||||
|
||||
# libdivide
|
||||
LIBDIVIDE_DOWNLOAD="https://github.com/ridiculousfish/libdivide/archive/refs/tags/v5.2.0.tar.gz"
|
||||
LIBDIVIDE_NAME="libdivide-v5.2.0.tar.gz"
|
||||
LIBDIVIDE_SOURCE="libdivide-5.2.0"
|
||||
LIBDIVIDE_MD5SUM="4ba77777192c295d6de2b86d88f3239a"
|
||||
|
||||
download_source() {
|
||||
local name="$1"
|
||||
local version="$2"
|
||||
|
|
@ -721,178 +698,6 @@ build_bitshuffle() {
|
|||
log_success "bitshuffle built successfully"
|
||||
}
|
||||
|
||||
# datasketches (header-only install)
|
||||
build_datasketches() {
|
||||
# Check if already installed (pick one representative header)
|
||||
if [[ -d "$INSTALL_DIR/include/datasketches" && -f "$INSTALL_DIR/include/datasketches/hll.hpp" ]]; then
|
||||
log_success "datasketches already installed, skipping"
|
||||
return 0
|
||||
fi
|
||||
|
||||
log_info "Installing datasketches ${DATASKETCHES_VERSION} headers..."
|
||||
|
||||
local src_dir="$THIRDPARTY_DIR/src"
|
||||
local build_dir="$THIRDPARTY_DIR/build/datasketches"
|
||||
|
||||
download_source "datasketches-cpp" "$DATASKETCHES_VERSION" \
|
||||
"$DATASKETCHES_DOWNLOAD" \
|
||||
"$DATASKETCHES_NAME"
|
||||
|
||||
mkdir -p "$build_dir"
|
||||
cd "$build_dir"
|
||||
|
||||
if [[ ! -d "$DATASKETCHES_SOURCE" ]]; then
|
||||
tar -xzf "$src_dir/$DATASKETCHES_NAME"
|
||||
fi
|
||||
|
||||
# Copy public headers into a flat include/datasketches directory, matching Linux build layout
|
||||
mkdir -p "$INSTALL_DIR/include/datasketches"
|
||||
cp -r "$DATASKETCHES_SOURCE"/common/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/cpc/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/fi/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/hll/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/kll/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/quantiles/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/req/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/sampling/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/theta/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
cp -r "$DATASKETCHES_SOURCE"/tuple/include/* "$INSTALL_DIR/include/datasketches/" || true
|
||||
|
||||
log_success "datasketches headers installed"
|
||||
}
|
||||
|
||||
# Build ryu from source and install into $INSTALL_DIR
|
||||
build_ryu() {
|
||||
# Check if already built
|
||||
if [[ -f "$INSTALL_DIR/lib/libryu.a" && -f "$INSTALL_DIR/include/ryu/ryu.h" ]]; then
|
||||
log_success "ryu already built, skipping"
|
||||
return 0
|
||||
fi
|
||||
|
||||
log_info "Building ryu from source..."
|
||||
|
||||
local src_dir="$THIRDPARTY_DIR/src"
|
||||
local build_dir="$THIRDPARTY_DIR/build/ryu"
|
||||
|
||||
download_source "ryu" "$RYU_SOURCE" "$RYU_DOWNLOAD" "$RYU_NAME"
|
||||
|
||||
mkdir -p "$build_dir"
|
||||
cd "$build_dir"
|
||||
|
||||
if [[ ! -d "$RYU_SOURCE" ]]; then
|
||||
unzip -q "$src_dir/$RYU_NAME"
|
||||
fi
|
||||
|
||||
cd "$RYU_SOURCE"
|
||||
|
||||
# Apply patch if exists (use shared thirdparty patch when available)
|
||||
local patch_file="$THIRDPARTY_DIR/patches/ryu.patch"
|
||||
if [[ -f "$patch_file" ]]; then
|
||||
log_info "Applying ryu patch..."
|
||||
patch -p1 < "$patch_file"
|
||||
fi
|
||||
|
||||
# Build and install
|
||||
cd ryu
|
||||
make -j"$PARALLEL_JOBS"
|
||||
make install DESTDIR="$INSTALL_DIR"
|
||||
|
||||
# Ensure headers path matches <ryu/ryu.h>
|
||||
mkdir -p "$INSTALL_DIR/include/ryu"
|
||||
if [[ -f "$INSTALL_DIR/include/ryu.h" ]]; then
|
||||
mv -f "$INSTALL_DIR/include/ryu.h" "$INSTALL_DIR/include/ryu/ryu.h"
|
||||
elif [[ -f "ryu.h" ]]; then
|
||||
cp -f "ryu.h" "$INSTALL_DIR/include/ryu/ryu.h"
|
||||
fi
|
||||
|
||||
# Ensure lib64 compatibility copy
|
||||
if [[ -f "$INSTALL_DIR/lib/libryu.a" ]]; then
|
||||
mkdir -p "$INSTALL_DIR/lib64"
|
||||
cp -f "$INSTALL_DIR/lib/libryu.a" "$INSTALL_DIR/lib64/libryu.a"
|
||||
fi
|
||||
|
||||
log_success "ryu built successfully"
|
||||
}
|
||||
|
||||
build_libdivide() {
|
||||
# Check if already built
|
||||
if [[ -f "$INSTALL_DIR/include/libdivide.h" ]]; then
|
||||
log_success "libdivide already built, skipping"
|
||||
return 0
|
||||
fi
|
||||
|
||||
log_info "Building libdivide..."
|
||||
|
||||
local src_dir="$THIRDPARTY_DIR/src"
|
||||
local build_dir="$THIRDPARTY_DIR/build/libdivide"
|
||||
|
||||
download_source "libdivide" "v5.2.0" \
|
||||
"$LIBDIVIDE_DOWNLOAD" \
|
||||
"$LIBDIVIDE_NAME"
|
||||
|
||||
mkdir -p "$build_dir"
|
||||
cd "$build_dir"
|
||||
|
||||
if [[ ! -d "$LIBDIVIDE_SOURCE" ]]; then
|
||||
tar -xzf "$src_dir/$LIBDIVIDE_NAME"
|
||||
fi
|
||||
|
||||
cd "$LIBDIVIDE_SOURCE"
|
||||
|
||||
# libdivide is header-only, just copy the header
|
||||
cp libdivide.h "$INSTALL_DIR/include/"
|
||||
|
||||
log_success "libdivide built successfully"
|
||||
}
|
||||
|
||||
build_icu() {
|
||||
# Check if already built
|
||||
if [[ -f "$INSTALL_DIR/lib/libicuuc.a" && -f "$INSTALL_DIR/include/unicode/ucasemap.h" ]]; then
|
||||
log_success "icu already built, skipping"
|
||||
return 0
|
||||
fi
|
||||
|
||||
log_info "Building icu..."
|
||||
|
||||
local src_dir="$THIRDPARTY_DIR/src"
|
||||
local build_dir="$THIRDPARTY_DIR/build/icu"
|
||||
|
||||
download_source "icu" "76-1" \
|
||||
"$ICU_DOWNLOAD" \
|
||||
"$ICU_NAME"
|
||||
|
||||
mkdir -p "$build_dir"
|
||||
cd "$build_dir"
|
||||
|
||||
if [[ ! -d "$ICU_SOURCE" ]]; then
|
||||
unzip -q "$src_dir/$ICU_NAME"
|
||||
fi
|
||||
|
||||
cd "$ICU_SOURCE/source"
|
||||
|
||||
# Fix line endings for shell scripts
|
||||
sed -i '' 's/\r$//' ./runConfigureICU
|
||||
sed -i '' 's/\r$//' ./config.*
|
||||
sed -i '' 's/\r$//' ./configure
|
||||
sed -i '' 's/\r$//' ./mkinstalldirs
|
||||
|
||||
# Clear compile flags to use ICU defaults
|
||||
unset CPPFLAGS
|
||||
unset CXXFLAGS
|
||||
unset CFLAGS
|
||||
|
||||
# Use a subshell to prevent environment variable leakage
|
||||
(
|
||||
export CFLAGS="-O3 -fno-omit-frame-pointer -fPIC"
|
||||
export CXXFLAGS="-O3 -fno-omit-frame-pointer -fPIC"
|
||||
./runConfigureICU macOS --prefix="$INSTALL_DIR" --enable-static --disable-shared
|
||||
make -j"$PARALLEL_JOBS"
|
||||
make install
|
||||
)
|
||||
|
||||
log_success "icu built successfully"
|
||||
}
|
||||
|
||||
|
||||
detect_boost_version() {
|
||||
# Detect Boost version from Homebrew installation
|
||||
|
|
@ -1015,10 +820,6 @@ build_source_deps() {
|
|||
build_glog
|
||||
build_protobuf
|
||||
build_leveldb
|
||||
build_datasketches
|
||||
build_ryu
|
||||
build_libdivide
|
||||
build_icu
|
||||
|
||||
# Layer 2: Libraries that depend on Layer 1
|
||||
build_brpc
|
||||
|
|
@ -1037,9 +838,6 @@ build_source_deps() {
|
|||
ln -sf ../lib/libbrpc.a "$INSTALL_DIR/lib64/libbrpc.a" 2>/dev/null || true
|
||||
ln -sf ../lib/librocksdb.a "$INSTALL_DIR/lib64/librocksdb.a" 2>/dev/null || true
|
||||
ln -sf ../lib/libhs.a "$INSTALL_DIR/lib64/libhs.a" 2>/dev/null || true
|
||||
ln -sf ../lib/libryu.a "$INSTALL_DIR/lib64/libryu.a" 2>/dev/null || true
|
||||
ln -sf ../lib/libicuuc.a "$INSTALL_DIR/lib64/libicuuc.a" 2>/dev/null || true
|
||||
ln -sf ../lib/libicui18n.a "$INSTALL_DIR/lib64/libicui18n.a" 2>/dev/null || true
|
||||
|
||||
log_success "All source dependencies built successfully"
|
||||
}
|
||||
|
|
@ -1069,10 +867,6 @@ main() {
|
|||
"libvelocypack.a"
|
||||
"libbitshuffle.a"
|
||||
"libhs.a"
|
||||
"libhs_runtime.a"
|
||||
"libryu.a"
|
||||
"libicuuc.a"
|
||||
"libicui18n.a"
|
||||
)
|
||||
|
||||
local missing_libs=()
|
||||
|
|
|
|||
|
|
@ -1,75 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// macOS shim for missing <endian.h>
|
||||
#ifndef STARROCKS_MAC_ENDIAN_SHIM_H
|
||||
#define STARROCKS_MAC_ENDIAN_SHIM_H
|
||||
|
||||
#ifdef __APPLE__
|
||||
#include <machine/endian.h>
|
||||
#include <libkern/OSByteOrder.h>
|
||||
|
||||
#ifndef __BYTE_ORDER
|
||||
#define __BYTE_ORDER BYTE_ORDER
|
||||
#endif
|
||||
#ifndef __LITTLE_ENDIAN
|
||||
#define __LITTLE_ENDIAN LITTLE_ENDIAN
|
||||
#endif
|
||||
#ifndef __BIG_ENDIAN
|
||||
#define __BIG_ENDIAN BIG_ENDIAN
|
||||
#endif
|
||||
|
||||
#ifndef htobe16
|
||||
#define htobe16(x) OSSwapHostToBigInt16(x)
|
||||
#endif
|
||||
#ifndef htole16
|
||||
#define htole16(x) OSSwapHostToLittleInt16(x)
|
||||
#endif
|
||||
#ifndef be16toh
|
||||
#define be16toh(x) OSSwapBigToHostInt16(x)
|
||||
#endif
|
||||
#ifndef le16toh
|
||||
#define le16toh(x) OSSwapLittleToHostInt16(x)
|
||||
#endif
|
||||
|
||||
#ifndef htobe32
|
||||
#define htobe32(x) OSSwapHostToBigInt32(x)
|
||||
#endif
|
||||
#ifndef htole32
|
||||
#define htole32(x) OSSwapHostToLittleInt32(x)
|
||||
#endif
|
||||
#ifndef be32toh
|
||||
#define be32toh(x) OSSwapBigToHostInt32(x)
|
||||
#endif
|
||||
#ifndef le32toh
|
||||
#define le32toh(x) OSSwapLittleToHostInt32(x)
|
||||
#endif
|
||||
|
||||
#ifndef htobe64
|
||||
#define htobe64(x) OSSwapHostToBigInt64(x)
|
||||
#endif
|
||||
#ifndef htole64
|
||||
#define htole64(x) OSSwapHostToLittleInt64(x)
|
||||
#endif
|
||||
#ifndef be64toh
|
||||
#define be64toh(x) OSSwapBigToHostInt64(x)
|
||||
#endif
|
||||
#ifndef le64toh
|
||||
#define le64toh(x) OSSwapLittleToHostInt64(x)
|
||||
#endif
|
||||
|
||||
#endif // __APPLE__
|
||||
|
||||
#endif // STARROCKS_MAC_ENDIAN_SHIM_H
|
||||
|
||||
|
|
@ -1,40 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// macOS stub for AvroCppScanner when Avro is disabled
|
||||
#pragma once
|
||||
|
||||
#include "exec/file_scanner/file_scanner.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
class AvroCppScanner final : public FileScanner {
|
||||
public:
|
||||
AvroCppScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRange& scan_range,
|
||||
ScannerCounter* counter, bool schema_only = false)
|
||||
: FileScanner(state, profile, scan_range.params, counter, schema_only), _scan_range(scan_range) {}
|
||||
|
||||
Status open() override { return Status::NotSupported("Avro is disabled on macOS build"); }
|
||||
void close() override {}
|
||||
StatusOr<ChunkPtr> get_next() override { return Status::NotSupported("Avro is disabled on macOS build"); }
|
||||
Status get_schema(std::vector<SlotDescriptor>* /*schema*/) override {
|
||||
return Status::NotSupported("Avro is disabled on macOS build");
|
||||
}
|
||||
|
||||
private:
|
||||
const TBrokerScanRange& _scan_range;
|
||||
};
|
||||
|
||||
} // namespace starrocks
|
||||
|
||||
|
|
@ -1,40 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// macOS stub for ORCScanner when ORC is disabled
|
||||
#pragma once
|
||||
|
||||
#include "exec/file_scanner/file_scanner.h"
|
||||
|
||||
namespace starrocks {
|
||||
|
||||
class ORCScanner : public FileScanner {
|
||||
public:
|
||||
ORCScanner(RuntimeState* state, RuntimeProfile* profile, const TBrokerScanRange& scan_range,
|
||||
starrocks::ScannerCounter* counter, bool schema_only = false)
|
||||
: FileScanner(state, profile, scan_range.params, counter, schema_only), _scan_range(scan_range) {}
|
||||
|
||||
Status open() override { return Status::NotSupported("ORC is disabled on macOS build"); }
|
||||
StatusOr<ChunkPtr> get_next() override { return Status::NotSupported("ORC is disabled on macOS build"); }
|
||||
Status get_schema(std::vector<SlotDescriptor>* /*schema*/) override {
|
||||
return Status::NotSupported("ORC is disabled on macOS build");
|
||||
}
|
||||
void close() override {}
|
||||
|
||||
private:
|
||||
const TBrokerScanRange& _scan_range;
|
||||
};
|
||||
|
||||
} // namespace starrocks
|
||||
|
||||
|
|
@ -1,32 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// Minimal stub of libfiu control header for macOS build
|
||||
#pragma once
|
||||
|
||||
#ifdef __cplusplus
|
||||
extern "C" {
|
||||
#endif
|
||||
|
||||
// External enable API used by StarRocks when integrating failpoints.
|
||||
// This stub does nothing and returns 0.
|
||||
static inline int fiu_enable_external(const char* /*name*/, int /*action*/, void* /*failnum*/, unsigned int /*flags*/,
|
||||
int (*/*callback*/)(const char*, int*, void**, unsigned int*)) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
#ifdef __cplusplus
|
||||
}
|
||||
#endif
|
||||
|
||||
|
|
@ -1,27 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// Minimal stub of libfiu header for macOS build
|
||||
#pragma once
|
||||
|
||||
#ifdef __cplusplus
|
||||
extern "C" {
|
||||
#endif
|
||||
|
||||
static inline void fiu_init(int /*unused*/) {}
|
||||
|
||||
#ifdef __cplusplus
|
||||
}
|
||||
#endif
|
||||
|
||||
|
|
@ -1,46 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// macOS shim for missing <malloc.h>
|
||||
// Keep StarRocks source unchanged by providing a trampoline header.
|
||||
#ifndef STARROCKS_MAC_MALLOC_SHIM_H
|
||||
#define STARROCKS_MAC_MALLOC_SHIM_H
|
||||
|
||||
#ifdef __APPLE__
|
||||
#include <malloc/malloc.h>
|
||||
#include <stdlib.h>
|
||||
#include <unistd.h>
|
||||
|
||||
// Provide GNU extensions expected by some codepaths on Linux
|
||||
// Implement memalign using posix_memalign on macOS
|
||||
static inline void* memalign(size_t alignment, size_t size) {
|
||||
void* p = nullptr;
|
||||
if (posix_memalign(&p, alignment, size) != 0) return nullptr;
|
||||
return p;
|
||||
}
|
||||
|
||||
// Provide pvalloc: allocate page-aligned memory rounded up to page size
|
||||
static inline void* pvalloc(size_t size) {
|
||||
size_t page = (size_t)getpagesize();
|
||||
if (page == 0) page = 4096;
|
||||
size_t rounded = (size + page - 1) / page * page;
|
||||
// valloc is deprecated but still available; alternatives are posix_memalign
|
||||
// but valloc matches pvalloc semantics best here.
|
||||
return valloc(rounded);
|
||||
}
|
||||
#else
|
||||
#include <malloc.h>
|
||||
#endif
|
||||
|
||||
#endif // STARROCKS_MAC_MALLOC_SHIM_H
|
||||
|
|
@ -1,24 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// macOS shim header to adapt StarRocks include of Thrift's TDebugProtocol
|
||||
// StarRocks includes <protocol/TDebugProtocol.h> assuming the include path
|
||||
// points at the 'thrift' headers directory. On macOS, Homebrew installs
|
||||
// headers under <thrift/...>. This shim bridges the path without touching
|
||||
// StarRocks source files.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <thrift/protocol/TDebugProtocol.h>
|
||||
|
||||
|
|
@ -1,180 +0,0 @@
|
|||
// 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.
|
||||
|
||||
// Minimal fmt helpers for macOS build
|
||||
// Purpose: Enable fmt's ostream support so fmt::format can print StarRocks
|
||||
// types/enums that define operator<< in existing headers, without modifying core code.
|
||||
|
||||
#pragma once
|
||||
|
||||
#include <fmt/format.h>
|
||||
// Enable formatting via operator<< when available
|
||||
#include <fmt/ostream.h>
|
||||
// Enable fmt::join and range printing
|
||||
#include <fmt/ranges.h>
|
||||
|
||||
// Bring in StarRocks enum definitions and helpers
|
||||
#include "gen_cpp/Types_types.h"
|
||||
#include "gen_cpp/Exprs_types.h"
|
||||
#include "gen_cpp/PlanNodes_types.h"
|
||||
#include "types/logical_type.h"
|
||||
#include <atomic>
|
||||
#include <type_traits>
|
||||
#include <cstdlib> // ensure integer std::abs overloads are visible on libc++
|
||||
#include <string_view>
|
||||
|
||||
// Prefer the lightweight format_as customization point added in fmt >= 10.
|
||||
// It lets fmt format our types via ADL by converting them to a formattable type.
|
||||
namespace starrocks {
|
||||
|
||||
inline const char* format_as(LogicalType v) {
|
||||
return logical_type_to_string(v);
|
||||
}
|
||||
|
||||
template <typename E>
|
||||
inline auto format_as(E v) -> decltype(to_string(v)) {
|
||||
return to_string(v);
|
||||
}
|
||||
|
||||
} // namespace starrocks
|
||||
|
||||
// Provide a generic format_as for any enum declared under starrocks (and subnamespaces)
|
||||
// so that fmt can format enums via their underlying integer type without
|
||||
// needing per-enum formatters. This avoids touching core code.
|
||||
namespace starrocks {
|
||||
template <typename E, std::enable_if_t<std::is_enum_v<E>, int> = 0>
|
||||
inline auto format_as(E e) -> std::underlying_type_t<E> {
|
||||
using U = std::underlying_type_t<E>;
|
||||
return static_cast<U>(e);
|
||||
}
|
||||
} // namespace starrocks
|
||||
|
||||
namespace starrocks::pipeline {
|
||||
template <typename E, std::enable_if_t<std::is_enum_v<E>, int> = 0>
|
||||
inline auto format_as(E e) -> std::underlying_type_t<E> {
|
||||
using U = std::underlying_type_t<E>;
|
||||
return static_cast<U>(e);
|
||||
}
|
||||
} // namespace starrocks::pipeline
|
||||
|
||||
// libc++ on macOS has stricter template deduction for std::max than libstdc++.
|
||||
// Some StarRocks call sites pass a mix of long and long long which fails to deduce.
|
||||
// Provide interop overloads to resolve those calls without touching core sources.
|
||||
#include <algorithm>
|
||||
namespace std {
|
||||
inline constexpr long long max(long a, long long b) {
|
||||
return std::max<long long>(static_cast<long long>(a), b);
|
||||
}
|
||||
inline constexpr long long max(long long a, long b) {
|
||||
return std::max<long long>(a, static_cast<long long>(b));
|
||||
}
|
||||
// Provide std::abs overload for extended integer __int128 (missing in libc++)
|
||||
inline __int128 abs(__int128 v) { return v < 0 ? -v : v; }
|
||||
} // namespace std
|
||||
|
||||
// Provide fmt formatter for std::atomic<T> by formatting the loaded value.
|
||||
template <typename T>
|
||||
struct fmt::formatter<std::atomic<T>> : fmt::formatter<T> {
|
||||
template <typename FormatContext>
|
||||
auto format(const std::atomic<T>& v, FormatContext& ctx) const {
|
||||
return fmt::formatter<T>::format(v.load(), ctx);
|
||||
}
|
||||
};
|
||||
|
||||
// Generic fallback: format any enum as its underlying integer when
|
||||
// there is no dedicated formatter. This avoids errors like
|
||||
// "type_is_unformattable_for<Enum, char>" on macOS' libc++.
|
||||
// Note: We intentionally avoid specializing fmt::formatter for all enums here
|
||||
// because fmt::formatter has only two template parameters in most fmt versions.
|
||||
// The format_as() overloads above are sufficient and safer.
|
||||
|
||||
// However, some external enums (not found by ADL or older fmt versions)
|
||||
// still fail. Provide narrow specializations for the ones used with fmt::format.
|
||||
namespace fmt {
|
||||
template <>
|
||||
struct formatter<starrocks::TFileType::type> : formatter<int> {
|
||||
template <typename FormatContext>
|
||||
auto format(starrocks::TFileType::type v, FormatContext& ctx) const {
|
||||
return formatter<int>::format(static_cast<int>(v), ctx);
|
||||
}
|
||||
};
|
||||
|
||||
template <>
|
||||
struct formatter<starrocks::StreamSourceType::type> : formatter<int> {
|
||||
template <typename FormatContext>
|
||||
auto format(starrocks::StreamSourceType::type v, FormatContext& ctx) const {
|
||||
return formatter<int>::format(static_cast<int>(v), ctx);
|
||||
}
|
||||
};
|
||||
|
||||
// Formatter for thrift enum used in ArrowFunctionCall
|
||||
template <>
|
||||
struct formatter<starrocks::TFunctionBinaryType::type> : formatter<int> {
|
||||
template <typename FormatContext>
|
||||
auto format(starrocks::TFunctionBinaryType::type v, FormatContext& ctx) const {
|
||||
return formatter<int>::format(static_cast<int>(v), ctx);
|
||||
}
|
||||
};
|
||||
|
||||
// Formatter for TExprNodeType thrift enum used in expr.cpp
|
||||
template <>
|
||||
struct formatter<starrocks::TExprNodeType::type> : formatter<int> {
|
||||
template <typename FormatContext>
|
||||
auto format(starrocks::TExprNodeType::type v, FormatContext& ctx) const {
|
||||
return formatter<int>::format(static_cast<int>(v), ctx);
|
||||
}
|
||||
};
|
||||
|
||||
} // namespace fmt
|
||||
|
||||
|
||||
|
||||
|
||||
// Provide a minimal fallback for boost::algorithm::to_lower_copy used in one
|
||||
// code path without pulling the entire Boost algorithm headers which cause
|
||||
// template conflicts on macOS with certain containers.
|
||||
namespace boost { namespace algorithm {
|
||||
inline std::string to_lower_copy(const std::string& s) {
|
||||
std::string r = s;
|
||||
std::transform(r.begin(), r.end(), r.begin(), [](unsigned char c) { return static_cast<char>(std::tolower(c)); });
|
||||
return r;
|
||||
}
|
||||
}} // namespace boost::algorithm
|
||||
|
||||
// Fix for boost::algorithm::join issues on macOS
|
||||
// The issue is that boost::algorithm::join cannot handle Buffer<uint8_t> properly
|
||||
// We override the problematic functions to work with basic types
|
||||
|
||||
namespace boost { namespace algorithm {
|
||||
// Override boost::algorithm::join only for non-string sequences to avoid template issues
|
||||
template <typename Sequence, typename Separator>
|
||||
auto join(const Sequence& sequence, const Separator& separator)
|
||||
-> std::enable_if_t<!std::is_same_v<typename Sequence::value_type, std::string>, std::string> {
|
||||
std::string result;
|
||||
auto it = sequence.begin();
|
||||
auto end = sequence.end();
|
||||
|
||||
if (it != end) {
|
||||
result += std::to_string(*it);
|
||||
++it;
|
||||
}
|
||||
|
||||
for (; it != end; ++it) {
|
||||
result += separator;
|
||||
result += std::to_string(*it);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
}} // namespace boost::algorithm
|
||||
|
|
@ -277,7 +277,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### enable_datacache_mem_auto_adjust
|
||||
##### enable_auto_adjust_pagecache
|
||||
|
||||
- Default: true
|
||||
- Type: Boolean
|
||||
|
|
@ -310,7 +310,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_mem_adjust_period
|
||||
##### pagecache_adjust_period
|
||||
|
||||
- Default: 20
|
||||
- Type: Int
|
||||
|
|
@ -321,7 +321,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_mem_adjust_interval_seconds
|
||||
##### auto_adjust_pagecache_interval_seconds
|
||||
|
||||
- Default: 10
|
||||
- Type: Int
|
||||
|
|
@ -1271,7 +1271,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
|
||||
##### enable_pk_parallel_execution
|
||||
|
||||
- Default: true
|
||||
- Default: false
|
||||
- Type: Boolean
|
||||
- Unit: -
|
||||
- Is mutable: Yes
|
||||
|
|
@ -4070,40 +4070,40 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
- Description: The maximum amount of data that can be cached on a single disk. You can set it as a percentage (for example, `80%`) or a physical limit (for example, `2T`, `500G`). For example, if you use two disks and set the value of the `datacache_disk_size` parameter as `21474836480` (20 GB), a maximum of 40 GB data can be cached on these two disks. The default value is `0`, which indicates that only memory is used to cache data.
|
||||
- Introduced in: -
|
||||
|
||||
##### enable_datacache_disk_auto_adjust
|
||||
##### datacache_auto_adjust_enable
|
||||
|
||||
- Default: true
|
||||
- Default: false
|
||||
- Type: Boolean
|
||||
- Unit: -
|
||||
- Is mutable: Yes
|
||||
- Description: Whether to enable Automatic Scaling for Data Cache disk capacity. When it is enabled, the system dynamically adjusts the cache capacity based on the current disk usage rate. This item is renamed from `datacache_auto_adjust_enable` to `enable_datacache_disk_auto_adjust` from v4.0 onwards.
|
||||
- Description: Whether to enable Automatic Scaling for Data Cache disk capacity. When it is enabled, the system dynamically adjusts the cache capacity based on the current disk usage rate.
|
||||
- Introduced in: v3.3.0
|
||||
|
||||
##### disk_high_level
|
||||
##### datacache_disk_high_level
|
||||
|
||||
- Default: 90
|
||||
- Type: Int
|
||||
- Unit: -
|
||||
- Is mutable: Yes
|
||||
- Description: The upper limit of disk usage (in percentage) that triggers the automatic scaling up of the cache capacity. When the disk usage exceeds this value, the system automatically evicts cache data from the Data Cache. From v3.4.0 onwards, the default value is changed from `80` to `90`. This item is renamed from `datacache_disk_high_level` to `disk_high_level` from v4.0 onwards.
|
||||
- Description: The upper limit of disk usage (in percentage) that triggers the automatic scaling up of the cache capacity. When the disk usage exceeds this value, the system automatically evicts cache data from the Data Cache. From v3.4.0 onwards, the default value is changed from `80` to `90`.
|
||||
- Introduced in: v3.3.0
|
||||
|
||||
##### disk_safe_level
|
||||
##### datacache_disk_safe_level
|
||||
|
||||
- Default: 80
|
||||
- Type: Int
|
||||
- Unit: -
|
||||
- Is mutable: Yes
|
||||
- Description: The safe level of disk usage (in percentage) for Data Cache. When Data Cache performs automatic scaling, the system adjusts the cache capacity with the goal of maintaining disk usage as close to this value as possible. From v3.4.0 onwards, the default value is changed from `70` to `80`. This item is renamed from `datacache_disk_safe_level` to `disk_safe_level` from v4.0 onwards.
|
||||
- Description: The safe level of disk usage (in percentage) for Data Cache. When Data Cache performs automatic scaling, the system adjusts the cache capacity with the goal of maintaining disk usage as close to this value as possible. From v3.4.0 onwards, the default value is changed from `70` to `80`.
|
||||
- Introduced in: v3.3.0
|
||||
|
||||
##### disk_low_level
|
||||
##### datacache_disk_low_level
|
||||
|
||||
- Default: 60
|
||||
- Type: Int
|
||||
- Unit: -
|
||||
- Is mutable: Yes
|
||||
- Description: The lower limit of disk usage (in percentage) that triggers the automatic scaling down of the cache capacity. When the disk usage remains below this value for the period specified in `datacache_disk_idle_seconds_for_expansion`, and the space allocated for Data Cache is fully utilized, the system will automatically expand the cache capacity by increasing the upper limit. This item is renamed from `datacache_disk_low_level` to `disk_low_level` from v4.0 onwards.
|
||||
- Description: The lower limit of disk usage (in percentage) that triggers the automatic scaling down of the cache capacity. When the disk usage remains below this value for the period specified in `datacache_disk_idle_seconds_for_expansion`, and the space allocated for Data Cache is fully utilized, the system will automatically expand the cache capacity by increasing the upper limit.
|
||||
- Introduced in: v3.3.0
|
||||
|
||||
##### datacache_disk_adjust_interval_seconds
|
||||
|
|
@ -4142,6 +4142,15 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
- Description: Whether to enable Block Buffer to optimize Data Cache efficiency. When Block Buffer is enabled, the system reads the Block data from the Data Cache and caches it in a temporary buffer, thus reducing the extra overhead caused by frequent cache reads.
|
||||
- Introduced in: v3.2.0
|
||||
|
||||
##### datacache_tiered_cache_enable
|
||||
|
||||
- Default: false
|
||||
- Type: Boolean
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
- Description: Whether to enable tiered cache mode for Data Cache. When tiered cache mode is enabled, Data Cache is configured with two layers of caching, memory and disk. When disk data becomes hot data, it is automatically loaded into the memory cache, and when the data in the memory cache becomes cold, it is automatically flushed to disk. When tiered cache mode is not enabled, the memory and disk configured for Data Cache form two separate cache spaces and cache different types of data, with no data flow between them.
|
||||
- Introduced in: v3.2.5
|
||||
|
||||
##### datacache_eviction_policy
|
||||
|
||||
- Default: slru
|
||||
|
|
@ -4254,7 +4263,7 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
<!--
|
||||
##### datacache_max_flying_memory_mb
|
||||
|
||||
- Default: 2
|
||||
- Default: 256
|
||||
- Type: Int
|
||||
- Unit: MB
|
||||
- Is mutable: No
|
||||
|
|
@ -4262,6 +4271,17 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_adaptor_enable
|
||||
|
||||
- Default: true
|
||||
- Type: Boolean
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_skip_read_factor
|
||||
|
||||
|
|
@ -4284,6 +4304,17 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_engine
|
||||
|
||||
- Default: Empty string
|
||||
- Type: String
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### report_datacache_metrics_interval_ms
|
||||
|
||||
|
|
@ -4298,7 +4329,7 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
<!--
|
||||
##### block_cache_enable
|
||||
|
||||
- Default: true
|
||||
- Default: false
|
||||
- Type: Boolean
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
|
|
@ -4317,6 +4348,39 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_disk_path
|
||||
|
||||
- Default: `${STARROCKS_HOME}/block_cache/`
|
||||
- Type: String
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_meta_path
|
||||
|
||||
- Default: `${STARROCKS_HOME}/block_cache/`
|
||||
- Type: String
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_block_size
|
||||
|
||||
- Default: 262144
|
||||
- Type: Int
|
||||
- Unit:
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_mem_size
|
||||
|
||||
|
|
@ -4329,7 +4393,18 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_direct_io_enable
|
||||
##### block_cache_max_concurrent_inserts
|
||||
|
||||
- Default: 1500000
|
||||
- Type: Int
|
||||
- Unit:
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_checksum_enable
|
||||
|
||||
- Default: false
|
||||
- Type: Boolean
|
||||
|
|
@ -4339,6 +4414,28 @@ When this value is set to less than `0`, the system uses the product of its abso
|
|||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_direct_io_enable
|
||||
|
||||
- Default: false
|
||||
- Type: Boolean
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_engine
|
||||
|
||||
- Default: Empty string
|
||||
- Type: String
|
||||
- Unit: -
|
||||
- Is mutable: No
|
||||
- Description:
|
||||
- Introduced in: -
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### query_cache_num_lanes_per_driver
|
||||
|
||||
|
|
|
|||
|
|
@ -1077,21 +1077,6 @@ For more information on how to build a monitoring service for your StarRocks clu
|
|||
- Unit: Count
|
||||
- Description: Total number of hits in the storage page cache.
|
||||
|
||||
### page_cache_insert_count
|
||||
|
||||
- Unit: Count
|
||||
- Description: Total number of insert operations in the storage page cache.
|
||||
|
||||
### page_cache_insert_evict_count
|
||||
|
||||
- Unit: Count
|
||||
- Description: Total number of cache entries evicted during insert operations due to capacity constraints.
|
||||
|
||||
### page_cache_release_evict_count
|
||||
|
||||
- Unit: Count
|
||||
- Description: Total number of cache entries evicted during release operations when cache usage exceeds capacity.
|
||||
|
||||
### bytes_read_total (Deprecated)
|
||||
|
||||
### update_rowset_commit_request_total
|
||||
|
|
|
|||
|
|
@ -1315,7 +1315,7 @@ DROP DATABASE <database_name>;
|
|||
|
||||
### Create an Iceberg table
|
||||
|
||||
Similar to the internal databases of StarRocks, if you have the [CREATE TABLE](../../../administration/user_privs/authorization/privilege_item.md#database) privilege on an Iceberg database, you can use the [CREATE TABLE](../../../sql-reference/sql-statements/table_bucket_part_index/CREATE_TABLE.md) or [CREATE TABLE AS SELECT](../../../sql-reference/sql-statements/table_bucket_part_index/CREATE_TABLE_AS_SELECT.md) statement to create a table in that Iceberg database. This feature is supported from v3.1 onwards.
|
||||
Similar to the internal databases of StarRocks, if you have the [CREATE TABLE](../../../administration/user_privs/authorization/privilege_item.md#database) privilege on an Iceberg database, you can use the [CREATE TABLE](../../../sql-reference/sql-statements/table_bucket_part_index/CREATE_TABLE.md) or [CREATE TABLE AS SELECT ../../sql-reference/sql-statements/table_bucket_part_index/CREATE_TABLE_AS_SELECT.mdELECT.md) statement to create a table in that Iceberg database. This feature is supported from v3.1 onwards.
|
||||
|
||||
[Switch to an Iceberg catalog and a database in it](#switch-to-an-iceberg-catalog-and-a-database-in-it), and then use the following syntax to create an Iceberg table in that database.
|
||||
|
||||
|
|
|
|||
|
|
@ -1,37 +0,0 @@
|
|||
---
|
||||
displayed_sidebar: docs
|
||||
---
|
||||
|
||||
# sec_to_time
|
||||
|
||||
## Description
|
||||
|
||||
The SEC_TO_TIME function converts a value in seconds into a TIME type, returning the result in the format HH:MM:SS.
|
||||
The input seconds represent the time elapsed since the start of a day (00:00:00).
|
||||
|
||||
## Syntax
|
||||
|
||||
```Haskell
|
||||
TIME sec_to_time(BIGINT sec)
|
||||
```
|
||||
|
||||
## Parameters
|
||||
|
||||
`sec`: It must be of the INT type.
|
||||
|
||||
## Return value
|
||||
|
||||
Returns a TIME value in the format HH:MM:SS, representing the time calculated from the start of a day (00:00:00).
|
||||
If sec is NULL, the function returns NULL.
|
||||
|
||||
## Examples
|
||||
|
||||
```plain text
|
||||
select sec_to_time(43994);
|
||||
+-----------------------------+
|
||||
| sec_to_time(43994) |
|
||||
+-----------------------------+
|
||||
| '12:13:14'|
|
||||
+-----------------------------+
|
||||
```
|
||||
|
||||
|
|
@ -986,7 +986,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
|
||||
##### enable_pk_parallel_execution
|
||||
|
||||
- デフォルト: true
|
||||
- デフォルト: false
|
||||
- タイプ: Boolean
|
||||
- 単位: -
|
||||
- 可変: はい
|
||||
|
|
@ -1952,40 +1952,40 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 説明: 単一ディスクにキャッシュできるデータの最大量。パーセンテージ (例: `80%`) または物理的な制限 (例: `2T`、`500G`) として設定できます。たとえば、2 つのディスクを使用し、`datacache_disk_size` パラメータの値を `21474836480` (20 GB) に設定した場合、これらの 2 つのディスクに最大 40 GB のデータをキャッシュできます。デフォルト値は `0` で、これはメモリのみがデータをキャッシュするために使用されることを示します。
|
||||
- 導入バージョン: -
|
||||
|
||||
##### enable_datacache_disk_auto_adjust
|
||||
##### datacache_auto_adjust_enable
|
||||
|
||||
- デフォルト: true
|
||||
- デフォルト: false
|
||||
- タイプ: Boolean
|
||||
- 単位: -
|
||||
- 可変: はい
|
||||
- 説明: Data Cache ディスク容量の自動スケーリングを有効にするかどうか。これを有効にすると、システムは現在のディスク使用率に基づいてキャッシュ容量を動的に調整します。この項目はバージョン4.0以降、`datacache_auto_adjust_enable` から `enable_datacache_disk_auto_adjust` に名称変更されました。
|
||||
- 説明: Data Cache ディスク容量の自動スケーリングを有効にするかどうか。これを有効にすると、システムは現在のディスク使用率に基づいてキャッシュ容量を動的に調整します。
|
||||
- 導入バージョン: v3.3.0
|
||||
|
||||
##### disk_high_level
|
||||
##### datacache_disk_high_level
|
||||
|
||||
- デフォルト: 90
|
||||
- タイプ: Int
|
||||
- 単位: -
|
||||
- 可変: はい
|
||||
- 説明: キャッシュ容量の自動スケーリングをトリガーするディスク使用率 (パーセンテージ) の上限。この値を超えると、システムは Data Cache からキャッシュデータを自動的に削除します。v3.4.0 以降、デフォルト値は `80` から `90` に変更されました。この項目はバージョン4.0以降、`datacache_disk_high_level` から `disk_high_level` に名称変更されました。
|
||||
- 説明: キャッシュ容量の自動スケーリングをトリガーするディスク使用率 (パーセンテージ) の上限。この値を超えると、システムは Data Cache からキャッシュデータを自動的に削除します。v3.4.0 以降、デフォルト値は `80` から `90` に変更されました。
|
||||
- 導入バージョン: v3.3.0
|
||||
|
||||
##### disk_safe_level
|
||||
##### datacache_disk_safe_level
|
||||
|
||||
- デフォルト: 80
|
||||
- タイプ: Int
|
||||
- 単位: -
|
||||
- 可変: はい
|
||||
- 説明: Data Cache のディスク使用率 (パーセンテージ) の安全レベル。Data Cache が自動スケーリングを実行する際、システムはディスク使用率をこの値にできるだけ近づけることを目標にキャッシュ容量を調整します。v3.4.0 以降、デフォルト値は `70` から `80` に変更されました。この項目はバージョン4.0以降、`datacache_disk_safe_level` から `disk_safe_level` に名称変更されました。
|
||||
- 説明: Data Cache のディスク使用率 (パーセンテージ) の安全レベル。Data Cache が自動スケーリングを実行する際、システムはディスク使用率をこの値にできるだけ近づけることを目標にキャッシュ容量を調整します。v3.4.0 以降、デフォルト値は `70` から `80` に変更されました。
|
||||
- 導入バージョン: v3.3.0
|
||||
|
||||
##### disk_low_level
|
||||
##### datacache_disk_low_level
|
||||
|
||||
- デフォルト: 60
|
||||
- タイプ: Int
|
||||
- 単位: -
|
||||
- 可変: はい
|
||||
- 説明: キャッシュ容量の自動スケーリングをトリガーするディスク使用率 (パーセンテージ) の下限。ディスク使用率が `datacache_disk_idle_seconds_for_expansion` で指定された期間を超えてこの値を下回り、Data Cache に割り当てられたスペースが完全に利用される場合、システムは上限を増やしてキャッシュ容量を自動的に拡張します。この項目はバージョン4.0以降、`datacache_disk_low_level` から `disk_low_level` に名称変更されました。
|
||||
- 説明: キャッシュ容量の自動スケーリングをトリガーするディスク使用率 (パーセンテージ) の下限。ディスク使用率が `datacache_disk_idle_seconds_for_expansion` で指定された期間を超えてこの値を下回り、Data Cache に割り当てられたスペースが完全に利用される場合、システムは上限を増やしてキャッシュ容量を自動的に拡張します。
|
||||
- 導入バージョン: v3.3.0
|
||||
|
||||
##### datacache_disk_adjust_interval_seconds
|
||||
|
|
@ -2024,6 +2024,15 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 説明: Data Cache の効率を最適化するために Block Buffer を有効にするかどうか。Block Buffer が有効な場合、システムは Data Cache から Block データを読み取り、一時バッファにキャッシュし、頻繁なキャッシュ読み取りによる余分なオーバーヘッドを削減します。
|
||||
- 導入バージョン: v3.2.0
|
||||
|
||||
##### datacache_tiered_cache_enable
|
||||
|
||||
- デフォルト: false
|
||||
- タイプ: Boolean
|
||||
- 単位: -
|
||||
- 可変: いいえ
|
||||
- 説明: Data Cache の階層型キャッシュモードを有効にするかどうか。階層型キャッシュモードが有効な場合、Data Cache はメモリとディスクの 2 層のキャッシュで構成されます。ディスクデータがホットデータになると、自動的にメモリキャッシュにロードされ、メモリキャッシュ内のデータがコールドになると、自動的にディスクにフラッシュされます。階層型キャッシュモードが有効でない場合、Data Cache に設定されたメモリとディスクは 2 つの独立したキャッシュスペースを形成し、異なるタイプのデータをキャッシュし、データの流れはありません。
|
||||
- 導入バージョン: v3.2.5
|
||||
|
||||
##### datacache_eviction_policy
|
||||
|
||||
- デフォルト: slru
|
||||
|
|
|
|||
|
|
@ -1076,21 +1076,6 @@ StarRocks クラスタのモニタリングサービスの構築方法につい
|
|||
- 単位: Count
|
||||
- 説明: ストレージページキャッシュのヒット総数。
|
||||
|
||||
### page_cache_insert_count
|
||||
|
||||
- 単位: Count
|
||||
- 説明: ストレージページキャッシュの挿入操作総数。
|
||||
|
||||
### page_cache_insert_evict_count
|
||||
|
||||
- 単位: Count
|
||||
- 説明: 容量制限により、挿入操作中にエビクトされたキャッシュエントリの総数。
|
||||
|
||||
### page_cache_release_evict_count
|
||||
|
||||
- 単位: Count
|
||||
- 説明: キャッシュ使用量が容量を超えた場合、リリース操作中にエビクトされたキャッシュエントリの総数。
|
||||
|
||||
### bytes_read_total (Deprecated)
|
||||
|
||||
### update_rowset_commit_request_total
|
||||
|
|
|
|||
|
|
@ -264,7 +264,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### enable_datacache_mem_auto_adjust
|
||||
##### enable_auto_adjust_pagecache
|
||||
|
||||
- 默认值:true
|
||||
- 类型:Boolean
|
||||
|
|
@ -297,7 +297,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_mem_adjust_period
|
||||
##### pagecache_adjust_period
|
||||
|
||||
- 默认值:20
|
||||
- 类型:Int
|
||||
|
|
@ -308,7 +308,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_mem_adjust_interval_seconds
|
||||
##### auto_adjust_pagecache_interval_seconds
|
||||
|
||||
- 默认值:10
|
||||
- 类型:Int
|
||||
|
|
@ -1228,7 +1228,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
|
||||
##### enable_pk_parallel_execution
|
||||
|
||||
- 默认值:true
|
||||
- 默认值:false
|
||||
- 类型:Boolean
|
||||
- 单位:-
|
||||
- 是否动态:是
|
||||
|
|
@ -4011,40 +4011,40 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 描述:单个磁盘缓存数据量的上限,可设为比例上限(如 `80%`)或物理上限(如 `2T`, `500G` 等)。假设系统使用了两块磁盘进行缓存,并设置 `datacache_disk_size` 参数值为 `21474836480`,即 20 GB,那么最多可缓存 40 GB 的磁盘数据。默认值为 `0`,即仅使用内存作为缓存介质,不使用磁盘。
|
||||
- 引入版本:-
|
||||
|
||||
##### enable_datacache_disk_auto_adjust
|
||||
##### datacache_auto_adjust_enable
|
||||
|
||||
- 默认值:true
|
||||
- 默认值:false
|
||||
- 类型:Boolean
|
||||
- 单位:-
|
||||
- 是否动态:是
|
||||
- 描述:Data Cache 磁盘容量自动调整开关,启用后会根据当前磁盘使用率动态调整缓存容量。该参数自 v4.0 起由 `datacache_auto_adjust_enable` 更名为 `enable_datacache_disk_auto_adjust`。
|
||||
- 描述:Data Cache 磁盘容量自动调整开关,启用后会根据当前磁盘使用率动态调整缓存容量。
|
||||
- 引入版本:v3.3.0
|
||||
|
||||
##### disk_high_level
|
||||
##### datacache_disk_high_level
|
||||
|
||||
- 默认值:90
|
||||
- 类型:Int
|
||||
- 单位:-
|
||||
- 是否动态:是
|
||||
- 描述:磁盘高水位(百分比)。当磁盘使用率高于该值时,系统自动淘汰 Data Cache 中的缓存数据。自 v3.4.0 起,该参数默认值由 `80` 变更为 `90`。该参数自 v4.0 起由 `datacache_disk_high_level` 更名为 `disk_high_level`。
|
||||
- 描述:Data Cache 磁盘高水位(百分比)。当磁盘使用率高于该值时,系统自动淘汰 Data Cache 中的缓存数据。自 v3.4.0 起,该参数默认值由 `80` 变更为 `90`。
|
||||
- 引入版本:v3.3.0
|
||||
|
||||
##### disk_safe_level
|
||||
##### datacache_disk_safe_level
|
||||
|
||||
- 默认值:80
|
||||
- 类型:Int
|
||||
- 单位:-
|
||||
- 是否动态:是
|
||||
- 描述:磁盘安全水位(百分比)。当 Data Cache 进行缓存自动扩缩容时,系统将尽可能以该阈值为磁盘使用率目标调整缓存容量。自 v3.4.0 起,该参数默认值由 `70` 变更为 `80`。该参数自 v4.0 起由 `datacache_disk_safe_level` 更名为 `disk_safe_level`。
|
||||
- 描述:Data Cache 磁盘安全水位(百分比)。当 Data Cache 进行缓存自动扩缩容时,系统将尽可能以该阈值为磁盘使用率目标调整缓存容量。自 v3.4.0 起,该参数默认值由 `70` 变更为 `80`。
|
||||
- 引入版本:v3.3.0
|
||||
|
||||
##### disk_low_level
|
||||
##### datacache_disk_low_level
|
||||
|
||||
- 默认值:60
|
||||
- 类型:Int
|
||||
- 单位:-
|
||||
- 是否动态:是
|
||||
- 描述:磁盘低水位(百分比)。当磁盘使用率在 `datacache_disk_idle_seconds_for_expansion` 指定的时间内持续低于该值,且用于缓存数据的空间已经写满时,系统将自动进行缓存扩容,增加缓存上限。该参数自 v4.0 起由 `datacache_disk_low_level` 更名为 `disk_low_level`。
|
||||
- 描述:Data Cache 磁盘低水位(百分比)。当磁盘使用率在 `datacache_disk_idle_seconds_for_expansion` 指定的时间内持续低于该值,且用于缓存数据的空间已经写满时,系统将自动进行缓存扩容,增加缓存上限。
|
||||
- 引入版本:v3.3.0
|
||||
|
||||
##### datacache_disk_adjust_interval_seconds
|
||||
|
|
@ -4083,6 +4083,15 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 描述:是否启用 Block Buffer 优化 Data Cache 效率。当启用 Block Buffer 时,系统会从 Data Cache 中读取完整的 Block 数据并缓存在临时 Buffer 中,从而减少频繁读取缓存带来的额外开销。
|
||||
- 引入版本:v3.2.0
|
||||
|
||||
##### datacache_tiered_cache_enable
|
||||
|
||||
- 默认值:false
|
||||
- 类型:Boolean
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
- 描述:是否为 Data Cache 启用分层模式。当启用分层模式时,Data Cache 配置的的内存和磁盘构成两级缓存,磁盘数据变为热数据时会自动载入到内存缓存,内存缓存中的数据变冷时自动落至磁盘。当不启用分层模式时,为 Data Cache 配置的内存和磁盘构成两个独立的缓存空间,并分别缓存不同类型数据,两者之间不进行数据流动。
|
||||
- 引入版本:v3.2.5
|
||||
|
||||
##### datacache_eviction_policy
|
||||
|
||||
- 默认值:slru
|
||||
|
|
@ -4195,7 +4204,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
<!--
|
||||
##### datacache_max_flying_memory_mb
|
||||
|
||||
- 默认值:2
|
||||
- 默认值:256
|
||||
- 类型:Int
|
||||
- 单位:MB
|
||||
- 是否动态:否
|
||||
|
|
@ -4203,6 +4212,17 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_adaptor_enable
|
||||
|
||||
- 默认值:true
|
||||
- 类型:Boolean
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_skip_read_factor
|
||||
|
||||
|
|
@ -4225,6 +4245,17 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_engine
|
||||
|
||||
- 默认值:空字符串
|
||||
- 类型:String
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### report_datacache_metrics_interval_ms
|
||||
|
||||
|
|
@ -4239,7 +4270,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
<!--
|
||||
##### block_cache_enable
|
||||
|
||||
- 默认值:true
|
||||
- 默认值:false
|
||||
- 类型:Boolean
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
|
|
@ -4250,7 +4281,40 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
<!--
|
||||
##### block_cache_disk_size
|
||||
|
||||
- 默认值:-1
|
||||
- 默认值:0
|
||||
- 类型:Int
|
||||
- 单位:
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_disk_path
|
||||
|
||||
- 默认值:`${STARROCKS_HOME}/block_cache/`
|
||||
- 类型:String
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_meta_path
|
||||
|
||||
- 默认值:`${STARROCKS_HOME}/block_cache/`
|
||||
- 类型:String
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_block_size
|
||||
|
||||
- 默认值:262144
|
||||
- 类型:Int
|
||||
- 单位:
|
||||
- 是否动态:否
|
||||
|
|
@ -4261,7 +4325,7 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
<!--
|
||||
##### block_cache_mem_size
|
||||
|
||||
- 默认值:0
|
||||
- 默认值:2147483648
|
||||
- 类型:Int
|
||||
- 单位:
|
||||
- 是否动态:否
|
||||
|
|
@ -4270,7 +4334,18 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
-->
|
||||
|
||||
<!--
|
||||
##### datacache_direct_io_enable
|
||||
##### block_cache_max_concurrent_inserts
|
||||
|
||||
- 默认值:1500000
|
||||
- 类型:Int
|
||||
- 单位:
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_checksum_enable
|
||||
|
||||
- 默认值:false
|
||||
- 类型:Boolean
|
||||
|
|
@ -4280,6 +4355,28 @@ curl http://<BE_IP>:<BE_HTTP_PORT>/varz
|
|||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_direct_io_enable
|
||||
|
||||
- 默认值:false
|
||||
- 类型:Boolean
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### block_cache_engine
|
||||
|
||||
- 默认值:空字符串
|
||||
- 类型:String
|
||||
- 单位:-
|
||||
- 是否动态:否
|
||||
- 描述:
|
||||
- 引入版本:-
|
||||
-->
|
||||
|
||||
<!--
|
||||
##### query_cache_num_lanes_per_driver
|
||||
|
||||
|
|
|
|||
|
|
@ -1076,21 +1076,6 @@ displayed_sidebar: docs
|
|||
- 单位:个
|
||||
- 描述:Storage Page Cache 中的命中总次数。
|
||||
|
||||
### page_cache_insert_count
|
||||
|
||||
- 单位:个
|
||||
- 描述:Storage Page Cache 中的插入操作总次数。
|
||||
|
||||
### page_cache_insert_evict_count
|
||||
|
||||
- 单位:个
|
||||
- 描述:由于容量限制,在插入操作期间被淘汰的缓存项总次数。
|
||||
|
||||
### page_cache_release_evict_count
|
||||
|
||||
- 单位:个
|
||||
- 描述:当缓存使用量超过容量时,在释放操作期间被淘汰的缓存项总次数。
|
||||
|
||||
### bytes_read_total(已弃用)
|
||||
|
||||
### update_rowset_commit_request_total
|
||||
|
|
|
|||
|
|
@ -1,35 +0,0 @@
|
|||
---
|
||||
displayed_sidebar: docs
|
||||
---
|
||||
|
||||
# sec_to_time
|
||||
|
||||
## 功能
|
||||
|
||||
SEC_TO_TIME 函数将一个以秒为单位的值转换为 TIME 类型,返回格式为 HH:MM:SS。输入的秒数表示从一天的起点时间(00:00:00)开始计算的时间。
|
||||
|
||||
## 语法
|
||||
|
||||
```Haskell
|
||||
TIME sec_to_time(BIGINT sec)
|
||||
```
|
||||
|
||||
## 参数说明
|
||||
|
||||
`sec`:必填,输入的秒数,表示从一天起点时间(00:00:00)开始计算的秒数,支持正整数或负整数类型。
|
||||
|
||||
## 返回值说明
|
||||
|
||||
必填,输入的秒数,表示从一天起点时间(00:00:00)开始计算的秒数,支持正整数或负整数类型。
|
||||
如果输入的 sec 为 NULL,函数返回 NULL。
|
||||
|
||||
## 示例
|
||||
|
||||
```plain text
|
||||
select sec_to_time(43994);
|
||||
+-----------------------------+
|
||||
| sec_to_time(43994) |
|
||||
+-----------------------------+
|
||||
| '12:13:14'|
|
||||
+-----------------------------+
|
||||
```
|
||||
|
|
@ -61,7 +61,6 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class FunctionSet {
|
||||
|
|
@ -131,7 +130,6 @@ public class FunctionSet {
|
|||
public static final String QUARTER = "quarter";
|
||||
public static final String TIMESTAMP = "timestamp";
|
||||
public static final String TIME_TO_SEC = "time_to_sec";
|
||||
public static final String SEC_TO_TIME = "sec_to_time";
|
||||
public static final String STR2DATE = "str2date";
|
||||
public static final String MICROSECONDS_ADD = "microseconds_add";
|
||||
public static final String MICROSECONDS_SUB = "microseconds_sub";
|
||||
|
|
@ -395,8 +393,6 @@ public class FunctionSet {
|
|||
// Hash functions:
|
||||
public static final String MURMUR_HASH3_32 = "murmur_hash3_32";
|
||||
public static final String CRC32_HASH = "crc32_hash";
|
||||
public static final String XX_HASH3_64 = "xx_hash3_64";
|
||||
public static final String XX_HASH3_128 = "xx_hash3_128";
|
||||
|
||||
// Percentile functions:
|
||||
public static final String PERCENTILE_APPROX_RAW = "percentile_approx_raw";
|
||||
|
|
@ -1389,7 +1385,12 @@ public class FunctionSet {
|
|||
addBuiltin(AggregateFunction.createBuiltin(COLUMN_COMPRESSED_SIZE, Lists.newArrayList(Type.ANY_ELEMENT),
|
||||
Type.BIGINT, Type.BIGINT, false, false, false));
|
||||
|
||||
Consumer<Type> registerLeadLagFunctions = t -> {
|
||||
for (Type t : Type.getSupportedTypes()) {
|
||||
// null/char/time is handled through type promotion
|
||||
// TODO: array/json/pseudo is not supported yet
|
||||
if (!t.canBeWindowFunctionArgumentTypes()) {
|
||||
continue;
|
||||
}
|
||||
addBuiltin(AggregateFunction.createAnalyticBuiltin(
|
||||
FIRST_VALUE, Lists.newArrayList(t), t, t));
|
||||
// Implements FIRST_VALUE for some windows that require rewrites during planning.
|
||||
|
|
@ -1414,17 +1415,6 @@ public class FunctionSet {
|
|||
LEAD, Lists.newArrayList(t), t, t));
|
||||
addBuiltin(AggregateFunction.createAnalyticBuiltin(
|
||||
LEAD, Lists.newArrayList(t, Type.BIGINT), t, t));
|
||||
};
|
||||
for (Type t : Type.getSupportedTypes()) {
|
||||
// null/char/time is handled through type promotion
|
||||
// TODO: array/json/pseudo is not supported yet
|
||||
if (!t.canBeWindowFunctionArgumentTypes()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
Type arrayType = new ArrayType(t);
|
||||
registerLeadLagFunctions.accept(t);
|
||||
registerLeadLagFunctions.accept(arrayType);
|
||||
}
|
||||
|
||||
for (Type t : HISTOGRAM_TYPE) {
|
||||
|
|
|
|||
|
|
@ -159,16 +159,7 @@ public class MaterializedView extends OlapTable implements GsonPreProcessable, G
|
|||
ADAPTIVE;
|
||||
|
||||
public static PartitionRefreshStrategy defaultValue() {
|
||||
return ADAPTIVE;
|
||||
}
|
||||
|
||||
public static PartitionRefreshStrategy of(String val) {
|
||||
try {
|
||||
return PartitionRefreshStrategy.valueOf(val.trim().toUpperCase());
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Failed to get partition refresh strategy, use default value", e);
|
||||
return PartitionRefreshStrategy.defaultValue();
|
||||
}
|
||||
return STRICT;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1405,25 +1396,15 @@ public class MaterializedView extends OlapTable implements GsonPreProcessable, G
|
|||
* Return the partition refresh strategy of the materialized view.
|
||||
*/
|
||||
public PartitionRefreshStrategy getPartitionRefreshStrategy() {
|
||||
TableProperty tableProperty = getTableProperty();
|
||||
if (tableProperty == null) {
|
||||
if (tableProperty == null || tableProperty.getPartitionRefreshStrategy() == null) {
|
||||
return PartitionRefreshStrategy.defaultValue();
|
||||
}
|
||||
|
||||
// otherwise, use `partition_refresh_strategy` property.
|
||||
PartitionRefreshStrategy partitionRefreshStrategy =
|
||||
PartitionRefreshStrategy.of(tableProperty.getPartitionRefreshStrategy());
|
||||
if (tableProperty.isSetPartitionRefreshStrategy()
|
||||
&& !PartitionRefreshStrategy.STRICT.equals(partitionRefreshStrategy)) {
|
||||
return partitionRefreshStrategy;
|
||||
} else {
|
||||
// if mv has `partition_refresh_number` property, use it first.
|
||||
// for compatibility, if `partition_refresh_number` is set and not equal to 1,
|
||||
if (tableProperty.isSetPartitionRefreshNumber()) {
|
||||
return PartitionRefreshStrategy.STRICT;
|
||||
} else {
|
||||
return PartitionRefreshStrategy.defaultValue();
|
||||
}
|
||||
try {
|
||||
return PartitionRefreshStrategy.valueOf(tableProperty.getPartitionRefreshStrategy().trim().toUpperCase());
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Failed to get partition refresh strategy for materialized view: {}, use default value",
|
||||
this.getName(), e);
|
||||
return PartitionRefreshStrategy.defaultValue();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -1272,10 +1272,6 @@ public class OlapTable extends Table {
|
|||
if (partitionName.startsWith(ExpressionRangePartitionInfo.SHADOW_PARTITION_PREFIX)) {
|
||||
continue;
|
||||
}
|
||||
// ensure partitionName is in nameToPartition
|
||||
if (!nameToPartition.containsKey(partitionName)) {
|
||||
continue;
|
||||
}
|
||||
rangePartitionMap.put(partitionName, rangePartitionInfo.getRange(partitionId));
|
||||
}
|
||||
return rangePartitionMap;
|
||||
|
|
@ -1321,9 +1317,6 @@ public class OlapTable extends Table {
|
|||
if (partitionName.startsWith(ExpressionRangePartitionInfo.SHADOW_PARTITION_PREFIX)) {
|
||||
continue;
|
||||
}
|
||||
if (!nameToPartition.containsKey(partitionName)) {
|
||||
continue;
|
||||
}
|
||||
// one item
|
||||
List<LiteralExpr> literalValues = listPartitionInfo.getLiteralExprValues().get(partitionId);
|
||||
if (CollectionUtils.isNotEmpty(literalValues)) {
|
||||
|
|
|
|||
|
|
@ -209,15 +209,15 @@ public class TableProperty implements Writable, GsonPostProcessable {
|
|||
|
||||
// This property only applies to materialized views
|
||||
// It represents the maximum number of partitions that will be refreshed by a TaskRun refresh
|
||||
private int partitionRefreshNumber = INVALID;
|
||||
private int partitionRefreshNumber = Config.default_mv_partition_refresh_number;
|
||||
|
||||
// This property only applies to materialized views
|
||||
// It represents the mode selected to determine the number of partitions to refresh
|
||||
private String partitionRefreshStrategy = "";
|
||||
private String partitionRefreshStrategy = Config.default_mv_partition_refresh_strategy;
|
||||
|
||||
// This property only applies to materialized views/
|
||||
// It represents the mode selected to determine how to refresh the materialized view
|
||||
private String mvRefreshMode = "";
|
||||
private String mvRefreshMode = Config.default_mv_refresh_mode;
|
||||
|
||||
// This property only applies to materialized views
|
||||
// When using the system to automatically refresh, the maximum range of the most recent partitions will be refreshed.
|
||||
|
|
@ -994,21 +994,12 @@ public class TableProperty implements Writable, GsonPostProcessable {
|
|||
this.autoRefreshPartitionsLimit = autoRefreshPartitionsLimit;
|
||||
}
|
||||
|
||||
public boolean isSetPartitionRefreshNumber() {
|
||||
return partitionRefreshNumber != INVALID;
|
||||
}
|
||||
|
||||
public int getPartitionRefreshNumber() {
|
||||
return partitionRefreshNumber == INVALID ? Config.default_mv_partition_refresh_number : partitionRefreshNumber;
|
||||
}
|
||||
|
||||
public boolean isSetPartitionRefreshStrategy() {
|
||||
return !Strings.isNullOrEmpty(partitionRefreshStrategy);
|
||||
return partitionRefreshNumber;
|
||||
}
|
||||
|
||||
public String getPartitionRefreshStrategy() {
|
||||
return Strings.isNullOrEmpty(partitionRefreshStrategy) ? Config.default_mv_partition_refresh_strategy
|
||||
: partitionRefreshStrategy;
|
||||
return partitionRefreshStrategy;
|
||||
}
|
||||
|
||||
public void setPartitionRefreshNumber(int partitionRefreshNumber) {
|
||||
|
|
@ -1024,7 +1015,7 @@ public class TableProperty implements Writable, GsonPostProcessable {
|
|||
}
|
||||
|
||||
public String getMvRefreshMode() {
|
||||
return Strings.isNullOrEmpty(mvRefreshMode) ? Config.default_mv_refresh_mode : mvRefreshMode;
|
||||
return mvRefreshMode;
|
||||
}
|
||||
|
||||
public void setResourceGroup(String resourceGroup) {
|
||||
|
|
|
|||
|
|
@ -414,9 +414,6 @@ public class Config extends ConfigBase {
|
|||
@ConfField(mutable = true, comment = "task run ttl")
|
||||
public static int task_runs_ttl_second = 7 * 24 * 3600; // 7 day
|
||||
|
||||
@ConfField(mutable = true, comment = "task run execute timeout, default 4 hours")
|
||||
public static int task_runs_timeout_second = 4 * 3600; // 4 hour
|
||||
|
||||
@ConfField(mutable = true, comment = "max number of task run history. ")
|
||||
public static int task_runs_max_history_number = 10000;
|
||||
|
||||
|
|
@ -3396,8 +3393,8 @@ public class Config extends ConfigBase {
|
|||
@ConfField(mutable = true, comment = "The default try lock timeout for mv refresh to try base table/mv dbs' lock")
|
||||
public static int mv_refresh_try_lock_timeout_ms = 30 * 1000;
|
||||
|
||||
@ConfField(mutable = true, comment = "materialized view can refresh at most 64 partition at a time")
|
||||
public static int mv_max_partitions_num_per_refresh = 64;
|
||||
@ConfField(mutable = true, comment = "materialized view can refresh at most 10 partition at a time")
|
||||
public static int mv_max_partitions_num_per_refresh = 10;
|
||||
|
||||
@ConfField(mutable = true, comment = "materialized view can refresh at most 100_000_000 rows of data at a time")
|
||||
public static long mv_max_rows_per_refresh = 100_000_000L;
|
||||
|
|
@ -3428,9 +3425,8 @@ public class Config extends ConfigBase {
|
|||
@ConfField(mutable = true)
|
||||
public static int default_mv_partition_refresh_number = 1;
|
||||
|
||||
@ConfField(mutable = true, comment = "The default refresh strategy for materialized view partition refresh, " +
|
||||
"adaptive by default")
|
||||
public static String default_mv_partition_refresh_strategy = "adaptive";
|
||||
@ConfField(mutable = true)
|
||||
public static String default_mv_partition_refresh_strategy = "strict";
|
||||
|
||||
@ConfField(mutable = true)
|
||||
public static String default_mv_refresh_mode = "pct";
|
||||
|
|
|
|||
|
|
@ -133,6 +133,10 @@ public class IcebergCatalogProperties {
|
|||
return catalogType;
|
||||
}
|
||||
|
||||
public boolean enableIcebergMetadataCache() {
|
||||
return enableIcebergMetadataCache;
|
||||
}
|
||||
|
||||
public long getIcebergMetaCacheTtlSec() {
|
||||
return icebergMetaCacheTtlSec;
|
||||
}
|
||||
|
|
@ -141,6 +145,7 @@ public class IcebergCatalogProperties {
|
|||
return icebergTableCacheRefreshIntervalSec;
|
||||
}
|
||||
|
||||
|
||||
public int getIcebergJobPlanningThreadNum() {
|
||||
return icebergJobPlanningThreadNum;
|
||||
}
|
||||
|
|
|
|||
|
|
@ -111,7 +111,7 @@ public class IcebergConnector implements Connector {
|
|||
if (icebergNativeCatalog == null) {
|
||||
IcebergCatalog nativeCatalog = buildIcebergNativeCatalog();
|
||||
|
||||
if (icebergCatalogProperties.isEnableIcebergMetadataCache() && !isResourceMappingCatalog(catalogName)) {
|
||||
if (icebergCatalogProperties.enableIcebergMetadataCache() && !isResourceMappingCatalog(catalogName)) {
|
||||
nativeCatalog = new CachingIcebergCatalog(catalogName, nativeCatalog,
|
||||
icebergCatalogProperties, buildBackgroundJobPlanningExecutor());
|
||||
GlobalStateMgr.getCurrentState().getConnectorTableMetadataProcessor()
|
||||
|
|
@ -161,7 +161,7 @@ public class IcebergConnector implements Connector {
|
|||
|
||||
@Override
|
||||
public boolean supportMemoryTrack() {
|
||||
return icebergCatalogProperties.isEnableIcebergMetadataCache() && icebergNativeCatalog != null;
|
||||
return icebergCatalogProperties.enableIcebergMetadataCache() && icebergNativeCatalog != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -32,7 +32,6 @@ import com.starrocks.system.SystemInfoService;
|
|||
import com.starrocks.thrift.TUniqueId;
|
||||
import com.starrocks.warehouse.Warehouse;
|
||||
import com.starrocks.warehouse.cngroup.ComputeResource;
|
||||
import com.starrocks.warehouse.cngroup.ComputeResourceProvider;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
|
|
@ -43,27 +42,19 @@ public class DataCacheSelectExecutor {
|
|||
private static final Logger LOG = LogManager.getLogger(DataCacheSelectExecutor.class);
|
||||
|
||||
public static DataCacheSelectMetrics cacheSelect(DataCacheSelectStatement statement,
|
||||
ConnectContext connectContext) throws Exception {
|
||||
ConnectContext connectContext) throws Exception {
|
||||
InsertStmt insertStmt = statement.getInsertStmt();
|
||||
|
||||
final WarehouseManager warehouseManager = GlobalStateMgr.getCurrentState().getWarehouseMgr();
|
||||
final Warehouse wh = warehouseManager.getWarehouse(connectContext.getCurrentWarehouseName());
|
||||
final ComputeResourceProvider computeResourceProvider = warehouseManager.getComputeResourceProvider();
|
||||
final List<ComputeResource> computeResources = computeResourceProvider.getComputeResources(wh);
|
||||
|
||||
List<StmtExecutor> subStmtExecutors = Lists.newArrayList();
|
||||
boolean isFirstSubContext = true;
|
||||
for (ComputeResource computeResource : computeResources) {
|
||||
if (!computeResourceProvider.isResourceAvailable(computeResource)) {
|
||||
// skip if this compute resource is not available
|
||||
LOG.warn("skip cache select for compute resource {} because it is not available", computeResource);
|
||||
continue;
|
||||
}
|
||||
|
||||
ConnectContext subContext = buildCacheSelectConnectContext(statement, connectContext, isFirstSubContext);
|
||||
for (int workerGroupIdx = 0; workerGroupIdx < wh.getWorkerGroupIds().size(); workerGroupIdx++) {
|
||||
long workerGroupId = wh.getWorkerGroupIds().get(workerGroupIdx);
|
||||
ConnectContext subContext = buildCacheSelectConnectContext(statement, connectContext, workerGroupIdx == 0);
|
||||
ComputeResource computeResource = warehouseManager.getComputeResourceProvider().ofComputeResource(
|
||||
wh.getId(), workerGroupId);
|
||||
subContext.setCurrentComputeResource(computeResource);
|
||||
StmtExecutor subStmtExecutor = StmtExecutor.newInternalExecutor(subContext, insertStmt);
|
||||
isFirstSubContext = false;
|
||||
// Register new StmtExecutor into current ConnectContext's StmtExecutor, so we can handle ctrl+c command
|
||||
// If DataCacheSelect is forward to leader, connectContext's Executor is null
|
||||
if (connectContext.getExecutor() != null) {
|
||||
|
|
|
|||
|
|
@ -400,13 +400,11 @@ public class LoadMgr implements MemoryTrackable {
|
|||
|
||||
// 2. remove from dbIdToLabelToLoadJobs
|
||||
Map<String, List<LoadJob>> labelToLoadJobs = dbIdToLabelToLoadJobs.get(dbId);
|
||||
if (labelToLoadJobs != null) {
|
||||
List<LoadJob> sameLabelJobs = labelToLoadJobs.get(label);
|
||||
if (sameLabelJobs != null) {
|
||||
sameLabelJobs.remove(job);
|
||||
if (sameLabelJobs.isEmpty()) {
|
||||
labelToLoadJobs.remove(label);
|
||||
}
|
||||
List<LoadJob> sameLabelJobs = labelToLoadJobs.get(label);
|
||||
if (sameLabelJobs != null) {
|
||||
sameLabelJobs.remove(job);
|
||||
if (sameLabelJobs.isEmpty()) {
|
||||
labelToLoadJobs.remove(label);
|
||||
}
|
||||
if (labelToLoadJobs.isEmpty()) {
|
||||
dbIdToLabelToLoadJobs.remove(dbId);
|
||||
|
|
|
|||
|
|
@ -315,8 +315,7 @@ public class MVPartitionExprResolver {
|
|||
if (item instanceof SlotRef) {
|
||||
SlotRef result = (SlotRef) item;
|
||||
TableName actTableName = result.getTblNameWithoutAnalyzed();
|
||||
if (result.getColumnName() != null
|
||||
&& result.getColumnName().equalsIgnoreCase(slot.getColumnName())
|
||||
if (result.getColumnName().equalsIgnoreCase(slot.getColumnName())
|
||||
&& (expTableName == null || expTableName.equals(actTableName))) {
|
||||
return visitExpr(context.withExpr(result).withRelation(relation));
|
||||
}
|
||||
|
|
@ -338,7 +337,7 @@ public class MVPartitionExprResolver {
|
|||
SlotRef slot = context.getSlotRef();
|
||||
if (slot.getTblNameWithoutAnalyzed() != null) {
|
||||
String tableName = slot.getTblNameWithoutAnalyzed().getTbl();
|
||||
if (node.getAlias() != null && !node.getAlias().getTbl().equalsIgnoreCase(tableName)) {
|
||||
if (!node.getAlias().getTbl().equalsIgnoreCase(tableName)) {
|
||||
return null;
|
||||
}
|
||||
slot = (SlotRef) slot.clone();
|
||||
|
|
@ -519,7 +518,7 @@ public class MVPartitionExprResolver {
|
|||
if (slot.getTblNameWithoutAnalyzed() != null) {
|
||||
String tableName = slot.getTblNameWithoutAnalyzed().getTbl();
|
||||
String cteName = node.getAlias() != null ? node.getAlias().getTbl() : node.getName();
|
||||
if (cteName != null && !cteName.equalsIgnoreCase(tableName)) {
|
||||
if (!cteName.equalsIgnoreCase(tableName)) {
|
||||
return null;
|
||||
}
|
||||
slot = (SlotRef) slot.clone();
|
||||
|
|
|
|||
|
|
@ -187,13 +187,6 @@ public class MVTaskRunProcessor extends BaseTaskRunProcessor implements MVRefres
|
|||
Preconditions.checkState(mv != null);
|
||||
mvMetricsEntity = MaterializedViewMetricsRegistry.getInstance().getMetricsEntity(mv.getMvId());
|
||||
this.taskRunState = retryProcessTaskRun(context);
|
||||
if (this.taskRunState == Constants.TaskRunState.SUCCESS) {
|
||||
logger.info("Refresh materialized view {} finished successfully.", mv.getName());
|
||||
// if success, try to generate next task run
|
||||
mvRefreshProcessor.generateNextTaskRunIfNeeded();
|
||||
} else {
|
||||
logger.warn("Refresh materialized view {} failed with state: {}.", mv.getName(), taskRunState);
|
||||
}
|
||||
// update metrics
|
||||
mvMetricsEntity.increaseRefreshJobStatus(taskRunState);
|
||||
connectContext.getState().setOk();
|
||||
|
|
|
|||
|
|
@ -965,30 +965,7 @@ public class TaskManager implements MemoryTrackable {
|
|||
}
|
||||
|
||||
public void removeExpiredTaskRuns(boolean archiveHistory) {
|
||||
// remove expired task run records
|
||||
taskRunManager.getTaskRunHistory().vacuum(archiveHistory);
|
||||
|
||||
// cancel long-running task runs to avoid resource waste
|
||||
long currentTimeMs = System.currentTimeMillis();
|
||||
Set<TaskRun> runningTaskRuns = taskRunManager.getTaskRunScheduler().getCopiedRunningTaskRuns();
|
||||
for (TaskRun taskRun : runningTaskRuns) {
|
||||
int taskRunTimeout = taskRun.getExecuteTimeoutS();
|
||||
if (taskRunTimeout <= 0) {
|
||||
continue;
|
||||
}
|
||||
if (taskRun.getStatus() == null) {
|
||||
continue;
|
||||
}
|
||||
TaskRunStatus taskRunStatus = taskRun.getStatus();
|
||||
long taskRunCreatedTime = taskRunStatus.getCreateTime();
|
||||
if (currentTimeMs - taskRunCreatedTime < taskRunTimeout * 1000L) {
|
||||
continue;
|
||||
}
|
||||
// if the task run has been running for a long time, cancel it directly
|
||||
LOG.warn("task run [{}] has been running for a long time, cancel it," +
|
||||
" created(ms):{}, timeout(s):{}", taskRun, taskRunCreatedTime, taskRunTimeout);
|
||||
taskRunManager.killRunningTaskRun(taskRun, true);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -39,7 +39,6 @@ import com.starrocks.common.util.UUIDUtil;
|
|||
import com.starrocks.load.loadv2.InsertLoadJob;
|
||||
import com.starrocks.qe.ConnectContext;
|
||||
import com.starrocks.qe.QueryState;
|
||||
import com.starrocks.qe.SessionVariable;
|
||||
import com.starrocks.qe.StmtExecutor;
|
||||
import com.starrocks.scheduler.persist.TaskRunStatus;
|
||||
import com.starrocks.server.GlobalStateMgr;
|
||||
|
|
@ -197,36 +196,6 @@ public class TaskRun implements Comparable<TaskRun> {
|
|||
return isKilled;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the execute timeout in seconds.
|
||||
*/
|
||||
public int getExecuteTimeoutS() {
|
||||
// if `query_timeout`/`insert_timeout` is set in the execute option, use it
|
||||
int defaultTimeoutS = Config.task_runs_timeout_second;
|
||||
if (properties != null) {
|
||||
for (Map.Entry<String, String> entry : properties.entrySet()) {
|
||||
if (entry.getKey().equalsIgnoreCase(SessionVariable.QUERY_TIMEOUT)
|
||||
|| entry.getKey().equalsIgnoreCase(SessionVariable.INSERT_TIMEOUT)) {
|
||||
try {
|
||||
int timeout = Integer.parseInt(entry.getValue());
|
||||
if (timeout > 0) {
|
||||
defaultTimeoutS = Math.max(timeout, defaultTimeoutS);
|
||||
}
|
||||
} catch (NumberFormatException e) {
|
||||
LOG.warn("invalid timeout value: {}, task run:{}", entry.getValue(), this);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// The timeout of task run should not be longer than the ttl of task runs and task
|
||||
return Math.min(Math.min(defaultTimeoutS, Config.task_runs_ttl_second), Config.task_ttl_second);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setStatus(TaskRunStatus status) {
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
public Map<String, String> refreshTaskProperties(ConnectContext ctx) {
|
||||
Map<String, String> newProperties = Maps.newHashMap();
|
||||
if (task.getSource() != Constants.TaskSource.MV) {
|
||||
|
|
|
|||
|
|
@ -28,7 +28,6 @@ import com.starrocks.scheduler.history.TaskRunHistory;
|
|||
import com.starrocks.scheduler.persist.TaskRunStatus;
|
||||
import com.starrocks.scheduler.persist.TaskRunStatusChange;
|
||||
import com.starrocks.server.GlobalStateMgr;
|
||||
import org.apache.arrow.util.VisibleForTesting;
|
||||
import org.apache.commons.collections4.CollectionUtils;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
|
@ -59,11 +58,6 @@ public class TaskRunManager implements MemoryTrackable {
|
|||
this.taskRunScheduler = taskRunScheduler;
|
||||
}
|
||||
|
||||
public SubmitResult submitTaskRun(TaskRun taskRun) {
|
||||
return submitTaskRun(taskRun, taskRun.getExecuteOption());
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public SubmitResult submitTaskRun(TaskRun taskRun, ExecuteOption option) {
|
||||
LOG.info("submit task run:{}", taskRun);
|
||||
|
||||
|
|
@ -93,11 +87,8 @@ public class TaskRunManager implements MemoryTrackable {
|
|||
return new SubmitResult(queryId, SubmitResult.SubmitStatus.SUBMITTED, taskRun.getFuture());
|
||||
}
|
||||
|
||||
/**
|
||||
* Kill the running task run. If force is true, it will always clear the task run from task run scheduler whether it's
|
||||
* canceled or not so can trigger the pending task runs as soon as possible.
|
||||
*/
|
||||
public boolean killRunningTaskRun(TaskRun taskRun, boolean force) {
|
||||
public boolean killTaskRun(Long taskId, boolean force) {
|
||||
TaskRun taskRun = taskRunScheduler.getRunningTaskRun(taskId);
|
||||
if (taskRun == null) {
|
||||
return false;
|
||||
}
|
||||
|
|
@ -110,6 +101,15 @@ public class TaskRunManager implements MemoryTrackable {
|
|||
if (future != null && !future.completeExceptionally(new RuntimeException("TaskRun killed"))) {
|
||||
LOG.warn("failed to complete future for task run: {}", taskRun);
|
||||
}
|
||||
|
||||
// mark pending tasks as failed
|
||||
Set<TaskRun> pendingTaskRuns = taskRunScheduler.getPendingTaskRunsByTaskId(taskId);
|
||||
if (CollectionUtils.isNotEmpty(pendingTaskRuns)) {
|
||||
for (TaskRun pendingTaskRun : pendingTaskRuns) {
|
||||
taskRunScheduler.removePendingTaskRun(pendingTaskRun, Constants.TaskRunState.FAILED);
|
||||
}
|
||||
}
|
||||
|
||||
// kill the task run
|
||||
ConnectContext runCtx = taskRun.getRunCtx();
|
||||
if (runCtx != null) {
|
||||
|
|
@ -126,30 +126,6 @@ public class TaskRunManager implements MemoryTrackable {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Kill all pending task runs of the input task id.
|
||||
*/
|
||||
public void killPendingTaskRuns(Long taskId) {
|
||||
// mark pending tasks as failed
|
||||
Set<TaskRun> pendingTaskRuns = taskRunScheduler.getPendingTaskRunsByTaskId(taskId);
|
||||
if (CollectionUtils.isNotEmpty(pendingTaskRuns)) {
|
||||
for (TaskRun pendingTaskRun : pendingTaskRuns) {
|
||||
taskRunScheduler.removePendingTaskRun(pendingTaskRun, Constants.TaskRunState.FAILED);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public boolean killTaskRun(Long taskId, boolean force) {
|
||||
// kill all pending task runs of the task id
|
||||
killPendingTaskRuns(taskId);
|
||||
// kill the running task run
|
||||
TaskRun taskRun = taskRunScheduler.getRunningTaskRun(taskId);
|
||||
if (taskRun == null) {
|
||||
return false;
|
||||
}
|
||||
return killRunningTaskRun(taskRun, force);
|
||||
}
|
||||
|
||||
// At present, only the manual and automatic tasks of the materialized view have different priorities.
|
||||
// The manual priority is higher. For manual tasks, we do not merge operations.
|
||||
// For automatic tasks, we will compare the definition, and if they are the same,
|
||||
|
|
|
|||
|
|
@ -169,11 +169,6 @@ public abstract class BaseMVRefreshProcessor {
|
|||
protected abstract BaseTableSnapshotInfo buildBaseTableSnapshotInfo(BaseTableInfo baseTableInfo,
|
||||
Table table);
|
||||
|
||||
/**
|
||||
* Generate the next task run to be processed and set it to the nextTaskRun field.
|
||||
*/
|
||||
public abstract void generateNextTaskRunIfNeeded();
|
||||
|
||||
/**
|
||||
* Get the retry times for the mv refresh processor.
|
||||
*
|
||||
|
|
@ -360,11 +355,9 @@ public abstract class BaseMVRefreshProcessor {
|
|||
String partitionName = toRefreshPartitions.partitions().iterator().next().name();
|
||||
Partition partition = mv.getPartition(partitionName);
|
||||
dataProperty = partitionInfo.getDataProperty(partition.getId());
|
||||
mv.dropPartition(db.getId(), partitionName, false);
|
||||
} else {
|
||||
for (PCellWithName partName : toRefreshPartitions.partitions()) {
|
||||
mvRefreshPartitioner.dropPartition(db, mv, partName.name());
|
||||
}
|
||||
}
|
||||
for (PCellWithName partName : toRefreshPartitions.partitions()) {
|
||||
mv.dropPartition(db.getId(), partName.name(), false);
|
||||
}
|
||||
|
||||
// for non-partitioned table, we need to build the partition here
|
||||
|
|
@ -424,6 +417,12 @@ public abstract class BaseMVRefreshProcessor {
|
|||
|
||||
// insert overwrite mv must set system = true
|
||||
insertStmt.setSystem(true);
|
||||
// set a unique load label for MV refresh: mv_{uuid}
|
||||
if (insertStmt.getLabel() == null || insertStmt.getLabel().isEmpty()) {
|
||||
insertStmt.setLabel(
|
||||
com.starrocks.sql.common.MetaUtils.genMVLabel(
|
||||
com.starrocks.common.util.UUIDUtil.toTUniqueId(ctx.getQueryId())));
|
||||
}
|
||||
// if mv has set sort keys, materialized view's output columns
|
||||
// may be different from the defined query's output.
|
||||
// so set materialized view's defined outputs as target columns.
|
||||
|
|
|
|||
|
|
@ -159,6 +159,11 @@ public final class MVPCTBasedRefreshProcessor extends BaseMVRefreshProcessor {
|
|||
updatePCTMeta(mvExecPlan, pctMVToRefreshedPartitions, pctRefTableRefreshPartitions);
|
||||
}
|
||||
|
||||
// do not generate next task run if the current task run is killed
|
||||
if (mvContext.hasNextBatchPartition() && !mvContext.getTaskRun().isKilled()) {
|
||||
generateNextTaskRun();
|
||||
}
|
||||
|
||||
return Constants.TaskRunState.SUCCESS;
|
||||
}
|
||||
|
||||
|
|
@ -243,12 +248,7 @@ public final class MVPCTBasedRefreshProcessor extends BaseMVRefreshProcessor {
|
|||
return insertStmt;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void generateNextTaskRunIfNeeded() {
|
||||
if (!mvContext.hasNextBatchPartition() || mvContext.getTaskRun().isKilled()) {
|
||||
return;
|
||||
}
|
||||
|
||||
private void generateNextTaskRun() {
|
||||
TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager();
|
||||
Map<String, String> properties = mvContext.getProperties();
|
||||
long mvId = Long.parseLong(properties.get(MV_ID));
|
||||
|
|
|
|||
|
|
@ -423,40 +423,40 @@ public final class MVPCTRefreshListPartitioner extends MVPCTRefreshPartitioner {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void filterPartitionByRefreshNumber(PCellSortedSet toRefreshPartitions,
|
||||
MaterializedView.PartitionRefreshStrategy refreshStrategy) {
|
||||
public void filterPartitionByRefreshNumber(PCellSortedSet mvPartitionsToRefresh) {
|
||||
filterPartitionByRefreshNumberInternal(mvPartitionsToRefresh, MaterializedView.PartitionRefreshStrategy.STRICT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void filterPartitionByAdaptiveRefreshNumber(PCellSortedSet mvPartitionsToRefresh) {
|
||||
filterPartitionByRefreshNumberInternal(mvPartitionsToRefresh, MaterializedView.PartitionRefreshStrategy.ADAPTIVE);
|
||||
}
|
||||
|
||||
public void filterPartitionByRefreshNumberInternal(PCellSortedSet toRefreshPartitions,
|
||||
MaterializedView.PartitionRefreshStrategy refreshStrategy) {
|
||||
|
||||
// filter by partition ttl
|
||||
filterPartitionsByTTL(toRefreshPartitions, false);
|
||||
if (toRefreshPartitions == null || toRefreshPartitions.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
// filter invalid cells from input
|
||||
toRefreshPartitions.stream()
|
||||
.filter(cell -> !mv.getListPartitionItems().containsKey(cell.name()))
|
||||
.forEach(toRefreshPartitions::remove);
|
||||
|
||||
Iterator<PCellWithName> iterator = toRefreshPartitions.iterator();
|
||||
// dynamically get the number of partitions to be refreshed this time
|
||||
int partitionRefreshNumber = getPartitionRefreshNumberAdaptive(toRefreshPartitions, refreshStrategy);
|
||||
if (partitionRefreshNumber <= 0 || partitionRefreshNumber >= toRefreshPartitions.size()) {
|
||||
int refreshNumber = getRefreshNumberByMode(iterator, refreshStrategy);
|
||||
if (toRefreshPartitions.size() <= refreshNumber) {
|
||||
return;
|
||||
}
|
||||
int i = 0;
|
||||
// refresh the recent partitions first
|
||||
Iterator<PCellWithName> iterator = getToRefreshPartitionsIterator(toRefreshPartitions, false);
|
||||
while (i++ < partitionRefreshNumber && iterator.hasNext()) {
|
||||
PCellWithName pCell = iterator.next();
|
||||
logger.debug("Materialized view [{}] to refresh partition name {}, value {}",
|
||||
mv.getName(), pCell.name(), pCell.cell());
|
||||
}
|
||||
|
||||
// get next partition values for next refresh
|
||||
// iterate refreshNumber times
|
||||
Set<PListCell> nextPartitionValues = Sets.newHashSet();
|
||||
while (iterator.hasNext()) {
|
||||
int i = 0;
|
||||
while (iterator.hasNext() && i++ < refreshNumber) {
|
||||
PCellWithName pCell = iterator.next();
|
||||
nextPartitionValues.add((PListCell) pCell.cell());
|
||||
iterator.remove();
|
||||
toRefreshPartitions.remove(pCell);
|
||||
}
|
||||
logger.info("Filter partitions by refresh number, ttl_number:{}, result:{}, remains:{}",
|
||||
partitionRefreshNumber, toRefreshPartitions, nextPartitionValues);
|
||||
refreshNumber, toRefreshPartitions, nextPartitionValues);
|
||||
if (CollectionUtils.isEmpty(nextPartitionValues)) {
|
||||
return;
|
||||
}
|
||||
|
|
@ -467,6 +467,26 @@ public final class MVPCTRefreshListPartitioner extends MVPCTRefreshPartitioner {
|
|||
}
|
||||
}
|
||||
|
||||
public int getAdaptivePartitionRefreshNumber(Iterator<PCellWithName> iterator) throws MVAdaptiveRefreshException {
|
||||
Map<String, Map<Table, Set<String>>> mvToBaseNameRefs = mvContext.getMvRefBaseTableIntersectedPartitions();
|
||||
MVRefreshPartitionSelector mvRefreshPartitionSelector =
|
||||
new MVRefreshPartitionSelector(Config.mv_max_rows_per_refresh, Config.mv_max_bytes_per_refresh,
|
||||
Config.mv_max_partitions_num_per_refresh, mvContext.getExternalRefBaseTableMVPartitionMap());
|
||||
int adaptiveRefreshNumber = 0;
|
||||
while (iterator.hasNext()) {
|
||||
PCellWithName cellWithName = iterator.next();
|
||||
String partitionName = cellWithName.name();
|
||||
Map<Table, Set<String>> refPartitionInfos = mvToBaseNameRefs.get(partitionName);
|
||||
if (mvRefreshPartitionSelector.canAddPartition(refPartitionInfos)) {
|
||||
mvRefreshPartitionSelector.addPartition(refPartitionInfos);
|
||||
adaptiveRefreshNumber++;
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return adaptiveRefreshNumber;
|
||||
}
|
||||
|
||||
private void addListPartitions(Database database, MaterializedView materializedView,
|
||||
Map<String, PCell> adds, Map<String, String> partitionProperties,
|
||||
DistributionDesc distributionDesc) {
|
||||
|
|
|
|||
|
|
@ -27,6 +27,7 @@ import com.starrocks.sql.common.PCellNone;
|
|||
import com.starrocks.sql.common.PCellSortedSet;
|
||||
import com.starrocks.sql.common.PCellWithName;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
|
@ -93,11 +94,20 @@ public final class MVPCTRefreshNonPartitioner extends MVPCTRefreshPartitioner {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void filterPartitionByRefreshNumber(PCellSortedSet mvPartitionsToRefresh,
|
||||
MaterializedView.PartitionRefreshStrategy partitionRefreshStrategy) {
|
||||
public void filterPartitionByRefreshNumber(PCellSortedSet mvPartitionsToRefresh) {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
public void filterPartitionByAdaptiveRefreshNumber(PCellSortedSet mvPartitionsToRefresh) {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getAdaptivePartitionRefreshNumber(Iterator<PCellWithName> partitionNameIter) throws MVAdaptiveRefreshException {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCalcPotentialRefreshPartition(Map<Table, PCellSortedSet> baseChangedPartitionNames,
|
||||
PCellSortedSet mvPartitions) {
|
||||
|
|
|
|||
|
|
@ -172,14 +172,18 @@ public abstract class MVPCTRefreshPartitioner {
|
|||
*/
|
||||
public abstract PCellSortedSet getMVPartitionNamesWithTTL(boolean isAutoRefresh) throws AnalysisException;
|
||||
|
||||
/**
|
||||
* Filter to refresh partitions by adaptive refresh number.
|
||||
* @param mvPartitionsToRefresh: mv partitions to refresh.
|
||||
*/
|
||||
public abstract void filterPartitionByAdaptiveRefreshNumber(PCellSortedSet mvPartitionsToRefresh);
|
||||
|
||||
/**
|
||||
* Filter to refresh partitions by partition refresh number.
|
||||
* @param partitionsToRefresh: partitions to refresh.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public abstract void filterPartitionByRefreshNumber(
|
||||
PCellSortedSet partitionsToRefresh,
|
||||
MaterializedView.PartitionRefreshStrategy refreshStrategy);
|
||||
public abstract void filterPartitionByRefreshNumber(PCellSortedSet partitionsToRefresh);
|
||||
|
||||
/**
|
||||
* Check whether to calculate the potential partitions to refresh or not. When the base table changed partitions
|
||||
|
|
@ -304,6 +308,51 @@ public abstract class MVPCTRefreshPartitioner {
|
|||
if (mvToRefreshedPartitions.isEmpty() || mvToRefreshedPartitions.size() <= 1) {
|
||||
return;
|
||||
}
|
||||
|
||||
// filter partitions by partition refresh strategy
|
||||
final MaterializedView.PartitionRefreshStrategy partitionRefreshStrategy = mv.getPartitionRefreshStrategy();
|
||||
boolean hasUnsupportedTableType = mv.getBaseTableTypes().stream()
|
||||
.anyMatch(type -> !SUPPORTED_TABLE_TYPES_FOR_ADAPTIVE_MV_REFRESH.contains(type));
|
||||
if (hasUnsupportedTableType) {
|
||||
logger.warn("Materialized view {} contains unsupported external tables. Using default refresh strategy.",
|
||||
mv.getId());
|
||||
filterPartitionWithStrict(mvToRefreshedPartitions);
|
||||
} else {
|
||||
switch (partitionRefreshStrategy) {
|
||||
case ADAPTIVE:
|
||||
filterPartitionWithAdaptive(mvToRefreshedPartitions);
|
||||
break;
|
||||
case STRICT:
|
||||
default:
|
||||
// Only refresh the first partition refresh number partitions, other partitions will generate new tasks
|
||||
filterPartitionWithStrict(mvToRefreshedPartitions);
|
||||
}
|
||||
}
|
||||
logger.info("after filterPartitionByAdaptive, partitionsToRefresh: {}",
|
||||
mvToRefreshedPartitions);
|
||||
}
|
||||
|
||||
public void filterPartitionWithStrict(PCellSortedSet partitionsToRefresh) {
|
||||
// refresh all partition when it's a sync refresh, otherwise updated partitions may be lost.
|
||||
ExecuteOption executeOption = mvContext.getExecuteOption();
|
||||
if (executeOption != null && executeOption.getIsSync()) {
|
||||
return;
|
||||
}
|
||||
// ignore if mv is not partitioned.
|
||||
if (!mv.isPartitionedTable()) {
|
||||
return;
|
||||
}
|
||||
// ignore if partition_fresh_limit is not set
|
||||
int partitionRefreshNumber = mv.getTableProperty().getPartitionRefreshNumber();
|
||||
if (partitionRefreshNumber <= 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
// do filter actions
|
||||
filterPartitionByRefreshNumber(partitionsToRefresh);
|
||||
}
|
||||
|
||||
public void filterPartitionWithAdaptive(PCellSortedSet partitionsToRefresh) {
|
||||
// refresh all partition when it's a sync refresh, otherwise updated partitions may be lost.
|
||||
ExecuteOption executeOption = mvContext.getExecuteOption();
|
||||
if (executeOption != null && executeOption.getIsSync()) {
|
||||
|
|
@ -314,17 +363,8 @@ public abstract class MVPCTRefreshPartitioner {
|
|||
return;
|
||||
}
|
||||
|
||||
// filter partitions by partition refresh strategy
|
||||
boolean hasUnsupportedTableType = mv.getBaseTableTypes().stream()
|
||||
.anyMatch(type -> !SUPPORTED_TABLE_TYPES_FOR_ADAPTIVE_MV_REFRESH.contains(type));
|
||||
if (hasUnsupportedTableType) {
|
||||
filterPartitionByRefreshNumber(mvToRefreshedPartitions, MaterializedView.PartitionRefreshStrategy.STRICT);
|
||||
} else {
|
||||
MaterializedView.PartitionRefreshStrategy partitionRefreshStrategy = mv.getPartitionRefreshStrategy();
|
||||
filterPartitionByRefreshNumber(mvToRefreshedPartitions, partitionRefreshStrategy);
|
||||
}
|
||||
logger.info("after filterPartitionByAdaptive, partitionsToRefresh: {}",
|
||||
mvToRefreshedPartitions);
|
||||
// do filter actions
|
||||
filterPartitionByAdaptiveRefreshNumber(partitionsToRefresh);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -346,86 +386,29 @@ public abstract class MVPCTRefreshPartitioner {
|
|||
* may fail due to missing or invalid metadata. In such cases, the method automatically
|
||||
* falls back to the STRICT strategy to ensure the MV can still be refreshed correctly.
|
||||
*
|
||||
* @param toRefreshPartitions sorted partition names to be refreshed.
|
||||
* @param refreshStrategy The refresh strategy: either ADAPTIVE or STRICT.
|
||||
* @param sortedPartitionIterator Iterator over sorted partition names to be refreshed.
|
||||
* @param refreshStrategy The refresh strategy: either ADAPTIVE or STRICT.
|
||||
* @return The number of partitions to refresh.
|
||||
*/
|
||||
public int getPartitionRefreshNumberAdaptive(PCellSortedSet toRefreshPartitions,
|
||||
MaterializedView.PartitionRefreshStrategy refreshStrategy) {
|
||||
public int getRefreshNumberByMode(Iterator<PCellWithName> sortedPartitionIterator,
|
||||
MaterializedView.PartitionRefreshStrategy refreshStrategy) {
|
||||
try {
|
||||
switch (refreshStrategy) {
|
||||
case ADAPTIVE:
|
||||
return getAdaptivePartitionRefreshNumber(toRefreshPartitions);
|
||||
return getAdaptivePartitionRefreshNumber(sortedPartitionIterator);
|
||||
case STRICT:
|
||||
default:
|
||||
return getRefreshNumberByDefaultMode(toRefreshPartitions);
|
||||
return mv.getTableProperty().getPartitionRefreshNumber();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
} catch (MVAdaptiveRefreshException e) {
|
||||
logger.warn("Adaptive refresh failed for mode '{}', falling back to STRICT mode. Reason: {}",
|
||||
refreshStrategy, e.getMessage(), e);
|
||||
return getRefreshNumberByDefaultMode(toRefreshPartitions);
|
||||
return mv.getTableProperty().getPartitionRefreshNumber();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get partition refresh number by default mode, which is used to be compatible with old versions and the following
|
||||
* conditions are met:
|
||||
* - partition_refresh_strategy is not set strict by user
|
||||
* - partition_refresh_number is not set by user
|
||||
*/
|
||||
private int getRefreshNumberByDefaultMode(PCellSortedSet torRefreshPartitions) {
|
||||
int defaultPartitionRefreshNumber = mv.getTableProperty().getPartitionRefreshNumber();
|
||||
TableProperty tableProperty = mv.getTableProperty();
|
||||
// if user has set partition_refresh_number, use it directly
|
||||
if (tableProperty == null || tableProperty.isSetPartitionRefreshNumber()
|
||||
|| defaultPartitionRefreshNumber != Config.default_mv_partition_refresh_number) {
|
||||
return defaultPartitionRefreshNumber;
|
||||
}
|
||||
MaterializedView.PartitionRefreshStrategy partitionRefreshStrategy =
|
||||
MaterializedView.PartitionRefreshStrategy.of(tableProperty.getPartitionRefreshStrategy());
|
||||
// if partition_refresh_strategy is strict, use partition_refresh_number directly
|
||||
if (tableProperty.isSetPartitionRefreshStrategy()
|
||||
&& MaterializedView.PartitionRefreshStrategy.STRICT.equals(partitionRefreshStrategy)) {
|
||||
return defaultPartitionRefreshNumber;
|
||||
}
|
||||
// if the number of partitions to refresh is not too many, use it directly
|
||||
int toRefreshPartitionNum = torRefreshPartitions.size();
|
||||
if (toRefreshPartitionNum <= Config.mv_max_partitions_num_per_refresh) {
|
||||
return defaultPartitionRefreshNumber;
|
||||
}
|
||||
// to be compatible with old version, use adaptive mode if partition_refresh_strategy is not set
|
||||
int toRefreshPartitionNumPerTaskRun =
|
||||
(int) Math.ceil((double) toRefreshPartitionNum / Math.max(1, Config.task_runs_concurrency));
|
||||
|
||||
// if there are too many partitions to refresh, limit the number of partitions to refresh per task run
|
||||
int finalToRefreshPartitionNumPerTaskRun = Math.min(toRefreshPartitionNumPerTaskRun,
|
||||
Config.mv_max_partitions_num_per_refresh);
|
||||
return finalToRefreshPartitionNumPerTaskRun;
|
||||
}
|
||||
|
||||
public int getAdaptivePartitionRefreshNumber(PCellSortedSet toRefreshPartitions)
|
||||
throws MVAdaptiveRefreshException {
|
||||
if (!mv.isPartitionedTable()) {
|
||||
return toRefreshPartitions.size();
|
||||
}
|
||||
|
||||
Map<String, Map<Table, Set<String>>> mvToBaseNameRefs = mvContext.getMvRefBaseTableIntersectedPartitions();
|
||||
MVRefreshPartitionSelector mvRefreshPartitionSelector =
|
||||
new MVRefreshPartitionSelector(Config.mv_max_rows_per_refresh, Config.mv_max_bytes_per_refresh,
|
||||
Config.mv_max_partitions_num_per_refresh, mvContext.getExternalRefBaseTableMVPartitionMap());
|
||||
int adaptiveRefreshNumber = 0;
|
||||
for (PCellWithName pCellWithName : toRefreshPartitions.partitions()) {
|
||||
String mvRefreshPartition = pCellWithName.name();
|
||||
Map<Table, Set<String>> refBaseTablesPartitions = mvToBaseNameRefs.get(mvRefreshPartition);
|
||||
if (mvRefreshPartitionSelector.canAddPartition(refBaseTablesPartitions)) {
|
||||
mvRefreshPartitionSelector.addPartition(refBaseTablesPartitions);
|
||||
adaptiveRefreshNumber++;
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return adaptiveRefreshNumber;
|
||||
}
|
||||
protected abstract int getAdaptivePartitionRefreshNumber(Iterator<PCellWithName> partitionNameIter)
|
||||
throws MVAdaptiveRefreshException;
|
||||
|
||||
/**
|
||||
* Check whether the base table is supported partition refresh or not.
|
||||
|
|
@ -561,7 +544,7 @@ public abstract class MVPCTRefreshPartitioner {
|
|||
return false;
|
||||
}
|
||||
|
||||
public void dropPartition(Database db, MaterializedView materializedView, String mvPartitionName) {
|
||||
protected void dropPartition(Database db, MaterializedView materializedView, String mvPartitionName) {
|
||||
String dropPartitionName = materializedView.getPartition(mvPartitionName).getName();
|
||||
Locker locker = new Locker();
|
||||
if (!locker.tryLockTableWithIntensiveDbLock(db.getId(), materializedView.getId(), LockType.WRITE,
|
||||
|
|
@ -674,9 +657,7 @@ public abstract class MVPCTRefreshPartitioner {
|
|||
getExpiredPartitionsWithRetention(ttlCondition, toRefreshPartitionMap, isMockPartitionIds);
|
||||
if (CollectionUtils.isNotEmpty(toRemovePartitions)) {
|
||||
toRemovePartitions.stream()
|
||||
.filter(p -> toRefreshPartitionMap.containsKey(p))
|
||||
.map(p -> PCellWithName.of(p, toRefreshPartitionMap.get(p)))
|
||||
.forEach(toRefreshPartitions::remove);
|
||||
.forEach(p -> toRefreshPartitions.remove(PCellWithName.of(p, toRefreshPartitionMap.get(p))));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -741,15 +722,4 @@ public abstract class MVPCTRefreshPartitioner {
|
|||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the iterator of to refresh partitions according to config.
|
||||
*/
|
||||
public Iterator<PCellWithName> getToRefreshPartitionsIterator(PCellSortedSet toRefreshPartitions,
|
||||
boolean isAscending) {
|
||||
if (isAscending) {
|
||||
return toRefreshPartitions.iterator();
|
||||
} else {
|
||||
return toRefreshPartitions.descendingIterator();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -64,6 +64,7 @@ import org.apache.logging.log4j.Logger;
|
|||
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
|
@ -412,33 +413,51 @@ public final class MVPCTRefreshRangePartitioner extends MVPCTRefreshPartitioner
|
|||
}
|
||||
|
||||
@Override
|
||||
public void filterPartitionByRefreshNumber(PCellSortedSet mvPartitionsToRefresh,
|
||||
MaterializedView.PartitionRefreshStrategy refreshStrategy) {
|
||||
public void filterPartitionByRefreshNumber(PCellSortedSet mvPartitionsToRefresh) {
|
||||
filterPartitionByRefreshNumberInternal(mvPartitionsToRefresh,
|
||||
MaterializedView.PartitionRefreshStrategy.STRICT);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void filterPartitionByAdaptiveRefreshNumber(PCellSortedSet mvPartitionsToRefresh) {
|
||||
filterPartitionByRefreshNumberInternal(mvPartitionsToRefresh, MaterializedView.PartitionRefreshStrategy.ADAPTIVE);
|
||||
}
|
||||
|
||||
public void filterPartitionByRefreshNumberInternal(PCellSortedSet mvPartitionsToRefresh,
|
||||
MaterializedView.PartitionRefreshStrategy refreshStrategy) {
|
||||
int partitionRefreshNumber = mv.getTableProperty().getPartitionRefreshNumber();
|
||||
Map<String, Range<PartitionKey>> mvRangePartitionMap = mv.getRangePartitionMap();
|
||||
if (partitionRefreshNumber <= 0 || partitionRefreshNumber >= mvRangePartitionMap.size()) {
|
||||
return;
|
||||
}
|
||||
|
||||
// remove invalid cells from the input to-refresh partitions
|
||||
mvPartitionsToRefresh
|
||||
.stream()
|
||||
.filter(pCell -> !mvRangePartitionMap.containsKey(pCell.name()))
|
||||
.forEach(mvPartitionsToRefresh::remove);
|
||||
|
||||
Iterator<PCellWithName> iterator = getToRefreshPartitionsIterator(mvPartitionsToRefresh,
|
||||
Config.materialized_view_refresh_ascending);
|
||||
// dynamically get the number of partitions to be refreshed this time
|
||||
partitionRefreshNumber = getPartitionRefreshNumberAdaptive(mvPartitionsToRefresh, refreshStrategy);
|
||||
if (partitionRefreshNumber <= 0 || mvPartitionsToRefresh.size() <= partitionRefreshNumber) {
|
||||
return;
|
||||
Iterator<PCellWithName> inputIter = mvPartitionsToRefresh.iterator();
|
||||
while (inputIter.hasNext()) {
|
||||
PCellWithName pCellWithName = inputIter.next();
|
||||
String mvPartitionName = pCellWithName.name();
|
||||
// skip if partition is not in the mv's partition range map
|
||||
if (!mvRangePartitionMap.containsKey(mvPartitionName)) {
|
||||
logger.warn("Partition {} is not in the materialized view's partition range map, " +
|
||||
"remove it from refresh list", mvPartitionName);
|
||||
mvPartitionsToRefresh.remove(pCellWithName);
|
||||
}
|
||||
}
|
||||
|
||||
int i = 0;
|
||||
while (i++ < partitionRefreshNumber && iterator.hasNext()) {
|
||||
PCellWithName pCell = iterator.next();
|
||||
logger.debug("Materialized view [{}] to refresh partition name {}, value {}",
|
||||
mv.getName(), pCell.name(), pCell.cell());
|
||||
LinkedList<PCellWithName> sortedPartition = mvPartitionsToRefresh
|
||||
.stream()
|
||||
.collect(Collectors.toCollection(LinkedList::new));
|
||||
Iterator<PCellWithName> sortedIterator = Config.materialized_view_refresh_ascending
|
||||
? sortedPartition.iterator() : sortedPartition.descendingIterator();
|
||||
// dynamically get the number of partitions to be refreshed this time
|
||||
partitionRefreshNumber = getRefreshNumberByMode(sortedIterator, refreshStrategy);
|
||||
|
||||
PCellWithName tmpPCellWithName = null;
|
||||
for (int i = 0; i < partitionRefreshNumber; i++) {
|
||||
if (sortedIterator.hasNext()) {
|
||||
tmpPCellWithName = sortedIterator.next();
|
||||
sortedIterator.remove();
|
||||
}
|
||||
|
||||
// NOTE: if mv's need to refresh partitions in the many-to-many mappings, no need to filter to
|
||||
// avoid data lose.
|
||||
// eg:
|
||||
|
|
@ -458,11 +477,41 @@ public final class MVPCTRefreshRangePartitioner extends MVPCTRefreshPartitioner
|
|||
// BTW, since the refresh has already scanned the needed base tables' data, it's better to update
|
||||
// more mv's partitions as more as possible.
|
||||
// TODO: But it may cause much memory to refresh many partitions, support fine-grained partition refresh later.
|
||||
if (!mvToRefreshPotentialPartitions.isEmpty() && mvToRefreshPotentialPartitions.contains(pCell.name())) {
|
||||
if (!mvToRefreshPotentialPartitions.isEmpty() && mvToRefreshPotentialPartitions.contains(tmpPCellWithName.name())) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
if (!Config.materialized_view_refresh_ascending) {
|
||||
sortedIterator = sortedPartition.iterator();
|
||||
}
|
||||
setNextPartitionStartAndEnd(mvPartitionsToRefresh, sortedIterator);
|
||||
}
|
||||
|
||||
public int getAdaptivePartitionRefreshNumber(
|
||||
Iterator<PCellWithName> iterator) throws MVAdaptiveRefreshException {
|
||||
Map<String, Map<Table, Set<String>>> mvToBaseNameRefs = mvContext.getMvRefBaseTableIntersectedPartitions();
|
||||
MVRefreshPartitionSelector mvRefreshPartitionSelector =
|
||||
new MVRefreshPartitionSelector(Config.mv_max_rows_per_refresh, Config.mv_max_bytes_per_refresh,
|
||||
Config.mv_max_partitions_num_per_refresh, mvContext.getExternalRefBaseTableMVPartitionMap());
|
||||
|
||||
int adaptiveRefreshNumber = 0;
|
||||
while (iterator.hasNext()) {
|
||||
PCellWithName pCellWithName = iterator.next();
|
||||
String mvRefreshPartition = pCellWithName.name();
|
||||
Map<Table, Set<String>> refBaseTablesPartitions = mvToBaseNameRefs.get(mvRefreshPartition);
|
||||
if (mvRefreshPartitionSelector.canAddPartition(refBaseTablesPartitions)) {
|
||||
mvRefreshPartitionSelector.addPartition(refBaseTablesPartitions);
|
||||
iterator.remove();
|
||||
adaptiveRefreshNumber++;
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return adaptiveRefreshNumber;
|
||||
}
|
||||
|
||||
private void setNextPartitionStartAndEnd(PCellSortedSet partitionsToRefresh,
|
||||
Iterator<PCellWithName> iterator) {
|
||||
String nextPartitionStart = null;
|
||||
PCellWithName end = null;
|
||||
if (iterator.hasNext()) {
|
||||
|
|
@ -471,11 +520,12 @@ public final class MVPCTRefreshRangePartitioner extends MVPCTRefreshPartitioner
|
|||
Range<PartitionKey> partitionKeyRange = pRangeCell.getRange();
|
||||
nextPartitionStart = AnalyzerUtils.parseLiteralExprToDateString(partitionKeyRange.lowerEndpoint(), 0);
|
||||
end = start;
|
||||
iterator.remove();
|
||||
partitionsToRefresh.remove(end);
|
||||
}
|
||||
while (iterator.hasNext()) {
|
||||
end = iterator.next();
|
||||
iterator.remove();
|
||||
partitionsToRefresh.remove(end);
|
||||
}
|
||||
|
||||
if (!mvRefreshParams.isTentative()) {
|
||||
|
|
|
|||
|
|
@ -180,6 +180,11 @@ public class MVIVMBasedMVRefreshProcessor extends BaseMVRefreshProcessor {
|
|||
}
|
||||
}
|
||||
|
||||
// generate the next task run state
|
||||
if (hasNextTaskRun && !mvContext.getTaskRun().isKilled()) {
|
||||
generateNextTaskRun();
|
||||
}
|
||||
|
||||
return Constants.TaskRunState.SUCCESS;
|
||||
}
|
||||
|
||||
|
|
@ -333,11 +338,7 @@ public class MVIVMBasedMVRefreshProcessor extends BaseMVRefreshProcessor {
|
|||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void generateNextTaskRunIfNeeded() {
|
||||
if (!hasNextTaskRun || mvContext.getTaskRun().isKilled()) {
|
||||
return;
|
||||
}
|
||||
private void generateNextTaskRun() {
|
||||
TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager();
|
||||
Map<String, String> properties = mvContext.getProperties();
|
||||
long mvId = Long.parseLong(properties.get(MV_ID));
|
||||
|
|
|
|||
|
|
@ -154,6 +154,14 @@ public class MetaUtils {
|
|||
return "update_" + DebugUtil.printId(executionId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a transaction label for MV refresh loads.
|
||||
* The label format is: mv_{uuid}
|
||||
*/
|
||||
public static String genMVLabel(TUniqueId executionId) {
|
||||
return "mv_" + DebugUtil.printId(executionId);
|
||||
}
|
||||
|
||||
public static ExternalOlapTable syncOLAPExternalTableMeta(ExternalOlapTable externalOlapTable) {
|
||||
ExternalOlapTable copiedTable = new ExternalOlapTable();
|
||||
externalOlapTable.copyOnlyForQuery(copiedTable);
|
||||
|
|
|
|||
|
|
@ -14,12 +14,11 @@
|
|||
|
||||
package com.starrocks.sql.common;
|
||||
|
||||
import com.starrocks.common.Config;
|
||||
import com.google.common.base.Joiner;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
|
|
@ -93,10 +92,6 @@ public record PCellSortedSet(SortedSet<PCellWithName> partitions) {
|
|||
partitions.forEach(action);
|
||||
}
|
||||
|
||||
public Iterator<PCellWithName> descendingIterator() {
|
||||
return ((NavigableSet<PCellWithName>) partitions).descendingIterator();
|
||||
}
|
||||
|
||||
public Iterator<PCellWithName> iterator() {
|
||||
return partitions.iterator();
|
||||
}
|
||||
|
|
@ -128,9 +123,8 @@ public record PCellSortedSet(SortedSet<PCellWithName> partitions) {
|
|||
}
|
||||
SortedSet<PCellWithName> limitedPartitions = new TreeSet<>(partitions);
|
||||
Iterator<PCellWithName> iterator = limitedPartitions.iterator();
|
||||
int removeCount = limitedPartitions.size() - limit;
|
||||
int count = 0;
|
||||
while (iterator.hasNext() && count < removeCount) {
|
||||
while (iterator.hasNext() && count < limitedPartitions.size() - limit) {
|
||||
iterator.next();
|
||||
iterator.remove();
|
||||
count++;
|
||||
|
|
@ -171,29 +165,7 @@ public record PCellSortedSet(SortedSet<PCellWithName> partitions) {
|
|||
if (partitions == null || partitions.isEmpty()) {
|
||||
return "";
|
||||
}
|
||||
int maxLen = Config.max_mv_task_run_meta_message_values_length;
|
||||
int size = partitions.size();
|
||||
|
||||
if (size <= maxLen) {
|
||||
// Join all names if under limit
|
||||
return partitions.stream()
|
||||
.map(PCellWithName::name)
|
||||
.collect(Collectors.joining(","));
|
||||
} else {
|
||||
int half = maxLen / 2;
|
||||
List<String> names = partitions.stream()
|
||||
.map(PCellWithName::name)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
String prefix = names.stream()
|
||||
.limit(half)
|
||||
.collect(Collectors.joining(","));
|
||||
String suffix = names.stream()
|
||||
.skip(size - half)
|
||||
.collect(Collectors.joining(","));
|
||||
|
||||
return prefix + ",...," + suffix;
|
||||
}
|
||||
return Joiner.on(",").join(getPartitionNames());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
|||
|
|
@ -117,14 +117,9 @@ public final class PRangeCell extends PCell {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "[%s-%s]".formatted(toString(range.lowerEndpoint()), toString(range.upperEndpoint()));
|
||||
}
|
||||
|
||||
private String toString(PartitionKey partitionKey) {
|
||||
if (partitionKey == null) {
|
||||
return "null";
|
||||
}
|
||||
return partitionKey.toSql();
|
||||
return "PRangeCell{" +
|
||||
"range=" + range +
|
||||
'}';
|
||||
}
|
||||
|
||||
public static Map<String, Range<PartitionKey>> toRangeMap(Map<String, PCell> input) {
|
||||
|
|
|
|||
|
|
@ -1061,11 +1061,6 @@ public class ScalarOperatorFunctions {
|
|||
/**
|
||||
* Arithmetic function
|
||||
*/
|
||||
|
||||
@ConstantFunction(name = "add", argTypes = {TINYINT, TINYINT}, returnType = TINYINT, isMonotonic = true)
|
||||
public static ConstantOperator addTinyInt(ConstantOperator first, ConstantOperator second) {
|
||||
return ConstantOperator.createTinyInt((byte) Math.addExact(first.getTinyInt(), second.getTinyInt()));
|
||||
}
|
||||
@ConstantFunction(name = "add", argTypes = {SMALLINT, SMALLINT}, returnType = SMALLINT, isMonotonic = true)
|
||||
public static ConstantOperator addSmallInt(ConstantOperator first, ConstantOperator second) {
|
||||
return ConstantOperator.createSmallInt((short) Math.addExact(first.getSmallint(), second.getSmallint()));
|
||||
|
|
|
|||
|
|
@ -14,23 +14,9 @@
|
|||
|
||||
package com.starrocks.sql.optimizer.rewrite.scalar;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.ImmutableSortedMap;
|
||||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.starrocks.catalog.ArrayType;
|
||||
import com.starrocks.catalog.PrimitiveType;
|
||||
import com.starrocks.catalog.ScalarFunction;
|
||||
import com.starrocks.catalog.ScalarType;
|
||||
import com.starrocks.catalog.Type;
|
||||
import com.starrocks.sql.analyzer.SemanticException;
|
||||
import com.starrocks.sql.ast.expression.ArithmeticExpr;
|
||||
import com.starrocks.sql.ast.expression.BinaryType;
|
||||
import com.starrocks.sql.ast.expression.Expr;
|
||||
import com.starrocks.sql.optimizer.operator.scalar.ArrayOperator;
|
||||
import com.starrocks.sql.optimizer.operator.scalar.BinaryPredicateOperator;
|
||||
import com.starrocks.sql.optimizer.operator.scalar.CallOperator;
|
||||
import com.starrocks.sql.optimizer.operator.scalar.CastOperator;
|
||||
|
|
@ -41,19 +27,12 @@ import com.starrocks.sql.optimizer.operator.scalar.LikePredicateOperator;
|
|||
import com.starrocks.sql.optimizer.operator.scalar.ScalarOperator;
|
||||
import com.starrocks.sql.optimizer.rewrite.ScalarOperatorEvaluator;
|
||||
import com.starrocks.sql.optimizer.rewrite.ScalarOperatorRewriteContext;
|
||||
import com.starrocks.sql.optimizer.rewrite.ScalarOperatorRewriter;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.BinaryOperator;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
public class FoldConstantsRule extends BottomUpScalarOperatorRewriteRule {
|
||||
private static final Logger LOG = LogManager.getLogger(FoldConstantsRule.class);
|
||||
|
|
@ -68,258 +47,12 @@ public class FoldConstantsRule extends BottomUpScalarOperatorRewriteRule {
|
|||
this.needMonotonicFunc = needMonotonicFunc;
|
||||
}
|
||||
|
||||
private <T extends ScalarOperator> Optional<ScalarOperator> arrayScalarFun(
|
||||
CallOperator call, BiFunction<List<ScalarOperator>, ScalarOperator, T> cb) {
|
||||
Preconditions.checkArgument(call.getArguments().size() == 2);
|
||||
ScalarOperator arg0 = call.getArguments().get(0);
|
||||
ScalarOperator arg1 = call.getArguments().get(1);
|
||||
if (arg0.isConstantNull()) {
|
||||
return Optional.of(ConstantOperator.createNull(call.getType()));
|
||||
}
|
||||
if (!(arg0 instanceof ArrayOperator) || !arg1.isConstantRef() ||
|
||||
!arg0.getChildren().stream().allMatch(ScalarOperator::isConstantRef)) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(cb.apply(arg0.getChildren(), arg1));
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> arrayUnaryFun(CallOperator call,
|
||||
Function<List<ScalarOperator>, ScalarOperator> cb) {
|
||||
Preconditions.checkArgument(call.getArguments().size() == 1);
|
||||
ScalarOperator arg0 = call.getChild(0);
|
||||
if (arg0 instanceof CastOperator && arg0.getChild(0) instanceof ArrayOperator) {
|
||||
arg0 = call.getChild(0).getChild(0);
|
||||
}
|
||||
if (arg0.isConstantNull()) {
|
||||
return Optional.of(ConstantOperator.createNull(arg0.getType()));
|
||||
}
|
||||
if (!(arg0 instanceof ArrayOperator) || !arg0.getChildren().stream().allMatch(ScalarOperator::isConstantRef)) {
|
||||
return Optional.empty();
|
||||
}
|
||||
try {
|
||||
return Optional.of(cb.apply(arg0.getChildren()));
|
||||
} catch (IllegalArgumentException ex) {
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayAppend(CallOperator call) {
|
||||
return arrayScalarFun(call, (arrayElements, target) ->
|
||||
new ArrayOperator(call.getType(), call.isNullable(),
|
||||
Lists.newArrayList(Iterables.concat(arrayElements, List.of(target)))));
|
||||
}
|
||||
|
||||
boolean constantEqual(ScalarOperator lhs, ScalarOperator rhs) {
|
||||
Preconditions.checkArgument(lhs.isConstantRef() && rhs.isConstantRef());
|
||||
ConstantOperator constLhs = (ConstantOperator) lhs;
|
||||
ConstantOperator constRhs = (ConstantOperator) rhs;
|
||||
if (constLhs.getType().isDecimalV3()) {
|
||||
return constLhs.getDecimal().compareTo(constRhs.getDecimal()) == 0 &&
|
||||
constLhs.isNull() == constRhs.isNull();
|
||||
} else {
|
||||
return lhs.equals(rhs);
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayContains(CallOperator call) {
|
||||
|
||||
return arrayScalarFun(call, (arrayElements, target) ->
|
||||
ConstantOperator.createBoolean(arrayElements.stream().anyMatch(elem -> constantEqual(elem, target))));
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayRemove(CallOperator call) {
|
||||
return arrayScalarFun(call, (arrayElements, target) ->
|
||||
new ArrayOperator(call.getType(), call.isNullable(),
|
||||
arrayElements.stream().filter((elem) -> !constantEqual(elem, target))
|
||||
.collect(Collectors.toList())));
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayPosition(CallOperator call) {
|
||||
return arrayScalarFun(call, (arrayElements, target) ->
|
||||
ConstantOperator.createInt(IntStream.range(0, arrayElements.size())
|
||||
.boxed()
|
||||
.filter(i -> constantEqual(arrayElements.get(i), target))
|
||||
.findAny().map(i -> i + 1).orElse(0)));
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayLength(CallOperator call) {
|
||||
ScalarOperator arg0 = call.getChild(0);
|
||||
if (arg0.isConstantNull()) {
|
||||
return Optional.of(ConstantOperator.createNull(call.getType()));
|
||||
} else if (arg0 instanceof ArrayOperator) {
|
||||
return Optional.of(ConstantOperator.createInt(arg0.getChildren().size()));
|
||||
} else if (arg0 instanceof CastOperator && (arg0.getChild(0) instanceof ArrayOperator)) {
|
||||
// array_length([]) and array_length([NULL, NULL]) would be
|
||||
// array_length(cast([] as ARRAY<BOOL>)) and array_length(cast([NULL, NULL] as ARRAY<BOOL>))
|
||||
return Optional.of(ConstantOperator.createInt(arg0.getChild(0).getChildren().size()));
|
||||
} else {
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArraySum(CallOperator call) {
|
||||
BinaryOperator<ConstantOperator> add = (lhs, rhs) -> {
|
||||
String opName = ArithmeticExpr.Operator.ADD.getName();
|
||||
com.starrocks.catalog.Function fn =
|
||||
Expr.getBuiltinFunction(opName, new Type[] {lhs.getType(), rhs.getType()},
|
||||
com.starrocks.catalog.Function.CompareMode.IS_SUPERTYPE_OF);
|
||||
CallOperator addition =
|
||||
new CallOperator(ArithmeticExpr.Operator.ADD.getName(), lhs.getType(), List.of(lhs, rhs), fn);
|
||||
ScalarOperator result = addition.accept(this, new ScalarOperatorRewriteContext());
|
||||
if (result instanceof ConstantOperator) {
|
||||
return (ConstantOperator) result;
|
||||
} else {
|
||||
throw new IllegalArgumentException();
|
||||
}
|
||||
};
|
||||
return arrayUnaryFun(call, (elements) -> elements.stream().filter(elem -> !elem.isConstantNull())
|
||||
.map(elem -> (ConstantOperator) elem).reduce(add).orElse(ConstantOperator.NULL));
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayMinMax(CallOperator call, boolean lessThan) {
|
||||
Comparator<ConstantOperator> comparator = (lhs, rhs) -> {
|
||||
BinaryPredicateOperator cmp =
|
||||
lessThan ? BinaryPredicateOperator.lt(lhs, rhs) : BinaryPredicateOperator.gt(lhs, rhs);
|
||||
ScalarOperator result = cmp.accept(this, new ScalarOperatorRewriteContext());
|
||||
if (result.isConstant() && result.getType().isBoolean()) {
|
||||
return ((ConstantOperator) result).getBoolean() ? -1 : 1;
|
||||
} else {
|
||||
throw new IllegalArgumentException();
|
||||
}
|
||||
};
|
||||
return arrayUnaryFun(call, (elements) -> elements.stream().filter(elem -> !elem.isConstantNull())
|
||||
.map(elem -> (ConstantOperator) elem).min(comparator).orElse(ConstantOperator.NULL));
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayMin(CallOperator call) {
|
||||
return constArrayMinMax(call, true);
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayMax(CallOperator call) {
|
||||
return constArrayMinMax(call, false);
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> constArrayAvg(CallOperator call) {
|
||||
Optional<ScalarOperator> optSum = constArraySum(call);
|
||||
Optional<ScalarOperator> optLength = constArrayLength(call);
|
||||
if (!optSum.isPresent() || !optLength.isPresent() ||
|
||||
!optSum.get().isConstant() || !optLength.get().isConstant()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
ConstantOperator sum = (ConstantOperator) optSum.get();
|
||||
ConstantOperator length = (ConstantOperator) optLength.get();
|
||||
if (sum.isNull() || length.isNull()) {
|
||||
return Optional.of(ConstantOperator.createNull(call.getType()));
|
||||
}
|
||||
|
||||
String opName = ArithmeticExpr.Operator.DIVIDE.getName();
|
||||
com.starrocks.catalog.Function
|
||||
fn = Expr.getBuiltinFunction(opName, new Type[] {call.getType(), call.getType()},
|
||||
com.starrocks.catalog.Function.CompareMode.IS_SUPERTYPE_OF);
|
||||
|
||||
// for decimal types, divide function should be rectified.
|
||||
if (call.getType().isDecimalV3()) {
|
||||
Preconditions.checkArgument(sum.getType().isDecimalV3());
|
||||
ScalarType sumType = (ScalarType) sum.getType();
|
||||
sumType = ScalarType.createDecimalV3Type(call.getType().getPrimitiveType(), sumType.getScalarPrecision(),
|
||||
sumType.getScalarScale());
|
||||
int precision = PrimitiveType.getMaxPrecisionOfDecimal(call.getType().getPrimitiveType());
|
||||
ScalarType countType = ScalarType.createDecimalV3Type(call.getType().getPrimitiveType(), precision, 0);
|
||||
fn = new ScalarFunction(fn.getFunctionName(), new Type[] {sumType, countType}, call.getType(),
|
||||
fn.hasVarArgs());
|
||||
}
|
||||
|
||||
CallOperator avg =
|
||||
new CallOperator(opName, call.getType(), List.of(sum, length), fn);
|
||||
ScalarOperatorRewriter rewriter = new ScalarOperatorRewriter();
|
||||
ScalarOperator result = rewriter.rewrite(avg, ScalarOperatorRewriter.DEFAULT_REWRITE_RULES);
|
||||
if (result.isConstant()) {
|
||||
return Optional.of(result);
|
||||
} else {
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> returnNullIfExistsNullArg(CallOperator call) {
|
||||
ImmutableSet<String> functions = new ImmutableSortedSet.Builder<String>(String.CASE_INSENSITIVE_ORDER)
|
||||
.add("split")
|
||||
.add("str_to_map")
|
||||
.add("regexp_extract_all")
|
||||
.add("regexp_split")
|
||||
.add("array_length")
|
||||
.add("array_sum")
|
||||
.add("array_avg")
|
||||
.add("array_min")
|
||||
.add("array_max")
|
||||
.add("array_distinct")
|
||||
.add("array_sort")
|
||||
.add("reverse")
|
||||
.add("array_join")
|
||||
.add("array_difference")
|
||||
.add("array_slice")
|
||||
.add("array_concat")
|
||||
.add("arrays_overlap")
|
||||
.add("array_intersect")
|
||||
.add("array_cum_sum")
|
||||
.add("array_contains_all")
|
||||
.add("array_contains_seq")
|
||||
.add("all_match")
|
||||
.add("any_match")
|
||||
.add("array_generate")
|
||||
.add("array_repeat")
|
||||
.add("array_flatten")
|
||||
.add("array_map")
|
||||
.add("map_size")
|
||||
.add("map_keys")
|
||||
.add("map_values")
|
||||
.add("map_from_arrays")
|
||||
.add("distinct_map_keys")
|
||||
.add("cardinality")
|
||||
.add("tokenize")
|
||||
.build();
|
||||
if (functions.contains(call.getFnName()) && call.getArguments().stream().anyMatch(
|
||||
ScalarOperator::isConstantNull)) {
|
||||
return Optional.of(ConstantOperator.createNull(call.getType()));
|
||||
} else {
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<ScalarOperator> tryToProcessConstantArrayFunctions(CallOperator call) {
|
||||
Optional<ScalarOperator> result = returnNullIfExistsNullArg(call);
|
||||
if (result.isPresent()) {
|
||||
return result;
|
||||
}
|
||||
ImmutableMap<String, Function<CallOperator, Optional<ScalarOperator>>> handlers =
|
||||
new ImmutableSortedMap.Builder<String, Function<CallOperator, Optional<ScalarOperator>>>(
|
||||
String.CASE_INSENSITIVE_ORDER)
|
||||
.put("array_length", this::constArrayLength)
|
||||
.put("array_sum", this::constArraySum)
|
||||
.put("array_min", this::constArrayMin)
|
||||
.put("array_max", this::constArrayMax)
|
||||
.put("array_avg", this::constArrayAvg)
|
||||
.put("array_contains", this::constArrayContains)
|
||||
.put("array_append", this::constArrayAppend)
|
||||
.put("array_remove", this::constArrayRemove)
|
||||
.put("array_position", this::constArrayPosition)
|
||||
.build();
|
||||
if (handlers.containsKey(call.getFnName())) {
|
||||
return handlers.get(call.getFnName()).apply(call);
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ScalarOperator visitCall(CallOperator call, ScalarOperatorRewriteContext context) {
|
||||
if (call.isAggregate()) {
|
||||
return call;
|
||||
}
|
||||
|
||||
Optional<ScalarOperator> mayBeConstant = tryToProcessConstantArrayFunctions(call);
|
||||
if (mayBeConstant.isPresent()) {
|
||||
return mayBeConstant.get();
|
||||
}
|
||||
|
||||
if (notAllConstant(call.getChildren())) {
|
||||
if (call.getFunction() != null && call.getFunction().isMetaFunction()) {
|
||||
String errMsg = String.format("Meta function %s does not support non-constant arguments",
|
||||
|
|
@ -410,25 +143,6 @@ public class FoldConstantsRule extends BottomUpScalarOperatorRewriteRule {
|
|||
return ConstantOperator.createNull(operator.getType());
|
||||
}
|
||||
|
||||
ScalarOperator arg0 = operator.getChild(0);
|
||||
if (arg0 instanceof ArrayOperator
|
||||
&& arg0.getChildren().stream().allMatch(ScalarOperator::isConstantRef)
|
||||
&& !arg0.getChildren().isEmpty() && operator.getType().isArrayType()
|
||||
&& ((ArrayType) operator.getType()).getItemType().isScalarType()) {
|
||||
if (notAllConstant(arg0.getChildren())) {
|
||||
return operator;
|
||||
}
|
||||
Type arrayElemType = ((ArrayType) operator.getType()).getItemType();
|
||||
ScalarOperatorRewriteContext subContext = new ScalarOperatorRewriteContext();
|
||||
List<ScalarOperator> newArguments = arg0.getChildren().stream()
|
||||
.map(arg -> visitCastOperator(new CastOperator(arrayElemType, arg), subContext))
|
||||
.collect(Collectors.toList());
|
||||
if (newArguments.stream().allMatch(arg -> arg != null && arg.isConstantRef())) {
|
||||
return new ArrayOperator(operator.getType(), operator.isNullable(), newArguments);
|
||||
}
|
||||
return operator;
|
||||
}
|
||||
|
||||
if (notAllConstant(operator.getChildren())) {
|
||||
return operator;
|
||||
}
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue