[Tool] Support compilation on Mac 7: be code 1 (#63815)

This commit is contained in:
kangkaisen 2025-10-09 10:28:57 +08:00 committed by GitHub
parent 4918a1fadc
commit 05a903eecc
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
22 changed files with 371 additions and 44 deletions

View File

@ -16,8 +16,17 @@
#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 {
@ -45,7 +54,9 @@ 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;

View File

@ -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) {
auto format(const starrocks::config::MutableString& s, format_context& ctx) const {
return formatter<std::string>::format(s.value(), ctx);
}
};

View File

@ -44,6 +44,13 @@
#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"
@ -164,6 +171,12 @@ 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);

View File

@ -18,6 +18,12 @@
#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>
@ -128,7 +134,12 @@ 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
@ -136,7 +147,13 @@ 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 = je_mallctl(buffer, nullptr, nullptr, nullptr, 0);
int ret =
#ifdef __APPLE__
mallctl
#else
je_mallctl
#endif
(buffer, nullptr, nullptr, nullptr, 0);
if (ret != 0) {
FMT_LOG("je_mallctl execute purge failed, errno:{}", ret);
@ -146,7 +163,13 @@ static void dontdump_unused_pages() {
res = snprintf(buffer, MAX_BUFFER_SIZE, "arena.%d.dontdump", MALLCTL_ARENAS_ALL);
buffer[res] = '\0';
ret = je_mallctl(buffer, nullptr, nullptr, nullptr, 0);
ret =
#ifdef __APPLE__
mallctl
#else
je_mallctl
#endif
(buffer, nullptr, nullptr, nullptr, 0);
if (ret != 0) {
FMT_LOG("je_mallctl execute dontdump failed, errno:{}", ret);
@ -199,8 +222,10 @@ 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.
// Set roll num. Not available with Homebrew glog on macOS.
#ifndef __APPLE__
FLAGS_log_filenum_quota = config::sys_log_roll_num;
#endif
// Set log level.
std::string loglevel = config::sys_log_level;
@ -230,13 +255,19 @@ 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;
@ -278,7 +309,10 @@ 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;

View File

@ -27,21 +27,45 @@
namespace starrocks {
// detail implements for allocator
static int set_jemalloc_profiling(bool enable) {
int ret = je_mallctl("prof.active", nullptr, nullptr, &enable, 1);
ret |= je_mallctl("prof.thread_active_init", nullptr, nullptr, &enable, 1);
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);
return ret;
}
static int has_enable_heap_profile() {
int value = 0;
size_t size = sizeof(value);
je_mallctl("prof.active", &value, &size, nullptr, 0);
#ifdef __APPLE__
mallctl
#else
je_mallctl
#endif
("prof.active", &value, &size, nullptr, 0);
return value;
}
bool dump_snapshot(const std::string& filename) {
const char* fname = filename.c_str();
return je_mallctl("prof.dump", nullptr, nullptr, &fname, sizeof(const char*)) == 0;
return (
#ifdef __APPLE__
mallctl
#else
je_mallctl
#endif
("prof.dump", nullptr, nullptr, &fname, sizeof(const char*))) == 0;
}
// declare exec from script
@ -85,4 +109,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

View File

@ -18,6 +18,14 @@
#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"

View File

@ -14,7 +14,11 @@
#pragma once
#ifdef __APPLE__
#include <sys/mount.h>
#else
#include <sys/statfs.h>
#endif
#include <atomic>
#include <memory>
@ -101,7 +105,11 @@ 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;
@ -117,4 +125,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

View File

@ -28,6 +28,11 @@
#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__)

View File

@ -44,6 +44,13 @@ 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 {
@ -295,7 +302,8 @@ public:
if constexpr (rule == ROUND_HALF_UP || rule == ROUND_HALF_EVEN) {
//TODO(by satanson): ROUND_HALF_UP is different from ROUND_HALF_EVEN
need_round = std::abs(remainder) >= (divisor >> 1);
auto abs_remainder = (remainder >= 0) ? remainder : -remainder;
need_round = abs_remainder >= (divisor >> 1);
} else if constexpr (rule == ROUND_FLOOR) {
need_round = remainder > 0 && quotient > 0;
} else if constexpr (rule == ROUND_CEILING) {

View File

@ -713,29 +713,6 @@ 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(); }) {

View File

@ -187,7 +187,22 @@ 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;
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 RuntimeFilterWorkerMetrics {
void update_event_nums(EventType event_type, int64_t delta) { event_nums[event_type] += delta; }

View File

@ -181,6 +181,12 @@ 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
// =============================================================================
@ -913,6 +919,8 @@ 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;
@ -923,6 +931,7 @@ template <>
struct make_unsigned<starrocks::int256_t> {
using type = starrocks::int256_t;
};
#endif // !__APPLE__
/// Hash function specialization for int256_t
template <>
@ -961,8 +970,9 @@ 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) {
auto format(const starrocks::int256_t& value, FormatContext& ctx) const {
return formatter<std::string>::format(value.to_string(), ctx);
}
};

View File

@ -24,6 +24,22 @@
#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"

View File

@ -328,8 +328,8 @@ public:
void set_capacity(size_t capacity) override;
size_t get_memory_usage() const override;
size_t get_capacity() const override;
uint64_t get_lookup_count() const override;
uint64_t get_hit_count() const override;
size_t get_lookup_count() const override;
size_t get_hit_count() const override;
bool adjust_capacity(int64_t delta, size_t min_capacity = 0) override;
private:

View File

@ -278,6 +278,7 @@ 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()) {
@ -286,6 +287,7 @@ void StarRocksMetrics::initialize(const std::vector<std::string>& paths, bool in
}
_jvm_metrics.install(&_metrics);
}
#endif
}
void StarRocksMetrics::_update() {

View File

@ -40,7 +40,9 @@
#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"
@ -437,7 +439,9 @@ private:
MetricRegistry _metrics;
SystemMetrics _system_metrics;
#ifndef MACOS_DISABLE_JAVA
JVMMetrics _jvm_metrics;
#endif
TableMetricsManager _table_metrics_mgr;
};

View File

@ -35,7 +35,11 @@
#pragma once
#include <pthread.h>
#ifdef __APPLE__
#include <sys/syscall.h>
#else
#include <syscall.h>
#endif
#include <atomic>
#include <thread>

View File

@ -204,6 +204,11 @@ 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)
# 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")
@ -413,10 +418,17 @@ set_target_properties(zlib PROPERTIES
IMPORTED_LOCATION ${ZLIB_LIB}
INTERFACE_INCLUDE_DIRECTORIES /opt/homebrew/include)
# vectorscan (hyperscan replacement) from thirdparty
# vectorscan (hyperscan-compatible) from thirdparty
# Vectorscan installs libhs.a and libhs_runtime.a (hyperscan ABI)
add_library(vectorscan STATIC IMPORTED GLOBAL)
set_target_properties(vectorscan PROPERTIES
IMPORTED_LOCATION ${THIRDPARTY_DIR}/installed/lib/libvectorscan.a
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
INTERFACE_INCLUDE_DIRECTORIES ${THIRDPARTY_DIR}/installed/include)
# roaring bitmap from homebrew (dynamic library)
@ -516,7 +528,8 @@ set(CMAKE_SKIP_RPATH TRUE)
set(CMAKE_EXPORT_COMPILE_COMMANDS ON)
# Base compiler flags for ARM64
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -march=armv8-a")
# Enable CRC32 intrinsics on ARM64
set(CXX_COMMON_FLAGS "${CXX_COMMON_FLAGS} -march=armv8-a+crc")
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")
@ -537,6 +550,12 @@ 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")
@ -559,7 +578,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}")
set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${CXX_COMMON_FLAGS} -Wno-everything")
# ==============================================================================
# LINK LIBRARIES CONFIGURATION
@ -586,6 +605,7 @@ list(APPEND STARROCKS_LINK_LIBS
snappy
zlib
vectorscan
hs_runtime
roaring
arrow
parquet
@ -637,6 +657,7 @@ 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)
@ -702,6 +723,11 @@ 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 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$")
@ -744,6 +770,7 @@ 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$")
@ -779,6 +806,9 @@ 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$")
@ -873,4 +903,4 @@ message(STATUS "To build:")
message(STATUS " ninja starrocks_be")
message(STATUS " ninja install")
message(STATUS "========================================")
message(STATUS "")
message(STATUS "")

