From bd3b311d7be525b44658e2abbe7c98d2a9a641fa Mon Sep 17 00:00:00 2001 From: kangkaisen Date: Thu, 9 Oct 2025 16:36:20 +0800 Subject: [PATCH] [Tool] Support compilation on Mac 8: be code 2 (#63822) --- .../pipeline/scan/olap_meta_scan_operator.cpp | 110 ----------- .../pipeline/scan/olap_meta_scan_operator.h | 73 ------- be/src/exec/tablet_sink.cpp | 2 +- be/src/exec/tablet_sink_index_channel.cpp | 2 +- be/src/exprs/cast_expr_array.cpp | 3 +- be/src/exprs/function_context.cpp | 10 +- be/src/exprs/function_context.h | 6 + be/src/formats/parquet/page_reader.cpp | 2 +- .../formats/parquet/parquet_file_writer.cpp | 3 +- be/src/runtime/decimalv2_value.h | 4 + be/src/types/int256.h | 5 + be/src/util/bthreads/future.h | 8 +- be/src/util/slice.h | 1 + build-mac/CMakeLists.txt | 14 ++ build-mac/build_thirdparty.sh | 144 ++++++++++++++ build-mac/shims/include/endian.h | 14 ++ .../exec/file_scanner/avro_cpp_scanner.h | 40 ++++ .../include/exec/file_scanner/orc_scanner.h | 40 ++++ build-mac/shims/include/fiu/fiu-control.h | 32 ++++ build-mac/shims/include/fiu/fiu.h | 27 +++ build-mac/shims/include/malloc.h | 14 ++ .../shims/include/protocol/TDebugProtocol.h | 24 +++ .../shims/include/starrocks_macos_fmt_shims.h | 180 ++++++++++++++++++ 23 files changed, 564 insertions(+), 194 deletions(-) delete mode 100644 be/src/exec/pipeline/scan/olap_meta_scan_operator.cpp delete mode 100644 be/src/exec/pipeline/scan/olap_meta_scan_operator.h create mode 100644 build-mac/shims/include/exec/file_scanner/avro_cpp_scanner.h create mode 100644 build-mac/shims/include/exec/file_scanner/orc_scanner.h create mode 100644 build-mac/shims/include/fiu/fiu-control.h create mode 100644 build-mac/shims/include/fiu/fiu.h create mode 100644 build-mac/shims/include/protocol/TDebugProtocol.h create mode 100644 build-mac/shims/include/starrocks_macos_fmt_shims.h diff --git a/be/src/exec/pipeline/scan/olap_meta_scan_operator.cpp b/be/src/exec/pipeline/scan/olap_meta_scan_operator.cpp deleted file mode 100644 index 060b2d06095..00000000000 --- a/be/src/exec/pipeline/scan/olap_meta_scan_operator.cpp +++ /dev/null @@ -1,110 +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. - -#include "exec/pipeline/scan/olap_meta_scan_operator.h" - -#include - -#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 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(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(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 diff --git a/be/src/exec/pipeline/scan/olap_meta_scan_operator.h b/be/src/exec/pipeline/scan/olap_meta_scan_operator.h deleted file mode 100644 index c5ef1bc8ca4..00000000000 --- a/be/src/exec/pipeline/scan/olap_meta_scan_operator.h +++ /dev/null @@ -1,73 +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. - -#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 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 _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 diff --git a/be/src/exec/tablet_sink.cpp b/be/src/exec/tablet_sink.cpp index 525a64a7646..c0e955a1098 100644 --- a/be/src/exec/tablet_sink.cpp +++ b/be/src/exec/tablet_sink.cpp @@ -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" diff --git a/be/src/exec/tablet_sink_index_channel.cpp b/be/src/exec/tablet_sink_index_channel.cpp index 776038acc5c..8c1089d063e 100644 --- a/be/src/exec/tablet_sink_index_channel.cpp +++ b/be/src/exec/tablet_sink_index_channel.cpp @@ -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" diff --git a/be/src/exprs/cast_expr_array.cpp b/be/src/exprs/cast_expr_array.cpp index 719aedf9f92..c0c56aab887 100644 --- a/be/src/exprs/cast_expr_array.cpp +++ b/be/src/exprs/cast_expr_array.cpp @@ -176,7 +176,8 @@ StatusOr 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: {}", str)); + return Status::InternalError( + fmt::format("invalid array input: {}", std::string_view(str.get_data(), str.get_size()))); } else { has_null = true; null_column->append(1); diff --git a/be/src/exprs/function_context.cpp b/be/src/exprs/function_context.cpp index 0fa3b107019..8ed8fe4f293 100644 --- a/be/src/exprs/function_context.cpp +++ b/be/src/exprs/function_context.cpp @@ -20,10 +20,14 @@ #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 { @@ -38,7 +42,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(BUILD_FORMAT_LIB) +#if !defined(MACOS_DISABLE_JAVA) && !defined(BUILD_FORMAT_LIB) ctx->_jvm_udaf_ctxs = std::make_unique(); #endif return ctx; @@ -54,7 +58,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(BUILD_FORMAT_LIB) +#if !defined(MACOS_DISABLE_JAVA) && !defined(BUILD_FORMAT_LIB) ctx->_jvm_udaf_ctxs = std::make_unique(); #endif ctx->_is_distinct = is_distinct; @@ -137,10 +141,12 @@ 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) { diff --git a/be/src/exprs/function_context.h b/be/src/exprs/function_context.h index 549dcb71e90..a57f6a3e83c 100644 --- a/be/src/exprs/function_context.h +++ b/be/src/exprs/function_context.h @@ -35,6 +35,12 @@ 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 has a complete type and can be destroyed +// without pulling in JNI headers. +struct JavaUDAFContext {}; +#endif struct NgramBloomFilterState; class FunctionContext { diff --git a/be/src/formats/parquet/page_reader.cpp b/be/src/formats/parquet/page_reader.cpp index 38e44e81588..84ca785abeb 100644 --- a/be/src/formats/parquet/page_reader.cpp +++ b/be/src/formats/parquet/page_reader.cpp @@ -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, uint64_t start_offset, uint64_t length, uint64_t num_values, +PageReader::PageReader(io::SeekableInputStream* stream, size_t start_offset, size_t length, size_t num_values, const ColumnReaderOptions& opts, const tparquet::CompressionCodec::type codec) : _stream(stream), _finish_offset(start_offset + length), diff --git a/be/src/formats/parquet/parquet_file_writer.cpp b/be/src/formats/parquet/parquet_file_writer.cpp index e3d754d5c6a..cfe2a483e67 100644 --- a/be/src/formats/parquet/parquet_file_writer.cpp +++ b/be/src/formats/parquet/parquet_file_writer.cpp @@ -266,7 +266,8 @@ 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 {}", converted_type)); + return Status::NotSupported( + fmt::format("not supported compression codec {}", static_cast(converted_type))); } return converted_type; diff --git a/be/src/runtime/decimalv2_value.h b/be/src/runtime/decimalv2_value.h index 00607688ccc..d321eed65c5 100644 --- a/be/src/runtime/decimalv2_value.h +++ b/be/src/runtime/decimalv2_value.h @@ -149,6 +149,10 @@ public: operator int32_t() const { return static_cast(operator int64_t()); } operator size_t() const { return static_cast(operator int64_t()); } +#if defined(__APPLE__) + // Disambiguate C-style casts to uint64_t on libc++ + operator uint64_t() const { return static_cast(operator int64_t()); } +#endif operator float() const { return (float)operator double(); } diff --git a/be/src/types/int256.h b/be/src/types/int256.h index d781857d5dc..b8c13ab1e4f 100644 --- a/be/src/types/int256.h +++ b/be/src/types/int256.h @@ -170,6 +170,11 @@ public: explicit operator long() const { return static_cast(static_cast(low)); } +#if defined(__APPLE__) + /// macOS/libc++: provide explicit conversion to 64-bit signed + explicit operator long long() const { return static_cast(static_cast(low)); } +#endif + explicit operator int128_t() const { return static_cast<__int128>(low); } /// Convert to signed char diff --git a/be/src/util/bthreads/future.h b/be/src/util/bthreads/future.h index 9886fd4ba89..9db88d00c6a 100644 --- a/be/src/util/bthreads/future.h +++ b/be/src/util/bthreads/future.h @@ -359,7 +359,7 @@ public: DISALLOW_COPY(Promise); ~Promise() { - if (static_cast(_state) && !_state.unique()) _state->break_promise(); + if (static_cast(_state) && _state.use_count() > 1) _state->break_promise(); } // Assignment @@ -407,7 +407,7 @@ public: DISALLOW_COPY(Promise); ~Promise() { - if (static_cast(_state) && !_state.unique()) _state->break_promise(); + if (static_cast(_state) && _state.use_count() > 1) _state->break_promise(); } // Assignment @@ -451,7 +451,7 @@ public: DISALLOW_COPY(Promise); ~Promise() { - if (static_cast(_state) && !_state.unique()) _state->break_promise(); + if (static_cast(_state) && _state.use_count() > 1) _state->break_promise(); } // Assignment @@ -484,4 +484,4 @@ inline void swap(Promise& x, Promise& y) noexcept { x.swap(y); } -} // namespace starrocks::bthreads \ No newline at end of file +} // namespace starrocks::bthreads diff --git a/be/src/util/slice.h b/be/src/util/slice.h index b7ab5119c57..583a8069c34 100644 --- a/be/src/util/slice.h +++ b/be/src/util/slice.h @@ -41,6 +41,7 @@ #include #include #include +#include #include #include #include diff --git a/build-mac/CMakeLists.txt b/build-mac/CMakeLists.txt index 18dda3eb4c0..b810bf9b1ed 100644 --- a/build-mac/CMakeLists.txt +++ b/build-mac/CMakeLists.txt @@ -207,6 +207,12 @@ 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) @@ -723,6 +729,13 @@ 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$") @@ -761,6 +774,7 @@ 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$") diff --git a/build-mac/build_thirdparty.sh b/build-mac/build_thirdparty.sh index 14de88ac758..b89cb5308d3 100755 --- a/build-mac/build_thirdparty.sh +++ b/build-mac/build_thirdparty.sh @@ -204,11 +204,29 @@ 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" @@ -703,6 +721,46 @@ 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 @@ -756,6 +814,85 @@ build_ryu() { 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 @@ -878,7 +1015,10 @@ 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 @@ -898,6 +1038,8 @@ build_source_deps() { 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" } @@ -929,6 +1071,8 @@ main() { "libhs.a" "libhs_runtime.a" "libryu.a" + "libicuuc.a" + "libicui18n.a" ) local missing_libs=() diff --git a/build-mac/shims/include/endian.h b/build-mac/shims/include/endian.h index e70daab4b55..dd419ec0a3c 100644 --- a/build-mac/shims/include/endian.h +++ b/build-mac/shims/include/endian.h @@ -1,3 +1,17 @@ +// 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 #ifndef STARROCKS_MAC_ENDIAN_SHIM_H #define STARROCKS_MAC_ENDIAN_SHIM_H diff --git a/build-mac/shims/include/exec/file_scanner/avro_cpp_scanner.h b/build-mac/shims/include/exec/file_scanner/avro_cpp_scanner.h new file mode 100644 index 00000000000..f27eb237f84 --- /dev/null +++ b/build-mac/shims/include/exec/file_scanner/avro_cpp_scanner.h @@ -0,0 +1,40 @@ +// 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 get_next() override { return Status::NotSupported("Avro is disabled on macOS build"); } + Status get_schema(std::vector* /*schema*/) override { + return Status::NotSupported("Avro is disabled on macOS build"); + } + +private: + const TBrokerScanRange& _scan_range; +}; + +} // namespace starrocks + diff --git a/build-mac/shims/include/exec/file_scanner/orc_scanner.h b/build-mac/shims/include/exec/file_scanner/orc_scanner.h new file mode 100644 index 00000000000..014fbdf4c8f --- /dev/null +++ b/build-mac/shims/include/exec/file_scanner/orc_scanner.h @@ -0,0 +1,40 @@ +// 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 get_next() override { return Status::NotSupported("ORC is disabled on macOS build"); } + Status get_schema(std::vector* /*schema*/) override { + return Status::NotSupported("ORC is disabled on macOS build"); + } + void close() override {} + +private: + const TBrokerScanRange& _scan_range; +}; + +} // namespace starrocks + diff --git a/build-mac/shims/include/fiu/fiu-control.h b/build-mac/shims/include/fiu/fiu-control.h new file mode 100644 index 00000000000..bf4247bef49 --- /dev/null +++ b/build-mac/shims/include/fiu/fiu-control.h @@ -0,0 +1,32 @@ +// 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 + diff --git a/build-mac/shims/include/fiu/fiu.h b/build-mac/shims/include/fiu/fiu.h new file mode 100644 index 00000000000..96295f6b5e2 --- /dev/null +++ b/build-mac/shims/include/fiu/fiu.h @@ -0,0 +1,27 @@ +// 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 + diff --git a/build-mac/shims/include/malloc.h b/build-mac/shims/include/malloc.h index 8bb879ebdbf..8a31b559dad 100644 --- a/build-mac/shims/include/malloc.h +++ b/build-mac/shims/include/malloc.h @@ -1,3 +1,17 @@ +// 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 // Keep StarRocks source unchanged by providing a trampoline header. #ifndef STARROCKS_MAC_MALLOC_SHIM_H diff --git a/build-mac/shims/include/protocol/TDebugProtocol.h b/build-mac/shims/include/protocol/TDebugProtocol.h new file mode 100644 index 00000000000..d037826db64 --- /dev/null +++ b/build-mac/shims/include/protocol/TDebugProtocol.h @@ -0,0 +1,24 @@ +// 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 assuming the include path +// points at the 'thrift' headers directory. On macOS, Homebrew installs +// headers under . This shim bridges the path without touching +// StarRocks source files. + +#pragma once + +#include + diff --git a/build-mac/shims/include/starrocks_macos_fmt_shims.h b/build-mac/shims/include/starrocks_macos_fmt_shims.h new file mode 100644 index 00000000000..f145abb88d7 --- /dev/null +++ b/build-mac/shims/include/starrocks_macos_fmt_shims.h @@ -0,0 +1,180 @@ +// 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 +// Enable formatting via operator<< when available +#include +// Enable fmt::join and range printing +#include + +// 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 +#include +#include // ensure integer std::abs overloads are visible on libc++ +#include + +// 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 +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 , int> = 0> +inline auto format_as(E e) -> std::underlying_type_t { + using U = std::underlying_type_t; + return static_cast(e); +} +} // namespace starrocks + +namespace starrocks::pipeline { +template , int> = 0> +inline auto format_as(E e) -> std::underlying_type_t { + using U = std::underlying_type_t; + return static_cast(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 +namespace std { +inline constexpr long long max(long a, long long b) { + return std::max(static_cast(a), b); +} +inline constexpr long long max(long long a, long b) { + return std::max(a, static_cast(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 by formatting the loaded value. +template +struct fmt::formatter> : fmt::formatter { + template + auto format(const std::atomic& v, FormatContext& ctx) const { + return fmt::formatter::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" 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 : formatter { + template + auto format(starrocks::TFileType::type v, FormatContext& ctx) const { + return formatter::format(static_cast(v), ctx); + } +}; + +template <> +struct formatter : formatter { + template + auto format(starrocks::StreamSourceType::type v, FormatContext& ctx) const { + return formatter::format(static_cast(v), ctx); + } +}; + +// Formatter for thrift enum used in ArrowFunctionCall +template <> +struct formatter : formatter { + template + auto format(starrocks::TFunctionBinaryType::type v, FormatContext& ctx) const { + return formatter::format(static_cast(v), ctx); + } +}; + +// Formatter for TExprNodeType thrift enum used in expr.cpp +template <> +struct formatter : formatter { + template + auto format(starrocks::TExprNodeType::type v, FormatContext& ctx) const { + return formatter::format(static_cast(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(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 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 +auto join(const Sequence& sequence, const Separator& separator) + -> std::enable_if_t, 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