[Tool] Support compilation on Mac 8: be code 2 (#63822)

This commit is contained in:
kangkaisen 2025-10-09 16:36:20 +08:00 committed by GitHub
parent 53eb82341c
commit bd3b311d7b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
23 changed files with 564 additions and 194 deletions

View File

@ -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 <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

View File

@ -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<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

View File

@ -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"

View File

@ -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"

View File

@ -176,7 +176,8 @@ 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: {}", 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);

View File

@ -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<JavaUDAFContext>();
#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<JavaUDAFContext>();
#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) {

View File

@ -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<JavaUDAFContext> has a complete type and can be destroyed
// without pulling in JNI headers.
struct JavaUDAFContext {};
#endif
struct NgramBloomFilterState;
class FunctionContext {

View File

@ -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),

View File

@ -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<int>(converted_type)));
}
return converted_type;

View File

@ -149,6 +149,10 @@ 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(); }

View File

@ -170,6 +170,11 @@ 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

View File

@ -359,7 +359,7 @@ public:
DISALLOW_COPY(Promise);
~Promise() {
if (static_cast<bool>(_state) && !_state.unique()) _state->break_promise();
if (static_cast<bool>(_state) && _state.use_count() > 1) _state->break_promise();
}
// Assignment
@ -407,7 +407,7 @@ public:
DISALLOW_COPY(Promise);
~Promise() {
if (static_cast<bool>(_state) && !_state.unique()) _state->break_promise();
if (static_cast<bool>(_state) && _state.use_count() > 1) _state->break_promise();
}
// Assignment
@ -451,7 +451,7 @@ public:
DISALLOW_COPY(Promise);
~Promise() {
if (static_cast<bool>(_state) && !_state.unique()) _state->break_promise();
if (static_cast<bool>(_state) && _state.use_count() > 1) _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

View File

@ -41,6 +41,7 @@
#include <cstring>
#include <iostream>
#include <map>
#include <ostream>
#include <string>
#include <string_view>
#include <vector>

View File

@ -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$")

View File

@ -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=()

View File

@ -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 <endian.h>
#ifndef STARROCKS_MAC_ENDIAN_SHIM_H
#define STARROCKS_MAC_ENDIAN_SHIM_H

View File

@ -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<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

View File

@ -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<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

View File

@ -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

View File

@ -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

View File

@ -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 <malloc.h>
// Keep StarRocks source unchanged by providing a trampoline header.
#ifndef STARROCKS_MAC_MALLOC_SHIM_H

View File

@ -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 <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>

View File

@ -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 <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