View File

@ -205,6 +205,9 @@ if [[ $SKIP_THIRDPARTY -eq 0 ]]; then
"librocksdb.a"
"libvelocypack.a"
"libbitshuffle.a"
# vectorscan (hyperscan ABI)
"libhs.a"
"libhs_runtime.a"
)
missing_libs=()
@ -430,4 +433,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!"

View File

@ -204,6 +204,11 @@ BITSHUFFLE_VERSION="0.5.1"
VECTORSCAN_VERSION="5.4.12"
VELOCYPACK_VERSION="XYZ1.0"
# 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"
download_source() {
local name="$1"
local version="$2"
@ -698,6 +703,59 @@ build_bitshuffle() {
log_success "bitshuffle built successfully"
}
# 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"
}
detect_boost_version() {
# Detect Boost version from Homebrew installation
@ -820,6 +878,7 @@ build_source_deps() {
build_glog
build_protobuf
build_leveldb
build_ryu
# Layer 2: Libraries that depend on Layer 1
build_brpc
@ -838,6 +897,7 @@ 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
log_success "All source dependencies built successfully"
}
@ -867,6 +927,8 @@ main() {
"libvelocypack.a"
"libbitshuffle.a"
"libhs.a"
"libhs_runtime.a"
"libryu.a"
)
local missing_libs=()

View File

@ -0,0 +1,61 @@
// 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

View File

@ -0,0 +1,32 @@
// 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