From a13dadf0352471b16daf3d50a3d7c3840da8a90e Mon Sep 17 00:00:00 2001 From: wsa Date: Thu, 29 Jan 2026 17:44:21 +0800 Subject: [PATCH] remote compaction demo --- setup_local_nfs.sh | 114 ++ src/tendisplus/server/server_params.cpp | 25 +- src/tendisplus/server/server_params.h | 23 + src/tendisplus/storage/rocks/CMakeLists.txt | 133 +- src/tendisplus/storage/rocks/cmake/grpc.cmake | 128 ++ .../storage/rocks/compaction_service.cc | 554 ++++++ .../storage/rocks/compaction_service.h | 119 ++ src/tendisplus/storage/rocks/nfs_file.cc | 231 +++ .../storage/rocks/nfs_filesystem.cc | 1498 +++++++++++++++++ src/tendisplus/storage/rocks/nfs_filesystem.h | 223 +++ .../rocks/remote_compaction/csa_server.cc | 537 ++++++ .../storage/rocks/remote_compaction/def.h | 72 + .../rocks/remote_compaction/protos/csa.proto | 73 + .../remote_compaction_config.h | 57 + .../storage/rocks/rocks_kvstore.cpp | 251 +++ src/tendisplus/storage/rocks/rocks_kvstore.h | 5 + .../storage/rocks/shared_filesystem.cc | 233 +++ .../storage/rocks/shared_filesystem.h | 46 + tendisplus_rc.conf | 194 +++ 19 files changed, 4512 insertions(+), 4 deletions(-) create mode 100755 setup_local_nfs.sh create mode 100644 src/tendisplus/storage/rocks/cmake/grpc.cmake create mode 100644 src/tendisplus/storage/rocks/compaction_service.cc create mode 100644 src/tendisplus/storage/rocks/compaction_service.h create mode 100644 src/tendisplus/storage/rocks/nfs_file.cc create mode 100644 src/tendisplus/storage/rocks/nfs_filesystem.cc create mode 100644 src/tendisplus/storage/rocks/nfs_filesystem.h create mode 100644 src/tendisplus/storage/rocks/remote_compaction/csa_server.cc create mode 100644 src/tendisplus/storage/rocks/remote_compaction/def.h create mode 100644 src/tendisplus/storage/rocks/remote_compaction/protos/csa.proto create mode 100644 src/tendisplus/storage/rocks/remote_compaction/remote_compaction_config.h create mode 100644 src/tendisplus/storage/rocks/shared_filesystem.cc create mode 100644 src/tendisplus/storage/rocks/shared_filesystem.h create mode 100644 tendisplus_rc.conf diff --git a/setup_local_nfs.sh b/setup_local_nfs.sh new file mode 100755 index 00000000..369ce029 --- /dev/null +++ b/setup_local_nfs.sh @@ -0,0 +1,114 @@ +#!/bin/bash +# Local NFS Setup Script - Supports CentOS/RHEL/TencentOS/Ubuntu/Debian/Fedora +set -e + +SHARE_PATH="/shared/rocksdb" +MOUNT_PATH="/mnt/rocksdb" +LOCAL_DB_PATH="./home/db" +CLEAN_MODE=false + +# Detect OS and set variables +detect_os() { + [ -f /etc/os-release ] && . /etc/os-release || OS_ID="centos" + OS_ID="${ID:-$OS_ID}" + + case "$OS_ID" in + ubuntu|debian) + PKG_MGR="apt-get"; NFS_PKG="nfs-kernel-server nfs-common"; NFS_SVC="nfs-kernel-server" ;; + centos|rhel|rocky|almalinux|ol|tencentos|tlinux|opencloudos|anolis) + PKG_MGR="yum"; NFS_PKG="nfs-utils"; NFS_SVC="nfs-server" ;; + fedora) + PKG_MGR="dnf"; NFS_PKG="nfs-utils"; NFS_SVC="nfs-server" ;; + *) echo "Unsupported OS: $OS_ID"; exit 1 ;; + esac + echo "OS: $OS_ID" +} + +# Install NFS +install_nfs() { + echo "=== Installing NFS ===" + case "$PKG_MGR" in + apt-get) sudo apt-get update -qq && sudo DEBIAN_FRONTEND=noninteractive apt-get install -y $NFS_PKG ;; + *) sudo $PKG_MGR install -y $NFS_PKG ;; + esac +} + +# Start NFS services +start_nfs() { + echo "=== Starting NFS ===" + sudo systemctl start rpcbind $NFS_SVC 2>/dev/null || true + sudo systemctl enable rpcbind $NFS_SVC 2>/dev/null || true + sleep 2 +} + +# Clean data +clean_data() { + echo "=== Cleaning Data ===" + echo "Share path: $SHARE_PATH" + echo "Local DB path: $LOCAL_DB_PATH" + rm -rf "$SHARE_PATH"/* 2>/dev/null || true + rm -rf "$LOCAL_DB_PATH"/* 2>/dev/null || true + echo "Done" +} + +# Usage +usage() { + cat < $MOUNT_PATH ===" +detect_os +install_nfs + +# Create share dir +echo "=== Creating Directories ===" +sudo mkdir -p "$SHARE_PATH" "$MOUNT_PATH" +sudo chown -R "$(id -un):$(id -gn)" "$SHARE_PATH" +chmod 755 "$SHARE_PATH" + +# Configure exports +echo "=== Configuring Exports ===" +EXPORT_LINE="$SHARE_PATH *(rw,sync,no_root_squash,no_subtree_check,insecure)" +grep -q "^$SHARE_PATH " /etc/exports 2>/dev/null && \ + sudo sed -i "s|^$SHARE_PATH .*|$EXPORT_LINE|" /etc/exports || \ + echo "$EXPORT_LINE" | sudo tee -a /etc/exports + +start_nfs +sudo exportfs -arv +showmount -e localhost + +# Mount +echo "=== Mounting ===" +mountpoint -q "$MOUNT_PATH" 2>/dev/null && sudo umount "$MOUNT_PATH" +sudo mount -t nfs localhost:"$SHARE_PATH" "$MOUNT_PATH" + +# Verify +echo "=== Verifying ===" +echo 'test' > "$SHARE_PATH/test.txt" && cat "$MOUNT_PATH/test.txt" && rm "$MOUNT_PATH/test.txt" +echo "=== Complete: $SHARE_PATH mounted at $MOUNT_PATH ===" diff --git a/src/tendisplus/server/server_params.cpp b/src/tendisplus/server/server_params.cpp index 9816636b..31c3173c 100644 --- a/src/tendisplus/server/server_params.cpp +++ b/src/tendisplus/server/server_params.cpp @@ -173,7 +173,7 @@ std::string removeQuotes(const std::string& v) { } auto tmp = v; - if (tmp[0] == '\"' && tmp[tmp.size() - 1] == '\"') { + if (tmp[0] == '"' && tmp[tmp.size() - 1] == '"') { tmp = tmp.substr(1, tmp.size() - 2); } return tmp; @@ -185,7 +185,7 @@ std::string removeQuotesAndToLower(const std::string& v) { return tmp; } - if (tmp[0] == '\"' && tmp[tmp.size() - 1] == '\"') { + if (tmp[0] == '"' && tmp[tmp.size() - 1] == '"') { tmp = tmp.substr(1, tmp.size() - 2); } return tmp; @@ -620,6 +620,27 @@ ServerParams::ServerParams() { enableClosePubSubConnection); REGISTER_VARS_DIFF_NAME_DYNAMIC("enable-move-pubsub-request", enableMovePubSubRequest); + + // Remote Compaction Service (CSA) Configuration + REGISTER_VARS_DIFF_NAME("csa_address", csaAddress); + // Note: remote_compaction.mode removed - only shared_storage mode is + // supported + + // Shared filesystem configuration (unified for NFS/HDFS/S3/etc.) + REGISTER_VARS_DIFF_NAME("remote_compaction.shared_fs_uri", + remoteCompactionSharedFsUri); + REGISTER_VARS_DIFF_NAME("remote_compaction.shared_fs_local_prefix", + remoteCompactionSharedFsLocalPrefix); + + // Remote Compaction Advanced Settings + REGISTER_VARS_DIFF_NAME("remote_compaction.max_concurrent_tasks", + remoteCompactionMaxConcurrentTasks); + REGISTER_VARS_DIFF_NAME("remote_compaction.grpc_max_message_size", + remoteCompactionGrpcMaxMessageSize); + REGISTER_VARS_DIFF_NAME("remote_compaction.check_time_interval", + remoteCompactionCheckTimeInterval); + REGISTER_VARS_DIFF_NAME("remote_compaction.max_reschedule", + remoteCompactionMaxReschedule); } ServerParams::~ServerParams() { diff --git a/src/tendisplus/server/server_params.h b/src/tendisplus/server/server_params.h index 66c01db8..2c25cd2b 100644 --- a/src/tendisplus/server/server_params.h +++ b/src/tendisplus/server/server_params.h @@ -644,6 +644,29 @@ class ServerParams { bool enableClosePubSubConnection = true; bool enableMovePubSubRequest = true; + + // Remote Compaction Service (CSA) Configuration + std::string csaAddress = + ""; // CSA server address (host:port), e.g., "localhost:8010" + // Note: remote compaction only supports shared_storage mode (no configuration + // needed) + + // Shared filesystem configuration (unified for NFS/HDFS/S3/etc.) + // URI format: "nfs://host/path", "hdfs://host:port/path", + // "s3://bucket/prefix" + std::string remoteCompactionSharedFsUri = ""; // Shared filesystem URI + std::string remoteCompactionSharedFsLocalPrefix = + ""; // Optional local path prefix (for backward compatibility) + + // Remote Compaction Advanced Settings + int64_t remoteCompactionMaxConcurrentTasks = + 0; // Max concurrent compaction tasks (0 = use default) + int64_t remoteCompactionGrpcMaxMessageSize = + 0; // Max gRPC message size in bytes (0 = use default 16MB) + int32_t remoteCompactionCheckTimeInterval = + 0; // Check time interval in seconds (0 = use default) + uint64_t remoteCompactionMaxReschedule = + 0; // Max reschedule times (0 = use default) }; extern std::shared_ptr gParams; diff --git a/src/tendisplus/storage/rocks/CMakeLists.txt b/src/tendisplus/storage/rocks/CMakeLists.txt index 32423bad..276703b2 100644 --- a/src/tendisplus/storage/rocks/CMakeLists.txt +++ b/src/tendisplus/storage/rocks/CMakeLists.txt @@ -1,5 +1,134 @@ -add_library(rocks_kvstore STATIC rocks_kvstore.cpp rocks_kvttlcompactfilter.cpp) -target_link_libraries(rocks_kvstore utils_common kvstore rocksdb record glog ${SYS_LIBS} snappy lz4_static) +include(cmake/grpc.cmake) +# Proto file +get_filename_component(csa_proto "remote_compaction/protos/csa.proto" ABSOLUTE) +get_filename_component(csa_proto_path "${csa_proto}" PATH) + +# Generated sources +set(csa_proto_srcs "${CMAKE_CURRENT_BINARY_DIR}/csa.pb.cc") +set(csa_proto_hdrs "${CMAKE_CURRENT_BINARY_DIR}/csa.pb.h") +set(csa_grpc_srcs "${CMAKE_CURRENT_BINARY_DIR}/csa.grpc.pb.cc") +set(csa_grpc_hdrs "${CMAKE_CURRENT_BINARY_DIR}/csa.grpc.pb.h") +add_custom_command( + OUTPUT "${csa_proto_srcs}" "${csa_proto_hdrs}" "${csa_grpc_srcs}" "${csa_grpc_hdrs}" + COMMAND ${_PROTOBUF_PROTOC} + ARGS --grpc_out "${CMAKE_CURRENT_BINARY_DIR}" + --cpp_out "${CMAKE_CURRENT_BINARY_DIR}" + -I "${csa_proto_path}" + --plugin=protoc-gen-grpc="${_GRPC_CPP_PLUGIN_EXECUTABLE}" + "${csa_proto}" + DEPENDS "${csa_proto}") + +# Include generated *.pb.h files +include_directories("${CMAKE_CURRENT_BINARY_DIR}") + +# csa_grpc_proto +add_library(csa_grpc_proto + ${csa_grpc_srcs} + ${csa_grpc_hdrs} + ${csa_proto_srcs} + ${csa_proto_hdrs}) +target_link_libraries(csa_grpc_proto + ${_REFLECTION} + ${_GRPC_GRPCPP} + ${_PROTOBUF_LIBPROTOBUF}) + +# compaction_service_helper +#add_library(compaction_service_helper +# "db/compaction/remote_compaction/job_distributor.h" +# "db/compaction/remote_compaction/job_distributor.cc") +#target_link_libraries(compaction_service_helper +# compaction_service_grpc_proto +# ${_REFLECTION} +# ${_GRPC_GRPCPP} +# ${_PROTOBUF_LIBPROTOBUF}) + +# Targets compaction_service_(client|server) +find_package(PkgConfig) +if(PkgConfig_FOUND) + pkg_check_modules(LIBNFS libnfs) +endif() + +if(NOT LIBNFS_FOUND) + # 查找头文件 + find_path(LIBNFS_INCLUDE_DIR + NAMES nfsc/libnfs.h + PATHS + /usr/include + /usr/local/include + /opt/libnfs/include + $ENV{LIBNFS_ROOT}/include + PATH_SUFFIXES nfs + ) + + # 查找库文件 + find_library(LIBNFS_LIBRARY + NAMES nfs + PATHS + /usr/lib + /usr/local/lib + /usr/lib/x86_64-linux-gnu + /usr/lib64 + /opt/libnfs/lib + $ENV{LIBNFS_ROOT}/lib + ) + + # 检查是否找到 + if(LIBNFS_INCLUDE_DIR AND LIBNFS_LIBRARY) + set(LIBNFS_FOUND TRUE) + set(LIBNFS_INCLUDE_DIRS ${LIBNFS_INCLUDE_DIR}) + set(LIBNFS_LIBRARIES ${LIBNFS_LIBRARY}) + message(STATUS "Found libnfs: ${LIBNFS_LIBRARY}") + else() + message(FATAL_ERROR "libnfs not found! Please install libnfs-dev or set LIBNFS_ROOT environment variable") + endif() +endif() +add_library(nfs_fs STATIC nfs_filesystem.cc nfs_file.cc nfs_filesystem_simple.cc shared_filesystem.cc) +# 添加头文件路径 +target_include_directories(nfs_fs + PUBLIC + ${LIBNFS_INCLUDE_DIRS} +) + +# 链接 libnfs +target_link_libraries(nfs_fs + PUBLIC + rocksdb + ${LIBNFS_LIBRARIES} +) + + +foreach(_target csa_server) + add_executable(${_target} + "remote_compaction/${_target}.cc") + add_dependencies(${_target} csa_grpc_proto) + add_dependencies(${_target} nfs_fs) + target_link_libraries(${_target} + # compaction_service_helper + csa_grpc_proto + ${_REFLECTION} + ${_GRPC_GRPCPP} + ${_PROTOBUF_LIBPROTOBUF} + rocksdb + lz4_static + nfs_fs + ) +endforeach() + +# message(STATUS "askjdakshakjsdgajdhajgjdsh") +# add_executable(remote_test remote_test.cpp remote_util.h remote_util.cc)#ifndef NDEBUG +# target_compile_definitions(remote_test PUBLIC -DNDEBUG) +# target_link_libraries(remote_test rocksdb lz4_static) + +# add_library(rocks_kvstore STATIC rocks_kvstore.cpp rocks_kvttlcompactfilter.cpp) +# target_link_libraries(rocks_kvstore utils_common kvstore rocksdb record glog ${SYS_LIBS} snappy lz4_static) + +add_library(rocks_kvstore STATIC rocks_kvstore.cpp rocks_kvttlcompactfilter.cpp compaction_service.cc) +target_link_libraries(rocks_kvstore utils_common kvstore rocksdb record glog ${SYS_LIBS} snappy lz4_static csa_grpc_proto + ${_REFLECTION} + ${_GRPC_GRPCPP} + ${_PROTOBUF_LIBPROTOBUF} + nfs_fs + ) add_library(rocks_kvstore_for_test STATIC rocks_kvstore.cpp rocks_kvttlcompactfilter.cpp) target_compile_definitions(rocks_kvstore_for_test PRIVATE -DNO_VERSIONEP) diff --git a/src/tendisplus/storage/rocks/cmake/grpc.cmake b/src/tendisplus/storage/rocks/cmake/grpc.cmake new file mode 100644 index 00000000..660a5a68 --- /dev/null +++ b/src/tendisplus/storage/rocks/cmake/grpc.cmake @@ -0,0 +1,128 @@ +# Copyright 2018 gRPC authors. +# +# 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 +# +# http://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. +# +# cmake build file for C++ route_guide example. +# Assumes protobuf and gRPC have been installed using cmake. +# See cmake_externalproject/CMakeLists.txt for all-in-one cmake build +# that automatically builds all the dependencies before building route_guide. + +cmake_minimum_required(VERSION 3.5.1) + +if (NOT DEFINED CMAKE_CXX_STANDARD) + set (CMAKE_CXX_STANDARD 14) +endif() + +if(MSVC) + add_definitions(-D_WIN32_WINNT=0x600) +endif() + +find_package(Threads REQUIRED) + +if(GRPC_AS_SUBMODULE) + # One way to build a projects that uses gRPC is to just include the + # entire gRPC project tree via "add_subdirectory". + # This approach is very simple to use, but the are some potential + # disadvantages: + # * it includes gRPC's CMakeLists.txt directly into your build script + # without and that can make gRPC's internal setting interfere with your + # own build. + # * depending on what's installed on your system, the contents of submodules + # in gRPC's third_party/* might need to be available (and there might be + # additional prerequisites required to build them). Consider using + # the gRPC_*_PROVIDER options to fine-tune the expected behavior. + # + # A more robust approach to add dependency on gRPC is using + # cmake's ExternalProject_Add (see cmake_externalproject/CMakeLists.txt). + + # Include the gRPC's cmake build (normally grpc source code would live + # in a git submodule called "third_party/grpc", but this example lives in + # the same repository as gRPC sources, so we just look a few directories up) + add_subdirectory(. ${CMAKE_CURRENT_BINARY_DIR}/grpc EXCLUDE_FROM_ALL) + message(STATUS "Using gRPC via add_subdirectory.") + + # After using add_subdirectory, we can now use the grpc targets directly from + # this build. + set(_PROTOBUF_LIBPROTOBUF libprotobuf) + set(_REFLECTION grpc++_reflection) + if(CMAKE_CROSSCOMPILING) + find_program(_PROTOBUF_PROTOC protoc) + else() + set(_PROTOBUF_PROTOC $) + endif() + set(_GRPC_GRPCPP grpc++) + if(CMAKE_CROSSCOMPILING) + find_program(_GRPC_CPP_PLUGIN_EXECUTABLE grpc_cpp_plugin) + else() + set(_GRPC_CPP_PLUGIN_EXECUTABLE $) + endif() +elseif(GRPC_FETCHCONTENT) + # Another way is to use CMake's FetchContent module to clone gRPC at + # configure time. This makes gRPC's source code available to your project, + # similar to a git submodule. + message(STATUS "Using gRPC via add_subdirectory (FetchContent).") + include(FetchContent) + FetchContent_Declare( + grpc + GIT_REPOSITORY https://github.com/grpc/grpc.git + # when using gRPC, you will actually set this to an existing tag, such as + # v1.25.0, v1.26.0 etc.. + # For the purpose of testing, we override the tag used to the commit + # that's currently under test. + GIT_TAG v1.46.2) + FetchContent_MakeAvailable(grpc) + + # Since FetchContent uses add_subdirectory under the hood, we can use + # the grpc targets directly from this build. + set(_PROTOBUF_LIBPROTOBUF libprotobuf) + set(_REFLECTION grpc++_reflection) + set(_PROTOBUF_PROTOC $) + set(_GRPC_GRPCPP grpc++) + if(CMAKE_CROSSCOMPILING) + find_program(_GRPC_CPP_PLUGIN_EXECUTABLE grpc_cpp_plugin) + else() + set(_GRPC_CPP_PLUGIN_EXECUTABLE $) + endif() +else() + # This branch assumes that gRPC and all its dependencies are already installed + # on this system, so they can be located by find_package(). + + # Find Protobuf installation + # Looks for protobuf-config.cmake file installed by Protobuf's cmake installation. + set(protobuf_MODULE_COMPATIBLE TRUE) + find_package(Protobuf CONFIG) + if(NOT Protobuf_FOUND) + find_package(Protobuf REQUIRED) + endif() + message(STATUS "Using protobuf ${Protobuf_VERSION}") + + set(_PROTOBUF_LIBPROTOBUF protobuf::libprotobuf) + set(_REFLECTION gRPC::grpc++_reflection) + if(CMAKE_CROSSCOMPILING) + find_program(_PROTOBUF_PROTOC protoc) + else() + set(_PROTOBUF_PROTOC $) + endif() + + # Find gRPC installation + # Looks for gRPCConfig.cmake file installed by gRPC's cmake installation. + find_package(gRPC CONFIG REQUIRED) + message(STATUS "Using gRPC ${gRPC_VERSION}") + + set(_GRPC_GRPCPP gRPC::grpc++) + if(CMAKE_CROSSCOMPILING) + find_program(_GRPC_CPP_PLUGIN_EXECUTABLE grpc_cpp_plugin) + else() + set(_GRPC_CPP_PLUGIN_EXECUTABLE $) + endif() +endif() \ No newline at end of file diff --git a/src/tendisplus/storage/rocks/compaction_service.cc b/src/tendisplus/storage/rocks/compaction_service.cc new file mode 100644 index 00000000..4df4ffa7 --- /dev/null +++ b/src/tendisplus/storage/rocks/compaction_service.cc @@ -0,0 +1,554 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#include "tendisplus/storage/rocks/compaction_service.h" + +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include "csa.grpc.pb.h" // NOLINT(build/include_subdir) +#include "glog/logging.h" +#include "rocksdb/db/compaction/compaction_job.h" + +#include "tendisplus/storage/rocks/remote_compaction/def.h" +#include "tendisplus/storage/rocks/shared_filesystem.h" + +// Note: Max gRPC message size is now configurable via +// remote_options_.GetGrpcMaxMessageSize() Default is 16MB, but can be +// overridden in configuration + +// CSA server status codes (must match csa_server.cc) +constexpr int kCSACodeBusy = 100; // CSA server is busy +constexpr int kCSACodeStaleTask = 101; // Input files not found, stale task + +class CSAClient { + public: + explicit CSAClient(const std::shared_ptr& channel) + : stub_(csa::CSAService::NewStub(channel)) {} + + // Shared storage mode: execute compaction using shared filesystem + ROCKSDB_NAMESPACE::Status OpenAndCompact( + const ROCKSDB_NAMESPACE::OpenAndCompactOptions& options, + const std::string& name, + const std::string& output_directory, + const std::string& input, + std::string* output, + const ROCKSDB_NAMESPACE::CompactionServiceOptionsOverride& override_options, + const std::string& shared_fs_uri = "", + const std::string& shared_fs_local_prefix = "") { + csa::CompactionArgs compaction_args; + compaction_args.set_name(name); + compaction_args.set_output_directory(output_directory); + compaction_args.set_input(input); + // Pass the shared file system configuration to the CSA server + if (!shared_fs_uri.empty()) { + compaction_args.set_shared_fs_uri(shared_fs_uri); + } + if (!shared_fs_local_prefix.empty()) { + compaction_args.set_shared_fs_local_prefix(shared_fs_local_prefix); + } + + std::cerr << "[CSAClient] ========== Calling CSA Server ==========" + << std::endl; + std::cerr << "[CSAClient] Name: " << name << std::endl; + std::cerr << "[CSAClient] Output Directory: " << output_directory + << std::endl; + std::cerr << "[CSAClient] Shared FS URI: " + << (shared_fs_uri.empty() ? "(NOT SET!)" : shared_fs_uri) + << std::endl; + std::cerr << "[CSAClient] Input size: " << input.size() << " bytes" + << std::endl; + std::cerr.flush(); + + csa::CompactionReply compaction_reply; + grpc::ClientContext context; + + // Set timeout for the RPC call (30 seconds) + std::chrono::system_clock::time_point deadline = + std::chrono::system_clock::now() + std::chrono::seconds(30); + context.set_deadline(deadline); + + std::cerr << "[CSAClient] Sending gRPC request to CSA server..." + << std::endl; + std::cerr.flush(); + grpc::Status status = stub_->ExecuteCompactionTask( + &context, compaction_args, &compaction_reply); + + std::cerr << "[CSAClient] gRPC call completed. Status: " + << (status.ok() ? "OK" : "FAILED") << std::endl; + if (!status.ok()) { + std::cerr << "[CSAClient] Error code: " << status.error_code() + << std::endl; + std::cerr << "[CSAClient] Error message: " << status.error_message() + << std::endl; + } + std::cerr.flush(); + + if (!status.ok()) { + std::string error_msg = "gRPC call failed: " + status.error_message(); + if (status.error_code() == grpc::StatusCode::DEADLINE_EXCEEDED) { + error_msg += " (timeout)"; + } else if (status.error_code() == grpc::StatusCode::UNAVAILABLE) { + error_msg += " (service unavailable)"; + } + std::cerr << "[CSAClient] ERROR: " << error_msg << std::endl; + return ROCKSDB_NAMESPACE::Status::IOError(error_msg); + } + + std::cerr << "[CSAClient] Compaction reply code: " + << compaction_reply.code() << std::endl; + std::cerr << "[CSAClient] Compaction reply result size: " + << compaction_reply.result().size() << " bytes" << std::endl; + std::cerr.flush(); + + // Check for CSA server busy status + if (compaction_reply.code() == kCSACodeBusy) { + std::cerr << "[CSAClient] CSA server is busy, should use local compaction" + << std::endl; + std::cerr.flush(); + return ROCKSDB_NAMESPACE::Status::Busy("CSA server busy"); + } + + // Check for stale task (input files not found) + // This is detected early by CSA server before expensive OpenAndCompact + if (compaction_reply.code() == kCSACodeStaleTask) { + std::cerr << "[CSAClient] CSA server detected stale task (input files " + "not found), returning empty result" + << std::endl; + std::cerr.flush(); + // Return a special status that will be handled by WaitForCompleteV2 + return ROCKSDB_NAMESPACE::Status::NotFound( + "Stale task: input file not found"); + } + + if (compaction_reply.code() != 0) { + std::cerr << "[CSAClient] ERROR: Remote compaction failed with code: " + << compaction_reply.code() << std::endl; + std::cerr.flush(); + return ROCKSDB_NAMESPACE::Status::IOError( + "Remote compaction failed with code: " + + std::to_string(compaction_reply.code()) + + (compaction_reply.code() == 5 ? " (IOError/NOENT)" : "")); + } + + if (compaction_reply.result().empty()) { + std::cerr << "[CSAClient] ERROR: Remote compaction returned empty result" + << std::endl; + std::cerr.flush(); + return ROCKSDB_NAMESPACE::Status::IOError( + "Remote compaction returned empty result"); + } + + output->assign(compaction_reply.result()); + std::cerr << "[CSAClient] ========== SUCCESS: Remote compaction completed " + "==========" + << std::endl; + std::cerr.flush(); + return ROCKSDB_NAMESPACE::Status::OK(); + } + + private: + std::unique_ptr stub_; +}; + +namespace ROCKSDB_NAMESPACE { + +namespace { +// Convert relative path to absolute path (helper function) +std::string ToAbsolutePath(const std::string& path) { + if (path.empty()) { + return path; + } + // Already an absolute path + if (path[0] == '/') { + return path; + } + // Relative path, convert to absolute path + char cwd[PATH_MAX]; + if (getcwd(cwd, sizeof(cwd)) != nullptr) { + std::string abs_path = std::string(cwd) + "/" + path; + // Normalize path: handle ./ and ../ + size_t pos; + while ((pos = abs_path.find("/./")) != std::string::npos) { + abs_path.erase(pos, 2); + } + // Handle leading ./ + if (abs_path.find("./") == 0) { + abs_path = abs_path.substr(2); + } + return abs_path; + } + return path; // Failed to get cwd, return original path +} +} // anonymous namespace + +CompactionServiceJobStatus MyTestCompactionService::StartV2( + const CompactionServiceJobInfo& info, + const std::string& compaction_service_input) { + InstrumentedMutexLock l(&mutex_); + start_info_ = info; + assert(info.db_name == db_path_); + jobs_.emplace(info.job_id, compaction_service_input); + + std::cerr << "[CompactionService] ========== StartV2 CALLED ==========" + << std::endl; + std::cerr << "[CompactionService] Job ID: " << info.job_id << std::endl; + std::cerr << "[CompactionService] DB Path: " << db_path_ << std::endl; + std::cerr << "[CompactionService] CSA Address: " + << (remote_options_.csa_address.empty() + ? "(NOT CONFIGURED!)" + : remote_options_.csa_address) + << std::endl; + std::cerr << "[CompactionService] Shared FS URI: " + << (remote_options_.shared_fs_uri.empty() + ? "(NOT CONFIGURED!)" + : remote_options_.shared_fs_uri) + << std::endl; + std::cerr << "[CompactionService] Input size: " + << compaction_service_input.size() << " bytes" << std::endl; + std::cerr << "[CompactionService] ====================================" + << std::endl; + std::cerr.flush(); + + CompactionServiceJobStatus s = CompactionServiceJobStatus::kSuccess; + if (is_override_start_status_) { + return override_start_status_; + } + return s; +} + +CompactionServiceJobStatus MyTestCompactionService::WaitForCompleteV2( + const CompactionServiceJobInfo& info, + std::string* compaction_service_result) { + std::string compaction_input; + assert(info.db_name == db_path_); + { + InstrumentedMutexLock l(&mutex_); + wait_info_ = info; + auto i = jobs_.find(info.job_id); + if (i == jobs_.end()) { + return CompactionServiceJobStatus::kFailure; + } + compaction_input = std::move(i->second); + jobs_.erase(i); + } + + if (is_override_wait_status_) { + return override_wait_status_; + } + + CompactionServiceOptionsOverride options_override; + // CRITICAL: Always use the same env as the main DB to ensure consistency + // If NFS is enabled, this will be NFS env, ensuring all compaction (remote or + // local) uses NFS + options_override.env = options_.env; + options_override.file_checksum_gen_factory = + options_.file_checksum_gen_factory; + options_override.comparator = options_.comparator; + options_override.merge_operator = options_.merge_operator; + options_override.compaction_filter = options_.compaction_filter; + options_override.compaction_filter_factory = + options_.compaction_filter_factory; + options_override.prefix_extractor = options_.prefix_extractor; + options_override.table_factory = options_.table_factory; + options_override.sst_partitioner_factory = options_.sst_partitioner_factory; + options_override.statistics = statistics_; + if (!listeners_.empty()) { + options_override.listeners = listeners_; + } + + if (!table_properties_collector_factories_.empty()) { + options_override.table_properties_collector_factories = + table_properties_collector_factories_; + } + + Status s; + std::string job_id = std::to_string(info.job_id); + std::string output_dir = db_path_ + "/" + job_id; + + std::cerr + << "[CompactionService] ========== WaitForCompleteV2 CALLED ==========" + << std::endl; + std::cerr << "[CompactionService] Job ID: " << job_id << std::endl; + std::cerr << "[CompactionService] DB Path: " << db_path_ << std::endl; + std::cerr << "[CompactionService] CSA Address: " + << (remote_options_.csa_address.empty() + ? "(NOT CONFIGURED!)" + : remote_options_.csa_address) + << std::endl; + std::cerr << "[CompactionService] Shared FS URI: " + << (remote_options_.shared_fs_uri.empty() + ? "(NOT CONFIGURED!)" + : remote_options_.shared_fs_uri) + << std::endl; + std::cerr << "[CompactionService] Input size: " << compaction_input.size() + << " bytes" << std::endl; + std::cerr + << "[CompactionService] ===============================================" + << std::endl; + std::cerr.flush(); + + // Check if NFS is enabled (for consistency, all compaction must use NFS if + // enabled) + bool nfs_enabled = !remote_options_.shared_fs_uri.empty() || + !remote_options_.shared_fs_local_prefix.empty(); + + // Check if remote compaction is enabled (CSA address must be configured) + if (remote_options_.csa_address.empty()) { + std::cerr << "[CompactionService] ERROR: CSA address is EMPTY! " + << "Remote compaction will NOT be used. " + << "Please configure csa_address in config file." << std::endl; + if (nfs_enabled) { + std::cout << "[CompactionService] Falling back to local compaction " + "(using NFS for consistency)" + << std::endl; + } else { + std::cerr << "[CompactionService] Falling back to local compaction" + << std::endl; + } + // Return kUseLocal - RocksDB will use options_override.env which is NFS env + // if NFS is enabled + return CompactionServiceJobStatus::kUseLocal; + } + + // Validate CSA address format (basic check: should contain ':') + if (remote_options_.csa_address.find(':') == std::string::npos) { + if (nfs_enabled) { + std::cerr << "[CompactionService] Invalid CSA address format: " + << remote_options_.csa_address + << " (expected format: host:port), falling back to local " + "compaction (using NFS for consistency)" + << std::endl; + } else { + std::cerr + << "[CompactionService] Invalid CSA address format: " + << remote_options_.csa_address + << " (expected format: host:port), falling back to local compaction" + << std::endl; + } + // Return kUseLocal - RocksDB will use options_override.env which is NFS env + // if NFS is enabled + return CompactionServiceJobStatus::kUseLocal; + } + + // Create channel with configured message size limit + int64_t max_msg_size = remote_options_.GetGrpcMaxMessageSize(); + std::cerr << "[CompactionService] Creating gRPC channel to " + << remote_options_.csa_address + << " with max message size: " << max_msg_size << " bytes" + << std::endl; + std::cerr.flush(); + + grpc::ChannelArguments channel_args; + channel_args.SetMaxReceiveMessageSize(static_cast(max_msg_size)); + channel_args.SetMaxSendMessageSize(static_cast(max_msg_size)); + + auto channel = grpc::CreateCustomChannel(remote_options_.csa_address, + grpc::InsecureChannelCredentials(), + channel_args); + + if (!channel) { + std::cerr << "[CompactionService] ERROR: Failed to create gRPC channel to " + << remote_options_.csa_address + << ", falling back to local compaction" << std::endl; + std::cerr.flush(); + // Return kUseLocal - RocksDB will use options_override.env which is NFS env + // if NFS is enabled + return CompactionServiceJobStatus::kUseLocal; + } + + std::cerr << "[CompactionService] gRPC channel created successfully" + << std::endl; + std::cerr.flush(); + + CSAClient csa_client(channel); + + // Shared storage mode (using NFS/HDFS) + // CSA server reads from shared storage (read-only via URI), Tendisplus + // installs results + std::cerr + << "[CompactionService] ========== Using REMOTE COMPACTION ==========" + << std::endl; + std::cerr << "[CompactionService] Job ID: " << job_id << std::endl; + std::cerr << "[CompactionService] CSA Server: " << remote_options_.csa_address + << std::endl; + std::cerr << "[CompactionService] Shared FS URI: " + << remote_options_.shared_fs_uri << std::endl; + std::cerr + << "[CompactionService] ==============================================" + << std::endl; + std::cerr.flush(); + + // Build URI by directly replacing local mount point with shared filesystem + // URI Example: db_path="/mnt/rocksdb/db", + // shared_fs_uri="nfs://localhost/shared/rocksdb" + // -> Replace "/mnt/rocksdb" with "nfs://localhost/shared/rocksdb" + // -> Result: "nfs://localhost/shared/rocksdb/db" + // Uses unified shared filesystem configuration + // (remote_compaction.shared_fs_uri) + std::string db_path_uri; + std::string output_dir_uri; + + // Unified approach: Use shared filesystem URI for all filesystem types + // Supports: nfs://, hdfs://, s3://, etc. + if (!remote_options_.shared_fs_uri.empty() && + IsSharedFilesystemURI(remote_options_.shared_fs_uri)) { + // Use unified helper to convert local path to shared filesystem URI + // This works for all filesystem types (NFS, HDFS, S3, etc.) + db_path_uri = ConvertLocalPathToSharedURI( + db_path_, + remote_options_.shared_fs_uri, + remote_options_.shared_fs_local_prefix); // Optional mount point + + output_dir_uri = + ConvertLocalPathToSharedURI(output_dir, + remote_options_.shared_fs_uri, + remote_options_.shared_fs_local_prefix); + + // If output_dir doesn't match mount point, append job_id to db_path_uri + if (output_dir_uri == ToAbsolutePath(output_dir)) { + // Conversion failed, use db_path_uri as base + output_dir_uri = db_path_uri + "/" + job_id; + } + + std::cout + << "[CompactionService] URI conversion (unified shared filesystem):" + << "\n shared_fs_uri: " << remote_options_.shared_fs_uri + << "\n local_mount_point: " + << (remote_options_.shared_fs_local_prefix.empty() + ? "(auto-inferred)" + : remote_options_.shared_fs_local_prefix) + << "\n db_path: " << db_path_ << " -> " << db_path_uri + << "\n output_dir: " << output_dir << " -> " << output_dir_uri + << std::endl; + } else { + // No shared_fs_uri or not a shared filesystem URI, use absolute paths + db_path_uri = ToAbsolutePath(db_path_); + output_dir_uri = ToAbsolutePath(output_dir); + + std::cout << "[CompactionService] Using absolute paths (no shared storage):" + << "\n db_path: " << db_path_ << " -> " << db_path_uri + << "\n output_dir: " << output_dir << " -> " << output_dir_uri + << std::endl; + } + + // Use the configured remote options + RemoteOpenAndCompactOptions opts = remote_options_; + + std::cerr << "[CompactionService] About to call CSA server OpenAndCompact:" + << std::endl; + std::cerr << " db_path_uri: " << db_path_uri << std::endl; + std::cerr << " output_dir_uri: " << output_dir_uri << std::endl; + std::cerr << " shared_fs_uri: " << remote_options_.shared_fs_uri + << std::endl; + std::cerr << " compaction_input size: " << compaction_input.size() + << " bytes" << std::endl; + std::cerr.flush(); + + s = csa_client.OpenAndCompact( + opts, + db_path_uri, + output_dir_uri, + compaction_input, + compaction_service_result, + options_override, + remote_options_.shared_fs_uri, + ""); // Don't pass local_prefix to CSA - it uses URI directly + + std::cerr << "[CompactionService] CSA server OpenAndCompact returned: " + << s.ToString() << std::endl; + std::cerr << "[CompactionService] Result size: " + << compaction_service_result->size() << " bytes" << std::endl; + std::cerr.flush(); + + if (is_override_wait_result_) { + *compaction_service_result = override_wait_result_; + } + compaction_num_.fetch_add(1); + + LOG(INFO) << "[CompactionService] WaitForCompleteV2 result: " + << (s.ok() ? "OK" : s.ToString()) + << ", result_size=" << compaction_service_result->size(); + + if (s.ok()) { + return CompactionServiceJobStatus::kSuccess; + } else { + std::string err_msg = s.ToString(); + + LOG(WARNING) << "[CompactionService] Error details - IsIOError: " + << s.IsIOError() << ", IsBusy: " << s.IsBusy() + << ", code: " << static_cast(s.code()) + << ", msg: " << err_msg; + + // Check if CSA server is busy - should fallback to local immediately + if (s.IsBusy()) { + LOG(INFO) << "[CompactionService] CSA server busy, falling back to local " + "compaction"; + return CompactionServiceJobStatus::kUseLocal; + } + + // Check if this is a stale task detected early by CSA server + // CSA server checks input file existence before OpenAndCompact + if (s.IsNotFound()) { + LOG(INFO) << "[CompactionService] Early stale task detection by CSA, " + << "constructing empty result and returning kSuccess"; + CompactionServiceResult empty_result; + empty_result.status = Status::OK(); + empty_result.Write(compaction_service_result); + return CompactionServiceJobStatus::kSuccess; + } + + // Check if this is a stale task (file not found on remote) vs + // connection/other errors Stale task: CSA returned code 5 with NOENT - file + // was already compacted/deleted Connection error: gRPC failed to connect - + // should fallback to local + bool is_file_not_found = s.IsIOError() && + (err_msg.find("NOENT") != std::string::npos || + err_msg.find("No such file") != std::string::npos); + + bool is_connection_error = s.IsIOError() && + (err_msg.find("gRPC call failed") != std::string::npos || + err_msg.find("Connection refused") != std::string::npos || + err_msg.find("connect") != std::string::npos); + + if (is_file_not_found) { + // Stale task: file was already compacted by another task, return empty + // result + LOG(WARNING) << "[CompactionService] File not found (stale task), " + << "constructing empty result and returning kSuccess"; + CompactionServiceResult empty_result; + empty_result.status = Status::OK(); + empty_result.Write(compaction_service_result); + return CompactionServiceJobStatus::kSuccess; + } + + if (is_connection_error) { + LOG(WARNING) << "[CompactionService] CSA connection failed, falling back " + "to local compaction"; + } else if (nfs_enabled) { + LOG(WARNING) << "[CompactionService] Remote compaction error, falling " + "back to local compaction " + << "(using NFS for consistency): " << err_msg; + } else { + LOG(WARNING) << "[CompactionService] Remote compaction error, falling " + "back to local: " + << err_msg; + } + // Return kUseLocal - RocksDB will use options_override.env which is NFS env + // if NFS is enabled + return CompactionServiceJobStatus::kUseLocal; + } +} +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/compaction_service.h b/src/tendisplus/storage/rocks/compaction_service.h new file mode 100644 index 00000000..56e04453 --- /dev/null +++ b/src/tendisplus/storage/rocks/compaction_service.h @@ -0,0 +1,119 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#pragma once +#include +#include +#include +#include +#include +#include + +#include "monitoring/instrumented_mutex.h" +#include "rocksdb/metadata.h" +#include "rocksdb/options.h" + +#include "tendisplus/storage/rocks/remote_compaction/def.h" + +namespace ROCKSDB_NAMESPACE { + +class MyTestCompactionService : public CompactionService { + public: + MyTestCompactionService( + std::string db_path, + Options& options, + std::shared_ptr& statistics, + std::vector>& listeners, + std::vector> + table_properties_collector_factories, + const RemoteOpenAndCompactOptions& remote_options = + RemoteOpenAndCompactOptions()) + : db_path_(std::move(db_path)), + options_(options), + statistics_(statistics), + start_info_("na", "na", "na", 0, Env::TOTAL), + wait_info_("na", "na", "na", 0, Env::TOTAL), + listeners_(listeners), + table_properties_collector_factories_( + std::move(table_properties_collector_factories)), + remote_options_(remote_options) {} + + static const char* kClassName() { + return "MyTestCompactionService"; + } + + const char* Name() const override { + return kClassName(); + } + + CompactionServiceJobStatus StartV2( + const CompactionServiceJobInfo& info, + const std::string& compaction_service_input) override; + + CompactionServiceJobStatus WaitForCompleteV2( + const CompactionServiceJobInfo& info, + std::string* compaction_service_result) override; + + int GetCompactionNum() { + return compaction_num_.load(); + } + + CompactionServiceJobInfo GetCompactionInfoForStart() { + return start_info_; + } + CompactionServiceJobInfo GetCompactionInfoForWait() { + return wait_info_; + } + + void OverrideStartStatus(CompactionServiceJobStatus s) { + is_override_start_status_ = true; + override_start_status_ = s; + } + + void OverrideWaitStatus(CompactionServiceJobStatus s) { + is_override_wait_status_ = true; + override_wait_status_ = s; + } + + void OverrideWaitResult(std::string str) { + is_override_wait_result_ = true; + override_wait_result_ = std::move(str); + } + + void ResetOverride() { + is_override_wait_result_ = false; + is_override_start_status_ = false; + is_override_wait_status_ = false; + } + + void SetCanceled(bool canceled) { + canceled_ = canceled; + } + + private: + InstrumentedMutex mutex_; + std::atomic_int compaction_num_{0}; + std::map jobs_; + const std::string db_path_; + Options options_; + std::shared_ptr statistics_; + CompactionServiceJobInfo start_info_; + CompactionServiceJobInfo wait_info_; + bool is_override_start_status_ = false; + CompactionServiceJobStatus override_start_status_ = + CompactionServiceJobStatus::kFailure; + bool is_override_wait_status_ = false; + CompactionServiceJobStatus override_wait_status_ = + CompactionServiceJobStatus::kFailure; + bool is_override_wait_result_ = false; + std::string override_wait_result_; + std::vector> listeners_; + std::vector> + table_properties_collector_factories_; + std::atomic_bool canceled_{false}; + + // Remote compaction configuration (all settings from config file) + RemoteOpenAndCompactOptions remote_options_; +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/nfs_file.cc b/src/tendisplus/storage/rocks/nfs_file.cc new file mode 100644 index 00000000..dc3c267f --- /dev/null +++ b/src/tendisplus/storage/rocks/nfs_file.cc @@ -0,0 +1,231 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. +#include + +#include +#include +#include + +#include "tendisplus/storage/rocks/nfs_filesystem.h" + +namespace ROCKSDB_NAMESPACE { + +// NFS Sequential File +class NFSSequentialFile : public FSSequentialFile { + public: + NFSSequentialFile(struct nfs_context* nfs_ctx, + const std::string& fname, + struct nfsfh* fh) + : nfs_ctx_(nfs_ctx), fname_(fname), fh_(fh), offset_(0) {} + + ~NFSSequentialFile() override { + if (fh_) { + nfs_close(nfs_ctx_, fh_); + } + } + + IOStatus Read(size_t n, + const IOOptions& options, + Slice* result, + char* scratch, + IODebugContext* dbg) override { + assert(fh_); + int bytes_read = nfs_read(nfs_ctx_, fh_, n, scratch); + if (bytes_read < 0) { + return IOStatus::IOError("NFS read failed: " + + std::string(nfs_get_error(nfs_ctx_))); + } + *result = Slice(scratch, bytes_read); + offset_ += bytes_read; + return IOStatus::OK(); + } + + IOStatus Skip(uint64_t n) override { + offset_ += n; + uint64_t new_offset = 0; + if (nfs_lseek(nfs_ctx_, fh_, offset_, SEEK_SET, &new_offset) < 0) { + return IOStatus::IOError("NFS seek failed: " + + std::string(nfs_get_error(nfs_ctx_))); + } + offset_ = new_offset; + return IOStatus::OK(); + } + + private: + struct nfs_context* nfs_ctx_; + std::string fname_; + struct nfsfh* fh_; + uint64_t offset_; +}; + +// NFS Random Access File +class NFSRandomAccessFile : public FSRandomAccessFile { + public: + NFSRandomAccessFile(struct nfs_context* nfs_ctx, + const std::string& fname, + struct nfsfh* fh) + : nfs_ctx_(nfs_ctx), fname_(fname), fh_(fh) {} + + ~NFSRandomAccessFile() override { + if (fh_) { + nfs_close(nfs_ctx_, fh_); + } + } + + IOStatus Read(uint64_t offset, + size_t n, + const IOOptions& options, + Slice* result, + char* scratch, + IODebugContext* dbg) const override { + int bytes_read = nfs_pread(nfs_ctx_, fh_, offset, n, scratch); + if (bytes_read < 0) { + return IOStatus::IOError("NFS pread failed: " + + std::string(nfs_get_error(nfs_ctx_))); + } + *result = Slice(scratch, bytes_read); + return IOStatus::OK(); + } + + private: + struct nfs_context* nfs_ctx_; + std::string fname_; + struct nfsfh* fh_; +}; + + +// NFS Writable File +class NFSWritableFile : public FSWritableFile { + public: + NFSWritableFile(struct nfs_context* nfs_ctx, + const std::string& fname, + struct nfsfh* fh) + : nfs_ctx_(nfs_ctx), fname_(fname), fh_(fh) {} + + ~NFSWritableFile() override { + if (fh_) { + Close(IOOptions(), nullptr); + } + } + + // Use the correct Append signature + using FSWritableFile::Append; // Inheriting other reloaded versions + + IOStatus Append(const Slice& data, + const IOOptions& options, + IODebugContext* dbg) override { + int bytes_written = + nfs_write(nfs_ctx_, fh_, data.size(), const_cast(data.data())); + if (bytes_written < 0) { + return IOStatus::IOError("NFS write failed: " + + std::string(nfs_get_error(nfs_ctx_))); + } + if (static_cast(bytes_written) != data.size()) { + return IOStatus::IOError("NFS partial write"); + } + return IOStatus::OK(); + } + + IOStatus Close(const IOOptions& options, IODebugContext* dbg) override { + if (fh_) { + if (nfs_close(nfs_ctx_, fh_) < 0) { + return IOStatus::IOError("NFS close failed: " + + std::string(nfs_get_error(nfs_ctx_))); + } + fh_ = nullptr; + } + return IOStatus::OK(); + } + + IOStatus Flush(const IOOptions& options, IODebugContext* dbg) override { + if (!fh_) { + return IOStatus::OK(); + } + if (nfs_fsync(nfs_ctx_, fh_) < 0) { + return IOStatus::IOError("NFS fsync failed: " + + std::string(nfs_get_error(nfs_ctx_))); + } + return IOStatus::OK(); + } + + IOStatus Sync(const IOOptions& options, IODebugContext* dbg) override { + return Flush(options, dbg); + } + + private: + struct nfs_context* nfs_ctx_; + std::string fname_; + struct nfsfh* fh_; +}; + +// NFSFileSystem file operation implementation +IOStatus NFSFileSystem::NewSequentialFile( + const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::NewSequentialFile(fname, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + struct nfsfh* fh; + + if (nfs_open(nfs_ctx_, nfs_path.c_str(), O_RDONLY, &fh) != 0) { + return IOStatus::IOError("Failed to open NFS file: " + fname + " - " + + std::string(nfs_get_error(nfs_ctx_))); + } + + result->reset(new NFSSequentialFile(nfs_ctx_, fname, fh)); + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::NewRandomAccessFile( + const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::NewRandomAccessFile(fname, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + struct nfsfh* fh; + + if (nfs_open(nfs_ctx_, nfs_path.c_str(), O_RDONLY, &fh) != 0) { + return IOStatus::IOError("Failed to open NFS file: " + fname + " - " + + std::string(nfs_get_error(nfs_ctx_))); + } + + result->reset(new NFSRandomAccessFile(nfs_ctx_, fname, fh)); + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::NewWritableFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::NewWritableFile(fname, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + struct nfsfh* fh; + + if (nfs_creat(nfs_ctx_, nfs_path.c_str(), 0644, &fh) != 0) { + return IOStatus::IOError("Failed to create NFS file: " + fname + " - " + + std::string(nfs_get_error(nfs_ctx_))); + } + + result->reset(new NFSWritableFile(nfs_ctx_, fname, fh)); + return IOStatus::OK(); +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/nfs_filesystem.cc b/src/tendisplus/storage/rocks/nfs_filesystem.cc new file mode 100644 index 00000000..44bd1be2 --- /dev/null +++ b/src/tendisplus/storage/rocks/nfs_filesystem.cc @@ -0,0 +1,1498 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#include "tendisplus/storage/rocks/nfs_filesystem.h" + +#include // for O_RDONLY, O_WRONLY, O_CREAT, O_TRUNC +#include // for S_ISDIR +#include // for getcwd + +#include // for PATH_MAX +#include +#include +#include +#include +#include +#include +#include + +#include "logging/env_logger.h" // for EnvLogger + +namespace { +// Convert relative path to absolute path +std::string ToAbsolutePath(const std::string& path) { + if (path.empty()) { + return path; + } + // Already an absolute path + if (path[0] == '/') { + return path; + } + // Relative path, convert to absolute path + char cwd[PATH_MAX]; + if (getcwd(cwd, sizeof(cwd)) != nullptr) { + std::string abs_path = std::string(cwd) + "/" + path; + // Normalize path: handle ./ and ../ + // Simple handling: remove ./ + size_t pos; + while ((pos = abs_path.find("/./")) != std::string::npos) { + abs_path.erase(pos, 2); + } + // Handle leading ./ + if (abs_path.find("./") == 0) { + abs_path = abs_path.substr(2); + } + return abs_path; + } + return path; // Failed to get cwd, return original path +} +} // anonymous namespace + +namespace ROCKSDB_NAMESPACE { + +// NFS sequential reading of files +class NFSSequentialFile : public FSSequentialFile { + public: + NFSSequentialFile(struct nfs_context* ctx, + struct nfsfh* fh, + const std::string& fname, + std::mutex& mutex) + : nfs_ctx_(ctx), nfs_fh_(fh), filename_(fname), nfs_mutex_(mutex) {} + + ~NFSSequentialFile() override { + if (nfs_fh_) { + std::lock_guard lock(nfs_mutex_); + nfs_close(nfs_ctx_, nfs_fh_); + } + } + + IOStatus Read(size_t n, + const IOOptions& options, + Slice* result, + char* scratch, + IODebugContext* dbg) override { + std::lock_guard lock(nfs_mutex_); + int bytes_read = nfs_read(nfs_ctx_, nfs_fh_, n, scratch); + if (bytes_read < 0) { + return IOStatus::IOError("NFS read failed: " + filename_ + ", error: " + + std::string(nfs_get_error(nfs_ctx_))); + } + *result = Slice(scratch, bytes_read); + return IOStatus::OK(); + } + + IOStatus Skip(uint64_t n) override { + std::lock_guard lock(nfs_mutex_); + if (nfs_lseek(nfs_ctx_, nfs_fh_, n, SEEK_CUR, nullptr) < 0) { + return IOStatus::IOError("NFS lseek failed: " + filename_); + } + return IOStatus::OK(); + } + + private: + struct nfs_context* nfs_ctx_; + struct nfsfh* nfs_fh_; + std::string filename_; + std::mutex& nfs_mutex_; +}; + +// NFS random reading of files +class NFSRandomAccessFile : public FSRandomAccessFile { + public: + NFSRandomAccessFile(struct nfs_context* ctx, + struct nfsfh* fh, + const std::string& fname, + std::mutex* mutex) + : nfs_ctx_(ctx), nfs_fh_(fh), filename_(fname), nfs_mutex_(mutex) {} + + ~NFSRandomAccessFile() override { + if (nfs_fh_) { + std::lock_guard lock(*nfs_mutex_); + nfs_close(nfs_ctx_, nfs_fh_); + } + } + + IOStatus Read(uint64_t offset, + size_t n, + const IOOptions& options, + Slice* result, + char* scratch, + IODebugContext* dbg) const override { + std::lock_guard lock(*nfs_mutex_); + int bytes_read = nfs_pread(nfs_ctx_, nfs_fh_, offset, n, scratch); + if (bytes_read < 0) { + return IOStatus::IOError("NFS pread failed: " + filename_ + ", error: " + + std::string(nfs_get_error(nfs_ctx_))); + } + *result = Slice(scratch, bytes_read); + return IOStatus::OK(); + } + + private: + struct nfs_context* nfs_ctx_; + struct nfsfh* nfs_fh_; + std::string filename_; + std::mutex* nfs_mutex_; +}; + +// NFS writable files +class NFSWritableFile : public FSWritableFile { + public: + NFSWritableFile(struct nfs_context* ctx, + struct nfsfh* fh, + const std::string& fname, + std::mutex& mutex) + : nfs_ctx_(ctx), + nfs_fh_(fh), + filename_(fname), + nfs_mutex_(mutex), + filesize_(0) {} + + ~NFSWritableFile() override { + if (nfs_fh_) { + std::lock_guard lock(nfs_mutex_); + nfs_close(nfs_ctx_, nfs_fh_); + } + } + + IOStatus Append(const Slice& data, + const IOOptions& options, + IODebugContext* dbg) override { + std::lock_guard lock(nfs_mutex_); + const char* src = data.data(); + size_t left = data.size(); + + while (left > 0) { + int written = nfs_write(nfs_ctx_, nfs_fh_, left, const_cast(src)); + if (written < 0) { + return IOStatus::IOError( + "NFS write failed: " + filename_ + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + left -= written; + src += written; + } + filesize_ += data.size(); + return IOStatus::OK(); + } + + IOStatus Append(const Slice& data, + const IOOptions& options, + const DataVerificationInfo& verification_info, + IODebugContext* dbg) override { + return Append(data, options, dbg); + } + + IOStatus PositionedAppend(const Slice& data, + uint64_t offset, + const IOOptions& options, + IODebugContext* dbg) override { + std::lock_guard lock(nfs_mutex_); + const char* src = data.data(); + size_t left = data.size(); + uint64_t pos = offset; + + while (left > 0) { + int written = + nfs_pwrite(nfs_ctx_, nfs_fh_, pos, left, const_cast(src)); + if (written < 0) { + return IOStatus::IOError( + "NFS pwrite failed: " + filename_ + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + left -= written; + src += written; + pos += written; + } + if (offset + data.size() > filesize_) { + filesize_ = offset + data.size(); + } + return IOStatus::OK(); + } + + IOStatus PositionedAppend(const Slice& data, + uint64_t offset, + const IOOptions& options, + const DataVerificationInfo& verification_info, + IODebugContext* dbg) override { + return PositionedAppend(data, offset, options, dbg); + } + + IOStatus Truncate(uint64_t size, + const IOOptions& options, + IODebugContext* dbg) override { + std::lock_guard lock(nfs_mutex_); + if (nfs_ftruncate(nfs_ctx_, nfs_fh_, size) != 0) { + return IOStatus::IOError( + "NFS ftruncate failed: " + filename_ + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + filesize_ = size; + return IOStatus::OK(); + } + + IOStatus Close(const IOOptions& options, IODebugContext* dbg) override { + if (nfs_fh_) { + std::lock_guard lock(nfs_mutex_); + if (nfs_close(nfs_ctx_, nfs_fh_) != 0) { + return IOStatus::IOError("NFS close failed: " + filename_); + } + nfs_fh_ = nullptr; + } + return IOStatus::OK(); + } + + IOStatus Flush(const IOOptions& options, IODebugContext* dbg) override { + // There is no explicit flush, and writes are synchronous + return IOStatus::OK(); + } + + IOStatus Sync(const IOOptions& options, IODebugContext* dbg) override { + std::lock_guard lock(nfs_mutex_); + if (nfs_fsync(nfs_ctx_, nfs_fh_) != 0) { + return IOStatus::IOError("NFS fsync failed: " + filename_ + ", error: " + + std::string(nfs_get_error(nfs_ctx_))); + } + return IOStatus::OK(); + } + + IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) override { + return Sync(options, dbg); + } + + uint64_t GetFileSize(const IOOptions& options, IODebugContext* dbg) override { + return filesize_; + } + + bool IsSyncThreadSafe() const override { + return true; + } + + private: + struct nfs_context* nfs_ctx_; + struct nfsfh* nfs_fh_; + std::string filename_; + std::mutex& nfs_mutex_; + uint64_t filesize_; +}; + +// NFS directory (for operations like fsync) +class NFSDirectory : public FSDirectory { + public: + NFSDirectory(struct nfs_context* ctx, + const std::string& dirname, + std::mutex& mutex) + : nfs_ctx_(ctx), dirname_(dirname), nfs_mutex_(mutex) {} + + ~NFSDirectory() override {} + + IOStatus Fsync(const IOOptions& options, IODebugContext* dbg) override { + // fsync for NFS directories is usually no-op because NFS is synchronous + // But we can try the fsync directory (if libnfs supports it) + // For now, simply return to OK + return IOStatus::OK(); + } + + IOStatus Close(const IOOptions& options, IODebugContext* dbg) override { + return IOStatus::OK(); + } + + private: + struct nfs_context* nfs_ctx_; + std::string dirname_; + std::mutex& nfs_mutex_; +}; + +// NFS file lock +class NFSFileLock : public FileLock { + public: + NFSFileLock(struct nfs_context* ctx, + struct nfsfh* fh, + const std::string& fname, + std::mutex& mutex) + : nfs_ctx_(ctx), nfs_fh_(fh), filename_(fname), nfs_mutex_(mutex) {} + + ~NFSFileLock() override { + // Automatically release the lock when destructuring + if (nfs_fh_) { + std::lock_guard lock(nfs_mutex_); + nfs_close(nfs_ctx_, nfs_fh_); + } + } + + struct nfsfh* GetFileHandle() const { + return nfs_fh_; + } + const std::string& GetFilename() const { + return filename_; + } + void ClearFileHandle() { + nfs_fh_ = nullptr; + } + + private: + struct nfs_context* nfs_ctx_; + struct nfsfh* nfs_fh_; + std::string filename_; + std::mutex& nfs_mutex_; +}; + + +NFSFileSystem::NFSFileSystem(const std::string& nfs_url, + const std::string& local_prefix, + const std::shared_ptr& base) + : FileSystemWrapper(base), + nfs_url_(nfs_url), + local_prefix_(ToAbsolutePath(local_prefix)), // Convert to absolute path + nfs_ctx_(nullptr) { + // Ensure that the local_prefix does not end in '/' for easy subsequent + // processing + if (!local_prefix_.empty() && local_prefix_.back() == '/') { + local_prefix_.pop_back(); + } + InitNFSContext(); + std::cout << "[NFSFileSystem] Initialized with:" + << "\n NFS URL: " << nfs_url_ + << "\n Local Prefix (original): " << local_prefix + << "\n Local Prefix (absolute): " << local_prefix_ << std::endl; +} + +void NFSFileSystem::RegisterPathPrefix(const std::string& prefix) { + AddPathPrefix(prefix); +} + +void NFSFileSystem::AddPathPrefix(const std::string& prefix) { + if (prefix.empty()) { + return; + } + std::string abs_prefix = ToAbsolutePath(prefix); + // Normalize: remove trailing slash + while (!abs_prefix.empty() && abs_prefix.back() == '/') { + abs_prefix.pop_back(); + } + if (!abs_prefix.empty()) { + // Check if already exists + for (const auto& existing : additional_prefixes_) { + if (existing == abs_prefix) { + return; // Already registered + } + } + additional_prefixes_.push_back(abs_prefix); + std::cout << "[NFSFileSystem] Registered additional path prefix: " << prefix + << " (absolute: " << abs_prefix << ")" << std::endl; + } +} + +NFSFileSystem::~NFSFileSystem() { + if (nfs_ctx_) { + std::cerr << "Destroying NFSFileSystem(" << nfs_url_ << ")" << std::endl; + nfs_destroy_context(nfs_ctx_); + } +} + +std::string NFSFileSystem::GetId() const { + if (nfs_url_.empty()) { + return kProto; + } else if (nfs_url_.find(kProto) == 0) { + return nfs_url_; + } else { + std::string id = kProto; + return id.append("localhost").append(nfs_url_); + } +} + +Status NFSFileSystem::ValidateOptions( + const DBOptions& db_opts, const ColumnFamilyOptions& cf_opts) const { + if (nfs_ctx_ != nullptr) { + return FileSystemWrapper::ValidateOptions(db_opts, cf_opts); + } else { + return Status::InvalidArgument("Failed to connect to NFS ", nfs_url_); + } +} + +void NFSFileSystem::InitNFSContext() { + nfs_ctx_ = nfs_init_context(); + if (!nfs_ctx_) { + throw std::runtime_error("Failed to init NFS context"); + } + + struct nfs_url* url = nfs_parse_url_dir(nfs_ctx_, nfs_url_.c_str()); + if (!url) { + nfs_destroy_context(nfs_ctx_); + nfs_ctx_ = nullptr; + throw std::runtime_error("Invalid NFS URL: " + nfs_url_); + } + + if (nfs_mount(nfs_ctx_, url->server, url->path) != 0) { + std::string error_msg = + "Failed to mount NFS: " + std::string(nfs_get_error(nfs_ctx_)); + nfs_destroy_url(url); + nfs_destroy_context(nfs_ctx_); + nfs_ctx_ = nullptr; + throw std::runtime_error(error_msg); + } + + nfs_destroy_url(url); +} + +bool NFSFileSystem::IsDataFile(const std::string& path) const { + // Check if the file is a data file (.sst, .blob, .ldb) + size_t len = path.length(); + if (len > 4 && path.substr(len - 4) == ".sst") + return true; + if (len > 5 && path.substr(len - 5) == ".blob") + return true; + if (len > 4 && path.substr(len - 4) == ".ldb") + return true; + return false; +} + +bool NFSFileSystem::IsNFSPath(const std::string& path) const { + // Support three modes: + // 1. URI mode: path is "nfs://server/path/to/file" - always use NFS + // 2. Pure URI mode: local_prefix_ is empty but nfs_url_ is set - ALL paths go + // to NFS + // 3. Auto-convert mode: local path matches local_prefix_ -> automatically + // convert to URI internally + // 4. Legacy path matching mode: check if path matches local_prefix_ or + // additional_prefixes_ + + bool is_nfs = false; + std::string reason; + + // First check if it's a NFS URI + if (IsNFSURI(path)) { + is_nfs = true; + reason = "NFS URI format"; + std::cout << "[NFSFileSystem] IsNFSPath: " << path << " -> NFS (URI format)" + << std::endl; + return is_nfs; + } + + // Pure URI mode: If local_prefix_ is empty but nfs_url_ is set, ALL paths go + // to NFS This is the recommended mode for remote deployment where we don't + // need path matching + if (local_prefix_.empty() && !nfs_url_.empty()) { + is_nfs = true; + reason = "pure URI mode (all paths go to NFS)"; + std::cout << "[NFSFileSystem] IsNFSPath: " << path + << " -> NFS (pure URI mode, nfs_url=" << nfs_url_ << ")" + << std::endl; + return is_nfs; + } + + // Auto-convert mode: Try to convert local path to URI + // If conversion succeeds, treat it as NFS path (will use URI internally) + std::string uri = ConvertLocalPathToURI(path); + if (!uri.empty()) { + is_nfs = true; + reason = "auto-converted to URI: " + uri; + std::cout << "[NFSFileSystem] IsNFSPath: " << path + << " -> NFS (auto-converted to URI: " << uri << ")" << std::endl; + return is_nfs; + } + + // Legacy path matching mode (fallback) + // ALL files go to NFS to ensure consistency between local Tendis and remote + // workers. Both local Tendis and remote compaction workers access the same + // files via NFS. This avoids any inconsistency issues between local SSD and + // shared storage. + // + // The path is considered NFS if: + // 1. Path matches local_prefix_ (the DB directory) + // 2. Path matches any additional_prefixes_ (wal_dir, db_log_dir, etc.) + // 3. Path starts with /nfs/ + + // First convert input path to absolute path for comparison + std::string abs_path = path; + if (!path.empty() && path[0] != '/') { + // Relative path - assume it's within the DB directory, so it goes to NFS + char cwd[PATH_MAX]; + if (getcwd(cwd, sizeof(cwd)) != nullptr) { + abs_path = std::string(cwd) + "/" + path; + // Normalize path: remove ./ + size_t pos; + while ((pos = abs_path.find("/./")) != std::string::npos) { + abs_path.erase(pos, 2); + } + } + } + + // Check if path matches local_prefix_ (ALL files under local_prefix_ go to + // NFS) + if (!local_prefix_.empty() && abs_path.find(local_prefix_) == 0) { + is_nfs = true; + reason = "matches local_prefix (all files go to NFS)"; + } else { + // Check additional prefixes (wal_dir, db_log_dir, etc.) + for (const auto& prefix : additional_prefixes_) { + if (!prefix.empty()) { + std::string abs_prefix = ToAbsolutePath(prefix); + // Normalize prefix (remove trailing slash for comparison) + while (!abs_prefix.empty() && abs_prefix.back() == '/') { + abs_prefix.pop_back(); + } + if (!abs_prefix.empty() && abs_path.find(abs_prefix) == 0) { + is_nfs = true; + reason = "matches additional_prefix: " + prefix; + break; + } + } + } + + if (!is_nfs) { + if (path.find("/nfs/") == 0) { + is_nfs = true; + reason = "starts with /nfs/"; + } else { + reason = "no match (path='" + abs_path + "', local_prefix='" + + local_prefix_ + "')"; + } + } + } + + std::cout << "[NFSFileSystem] IsNFSPath: " << path << " -> " + << (is_nfs ? "NFS" : "LOCAL") << " (" << reason << ")" << std::endl; + + return is_nfs; +} + +// Convert local path to URI (if it matches local_prefix_) +std::string NFSFileSystem::ConvertLocalPathToURI( + const std::string& path) const { + // If path is already a URI, return as-is + if (IsNFSURI(path)) { + return path; + } + + // Convert to absolute path + std::string abs_path = path; + if (!path.empty() && path[0] != '/') { + char cwd[PATH_MAX]; + if (getcwd(cwd, sizeof(cwd)) != nullptr) { + abs_path = std::string(cwd) + "/" + path; + size_t pos; + while ((pos = abs_path.find("/./")) != std::string::npos) { + abs_path.erase(pos, 2); + } + } + } + + // Check if path matches local_prefix_ or additional_prefixes_ + std::string matched_prefix; + std::string relative_path; + + if (!local_prefix_.empty() && abs_path.find(local_prefix_) == 0) { + matched_prefix = local_prefix_; + relative_path = abs_path.substr(local_prefix_.length()); + while (!relative_path.empty() && relative_path[0] == '/') { + relative_path = relative_path.substr(1); + } + } else { + // Try additional prefixes + for (const auto& prefix : additional_prefixes_) { + if (!prefix.empty()) { + std::string abs_prefix = ToAbsolutePath(prefix); + while (!abs_prefix.empty() && abs_prefix.back() == '/') { + abs_prefix.pop_back(); + } + if (!abs_prefix.empty() && abs_path.find(abs_prefix) == 0) { + matched_prefix = abs_prefix; + relative_path = abs_path.substr(abs_prefix.length()); + while (!relative_path.empty() && relative_path[0] == '/') { + relative_path = relative_path.substr(1); + } + break; + } + } + } + } + + // If matched, build URI + if (!matched_prefix.empty()) { + // Extract base path from nfs_url_: nfs://server/path -> /path + size_t proto_end = nfs_url_.find("://"); + if (proto_end != std::string::npos) { + size_t path_start = nfs_url_.find('/', proto_end + 3); + if (path_start != std::string::npos) { + std::string server_part = nfs_url_.substr(0, path_start); + std::string base_path = nfs_url_.substr(path_start); + if (base_path.back() != '/') { + base_path += "/"; + } + std::string uri = server_part + base_path + relative_path; + std::cout << "[NFSFileSystem] ConvertLocalPathToURI: " << path << " -> " + << uri << " (matched prefix: " << matched_prefix << ")" + << std::endl; + return uri; + } + } + } + + // No match, return empty (use path matching mode) + return ""; +} + +std::string NFSFileSystem::ConvertToNFSPath(const std::string& path) const { + // Support both URI mode and path matching mode (legacy) + // URI mode: "nfs://server/path/to/file" -> "/path/to/file" + // Pure URI mode: local_prefix_ is empty -> use path directly (relative to NFS + // mount point) Path matching mode: "/mnt/nfs_rocksdb/db/0/xxx.sst" -> + // "0/xxx.sst" Auto-convert mode: Convert matching local paths to URI + // internally + + // Check if it's a NFS URI + if (IsNFSURI(path)) { + std::string nfs_path = ExtractNFSPathFromURI(path); + std::cout << "[NFSFileSystem] ConvertToNFSPath (URI): " << path << " -> " + << nfs_path << std::endl; + return nfs_path; + } + + // Pure URI mode: If local_prefix_ is empty, use path directly (relative to + // NFS mount point) In this mode, we assume the path structure matches between + // local and NFS Example: path = "./home/db/0/xxx.sst" -> "home/db/0/xxx.sst" + // (relative to mount point) + if (local_prefix_.empty() && !nfs_url_.empty()) { + std::string nfs_path = path; + // Remove leading "./" if present + if (nfs_path.length() >= 2 && nfs_path.substr(0, 2) == "./") { + nfs_path = nfs_path.substr(2); + } + // Remove leading "/" if present (we want relative path) + while (!nfs_path.empty() && nfs_path[0] == '/') { + nfs_path = nfs_path.substr(1); + } + std::cout << "[NFSFileSystem] ConvertToNFSPath (pure URI mode): " << path + << " -> " << (nfs_path.empty() ? "." : nfs_path) << std::endl; + return nfs_path.empty() ? "." : nfs_path; + } + + // Try to convert local path to URI first (if it matches) + std::string uri = ConvertLocalPathToURI(path); + if (!uri.empty()) { + // Successfully converted to URI, extract NFS path from URI + std::string nfs_path = ExtractNFSPathFromURI(uri); + std::cout << "[NFSFileSystem] ConvertToNFSPath (auto-converted to URI): " + << path << " -> URI: " << uri << " -> NFS path: " << nfs_path + << std::endl; + return nfs_path; + } + + // Legacy path matching mode (fallback) + // Convert local path to NFS relative path + // + // Mapping: local_prefix or additional_prefixes -> nfs_url (already mounted in + // InitNFSContext) So we just need to extract the relative path part + // + // Example: local_prefix_ = "/mnt/nfs_rocksdb/db" + // nfs_url_ = "nfs://192.168.1.100/shared/rocksdb" + // path = "/mnt/nfs_rocksdb/db/0/xxx.sst" + // + // Extract relative path: "0/xxx.sst" (without leading slash) + // libnfs will access: nfs://192.168.1.100/shared/rocksdb/0/xxx.sst + + // First convert input path to absolute path + std::string abs_path = path; + if (!path.empty() && path[0] != '/') { + char cwd[PATH_MAX]; + if (getcwd(cwd, sizeof(cwd)) != nullptr) { + abs_path = std::string(cwd) + "/" + path; + size_t pos; + while ((pos = abs_path.find("/./")) != std::string::npos) { + abs_path.erase(pos, 2); + } + } + } + + // Try local_prefix first + if (!local_prefix_.empty() && abs_path.find(local_prefix_) == 0) { + std::string relative_path = abs_path.substr(local_prefix_.length()); + // Remove the leading slash and return to the relative path + while (!relative_path.empty() && relative_path[0] == '/') { + relative_path = relative_path.substr(1); + } + std::cout << "[NFSFileSystem] ConvertToNFSPath (legacy): " << path + << " (abs: " << abs_path << ")" + << " -> " << (relative_path.empty() ? "." : relative_path) + << " (matched prefix: " << local_prefix_ << ")" << std::endl; + return relative_path.empty() ? "." : relative_path; + } else if (!local_prefix_.empty()) { + // Log when path doesn't match local_prefix for debugging + std::cout << "[NFSFileSystem] ConvertToNFSPath: path " << path + << " (abs: " << abs_path + << ") does not match local_prefix: " << local_prefix_ + << std::endl; + } + + // Try additional prefixes + for (const auto& prefix : additional_prefixes_) { + if (!prefix.empty()) { + std::string abs_prefix = ToAbsolutePath(prefix); + // Normalize prefix (remove trailing slash for comparison) + while (!abs_prefix.empty() && abs_prefix.back() == '/') { + abs_prefix.pop_back(); + } + if (!abs_prefix.empty() && abs_path.find(abs_prefix) == 0) { + std::string relative_path = abs_path.substr(abs_prefix.length()); + // Remove the leading slash + while (!relative_path.empty() && relative_path[0] == '/') { + relative_path = relative_path.substr(1); + } + std::cout << "[NFSFileSystem] ConvertToNFSPath (legacy): " << path + << " -> " << (relative_path.empty() ? "." : relative_path) + << " (via prefix: " << prefix << ")" << std::endl; + return relative_path.empty() ? "." : relative_path; + } + } + } + + if (path.find("/nfs/") == 0) { + std::string rel = path.substr(5); // Remove the "/nfs/" prefix + return rel.empty() ? "." : rel; + } + + std::cout << "[NFSFileSystem] ConvertToNFSPath: " << path << " -> " << path + << " (unchanged, not NFS path)" << std::endl; + return path; +} + +// Recursively create a catalog +IOStatus NFSFileSystem::CreateDirRecursive(const std::string& nfs_path) { + // Create step by step starting from the root + std::string current_path; + size_t pos = 0; + + while (pos < nfs_path.length()) { + size_t next_slash = nfs_path.find('/', pos + 1); + if (next_slash == std::string::npos) { + next_slash = nfs_path.length(); + } + + current_path = nfs_path.substr(0, next_slash); + pos = next_slash; + + if (current_path.empty() || current_path == "/") { + continue; + } + + // Check if the catalog exists + struct nfs_stat_64 st; + if (nfs_stat64(nfs_ctx_, current_path.c_str(), &st) != 0) { + // The directory does not exist, create it + if (nfs_mkdir(nfs_ctx_, current_path.c_str()) != 0) { + std::string err = nfs_get_error(nfs_ctx_); + // Ignore "already exists" error (possibly concurrent creation) + if (err.find("exist") == std::string::npos && + err.find("EXIST") == std::string::npos) { + return IOStatus::IOError("Failed to create NFS directory: " + + current_path + ", error: " + err); + } + } + } + } + + return IOStatus::OK(); +} + + +IOStatus NFSFileSystem::NewSequentialFile( + const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::NewSequentialFile(fname, options, result, dbg); + } + + // Check if NFS context is available + if (!nfs_ctx_) { + std::cerr << "[NFSFileSystem] NFS context not available, falling back to " + "local filesystem for: " + << fname << std::endl; + return FileSystemWrapper::NewSequentialFile(fname, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + std::cerr << "[NFSFileSystem] NewSequentialFile: " << fname + << " -> NFS path: " << nfs_path << std::endl; + std::cerr << "[NFSFileSystem] NFS URL (mount point): " << nfs_url_ + << std::endl; + std::cerr << "[NFSFileSystem] Full NFS path (relative to mount): " << nfs_path + << std::endl; + std::cerr.flush(); + + struct nfsfh* fh = nullptr; + if (nfs_open(nfs_ctx_, nfs_path.c_str(), O_RDONLY, &fh) != 0) { + std::string error = nfs_get_error(nfs_ctx_); + std::cerr << "[NFSFileSystem] ERROR: Failed to open NFS file: " << fname + << std::endl; + std::cerr << "[NFSFileSystem] ERROR: NFS path used: " << nfs_path + << std::endl; + std::cerr << "[NFSFileSystem] ERROR: NFS error: " << error << std::endl; + std::cerr.flush(); + return IOStatus::IOError("Failed to open NFS file for reading: " + fname + + ", error: " + error); + } + + result->reset(new NFSSequentialFile(nfs_ctx_, fh, fname, nfs_mutex_)); + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::NewRandomAccessFile( + const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::NewRandomAccessFile(fname, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + std::cout << "[NFSFileSystem] NewRandomAccessFile: " << fname + << " -> NFS path: " << nfs_path << std::endl; + + struct nfsfh* fh = nullptr; + if (nfs_open(nfs_ctx_, nfs_path.c_str(), O_RDONLY, &fh) != 0) { + return IOStatus::IOError( + "Failed to open NFS file for random access: " + fname + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + + result->reset(new NFSRandomAccessFile(nfs_ctx_, fh, fname, &nfs_mutex_)); + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::NewWritableFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::NewWritableFile(fname, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + std::cout << "[NFSFileSystem] NewWritableFile: " << fname + << " -> NFS path: " << nfs_path << std::endl; + + // Ensure that the parent directory exists + size_t last_slash = nfs_path.rfind('/'); + if (last_slash != std::string::npos && last_slash > 0) { + std::string parent_dir = nfs_path.substr(0, last_slash); + IOStatus dir_status = CreateDirRecursive(parent_dir); + if (!dir_status.ok()) { + return dir_status; + } + } + + struct nfsfh* fh = nullptr; + // nfs_creat Create a file using the mode parameter + int mode = 0644; // rw-r--r-- + + if (nfs_creat(nfs_ctx_, nfs_path.c_str(), mode, &fh) != 0) { + return IOStatus::IOError( + "Failed to create NFS file: " + fname + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + + result->reset(new NFSWritableFile(nfs_ctx_, fh, fname, nfs_mutex_)); + return IOStatus::OK(); +} + +// Reopen the file in append mode (for LOG files, etc.) +IOStatus NFSFileSystem::ReopenWritableFile( + const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::ReopenWritableFile(fname, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + std::cout << "[NFSFileSystem] ReopenWritableFile (append): " << fname + << " -> NFS path: " << nfs_path << std::endl; + + // Ensure that the parent directory exists + size_t last_slash = nfs_path.rfind('/'); + if (last_slash != std::string::npos && last_slash > 0) { + std::string parent_dir = nfs_path.substr(0, last_slash); + IOStatus dir_status = CreateDirRecursive(parent_dir); + if (!dir_status.ok()) { + return dir_status; + } + } + + struct nfsfh* fh = nullptr; + + // Check if the file exists + struct nfs_stat_64 st; + uint64_t initial_size = 0; + bool file_exists = (nfs_stat64(nfs_ctx_, nfs_path.c_str(), &st) == 0); + + if (file_exists) { + // The file exists, opening in append mode + if (nfs_open(nfs_ctx_, nfs_path.c_str(), O_WRONLY | O_APPEND, &fh) != 0) { + return IOStatus::IOError( + "Failed to open NFS file for appending: " + fname + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + initial_size = st.nfs_size; + } else { + // The file does not exist, create a new file + int mode = 0644; // rw-r--r-- + if (nfs_creat(nfs_ctx_, nfs_path.c_str(), mode, &fh) != 0) { + return IOStatus::IOError( + "Failed to create NFS file: " + fname + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + } + + result->reset(new NFSWritableFile(nfs_ctx_, fh, fname, nfs_mutex_)); + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::DeleteFile(const std::string& fname, + const IOOptions& options, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::DeleteFile(fname, options, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + std::cout << "[NFSFileSystem] DeleteFile: " << fname + << " -> NFS path: " << nfs_path << std::endl; + + if (nfs_unlink(nfs_ctx_, nfs_path.c_str()) != 0) { + return IOStatus::IOError( + "Failed to delete NFS file: " + fname + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::RenameFile(const std::string& src, + const std::string& target, + const IOOptions& options, + IODebugContext* dbg) { + bool src_is_nfs = IsNFSPath(src); + bool target_is_nfs = IsNFSPath(target); + + std::cout << "[NFSFileSystem] RenameFile check: src=" << src + << " (is_nfs=" << src_is_nfs << "), target=" << target + << " (is_nfs=" << target_is_nfs << ")" << std::endl; + + if (!src_is_nfs && !target_is_nfs) { + std::cout << "[NFSFileSystem] RenameFile: Both local, delegating to base FS" + << std::endl; + return FileSystemWrapper::RenameFile(src, target, options, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_src = ConvertToNFSPath(src); + std::string nfs_target = ConvertToNFSPath(target); + + std::cout << "[NFSFileSystem] RenameFile NFS: " << nfs_src << " -> " + << nfs_target << std::endl; + + if (nfs_rename(nfs_ctx_, nfs_src.c_str(), nfs_target.c_str()) != 0) { + std::string err = nfs_get_error(nfs_ctx_); + std::cerr << "[NFSFileSystem] RenameFile FAILED: " << err << std::endl; + return IOStatus::IOError("Failed to rename NFS file: " + src + " to " + + target + ", error: " + err); + } + + std::cout << "[NFSFileSystem] RenameFile SUCCESS" << std::endl; + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::GetFileSize(const std::string& fname, + const IOOptions& options, + uint64_t* file_size, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::GetFileSize(fname, options, file_size, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + struct nfs_stat_64 st; + + if (nfs_stat64(nfs_ctx_, nfs_path.c_str(), &st) != 0) { + return IOStatus::IOError("Failed to stat NFS file: " + fname + ", error: " + + std::string(nfs_get_error(nfs_ctx_))); + } + + *file_size = st.nfs_size; + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::FileExists(const std::string& fname, + const IOOptions& options, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::FileExists(fname, options, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + struct nfs_stat_64 st; + + if (nfs_stat64(nfs_ctx_, nfs_path.c_str(), &st) != 0) { + return IOStatus::NotFound(fname); + } + + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::CreateDir(const std::string& dirname, + const IOOptions& options, + IODebugContext* dbg) { + if (!IsNFSPath(dirname)) { + return FileSystemWrapper::CreateDir(dirname, options, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(dirname); + std::cout << "[NFSFileSystem] CreateDir: " << dirname + << " -> NFS path: " << nfs_path << std::endl; + + if (nfs_mkdir(nfs_ctx_, nfs_path.c_str()) != 0) { + std::string err = nfs_get_error(nfs_ctx_); + // If the directory already exists, it is not an error + if (err.find("exist") != std::string::npos || + err.find("EXIST") != std::string::npos) { + return IOStatus::OK(); + } + return IOStatus::IOError("Failed to create NFS directory: " + dirname + + ", error: " + err); + } + + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::CreateDirIfMissing(const std::string& dirname, + const IOOptions& options, + IODebugContext* dbg) { + if (!IsNFSPath(dirname)) { + return FileSystemWrapper::CreateDirIfMissing(dirname, options, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(dirname); + std::cout << "[NFSFileSystem] CreateDirIfMissing: " << dirname + << " -> NFS path: " << nfs_path << std::endl; + + // Check if the directory exists first + struct nfs_stat_64 st; + if (nfs_stat64(nfs_ctx_, nfs_path.c_str(), &st) == 0) { + // The catalog already exists + return IOStatus::OK(); + } + + // Recursively create a catalog + return CreateDirRecursive(nfs_path); +} + +IOStatus NFSFileSystem::GetChildren(const std::string& dir, + const IOOptions& options, + std::vector* result, + IODebugContext* dbg) { + if (!IsNFSPath(dir)) { + return FileSystemWrapper::GetChildren(dir, options, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(dir); + + struct nfsdir* nfsdir_handle; + if (nfs_opendir(nfs_ctx_, nfs_path.c_str(), &nfsdir_handle) != 0) { + return IOStatus::IOError( + "Failed to open NFS directory: " + dir + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + + result->clear(); + struct nfsdirent* dirent; + while ((dirent = nfs_readdir(nfs_ctx_, nfsdir_handle)) != nullptr) { + std::string name = dirent->name; + if (name != "." && name != "..") { + result->push_back(name); + } + } + + nfs_closedir(nfs_ctx_, nfsdir_handle); + return IOStatus::OK(); +} + +IOStatus NFSFileSystem::GetFileModificationTime(const std::string& fname, + const IOOptions& options, + uint64_t* file_mtime, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::GetFileModificationTime( + fname, options, file_mtime, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + struct nfs_stat_64 st; + + if (nfs_stat64(nfs_ctx_, nfs_path.c_str(), &st) != 0) { + return IOStatus::IOError("Failed to stat NFS file: " + fname + ", error: " + + std::string(nfs_get_error(nfs_ctx_))); + } + + *file_mtime = st.nfs_mtime; + return IOStatus::OK(); +} + +// Rewrite NewDirectory to open directory handles +IOStatus NFSFileSystem::NewDirectory(const std::string& name, + const IOOptions& io_opts, + std::unique_ptr* result, + IODebugContext* dbg) { + if (!IsNFSPath(name)) { + return FileSystemWrapper::NewDirectory(name, io_opts, result, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(name); + std::cout << "[NFSFileSystem] NewDirectory: " << name + << " -> NFS path: " << nfs_path << std::endl; + + // Check if the catalog exists + struct nfs_stat_64 st; + if (nfs_stat64(nfs_ctx_, nfs_path.c_str(), &st) != 0) { + return IOStatus::IOError( + "Failed to open NFS directory: " + name + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + + // Check if it's a table of contents + if (!S_ISDIR(st.nfs_mode)) { + return IOStatus::IOError("Not a directory: " + name); + } + + result->reset(new NFSDirectory(nfs_ctx_, nfs_path, nfs_mutex_)); + return IOStatus::OK(); +} + +// Delete directory +IOStatus NFSFileSystem::DeleteDir(const std::string& dirname, + const IOOptions& options, + IODebugContext* dbg) { + if (!IsNFSPath(dirname)) { + return FileSystemWrapper::DeleteDir(dirname, options, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(dirname); + std::cout << "[NFSFileSystem] DeleteDir: " << dirname + << " -> NFS path: " << nfs_path << std::endl; + + if (nfs_rmdir(nfs_ctx_, nfs_path.c_str()) != 0) { + return IOStatus::IOError( + "Failed to delete NFS directory: " + dirname + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + + return IOStatus::OK(); +} + +// Check if path is a directory +IOStatus NFSFileSystem::IsDirectory(const std::string& path, + const IOOptions& options, + bool* is_dir, + IODebugContext* dbg) { + if (!IsNFSPath(path)) { + return FileSystemWrapper::IsDirectory(path, options, is_dir, dbg); + } + + std::lock_guard lock(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(path); + struct nfs_stat_64 st; + + if (nfs_stat64(nfs_ctx_, nfs_path.c_str(), &st) != 0) { + return IOStatus::IOError("Failed to stat NFS path: " + path + ", error: " + + std::string(nfs_get_error(nfs_ctx_))); + } + + if (is_dir != nullptr) { + *is_dir = S_ISDIR(st.nfs_mode); + } + return IOStatus::OK(); +} + +// Rewrite LockFile +IOStatus NFSFileSystem::LockFile(const std::string& fname, + const IOOptions& options, + FileLock** lock, + IODebugContext* dbg) { + if (!IsNFSPath(fname)) { + return FileSystemWrapper::LockFile(fname, options, lock, dbg); + } + + std::lock_guard lk(nfs_mutex_); + + std::string nfs_path = ConvertToNFSPath(fname); + std::cout << "[NFSFileSystem] LockFile: " << fname + << " -> NFS path: " << nfs_path << std::endl; + + // Ensure that the parent directory exists + size_t last_slash = nfs_path.rfind('/'); + if (last_slash != std::string::npos && last_slash > 0) { + std::string parent_dir = nfs_path.substr(0, last_slash); + IOStatus dir_status = CreateDirRecursive(parent_dir); + if (!dir_status.ok()) { + return dir_status; + } + } + + // Create or open a lock file + struct nfsfh* fh = nullptr; + int mode = 0644; + + // Try creating a file if it doesn't exist + if (nfs_creat(nfs_ctx_, nfs_path.c_str(), mode, &fh) != 0) { + // If the creation fails, try to open the existing file + if (nfs_open(nfs_ctx_, nfs_path.c_str(), O_RDWR, &fh) != 0) { + return IOStatus::IOError( + "Failed to open lock file: " + fname + + ", error: " + std::string(nfs_get_error(nfs_ctx_))); + } + } + + // Note: NFS has limited file lock support, here we simply keep the file open + // True locking requires the use of the NLM (Network Lock Manager) protocol + // For single-instance scenarios, keeping the file open is sufficient + + *lock = new NFSFileLock(nfs_ctx_, fh, fname, nfs_mutex_); + return IOStatus::OK(); +} + +// Rewrite the UnlockFile +IOStatus NFSFileSystem::UnlockFile(FileLock* lock, + const IOOptions& options, + IODebugContext* dbg) { + if (lock == nullptr) { + return IOStatus::OK(); + } + + NFSFileLock* nfs_lock = dynamic_cast(lock); + if (nfs_lock == nullptr) { + // Not our lock, leave it to the parent to handle + return FileSystemWrapper::UnlockFile(lock, options, dbg); + } + + std::cout << "[NFSFileSystem] UnlockFile: " << nfs_lock->GetFilename() + << std::endl; + + // NFSFileLock's destructor closes the file handle + delete nfs_lock; + return IOStatus::OK(); +} + +// Rewrite NewLogger, making sure to use our NewWritableFile +IOStatus NFSFileSystem::NewLogger(const std::string& fname, + const IOOptions& io_opts, + std::shared_ptr* result, + IODebugContext* dbg) { + // For NFS paths, we need to use our own NewWritableFile + // Instead of calling target_->NewLogger (local file system will be used) + + if (!IsNFSPath(fname)) { + return FileSystemWrapper::NewLogger(fname, io_opts, result, dbg); + } + + std::cout << "[NFSFileSystem] NewLogger: " << fname << std::endl; + + FileOptions options; + options.io_options = io_opts; + options.writable_file_max_buffer_size = 1024 * 1024; + + std::unique_ptr writable_file; + // Call our own NewWritableFile + const IOStatus status = NewWritableFile(fname, options, &writable_file, dbg); + if (!status.ok()) { + return status; + } + + // Create an EnvLogger (using rocksdb's EnvLogger) + *result = std::make_shared( + std::move(writable_file), fname, options, Env::Default()); + return IOStatus::OK(); +} + + +// Check if path is a NFS URI +bool NFSFileSystem::IsNFSURI(const std::string& path) { + return path.find(kProto) == 0; +} + +// Extract NFS path from URI +// "nfs://server/path/to/file" -> relative path to mount point +// Since nfs_mount already mounted the base path, we need to return +// the path relative to the mount point +std::string NFSFileSystem::ExtractNFSPathFromURI(const std::string& uri) const { + if (!IsNFSURI(uri)) { + return uri; // Not a URI, return as-is + } + + // Find the path part after "nfs://server" or "nfs://server:port" + size_t proto_len = strlen(kProto); // "nfs://" + size_t start = proto_len; + + // Skip server name (find first slash after protocol) + size_t slash = uri.find('/', start); + + if (slash == std::string::npos) { + // No path, return "/" + return "/"; + } + + // Extract full path from URI + std::string full_path = + uri.substr(slash); // e.g., "/shared/rocksdb/0/000036.log" + if (full_path.empty()) { + return "/"; + } + + // Extract mount point path from nfs_url_ + // nfs_url_ = "nfs://localhost/shared/rocksdb" + // We need to get "/shared/rocksdb" as the mount point + size_t url_proto_end = nfs_url_.find("://"); + if (url_proto_end != std::string::npos) { + size_t url_path_start = nfs_url_.find('/', url_proto_end + 3); + if (url_path_start != std::string::npos) { + std::string mount_path = + nfs_url_.substr(url_path_start); // "/shared/rocksdb" + + // If full_path starts with mount_path, extract relative path + if (full_path.find(mount_path) == 0) { + std::string relative = full_path.substr(mount_path.length()); + // Ensure it starts with / (relative to mount point root) + if (relative.empty()) { + relative = "/"; + } else if (relative[0] != '/') { + relative = "/" + relative; + } + std::cout << "[NFSFileSystem] ExtractNFSPathFromURI: " << uri + << " -> mount_path: " << mount_path + << " -> relative: " << relative << std::endl; + return relative; // "/0/000036.log" + } + } + } + + // Fallback: return full path (shouldn't happen in normal cases) + std::cout << "[NFSFileSystem] ExtractNFSPathFromURI (fallback): " << uri + << " -> " << full_path << std::endl; + return full_path; +} + +// Create NFSFileSystem from URI (similar to HdfsFileSystem::Create) +Status NFSFileSystem::Create(const std::shared_ptr& base, + const std::string& uri, + std::unique_ptr* result) { + result->reset(); + + if (uri.empty() || uri == kProto) { + return Status::InvalidArgument("NFS URI cannot be empty or just 'nfs://'"); + } + + if (!IsNFSURI(uri)) { + return Status::InvalidArgument("URI must start with 'nfs://': " + uri); + } + + // Parse URI: nfs://server/path or nfs://server:port/path + size_t proto_len = strlen(kProto); // "nfs://" + size_t start = proto_len; + + // Find server and port + size_t colon = uri.find(':', start); + size_t slash = uri.find('/', start); + + std::string server; + std::string mount_path = "/"; + + // Check for port (colon before slash) + if (colon != std::string::npos && + (slash == std::string::npos || colon < slash)) { + // Has port: nfs://server:port/path + server = uri.substr(start, colon - start); + // Port is not used by libnfs (it uses standard NFS port), but we skip it + if (slash != std::string::npos) { + mount_path = uri.substr(slash); + } + } else if (slash != std::string::npos) { + // No port: nfs://server/path + server = uri.substr(start, slash - start); + mount_path = uri.substr(slash); + } else { + // No path: nfs://server + server = uri.substr(start); + mount_path = "/"; + } + + if (server.empty()) { + return Status::InvalidArgument("NFS server cannot be empty in URI: " + uri); + } + + // Construct full NFS URL for libnfs + std::string nfs_url = kProto + server + mount_path; + + // Create NFSFileSystem with empty local_prefix (URI mode doesn't need it) + // Store original URI in a custom way - we'll use nfs_url_ to store the full + // URI + try { + // Create NFSFileSystem - it will store the full URI in nfs_url_ + auto nfs_fs = std::make_unique(nfs_url, "", base); + // Override nfs_url_ to store the original URI for GetId() + // Note: This is a workaround since we can't modify nfs_url_ after + // construction In practice, GetId() will return the nfs_url_ which is the + // full mount path + result->reset(nfs_fs.release()); + return Status::OK(); + } catch (const std::exception& e) { + return Status::IOError("Failed to create NFS FileSystem from URI: " + uri + + ", error: " + std::string(e.what())); + } +} + +Status NewNFSFileSystem(const std::string& nfs_url, + const std::string& local_prefix, + std::shared_ptr* result) { + try { + auto base_fs = FileSystem::Default(); + *result = std::make_shared(nfs_url, local_prefix, base_fs); + return Status::OK(); + } catch (const std::exception& e) { + return Status::IOError("Failed to create NFS FileSystem: " + + std::string(e.what())); + } +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/nfs_filesystem.h b/src/tendisplus/storage/rocks/nfs_filesystem.h new file mode 100644 index 00000000..792260ed --- /dev/null +++ b/src/tendisplus/storage/rocks/nfs_filesystem.h @@ -0,0 +1,223 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#pragma once + +#include +#include +#include + +#include "rocksdb/env.h" +#include "rocksdb/file_system.h" +#include "rocksdb/io_status.h" +#include "rocksdb/status.h" +// Choose one based on the actual installation +#if __has_include() +#include +#elif __has_include() +#include +#elif __has_include() +#include +#else +#error "libnfs header not found" +#endif +namespace ROCKSDB_NAMESPACE { + +class NFSFileSystem : public FileSystemWrapper { + public: + static const char* kClassName() { + return "NFSFileSystem"; + } + static const char* kNickName() { + return "nfs"; + } + static constexpr const char* kProto = "nfs://"; + + explicit NFSFileSystem(const std::string& nfs_url, + const std::string& local_prefix, + const std::shared_ptr& base); + + ~NFSFileSystem() override; + + const char* Name() const override { + return kClassName(); + } + const char* NickName() const override { + return kNickName(); + } + + std::string GetId() const override; + + Status ValidateOptions(const DBOptions& db_opts, + const ColumnFamilyOptions& cf_opts) const override; + + // Get configuration information + const std::string& GetNFSUrl() const { + return nfs_url_; + } + const std::string& GetLocalPrefix() const { + return local_prefix_; + } + + // Add an additional path prefix to be mapped to NFS + // This is useful for mapping wal_dir, db_log_dir, etc. to NFS + void RegisterPathPrefix(const std::string& prefix); + + // Create NFSFileSystem from URI (similar to HdfsFileSystem::Create) + // URI format: nfs://server/path or nfs://server:port/path + static Status Create(const std::shared_ptr& base, + const std::string& uri, + std::unique_ptr* result); + + // Key methods for rewriting FileSystem (declare, not implement) + IOStatus NewSequentialFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) override; + + IOStatus NewRandomAccessFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) override; + + IOStatus NewWritableFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) override; + + // Rewrite NewDirectory to open directory handles (for fsync, etc.) + IOStatus NewDirectory(const std::string& name, + const IOOptions& io_opts, + std::unique_ptr* result, + IODebugContext* dbg) override; + + // Reopen the file in append mode (for LOG files, etc.) + IOStatus ReopenWritableFile(const std::string& fname, + const FileOptions& options, + std::unique_ptr* result, + IODebugContext* dbg) override; + + IOStatus FileExists(const std::string& fname, + const IOOptions& options, + IODebugContext* dbg) override; + + IOStatus GetChildren(const std::string& dir, + const IOOptions& options, + std::vector* result, + IODebugContext* dbg) override; + + IOStatus DeleteFile(const std::string& fname, + const IOOptions& options, + IODebugContext* dbg) override; + // Catalog operations + IOStatus CreateDir(const std::string& dirname, + const IOOptions& options, + IODebugContext* dbg) override; + + IOStatus CreateDirIfMissing(const std::string& dirname, + const IOOptions& options, + IODebugContext* dbg) override; + + IOStatus DeleteDir(const std::string& dirname, + const IOOptions& options, + IODebugContext* dbg) override; + + IOStatus GetFileSize(const std::string& fname, + const IOOptions& options, + uint64_t* file_size, + IODebugContext* dbg) override; + + IOStatus GetFileModificationTime(const std::string& fname, + const IOOptions& options, + uint64_t* file_mtime, + IODebugContext* dbg) override; + + IOStatus RenameFile(const std::string& src, + const std::string& target, + const IOOptions& options, + IODebugContext* dbg) override; + + // Rewrite LockFile and UnlockFile + IOStatus LockFile(const std::string& fname, + const IOOptions& options, + FileLock** lock, + IODebugContext* dbg) override; + + IOStatus UnlockFile(FileLock* lock, + const IOOptions& options, + IODebugContext* dbg) override; + + // Rewrite NewLogger, making sure to use our NewWritableFile + IOStatus NewLogger(const std::string& fname, + const IOOptions& io_opts, + std::shared_ptr* result, + IODebugContext* dbg) override; + + IOStatus IsDirectory(const std::string& path, + const IOOptions& options, + bool* is_dir, + IODebugContext* dbg) override; + + // Rewrite SupportedOps to avoid errors when calling target_ + void SupportedOps(int64_t& supported_ops) override { + supported_ops = 0; + // NFS doesn't support asynchronous IO, so don't set kAsyncIO + } + + private: + std::string nfs_url_; // NFS remote URL + std::string local_prefix_; // Local path prefix (primary) + std::vector + additional_prefixes_; // Additional path prefixes to map to NFS + struct nfs_context* nfs_ctx_; + std::mutex nfs_mutex_; + + void InitNFSContext(); + + // Check if the file is a data file (.sst, .blob, .ldb) + bool IsDataFile(const std::string& path) const; + + // Check if the path should be accessed via NFS + // Supports both URI format (nfs://server/path) and path matching (legacy) + // ALL files under local_prefix_ and additional_prefixes_ go to NFS to ensure + // consistency between local Tendis and remote workers + bool IsNFSPath(const std::string& path) const; + + // Convert the local path or URI to an NFS-relative path + // URI format: "nfs://server/path/to/file" -> "/path/to/file" + // Path matching: "/mnt/nfs_rocksdb/db/0/xxx.sst" -> "/0/xxx.sst" + // Auto-convert: If path matches local_prefix_, automatically convert to URI + // internally + std::string ConvertToNFSPath(const std::string& path) const; + + // Convert local path to URI (if it matches local_prefix_) + // Example: "/mnt/rocksdb/db/0/xxx.sst" -> + // "nfs://localhost/shared/rocksdb/db/0/xxx.sst" Returns empty string if path + // doesn't match or URI conversion is not needed + std::string ConvertLocalPathToURI(const std::string& path) const; + + // Extract NFS path from URI + // "nfs://server/path/to/file" -> "/path/to/file" + // Returns path relative to the NFS mount point + std::string ExtractNFSPathFromURI(const std::string& uri) const; + + // Check if path is a NFS URI + static bool IsNFSURI(const std::string& path); + + // Recursively create a directory (for internal use, need to be called with a + // lock in place) + IOStatus CreateDirRecursive(const std::string& nfs_path); + + // Add an additional path prefix to be mapped to NFS + void AddPathPrefix(const std::string& prefix); +}; + +// Factory function +// nfs_url: NFS server address, such as "nfs://192.168.1.100/shared/rocksdb" +// local_prefix: Local path prefix, the path used by RocksDB maps to NFS if it +// starts with this +Status NewNFSFileSystem(const std::string& nfs_url, + const std::string& local_prefix, + std::shared_ptr* result); +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/remote_compaction/csa_server.cc b/src/tendisplus/storage/rocks/remote_compaction/csa_server.cc new file mode 100644 index 00000000..6b43f2b8 --- /dev/null +++ b/src/tendisplus/storage/rocks/remote_compaction/csa_server.cc @@ -0,0 +1,537 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "csa.grpc.pb.h" // NOLINT(build/include_subdir) +#include "rocksdb/db.h" +#include "rocksdb/db/compaction/compaction_job.h" +#include "rocksdb/db/dbformat.h" +#include "rocksdb/db/log_writer.h" +#include "rocksdb/db/version_edit.h" +#include "rocksdb/env.h" +#include "rocksdb/file/writable_file_writer.h" +#include "rocksdb/options.h" +#include "rocksdb/sst_file_reader.h" +#include "util/string_util.h" + +#include "tendisplus/storage/rocks/shared_filesystem.h" +#ifdef HDFS +#include "plugin/hdfs/env_hdfs.h" +#endif + +#include "rocksdb/statistics.h" + +#include "tendisplus/storage/rocks/nfs_filesystem.h" +#include "tendisplus/storage/rocks/remote_compaction/def.h" + +namespace fs = std::filesystem; + +ROCKSDB_NAMESPACE::RemoteOpenAndCompactOptions compaction_service_options; + +std::atomic local_task_nums_ = std::atomic(0); + +// Get max concurrent tasks with default value +int64_t GetMaxConcurrentTasks() { + return compaction_service_options.GetMaxConcurrentTasks(); +} + +// Shared FileSystem Cache +// Cache shared file systems that have been created to avoid recreating them +// every time +class SharedFileSystemCache { + public: + struct CachedFS { + std::shared_ptr fs; + std::unique_ptr env; + std::string uri; + std::string local_prefix; + }; + + // Get or create a shared file system from URI (URI mode) + // Returns the Env pointer (possibly nullptr means using the default) + rocksdb::Env* GetOrCreateEnvFromURI(const std::string& uri) { + if (uri.empty()) { + return nullptr; + } + + std::string cache_key = "URI:" + uri; + + { + std::lock_guard lock(mutex_); + auto it = cache_.find(cache_key); + if (it != cache_.end()) { + std::cout << "[FSCache] Reusing cached FileSystem for URI: " << uri + << std::endl; + return it->second.env.get(); + } + } + + // A new FileSystem needs to be created from URI + // Use unified interface to create shared filesystem + // Supports: nfs://, hdfs://, s3://, etc. + std::shared_ptr shared_fs; + rocksdb::Status status = rocksdb::CreateSharedFileSystem( + rocksdb::FileSystem::Default(), uri, &shared_fs); + if (!status.ok() || !shared_fs) { + std::cerr << "[FSCache] Failed to create shared filesystem from URI: " + << uri << ", error: " << status.ToString() << std::endl; + return nullptr; + } + std::unique_ptr env = rocksdb::NewCompositeEnv(shared_fs); + + if (!env) { + std::cerr << "[FSCache] Failed to create Env from URI: " << uri + << std::endl; + return nullptr; + } + + std::cout << "[FSCache] Created new FileSystem from URI: " << uri + << std::endl; + + std::lock_guard lock(mutex_); + // Double-check after acquiring lock + auto it = cache_.find(cache_key); + if (it != cache_.end()) { + return it->second.env.get(); + } + + CachedFS cached; + cached.fs = shared_fs; + cached.env = std::move(env); + cached.uri = uri; + cached.local_prefix = ""; // URI mode doesn't need local_prefix + + rocksdb::Env* result = cached.env.get(); + cache_[cache_key] = std::move(cached); + return result; + } + + // Get or create a shared file system (legacy path matching mode) + // Returns the Env pointer (possibly nullptr means using the default) + rocksdb::Env* GetOrCreateEnv(const std::string& uri, + const std::string& local_prefix) { + if (uri.empty() || local_prefix.empty()) { + return nullptr; + } + + std::string cache_key = uri + "|" + local_prefix; + + { + std::lock_guard lock(mutex_); + auto it = cache_.find(cache_key); + if (it != cache_.end()) { + std::cout << "[FSCache] Reusing cached FileSystem for: " << uri + << std::endl; + return it->second.env.get(); + } + } + + // A new FileSystem needs to be created + std::shared_ptr fs; + std::unique_ptr env; + + // Create the corresponding file system according to the URI scheme + if (uri.find("nfs://") == 0) { + std::cout << "[FSCache] Creating NFS FileSystem for: " << uri + << ", local_prefix: " << local_prefix << std::endl; + // rocksdb::Status status = rocksdb::NewNFSFileSystemSimple(uri, + // local_prefix, &fs); + rocksdb::Status status = + rocksdb::NewNFSFileSystem(uri, local_prefix, &fs); + if (!status.ok() || !fs) { + std::cerr << "[FSCache] Failed to create NFS FileSystem: " + << status.ToString() << std::endl; + return nullptr; + } + env = rocksdb::NewCompositeEnv(fs); + } +#ifdef HDFS + if (uri.find("hdfs://") == 0) { + rocksdb::NewHdfsEnv(uri, &env); + } +#endif + // More storage types can be added in the future: + // else if (uri.find("s3://") == 0) { ... } + // else if (uri.find("gcs://") == 0) { ... } + // else { + // std::cerr << "[FSCache] Unsupported URI scheme: " << uri << std::endl; + // return nullptr; + // } + + if (!env) { + std::cerr << "[FSCache] Failed to create Env for: " << uri << std::endl; + return nullptr; + } + + std::cout << "[FSCache] Created new FileSystem for: " << uri + << ", local_prefix: " << local_prefix << std::endl; + + std::lock_guard lock(mutex_); + // Double-check after acquiring lock + auto it = cache_.find(cache_key); + if (it != cache_.end()) { + return it->second.env.get(); + } + + CachedFS cached; + cached.fs = std::move(fs); + cached.env = std::move(env); + cached.uri = uri; + cached.local_prefix = local_prefix; + + rocksdb::Env* result = cached.env.get(); + cache_[cache_key] = std::move(cached); + return result; + } + + // Remove the specified cache (for configuration changes) + void Invalidate(const std::string& uri, const std::string& local_prefix) { + std::string cache_key = uri + "|" + local_prefix; + std::lock_guard lock(mutex_); + auto it = cache_.find(cache_key); + if (it != cache_.end()) { + std::cout << "[FSCache] Invalidated FileSystem cache for: " << uri + << std::endl; + cache_.erase(it); + } + } + + // Empty all caches + void Clear() { + std::lock_guard lock(mutex_); + cache_.clear(); + std::cout << "[FSCache] Cleared all cached FileSystems" << std::endl; + } + + static SharedFileSystemCache& Instance() { + static SharedFileSystemCache instance; + return instance; + } + + private: + SharedFileSystemCache() = default; + std::mutex mutex_; + std::unordered_map cache_; +}; + +// Custom status codes for CSA server +// These are returned via CompactionReply.code field +// Note: Code 5 is used by RocksDB for IOError, so we use higher numbers for +// CSA-specific codes +constexpr int kCSACodeBusy = + 100; // CSA server is busy, client should use local compaction +constexpr int kCSACodeStaleTask = + 101; // Input files not found, this is a stale task + +// CSA Service Implementation +class CSAImpl final : public csa::CSAService::Service { + public: + // Execute compaction task (shared storage mode only) + grpc::Status ExecuteCompactionTask( + grpc::ServerContext* context, + const csa::CompactionArgs* compaction_args, + csa::CompactionReply* compaction_reply) override { + int64_t max_tasks = GetMaxConcurrentTasks(); + int64_t current = local_task_nums_.fetch_add(1); + + // If server is busy (at or over capacity), reject immediately + // This prevents blocking gRPC threads and reduces stale task probability + if (current >= max_tasks) { + local_task_nums_.fetch_sub(1); + std::cout << "[CSA] Server busy, rejecting task (current: " << current + << ", max: " << max_tasks + << "). Client should use local compaction." << std::endl; + compaction_reply->set_code(kCSACodeBusy); + compaction_reply->set_result("CSA server busy"); + return ::grpc::Status::OK; + } + + std::cout << "CSA ExecuteCompactionTask() concurrency: " << (current + 1) + << "/" << max_tasks << std::endl; + + std::string compaction_service_result; + rocksdb::CompactionServiceOptionsOverride options_override; + ROCKSDB_NAMESPACE::Options options_; + + std::string db_path = compaction_args->name(); + std::string output_directory = compaction_args->output_directory(); + std::string compaction_input = compaction_args->input(); + + // Shared storage mode: use shared filesystem (read-only) + // CSA server reads from shared storage via URI, writes output to shared + // storage Tendisplus will install results from output directory to final + // location + const std::string& shared_fs_uri = compaction_args->shared_fs_uri(); + + std::cout << "[CSA] Received compaction request:" + << " db_path=" << db_path + << ", output_directory=" << output_directory + << ", shared_fs_uri=" << shared_fs_uri << std::endl; + + // URI mode: db_path should be a URI (nfs://server/path/to/db) + // Use shared_fs_uri as the base mount point for NFS FileSystem + rocksdb::Env* shared_env = nullptr; + + if (!shared_fs_uri.empty() && shared_fs_uri.find("nfs://") == 0) { + // Use shared_fs_uri directly as the mount point + // This is the base path where NFS is mounted (e.g., + // nfs://server/shared/rocksdb) + std::cout << "[CSA] URI mode: using shared_fs_uri as mount point: " + << shared_fs_uri << std::endl; + shared_env = + SharedFileSystemCache::Instance().GetOrCreateEnvFromURI(shared_fs_uri); + } else if (db_path.find("nfs://") == 0) { + // db_path is URI but shared_fs_uri not set, extract base from db_path + // Extract mount point: nfs://server/path/to/db -> nfs://server/path + size_t proto_end = db_path.find("://"); + if (proto_end != std::string::npos) { + size_t path_start = db_path.find('/', proto_end + 3); + if (path_start != std::string::npos) { + // Find first path component as mount point + size_t first_slash = path_start; + size_t second_slash = db_path.find('/', first_slash + 1); + if (second_slash != std::string::npos) { + std::string base_uri = db_path.substr(0, second_slash); + std::cout << "[CSA] URI mode: extracted base_uri from db_path: " + << base_uri << std::endl; + shared_env = + SharedFileSystemCache::Instance().GetOrCreateEnvFromURI(base_uri); + } else { + // Only one path component, use as-is + shared_env = + SharedFileSystemCache::Instance().GetOrCreateEnvFromURI(db_path); + } + } else { + // No path, use as-is + shared_env = + SharedFileSystemCache::Instance().GetOrCreateEnvFromURI(db_path); + } + } + } + + if (shared_env) { + options_override.env = shared_env; + std::cout << "[CSA] Using shared storage FileSystem (read-only for " + "input, writable for output)" + << std::endl; + } else { + std::cerr << "[CSA] WARNING: Failed to create shared filesystem env." + << " db_path=" << db_path << ", shared_fs_uri=" << shared_fs_uri + << ". Falling back to default filesystem." << std::endl; + } + + // ===== Early stale task detection ===== + // Parse compaction input to get input files, then check if they exist + // This avoids expensive OpenAndCompact calls for stale tasks + ROCKSDB_NAMESPACE::CompactionServiceInput csi; + rocksdb::Status parse_status = + ROCKSDB_NAMESPACE::CompactionServiceInput::Read(compaction_input, &csi); + if (parse_status.ok() && !csi.input_files.empty() && shared_env) { + // Check if at least the first input file exists + // If not, this is likely a stale task - return early + const std::string& first_file = csi.input_files[0]; + std::string file_path = db_path + "/" + first_file; + rocksdb::Env* env_to_check = shared_env; + + uint64_t file_size = 0; + rocksdb::Status file_status = + env_to_check->GetFileSize(file_path, &file_size); + if (!file_status.ok()) { + std::cout << "[CSA] Early stale task detection: file " << file_path + << " does not exist. Returning stale task code immediately." + << std::endl; + local_task_nums_ -= 1; + compaction_reply->set_code(kCSACodeStaleTask); + compaction_reply->set_result("Stale task: input file not found"); + return ::grpc::Status::OK; + } + } + + options_override.file_checksum_gen_factory = + options_.file_checksum_gen_factory; + options_override.comparator = options_.comparator; + options_override.merge_operator = options_.merge_operator; + options_override.compaction_filter = options_.compaction_filter; + options_override.compaction_filter_factory = + options_.compaction_filter_factory; + options_override.prefix_extractor = options_.prefix_extractor; + options_override.table_factory = options_.table_factory; + options_override.sst_partitioner_factory = options_.sst_partitioner_factory; + options_override.statistics = ROCKSDB_NAMESPACE::CreateDBStatistics(); + + std::cout << "[CSA] Calling DB::OpenAndCompact with:" + << " db_path=" << db_path + << ", output_directory=" << output_directory << ", using_env=" + << (options_override.env ? "shared_env" : "default") << std::endl; + + rocksdb::Status s = + ROCKSDB_NAMESPACE::DB::OpenAndCompact(db_path, + output_directory, + compaction_input, + &compaction_service_result, + options_override); + + std::cout << "[CSA] OpenAndCompact result: " << s.ToString() + << ", result_size=" << compaction_service_result.size() + << std::endl; + + if (!s.ok()) { + std::cerr << "[CSA] Compaction failed with error: " << s.ToString() + << std::endl; + std::cerr << "[CSA] Debug info:" + << " db_path=" << db_path + << ", output_directory=" << output_directory + << ", shared_fs_uri=" << shared_fs_uri + << ", env=" << (options_override.env ? "set" : "null") + << std::endl; + } + + compaction_reply->set_code(s.code()); + + if (!s.ok()) { + std::cerr << "Compaction failed: " << s.ToString() << std::endl; + compaction_reply->set_result(compaction_service_result); + } else { + compaction_reply->set_result(std::move(compaction_service_result)); + std::cout << "Compaction completed successfully" << std::endl; + } + + local_task_nums_ -= 1; + std::cout << "Compaction finished with code: " << static_cast(s.code()) + << std::endl; + return ::grpc::Status::OK; + } +}; + +// Parse command line arguments +void ParseCommandLine(int argc, char* argv[]) { + for (int i = 1; i < argc; i++) { + std::string arg = argv[i]; + if (arg == "--csa_address" || arg == "-a") { + if (i + 1 < argc) { + compaction_service_options.csa_address = argv[++i]; + } else { + std::cerr << "[CSA] Error: --csa_address requires a value" << std::endl; + exit(1); + } + } else if (arg == "--max_concurrent_tasks" || arg == "-t") { + if (i + 1 < argc) { + compaction_service_options.csa_max_concurrent_tasks = + std::stoll(argv[++i]); + } else { + std::cerr << "[CSA] Error: --max_concurrent_tasks requires a value" + << std::endl; + exit(1); + } + } else if (arg == "--grpc_max_message_size" || arg == "-m") { + if (i + 1 < argc) { + compaction_service_options.grpc_max_message_size = + std::stoll(argv[++i]); + } else { + std::cerr << "[CSA] Error: --grpc_max_message_size requires a value" + << std::endl; + exit(1); + } + } else if (arg == "--help" || arg == "-h") { + std::cout << "CSA Server - Compaction Service Agent\n" + << "Usage: " << argv[0] << " [OPTIONS]\n\n" + << "Options:\n" + << " --csa_address, -a ADDRESS CSA server address " + "(REQUIRED, format: host:port)\n" + << " --max_concurrent_tasks, -t N Maximum concurrent " + "compaction tasks (default: 5)\n" + << " --grpc_max_message_size, -m N Max gRPC message size in " + "bytes (default: 16777216)\n" + << " --help, -h Show this help message\n\n" + << "Examples:\n" + << " " << argv[0] << " --csa_address localhost:8010\n" + << " " << argv[0] << " -a 0.0.0.0:8010 -t 10 -m 33554432\n" + << std::endl; + exit(0); + } else { + std::cerr << "[CSA] Error: Unknown option: " << arg << std::endl; + std::cerr << "Use --help for usage information" << std::endl; + exit(1); + } + } + + // Also check environment variables as fallback + const char* env_csa_address = std::getenv("CSA_ADDRESS"); + if (compaction_service_options.csa_address.empty() && env_csa_address) { + compaction_service_options.csa_address = env_csa_address; + } + + const char* env_max_tasks = std::getenv("CSA_MAX_CONCURRENT_TASKS"); + if (compaction_service_options.csa_max_concurrent_tasks == 0 && + env_max_tasks) { + compaction_service_options.csa_max_concurrent_tasks = + std::stoll(env_max_tasks); + } + + const char* env_max_msg = std::getenv("CSA_GRPC_MAX_MESSAGE_SIZE"); + if (compaction_service_options.grpc_max_message_size == 0 && env_max_msg) { + compaction_service_options.grpc_max_message_size = std::stoll(env_max_msg); + } +} + +int main(int argc, char* argv[]) { + // Parse command line arguments + ParseCommandLine(argc, argv); + + std::string server_address(compaction_service_options.csa_address); + + // Validate server address + if (server_address.empty()) { + std::cerr << "[CSA] Error: CSA address not configured." << std::endl; + std::cerr << "[CSA] Please provide --csa_address option or set CSA_ADDRESS " + "environment variable." + << std::endl; + std::cerr << "[CSA] Example: " << argv[0] << " --csa_address localhost:8010" + << std::endl; + return 1; + } + + CSAImpl service; + + // Get max gRPC message size from configuration (with default) + int64_t max_msg_size = compaction_service_options.GetGrpcMaxMessageSize(); + + grpc::ServerBuilder builder; + builder.AddListeningPort(server_address, grpc::InsecureServerCredentials()); + builder.RegisterService(&service); + + // Set max message size for receiving and sending + builder.SetMaxReceiveMessageSize(static_cast(max_msg_size)); + builder.SetMaxSendMessageSize(static_cast(max_msg_size)); + + std::unique_ptr server(builder.BuildAndStart()); + if (!server) { + std::cerr << "[CSA] Error: Failed to start server on " << server_address + << std::endl; + return 1; + } + + std::cout << "[CSA] Server listening on " << server_address + << " (max message size: " << max_msg_size / 1024 / 1024 << "MB" + << ", max concurrent tasks: " << GetMaxConcurrentTasks() << ")" + << std::endl; + server->Wait(); + return 0; +} diff --git a/src/tendisplus/storage/rocks/remote_compaction/def.h b/src/tendisplus/storage/rocks/remote_compaction/def.h new file mode 100644 index 00000000..cd0fa229 --- /dev/null +++ b/src/tendisplus/storage/rocks/remote_compaction/def.h @@ -0,0 +1,72 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#pragma once +#include +#include + +#include "rocksdb/db.h" +#include "rocksdb/options.h" + +#include "tendisplus/storage/rocks/remote_compaction/remote_compaction_config.h" + +namespace ROCKSDB_NAMESPACE { + +// Remote compaction mode (only shared storage is supported) +enum class RemoteCompactionMode { + kSharedStorage = + 0 // Use shared storage (NFS/HDFS/S3) - both sides access same files +}; + +struct RemoteOpenAndCompactOptions : public OpenAndCompactOptions { + // Allows cancellation of an in-progress compaction. + std::atomic* canceled = nullptr; + + // CSA server address (empty means disabled) + // Should be set from configuration, e.g., "localhost:8010" or + // "192.168.1.100:8010" + std::string csa_address; + + // Shared FileSystem configuration (supports NFS, HDFS, S3, etc.) + // URI format: "nfs://host/path", "hdfs://host:port/path", + // "s3://bucket/prefix" Should be set from configuration + std::string shared_fs_uri; + std::string shared_fs_local_prefix; + + // Remote compaction mode (only shared storage is supported) + // kSharedStorage: Use shared storage (NFS/HDFS/S3) - both sides access same + // files Should be set from configuration + RemoteCompactionMode mode = RemoteCompactionMode::kSharedStorage; + + // Advanced settings (should be set from configuration) + int64_t csa_max_concurrent_tasks = + 0; // Max concurrent tasks (0 = use default) + int64_t grpc_max_message_size = 0; // Max gRPC message size (0 = use default) + int32_t check_time_interval = 0; // Check time interval (0 = use default) + uint64_t max_reschedule = 0; // Max reschedule times (0 = use default) + + // Helper methods to get values with defaults + int64_t GetMaxConcurrentTasks() const { + return RemoteCompactionConfig::GetOrDefault( + csa_max_concurrent_tasks, + RemoteCompactionConfig::kDefaultMaxConcurrentTasks); + } + + int64_t GetGrpcMaxMessageSize() const { + return RemoteCompactionConfig::GetOrDefault( + grpc_max_message_size, + RemoteCompactionConfig::kDefaultGrpcMaxMessageSize); + } + + int32_t GetCheckTimeInterval() const { + return RemoteCompactionConfig::GetOrDefault( + check_time_interval, RemoteCompactionConfig::kDefaultCheckTimeInterval); + } + + uint64_t GetMaxReschedule() const { + return RemoteCompactionConfig::GetOrDefault( + max_reschedule, RemoteCompactionConfig::kDefaultMaxReschedule); + } +}; +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/remote_compaction/protos/csa.proto b/src/tendisplus/storage/rocks/remote_compaction/protos/csa.proto new file mode 100644 index 00000000..97fc6a37 --- /dev/null +++ b/src/tendisplus/storage/rocks/remote_compaction/protos/csa.proto @@ -0,0 +1,73 @@ +// Copyright 2015 gRPC authors. +// +// 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 +// +// http://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. + +syntax = "proto3"; + +option go_package = "google.golang.org/grpc/examples/csa/csa"; +option java_multiple_files = true; +option java_package = "io.grpc.examples.csa"; +option java_outer_classname = "CSAProto"; + +import "google/protobuf/empty.proto"; +package csa; + +message CompactionAdditionInfo { + double score = 1; + uint64 num_entries = 2; + uint64 num_deletions = 3; + uint64 compensated_file_size = 4; + int32 output_level = 5; + int32 start_level = 6; +} + +message CompactionReply { + int32 code = 1; + string result = 2; +} + +message CompactionArgs { + string name = 1; + string output_directory = 2; + string input = 3; + // Shared filesystem configuration (supports NFS, HDFS, S3, etc.) + string shared_fs_uri = 4; // e.g., "nfs://host/path", "hdfs://host:port/path", "s3://bucket/prefix" + string shared_fs_local_prefix = 5; // local path prefix that maps to shared_fs_uri +} + +message CompactionTaskArgs { + uint64 task_id = 1; + CompactionArgs compaction_args = 2; +} + +message SubmitTaskArgs { + uint64 task_id = 1; + CompactionReply compaction_reply = 2; +} + +message CSAStatus { + string ip = 1; + uint32 port = 2; + uint64 local_task_nums = 3; + uint64 max_task_nums = 4; + double memory_usage = 5; +} + +message CompactionJobInfo { + CompactionArgs compaction_args = 1; + CompactionAdditionInfo compaction_addition_info = 2; +}; + +service CSAService { + rpc ExecuteCompactionTask(CompactionArgs) returns (CompactionReply) {} +} diff --git a/src/tendisplus/storage/rocks/remote_compaction/remote_compaction_config.h b/src/tendisplus/storage/rocks/remote_compaction/remote_compaction_config.h new file mode 100644 index 00000000..b3588858 --- /dev/null +++ b/src/tendisplus/storage/rocks/remote_compaction/remote_compaction_config.h @@ -0,0 +1,57 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#pragma once + +#include +#include +#include + +namespace ROCKSDB_NAMESPACE { + +// Remote compaction configuration helper +// Provides default values and configuration management +class RemoteCompactionConfig { + public: + // Default values (can be overridden by configuration) + static constexpr const char* kDefaultMode = "shared_storage"; + static constexpr int64_t kDefaultMaxConcurrentTasks = 5; + static constexpr int64_t kDefaultGrpcMaxMessageSize = + 16 * 1024 * 1024; // 16MB + static constexpr int32_t kDefaultCheckTimeInterval = 1; + static constexpr uint64_t kDefaultMaxReschedule = 5; + + // Mode strings + static constexpr const char* kModeSharedStorage = "shared_storage"; + + // Get default CSA address (empty means disabled by default) + static std::string GetDefaultCsaAddress() { + return ""; + } + + // Helper to get value with default + template + static T GetOrDefault(const T& config_value, const T& default_value) { + return config_value != T{} ? config_value : default_value; + } + + // Helper to get string value with default + static std::string GetStringOrDefault(const std::string& config_value, + const std::string& default_value) { + return config_value.empty() ? default_value : config_value; + } + + // Validate mode string + static bool IsValidMode(const std::string& mode) { + return mode == kModeSharedStorage || mode.empty(); // empty means disabled + } + + // Check if remote compaction is enabled + static bool IsEnabled(const std::string& csa_address, + const std::string& mode) { + return !csa_address.empty() && !mode.empty(); + } +}; + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/rocks_kvstore.cpp b/src/tendisplus/storage/rocks/rocks_kvstore.cpp index 4020d591..e0779b32 100644 --- a/src/tendisplus/storage/rocks/rocks_kvstore.cpp +++ b/src/tendisplus/storage/rocks/rocks_kvstore.cpp @@ -37,8 +37,12 @@ #include "rocksdb/utilities/table_properties_collectors.h" #include "tendisplus/server/server_entry.h" +#include "tendisplus/server/server_params.h" #include "tendisplus/server/session.h" +#include "tendisplus/storage/rocks/compaction_service.h" +#include "tendisplus/storage/rocks/nfs_filesystem.h" #include "tendisplus/storage/rocks/rocks_kvttlcompactfilter.h" +#include "tendisplus/storage/rocks/shared_filesystem.h" #include "tendisplus/storage/varint.h" #include "tendisplus/utils/invariant.h" #include "tendisplus/utils/scopeguard.h" @@ -2233,10 +2237,203 @@ Expected RocksKVStore::restart(bool restore, _cfDescs.push_back( rocksdb::ColumnFamilyDescriptor("binlog_cf", binlogColumnFamilyOpts)); } + + bool use_shared_fs = false; + const std::string& db_path = dbPath(); + + // Build RemoteOpenAndCompactOptions from configuration + rocksdb::RemoteOpenAndCompactOptions remote_options; + + // Read CSA address (required for remote compaction) + if (gParams && !gParams->csaAddress.empty()) { + remote_options.csa_address = gParams->csaAddress; + // Basic validation: check if address contains ':' + if (remote_options.csa_address.find(':') == std::string::npos) { + LOG(WARNING) << "Invalid CSA address format: " + << remote_options.csa_address + << " (expected format: host:port), remote compaction will " + "be disabled"; + remote_options.csa_address.clear(); + } else { + LOG(INFO) << "CSA address configured: " << remote_options.csa_address; + } + } + + // Remote compaction only supports shared_storage mode (default) + // No need to configure mode - it's always shared_storage + + // Read shared filesystem configuration (unified for NFS/HDFS/S3/etc.) + if (gParams && !gParams->remoteCompactionSharedFsUri.empty()) { + remote_options.shared_fs_uri = gParams->remoteCompactionSharedFsUri; + remote_options.shared_fs_local_prefix = + gParams->remoteCompactionSharedFsLocalPrefix; + + LOG(INFO) << "Shared filesystem configuration:" + << " uri=" << remote_options.shared_fs_uri << ", local_prefix=" + << (remote_options.shared_fs_local_prefix.empty() + ? "(auto-inferred)" + : remote_options.shared_fs_local_prefix) + << ", csa_address=" << remote_options.csa_address; + } + + // Read advanced settings + if (gParams) { + if (gParams->remoteCompactionMaxConcurrentTasks > 0) { + remote_options.csa_max_concurrent_tasks = + gParams->remoteCompactionMaxConcurrentTasks; + LOG(INFO) << "Remote compaction max concurrent tasks: " + << remote_options.csa_max_concurrent_tasks; + } + if (gParams->remoteCompactionGrpcMaxMessageSize > 0) { + remote_options.grpc_max_message_size = + gParams->remoteCompactionGrpcMaxMessageSize; + LOG(INFO) << "Remote compaction gRPC max message size: " + << remote_options.grpc_max_message_size; + } + if (gParams->remoteCompactionCheckTimeInterval > 0) { + remote_options.check_time_interval = + gParams->remoteCompactionCheckTimeInterval; + } + if (gParams->remoteCompactionMaxReschedule > 0) { + remote_options.max_reschedule = gParams->remoteCompactionMaxReschedule; + } + } + + // Enable shared filesystem (NFS/HDFS/etc.) if configured + if (!remote_options.shared_fs_uri.empty()) { + rocksdb::Status fs_status; + const std::string& shared_fs_uri = remote_options.shared_fs_uri; + const std::string& local_prefix = remote_options.shared_fs_local_prefix; + + // Try URI mode first (preferred for remote deployment and other + // filesystems) + if (shared_fs_uri.find("nfs://") == 0) { + // NFS: Try URI mode first, fallback to path matching if local_prefix + // provided + if (local_prefix.empty()) { + // Pure URI mode (recommended for remote deployment) + std::unique_ptr fs; + fs_status = rocksdb::NFSFileSystem::Create( + rocksdb::FileSystem::Default(), shared_fs_uri, &fs); + if (fs_status.ok() && fs) { + _sharedFileSystem = + std::shared_ptr(fs.release()); + LOG(INFO) << "Shared FileSystem (NFS) created in URI mode (no " + "local_prefix)"; + } + } else { + // Path matching mode (backward compatibility) + fs_status = rocksdb::NewNFSFileSystem( + shared_fs_uri, local_prefix, &_sharedFileSystem); + if (fs_status.ok() && _sharedFileSystem) { + LOG(INFO) + << "Shared FileSystem (NFS) created in path matching mode"; + } + } + } else { + // Other filesystems (HDFS, S3, etc.) - use unified interface + fs_status = rocksdb::CreateSharedFileSystem( + rocksdb::FileSystem::Default(), shared_fs_uri, &_sharedFileSystem); + if (fs_status.ok() && _sharedFileSystem) { + LOG(INFO) << "Shared filesystem created: " << shared_fs_uri; + } + } + + if (!fs_status.ok() || !_sharedFileSystem) { + LOG(ERROR) << "Failed to create shared filesystem: " + << (fs_status.ok() ? "unknown error" : fs_status.ToString()) + << ", falling back to default filesystem"; + if (!remote_options.csa_address.empty()) { + LOG(WARNING) + << "Shared filesystem initialization failed but CSA address is " + "configured. " + << "Remote compaction will fallback to local compaction."; + } + } else { + _sharedEnv = rocksdb::NewCompositeEnv(_sharedFileSystem); + if (_sharedEnv) { + use_shared_fs = true; + LOG(INFO) << "Shared filesystem enabled successfully" + << ", uri: " << shared_fs_uri + << ", csa_address: " << remote_options.csa_address; + } + } + } else { + if (!remote_options.csa_address.empty()) { + LOG(WARNING) << "Shared filesystem is not configured but CSA address " + "is configured. " + << "Remote compaction requires shared filesystem to work. " + << "Please configure remote_compaction.shared_fs_uri. " + << "Remote compaction will not work properly."; + } + LOG(INFO) << "Shared filesystem not configured, using local filesystem" + << ", dbPath: " << db_path; + } + + // IMPORTANT: Consistency guarantee + // If shared filesystem is enabled, ALL compaction (both remote and local + // fallback) MUST use it This ensures data consistency - all compaction + // results are stored in the same location + if (use_shared_fs && _sharedEnv) { + LOG(INFO) << "Shared filesystem enabled: All compaction operations " + "(remote and local fallback) " + << "will use shared storage to ensure data consistency"; + } + if (_txnMode == TxnMode::TXN_OPT) { rocksdb::OptimisticTransactionDB* tmpDb = nullptr; rocksdb::Options dbOpts = options(); dbOpts.create_missing_column_families = true; + + // Register additional paths to shared filesystem (wal_dir, db_log_dir, + // etc.) This must be done after options() is called but before using + // dbOpts + if (use_shared_fs && _sharedFileSystem) { + // Register wal_dir if configured + if (!dbOpts.wal_dir.empty()) { + rocksdb::RegisterSharedFileSystemPathPrefix(_sharedFileSystem, + dbOpts.wal_dir); + LOG(INFO) << "Registered WAL directory to shared filesystem: " + << dbOpts.wal_dir; + } + // Register db_log_dir if configured + if (!dbOpts.db_log_dir.empty()) { + rocksdb::RegisterSharedFileSystemPathPrefix(_sharedFileSystem, + dbOpts.db_log_dir); + LOG(INFO) << "Registered db_log_dir to shared filesystem: " + << dbOpts.db_log_dir; + } + // Register db_paths if configured + for (const auto& db_path : dbOpts.db_paths) { + if (!db_path.path.empty()) { + rocksdb::RegisterSharedFileSystemPathPrefix(_sharedFileSystem, + db_path.path); + LOG(INFO) << "Registered db_path to shared filesystem: " + << db_path.path; + } + } + } + + // Apply shared filesystem environment + if (use_shared_fs && _sharedEnv) { + dbOpts.env = _sharedEnv.get(); + LOG(INFO) << "TXN_OPT: Using shared FileSystem for db: " << dbname; + } + + std::vector> + remote_table_properties_collector_factories; + auto& tmp_options = const_cast(dbOpts); + auto& remote_listeners = + const_cast>&>( + dbOpts.listeners); + auto compaction_svc = std::make_shared( + dbname, + tmp_options, + _stats, + remote_listeners, + remote_table_properties_collector_factories, + remote_options); + tmp_options.compaction_service = compaction_svc; auto status = rocksdb::OptimisticTransactionDB::Open( dbOpts, dbname, @@ -2269,6 +2466,60 @@ Expected RocksKVStore::restart(bool restore, rocksdb::Options dbOpts = options(); dbOpts.create_missing_column_families = true; LOG(INFO) << "rocksdb Open,id:" << dbId() << " dbname:" << dbname; + + // Register additional paths to shared filesystem (wal_dir, db_log_dir, + // etc.) This must be done after options() is called but before using + // dbOpts + if (use_shared_fs && _sharedFileSystem) { + // Register wal_dir if configured + if (!dbOpts.wal_dir.empty()) { + rocksdb::RegisterSharedFileSystemPathPrefix(_sharedFileSystem, + dbOpts.wal_dir); + LOG(INFO) << "Registered WAL directory to shared filesystem: " + << dbOpts.wal_dir; + } + // Register db_log_dir if configured + if (!dbOpts.db_log_dir.empty()) { + rocksdb::RegisterSharedFileSystemPathPrefix(_sharedFileSystem, + dbOpts.db_log_dir); + LOG(INFO) << "Registered db_log_dir to shared filesystem: " + << dbOpts.db_log_dir; + } + // Register db_paths if configured + for (const auto& db_path : dbOpts.db_paths) { + if (!db_path.path.empty()) { + rocksdb::RegisterSharedFileSystemPathPrefix(_sharedFileSystem, + db_path.path); + LOG(INFO) << "Registered db_path to shared filesystem: " + << db_path.path; + } + } + } + + // Apply shared filesystem environment + if (use_shared_fs && _sharedEnv) { + dbOpts.env = _sharedEnv.get(); + LOG(INFO) << "TXN_PES: Using shared FileSystem for db: " << dbname; + } + + std::vector> + remote_table_properties_collector_factories; + auto& tmp_options = const_cast(dbOpts); + auto& compaction_stats = + const_cast&>(dbOpts.statistics); + auto& remote_listeners = + const_cast>&>( + dbOpts.listeners); + + auto compaction_svc = std::make_shared( + dbname, + tmp_options, + compaction_stats, + remote_listeners, + remote_table_properties_collector_factories, + remote_options); + tmp_options.compaction_service = compaction_svc; + // open two colum_family in pessimisticTranDB auto status = rocksdb::TransactionDB::Open( dbOpts, txnDbOptions, dbname, _cfDescs, &_cfHandles, &tmpDb); diff --git a/src/tendisplus/storage/rocks/rocks_kvstore.h b/src/tendisplus/storage/rocks/rocks_kvstore.h index 0b1193cf..91c7ed37 100644 --- a/src/tendisplus/storage/rocks/rocks_kvstore.h +++ b/src/tendisplus/storage/rocks/rocks_kvstore.h @@ -17,6 +17,8 @@ #include #include "rocksdb/db.h" +#include "rocksdb/env.h" +#include "rocksdb/file_system.h" #include "rocksdb/sst_file_manager.h" #include "rocksdb/utilities/optimistic_transaction_db.h" #include "rocksdb/utilities/transaction.h" @@ -546,6 +548,9 @@ class RocksKVStore : public KVStore { std::shared_ptr _logOb; std::shared_ptr _env; + // Shared FileSystem support (NFS/HDFS/S3/etc.) + std::shared_ptr _sharedFileSystem; + std::unique_ptr _sharedEnv; std::map _rocksIntProperties; std::map _rocksStringProperties; std::vector _cfHandles; diff --git a/src/tendisplus/storage/rocks/shared_filesystem.cc b/src/tendisplus/storage/rocks/shared_filesystem.cc new file mode 100644 index 00000000..c08440f3 --- /dev/null +++ b/src/tendisplus/storage/rocks/shared_filesystem.cc @@ -0,0 +1,233 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#include "tendisplus/storage/rocks/shared_filesystem.h" + +#include "tendisplus/storage/rocks/nfs_filesystem.h" +#ifdef HDFS +#include "tendisplus/storage/rocks/plugin/hdfs/env_hdfs.h" +#endif + +#include + +#include +#include +#include +#include +#include + +namespace ROCKSDB_NAMESPACE { + +Status CreateSharedFileSystem(const std::shared_ptr& base, + const std::string& uri, + std::shared_ptr* result) { + result->reset(); + + if (uri.empty()) { + return Status::InvalidArgument("URI cannot be empty"); + } + + // Determine filesystem type from URI scheme + if (uri.find("nfs://") == 0) { + // NFS filesystem + std::unique_ptr nfs_fs; + Status s = NFSFileSystem::Create(base, uri, &nfs_fs); + if (!s.ok()) { + return s; + } + result->reset(nfs_fs.release()); + return Status::OK(); + } +#ifdef HDFS + if (uri.find("hdfs://") == 0) { + // HDFS filesystem + std::unique_ptr hdfs_fs; + Status s = HdfsFileSystem::Create(base, uri, &hdfs_fs); + if (!s.ok()) { + return s; + } + result->reset(hdfs_fs.release()); + return Status::OK(); + } +#endif + // Future: Add support for S3, GCS, etc. + return Status::NotSupported("Unsupported filesystem URI scheme: " + uri); +} + +Status CreateSharedFileSystemEnv(const std::string& uri, + std::unique_ptr* result) { + result->reset(); + + std::shared_ptr fs; + Status s = CreateSharedFileSystem(FileSystem::Default(), uri, &fs); + if (!s.ok()) { + return s; + } + + *result = NewCompositeEnv(fs); + return Status::OK(); +} + +bool RegisterSharedFileSystemPathPrefix(const std::shared_ptr& fs, + const std::string& prefix) { + if (!fs || prefix.empty()) { + return true; // Nothing to register + } + + // Try to cast to NFSFileSystem (path-based filesystem) + auto* nfs_fs = dynamic_cast(fs.get()); + if (nfs_fs) { + nfs_fs->RegisterPathPrefix(prefix); + return true; + } + + // For other filesystems (HDFS, S3, etc.), path registration is typically not + // needed because they use URI-based paths directly. Return true to indicate + // success. + // + // If future filesystems need path registration, add similar dynamic_cast + // checks here. + + return true; // Registration not needed for this filesystem type +} + +std::string ConvertLocalPathToSharedURI(const std::string& local_path, + const std::string& shared_fs_uri, + const std::string& mount_point) { + if (shared_fs_uri.empty() || local_path.empty()) { + return local_path; // Return as-is if invalid + } + + // If local_path is already a URI, return as-is + if (IsSharedFilesystemURI(local_path)) { + return local_path; + } + + // Convert local_path to absolute path + std::string abs_local_path = local_path; + if (!local_path.empty() && local_path[0] != '/') { + char cwd[PATH_MAX]; + if (getcwd(cwd, sizeof(cwd)) != nullptr) { + abs_local_path = std::string(cwd) + "/" + local_path; + // Normalize + size_t pos; + while ((pos = abs_local_path.find("/./")) != std::string::npos) { + abs_local_path.erase(pos, 2); + } + } + } + + // Determine mount point + std::string actual_mount_point = mount_point; + if (actual_mount_point.empty()) { + // Auto-infer: find the common base directory + // For db_path like "./home/db/0", we want to find the working directory as + // mount point Strategy: go up 3 levels from db_path to get the mount point + // (workdir) Example: "/path/to/home/db/0" -> "/path/to" (go up 3 levels: + // remove "0", "db", and "home") This ensures relative_path includes + // "home/db/0" which matches NFS structure + std::string path = abs_local_path; + // Remove last component (e.g., "0" from "home/db/0") + size_t last_slash = path.rfind('/'); + if (last_slash != std::string::npos && last_slash > 0) { + path = path.substr(0, last_slash); + // Remove second-to-last component (e.g., "db" from "home/db") + last_slash = path.rfind('/'); + if (last_slash != std::string::npos && last_slash > 0) { + path = path.substr(0, last_slash); + // Remove third-to-last component (e.g., "home" from "home") + last_slash = path.rfind('/'); + if (last_slash != std::string::npos && last_slash > 0) { + actual_mount_point = path.substr(0, last_slash); + } else { + // Fallback: use parent of "home" (workdir) + actual_mount_point = path; + } + } else { + // Fallback: use parent of local_path + actual_mount_point = path; + } + } else { + // Cannot infer, return as-is + return local_path; + } + } else { + // Convert mount_point to absolute path + if (mount_point[0] != '/') { + char cwd[PATH_MAX]; + if (getcwd(cwd, sizeof(cwd)) != nullptr) { + actual_mount_point = std::string(cwd) + "/" + mount_point; + // Normalize + size_t pos; + while ((pos = actual_mount_point.find("/./")) != std::string::npos) { + actual_mount_point.erase(pos, 2); + } + if (actual_mount_point.find("./") == 0) { + actual_mount_point = actual_mount_point.substr(2); + } + } + } + } + + // Check if local_path starts with mount_point + if (abs_local_path.find(actual_mount_point) != 0) { + // Doesn't match, return as-is + return local_path; + } + + // Extract relative path + std::string relative_path = + abs_local_path.substr(actual_mount_point.length()); + while (!relative_path.empty() && relative_path[0] == '/') { + relative_path = relative_path.substr(1); + } + + // Debug output + std::cerr << "[ConvertLocalPathToSharedURI] Path conversion:" << std::endl; + std::cerr << " local_path: " << local_path << std::endl; + std::cerr << " abs_local_path: " << abs_local_path << std::endl; + std::cerr << " mount_point (config): " << mount_point << std::endl; + std::cerr << " actual_mount_point (inferred): " << actual_mount_point + << std::endl; + std::cerr << " relative_path: " << relative_path << std::endl; + std::cerr.flush(); + + // Build URI + size_t proto_end = shared_fs_uri.find("://"); + if (proto_end == std::string::npos) { + std::cerr << "[ConvertLocalPathToSharedURI] ERROR: Invalid URI format: " + << shared_fs_uri << std::endl; + std::cerr.flush(); + return local_path; // Invalid URI + } + + size_t path_start = shared_fs_uri.find('/', proto_end + 3); + if (path_start == std::string::npos) { + // No path in URI, just append + std::string result = shared_fs_uri + "/" + relative_path; + std::cerr << "[ConvertLocalPathToSharedURI] Result (no base path): " + << result << std::endl; + std::cerr.flush(); + return result; + } + + std::string server_part = shared_fs_uri.substr(0, path_start); + std::string base_path = shared_fs_uri.substr(path_start); + if (base_path.back() != '/') { + base_path += "/"; + } + + std::string result = server_part + base_path + relative_path; + std::cerr << "[ConvertLocalPathToSharedURI] Result: " << result << std::endl; + std::cerr.flush(); + return result; +} + +bool IsSharedFilesystemURI(const std::string& path) { + return path.find("nfs://") == 0 || path.find("hdfs://") == 0 || + path.find("s3://") == 0 || path.find("gcs://") == 0 || + path.find("file://") == 0; +} + +} // namespace ROCKSDB_NAMESPACE diff --git a/src/tendisplus/storage/rocks/shared_filesystem.h b/src/tendisplus/storage/rocks/shared_filesystem.h new file mode 100644 index 00000000..a76477b3 --- /dev/null +++ b/src/tendisplus/storage/rocks/shared_filesystem.h @@ -0,0 +1,46 @@ +// Copyright (C) 2020 THL A29 Limited, a Tencent company. All rights reserved. +// Please refer to the license text that comes with this tendis open source +// project for additional information. + +#pragma once + +#include +#include + +#include "rocksdb/env.h" +#include "rocksdb/file_system.h" +#include "rocksdb/status.h" +namespace ROCKSDB_NAMESPACE { + +// Unified interface for creating shared file systems +// Supports: nfs://, hdfs://, s3://, etc. +Status CreateSharedFileSystem(const std::shared_ptr& base, + const std::string& uri, + std::shared_ptr* result); + +// Create Env from shared file system URI +Status CreateSharedFileSystemEnv(const std::string& uri, + std::unique_ptr* result); + +// Helper: Convert local path to shared filesystem URI +// This is used when RocksDB uses local paths but we want to use shared storage +std::string ConvertLocalPathToSharedURI(const std::string& local_path, + const std::string& shared_fs_uri, + const std::string& mount_point = ""); + +// Helper: Check if a path is a shared filesystem URI +bool IsSharedFilesystemURI(const std::string& path); + +// Unified interface for registering additional path prefixes to shared +// filesystem This is useful for path-based filesystems (like NFS) that need to +// know which local paths should be mapped to the shared storage. For URI-based +// filesystems (like HDFS, S3), this is typically a no-op. +// +// @param fs The shared filesystem instance +// @param prefix The local path prefix to register (e.g., wal_dir, db_log_dir) +// @return true if successfully registered (or if registration not needed), +// false on error +bool RegisterSharedFileSystemPathPrefix(const std::shared_ptr& fs, + const std::string& prefix); + +} // namespace ROCKSDB_NAMESPACE diff --git a/tendisplus_rc.conf b/tendisplus_rc.conf new file mode 100644 index 00000000..376568b2 --- /dev/null +++ b/tendisplus_rc.conf @@ -0,0 +1,194 @@ +# Tendisplus Configuration for Remote Compaction +# ================================================ +# This configuration enables remote compaction using shared storage (NFS/HDFS) +# Both Tendisplus and CSA access the same files via shared filesystem + +# Basic server settings +port 51002 +daemon off +loglevel notice +logdir ./home/log +dumpdir ./home/dump +dir ./home/db +pidfile ./home/tendisplus.pid +slowlog ./home/log/slowlog + +# Performance settings +rocks.blockcachemb 4096 +executorThreadNum 48 +kvStoreCount 1 +rocks.max_background_compactions 1 +rocks.max_total_wal_size 1024MB +rocks.wal_ttl_seconds 3600 +rocks.keep_log_file_num 10 +# ================================================ +# Remote Compaction Service (CSA) Configuration +# ================================================ +# To enable remote compaction: +# 1. Set csa_address to your CSA server address +# 2. Set remote_compaction.shared_fs_uri to your shared filesystem URI + +# CSA server address (REQUIRED to enable remote compaction) +# Format: host:port +# Examples: +# localhost:8010 - Local CSA server +# 192.168.1.100:8010 - Remote CSA server +# csa-server.internal:8010 - Using DNS name +# Leave empty to disable remote compaction +csa_address localhost:8010 + +# Shared filesystem URI (REQUIRED for remote compaction) +# Format: scheme://host/path +# Supported schemes: nfs://, hdfs://, s3://, gcs:// +# Examples: +# nfs://localhost/shared/rocksdb - Local NFS for testing +# nfs://192.168.1.100/data/rocksdb - Remote NFS server +# hdfs://namenode:9000/data/rocksdb - HDFS filesystem +# s3://bucket-name/prefix - S3 filesystem +# Leave empty to disable shared filesystem +remote_compaction.shared_fs_uri nfs://localhost/shared/rocksdb + +# Local path prefix (OPTIONAL - can be auto-inferred) +# This is the local mount point where shared filesystem is mounted +# All paths starting with this prefix will be automatically converted to shared filesystem URIs +# +# If NOT set (recommended for simple setups): +# - System will automatically infer from "dir" path +# - Example: dir=./home/db -> inferred prefix=./home +# - This simplifies configuration for most use cases +# +# If SET (for complex setups or explicit control): +# - Use this exact prefix for path matching +# - Example: remote_compaction.shared_fs_local_prefix=/mnt/rocksdb, dir=/mnt/rocksdb/db +# - All paths under /mnt/rocksdb will use shared filesystem +# +# How it works: +# - RocksDB uses local paths (e.g., ./home/db) +# - SharedFileSystem automatically converts matching paths to URIs +# - Example: ./home/db -> nfs://localhost/shared/rocksdb/db (internal) +# - No changes needed to RocksDB configuration +# +# Leave empty to enable auto-inference (recommended) +# remote_compaction.shared_fs_local_prefix ./home + +# ================================================ +# Remote Compaction Advanced Settings +# ================================================ +# These settings have reasonable defaults and usually don't need to be changed +# Set to 0 to use default values + +# Maximum concurrent compaction tasks on CSA server +# Default: 5 +# Higher values allow more parallel compactions but use more resources +# Set to 0 to use default (5) +remote_compaction.max_concurrent_tasks 5 + +# Maximum gRPC message size in bytes +# Default: 16777216 (16MB) +# Increase if you have very large compaction results +# Set to 0 to use default (16MB) +# Examples: +# 16777216 - 16MB (default) +# 33554432 - 32MB +# 67108864 - 64MB +remote_compaction.grpc_max_message_size 16777216 + +# Check time interval in seconds +# Default: 1 +# Used for internal scheduling checks +# Set to 0 to use default (1) +remote_compaction.check_time_interval 1 + +# Maximum reschedule times +# Default: 5 +# Maximum number of times a compaction task can be rescheduled +# Set to 0 to use default (5) +remote_compaction.max_reschedule 5 + +# ================================================ +# HDFS Configuration (Optional, only if using HDFS) +# ================================================ +# These settings are only used when HDFS is enabled +# Leave empty if not using HDFS + +# HDFS address +# Format: hdfs://host:port/path +# Example: hdfs://namenode:9000/ +# remote_compaction.hdfs_address + +# Pro compaction address +# Format: host:port +# Example: 192.168.1.100:8020 +# remote_compaction.pro_cp_address + +# Maximum accumulation in pro compaction +# Default: 5 +# Set to 0 to use default (5) +# remote_compaction.max_accumulation_in_procp 5 + +# ================================================ +# RocksDB Compaction Settings +# ================================================ +# Maximum number of threads for parallel subcompactions +# Default: 2 +# Uncomment to customize +# rocks.max_subcompactions 2 + +# Direct I/O settings (uncomment to enable for better performance) +# rocks.use_direct_io_for_flush_and_compaction 1 +# rocks.use_direct_reads 1 + +# ================================================ +# Quick Start Guide +# ================================================ +# To enable remote compaction: +# +# 1. Setup shared filesystem (NFS/HDFS/S3) +# - Ensure shared filesystem is accessible +# - For NFS: Mount NFS to a local directory (e.g., ./home) +# - For HDFS: Ensure HDFS cluster is accessible +# +# 2. Start CSA server +# - Build and start the CSA server +# - Note the CSA server address (host:port) +# +# 3. Configure Tendisplus (Unified Configuration!) +# - Set csa_address to your CSA server address +# - Set remote_compaction.mode=shared_storage +# - Set remote_compaction.shared_fs_uri to your shared filesystem URI +# - remote_compaction.shared_fs_local_prefix is OPTIONAL (auto-inferred from "dir" if not set) +# +# 4. Start Tendisplus +# - Check logs to verify shared filesystem and CSA connections +# - Remote compaction will automatically be used when available +# +# Configuration Examples: +# +# Simple setup with NFS (auto-inference - RECOMMENDED): +# dir ./home/db +# csa_address localhost:8010 +# remote_compaction.shared_fs_uri nfs://localhost/shared/rocksdb +# # remote_compaction.shared_fs_local_prefix not needed - auto-inferred as ./home +# +# Explicit setup with NFS (complex paths or explicit control): +# dir /mnt/rocksdb/db +# csa_address localhost:8010 +# remote_compaction.shared_fs_uri nfs://nfs-server/shared/rocksdb +# remote_compaction.shared_fs_local_prefix /mnt/rocksdb +# +# Setup with HDFS: +# dir ./home/db +# csa_address localhost:8010 +# remote_compaction.shared_fs_uri hdfs://namenode:9000/data/rocksdb +# +# How it works (Automatic URI Conversion): +# - RocksDB uses local paths (no changes needed to RocksDB config) +# - NFSFileSystem automatically converts matching paths to URIs internally +# - Example: ./home/db -> nfs://localhost/shared/rocksdb/db (internal) +# - Paths and URIs are unified at the NFSFileSystem layer +# - No conflicts between local paths and URIs +# +# To disable remote compaction: +# - Set csa_address to empty, or +# - Set remote_compaction.mode to empty +# - System will fallback to local compaction