From 1195e6ea597ab04111eb7e552b38bc5a73a7916e Mon Sep 17 00:00:00 2001 From: whzruc Date: Tue, 26 Aug 2025 12:11:32 +0800 Subject: [PATCH 1/2] BufferPool-v1.0:Functionality correct but performance issues still exist --- cpp/CMakeLists.txt | 2 +- cpp/Makefile | 6 +- cpp/include/PixelsReadGlobalState.hpp | 3 + cpp/pixels-common/CMakeLists.txt | 59 +--- .../include/physical/BufferPool.h | 133 ++++++- .../include/physical/BufferPool/Bitmap.h | 48 +++ .../physical/BufferPool/BufferPoolEntry.h | 46 +++ cpp/pixels-common/include/physical/Request.h | 1 + .../include/physical/RequestBatch.h | 2 + .../include/physical/Scheduler.h | 2 + .../include/physical/io/PhysicalLocalReader.h | 63 ++-- .../include/physical/natives/ByteBuffer.h | 5 + .../include/physical/natives/DirectIoLib.h | 4 + .../natives/DirectUringRandomAccessFile.h | 37 +- .../include/utils/MutexTracker.h | 129 +++++++ cpp/pixels-common/lib/physical/BufferPool.cpp | 276 ++++++++++++--- .../lib/physical/BufferPool/Bitmap.cpp | 7 + .../physical/BufferPool/BufferPoolEntry.cpp | 113 ++++++ cpp/pixels-common/lib/physical/Request.cpp | 1 + .../lib/physical/RequestBatch.cpp | 8 + .../lib/physical/io/PhysicalLocalReader.cpp | 38 +- .../lib/physical/natives/ByteBuffer.cpp | 20 ++ .../lib/physical/natives/DirectIoLib.cpp | 5 +- .../natives/DirectUringRandomAccessFile.cpp | 332 +++++++++++++++--- .../lib/physical/scheduler/NoopScheduler.cpp | 28 +- cpp/pixels-common/lib/utils/MutexTracker.cpp | 5 + .../include/reader/PixelsReaderOption.h | 5 + .../include/reader/PixelsRecordReaderImpl.h | 3 + cpp/pixels-core/include/vector/ColumnVector.h | 1 + .../lib/reader/PixelsReaderOption.cpp | 9 + .../lib/reader/PixelsRecordReaderImpl.cpp | 50 ++- .../lib/reader/StringColumnReader.cpp | 7 + cpp/pixels-core/lib/vector/ColumnVector.cpp | 8 + cpp/pixels-duckdb/PixelsScanFunction.cpp | 50 +-- cpp/pixels-duckdb/duckdb | 2 +- 35 files changed, 1282 insertions(+), 226 deletions(-) create mode 100644 cpp/pixels-common/include/physical/BufferPool/Bitmap.h create mode 100644 cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h create mode 100644 cpp/pixels-common/include/utils/MutexTracker.h create mode 100644 cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp create mode 100644 cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp create mode 100644 cpp/pixels-common/lib/utils/MutexTracker.cpp diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index e2a9f72867..05196c99b6 100755 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -20,7 +20,7 @@ include_directories(${CMAKE_CURRENT_BINARY_DIR}) add_subdirectory(pixels-common) add_subdirectory(pixels-core) -add_subdirectory(pixels-cli) +#add_subdirectory(pixels-cli) add_subdirectory(third-party/googletest) add_subdirectory(tests) diff --git a/cpp/Makefile b/cpp/Makefile index 418f175a2f..6fe36ee78b 100644 --- a/cpp/Makefile +++ b/cpp/Makefile @@ -57,4 +57,8 @@ debug: deps release: deps cmake $(GENERATOR) $(FORCE_COLOR) $(EXTENSION_FLAGS) ${CLIENT_FLAGS} -DEXTENSION_STATIC_BUILD=1 -DCMAKE_BUILD_TYPE=Release ${BUILD_FLAGS} -S pixels-duckdb/duckdb -B build/release && \ - cmake --build build/release --config Release \ No newline at end of file + cmake --build build/release --config Release + +relWithDebInfo: deps + cmake $(GENERATOR) $(FORCE_COLOR) $(EXTENSION_FLAGS) ${CLIENT_FLAGS} -DEXTENSION_STATIC_BUILD=1 -DCMAKE_BUILD_TYPE=RelWithDebInfo ${BUILD_FLAGS} -S pixels-duckdb/duckdb -B build/relWithDebInfo && \ + cmake --build build/relWithDebInfo --config RelWithDebInfo \ No newline at end of file diff --git a/cpp/include/PixelsReadGlobalState.hpp b/cpp/include/PixelsReadGlobalState.hpp index 0d23e1f855..901fef255d 100644 --- a/cpp/include/PixelsReadGlobalState.hpp +++ b/cpp/include/PixelsReadGlobalState.hpp @@ -40,6 +40,9 @@ namespace duckdb { mutex lock; + atomic active_threads; // 活跃线程数 + atomic all_done; // 是否所有线程都已完成 + //! The initial reader from the bind phase std::shared_ptr initialPixelsReader; diff --git a/cpp/pixels-common/CMakeLists.txt b/cpp/pixels-common/CMakeLists.txt index 1ed1f4f03d..3cba67bf36 100644 --- a/cpp/pixels-common/CMakeLists.txt +++ b/cpp/pixels-common/CMakeLists.txt @@ -4,50 +4,21 @@ set(CMAKE_CXX_STANDARD 17) include(ExternalProject) -set(pixels_common_cxx - lib/physical/storage/LocalFS.cpp - lib/physical/storage/LocalFSProvider.cpp - lib/physical/storage/PhysicalLocalWriter.cpp - lib/physical/PhysicalWriterOption.cpp - lib/physical/Status.cpp - lib/physical/Storage.cpp - lib/physical/FilePath.cpp - lib/physical/natives/PixelsRandomAccessFile.cpp - lib/physical/natives/DirectRandomAccessFile.cpp - lib/physical/natives/ByteBuffer.cpp - lib/physical/io/PhysicalLocalReader.cpp - lib/physical/StorageFactory.cpp - lib/physical/Request.cpp - lib/physical/RequestBatch.cpp - lib/physical/scheduler/NoopScheduler.cpp - lib/physical/SchedulerFactory.cpp - lib/exception/InvalidArgumentException.cpp - lib/utils/Constants.cpp - lib/utils/String.cpp - include/physical/natives/DirectIoLib.h - lib/physical/natives/DirectIoLib.cpp - include/utils/ConfigFactory.h - lib/utils/ConfigFactory.cpp - include/physical/MergedRequest.h - include/physical/scheduler/SortMergeScheduler.h - lib/physical/scheduler/SortMergeScheduler.cpp - lib/MergedRequest.cpp include/profiler/TimeProfiler.h - lib/profiler/TimeProfiler.cpp - include/profiler/CountProfiler.h - lib/profiler/CountProfiler.cpp - include/profiler/AbstractProfiler.h - include/physical/allocator/Allocator.h - include/physical/allocator/OrdinaryAllocator.h - lib/physical/allocator/OrdinaryAllocator.cpp - include/physical/allocator/BufferPoolAllocator.h - lib/physical/allocator/BufferPoolAllocator.cpp - include/physical/BufferPool.h - lib/physical/BufferPool.cpp - include/physical/natives/DirectUringRandomAccessFile.h - lib/physical/natives/DirectUringRandomAccessFile.cpp - include/utils/ColumnSizeCSVReader.h lib/utils/ColumnSizeCSVReader.cpp - include/physical/StorageArrayScheduler.h lib/physical/StorageArrayScheduler.cpp - include/physical/natives/ByteOrder.h + +file(GLOB_RECURSE pixels_common_cxx + "lib/physical/*.cpp" + "lib/physical/*.h" + "lib/exception/*.cpp" + "lib/exception/*.h" + "lib/utils/*.cpp" + "lib/utils/*.h" + "lib/profiler/*.cpp" + "lib/profiler/*.h" + "include/physical/*.h" + "include/profiler/*.h" + "include/utils/*.h" + "include/physical/BufferPool/*.h" + "lib/MergedRequest.cpp" ) include_directories(include) diff --git a/cpp/pixels-common/include/physical/BufferPool.h b/cpp/pixels-common/include/physical/BufferPool.h index ec810e29a9..f07003efd8 100644 --- a/cpp/pixels-common/include/physical/BufferPool.h +++ b/cpp/pixels-common/include/physical/BufferPool.h @@ -33,41 +33,154 @@ #include "exception/InvalidArgumentException.h" #include "utils/ColumnSizeCSVReader.h" #include - +#include "physical/BufferPool/Bitmap.h" +#include "physical/BufferPool/BufferPoolEntry.h" +#include +#include +#include // when allocating buffer pool, we use the size of the first pxl file. Consider that // the remaining pxl file has larger size than the first file, we allocate some extra // size (10MB) to each column. // TODO: how to evaluate the maximal pool size -#define EXTRA_POOL_SIZE 3*1024*1024 +#define EXTRA_POOL_SIZE 10*1024*1024 class DirectUringRandomAccessFile; // This class is global class. The variable is shared by each thread class BufferPool { public: + // 嵌套子类,用于管理缓冲区池条目及其属性 + class BufferPoolManagedEntry { + public: + enum class State{ + InitizaledNotAllocated, + AllocatedAndInUse, + UselessButNotFree + }; + private: + std::shared_ptr bufferPoolEntry; // 指向缓冲区池条目的智能指针 + int ring_index; // 环形缓冲区索引 + size_t current_size; // 当前使用大小 + int offset; // 偏移量 + State state; + + + + public: + + BufferPoolManagedEntry(std::shared_ptr entry, + int ringIdx, + size_t currSize, + off_t off) + : bufferPoolEntry(std::move(entry)), + ring_index(ringIdx), + current_size(currSize), + offset(off) , + state(State::InitizaledNotAllocated){} + + std::shared_ptr getBufferPoolEntry() const { + return bufferPoolEntry; + } + + int getRingIndex() const { + return ring_index; + } + + void setRingIndex(int index) { + ring_index = index; + } + + size_t getCurrentSize() const { + return current_size; + } + + void setCurrentSize(size_t size) { + current_size = size; + } + + int getOffset() const { + return offset; + } + + void setOffset(int off) { + offset = off; + } + + // 获取当前状态 + State getStatus() const { + return state; + } + + // 设置状态 + void setStatus(State newStatus) { + state = newStatus; + } + }; + static void Initialize(std::vector colIds, std::vector bytes, std::vector columnNames); - static std::shared_ptr GetBuffer(uint32_t colId); + static void + InitializeBuffers(); + + static std::shared_ptr GetBuffer(uint32_t colId,uint64_t byte,std::string columnName); - static int64_t GetBufferId(uint32_t index); + static int64_t GetBufferId(); static void Switch(); static void Reset(); + static std::shared_ptr AddNewBuffer(size_t size); + + static int getRingIndex(uint32_t colId); + + static std::shared_ptr AllocateNewBuffer(std::shared_ptr currentBufferManagedEntry, uint32_t colId,uint64_t byte,std::string columnName); + + static std::shared_ptr ReusePreviousBuffer(std::shared_ptr currentBufferManagedEntry,uint32_t colId,uint64_t byte,std::string columnName); + + static void PrintStats() { + // 打印当前线程 ID + std::thread::id tid = std::this_thread::get_id(); + + printf("线程 %zu -> 全局缓冲区使用: %ld / %ld\n", + std::hash{}(tid), // 转换成整数便于阅读 + global_used_size, global_free_size); + + // 线程局部统计 + printf("线程 %zu -> Buffer0使用: %zu, 缓冲区数量: %d\n", + std::hash{}(tid), + thread_local_used_size[0], thread_local_buffer_count[0]); + + printf("线程 %zu -> Buffer1使用: %zu, 缓冲区数量: %d\n", + std::hash{}(tid), + thread_local_used_size[1], thread_local_buffer_count[1]); + } private: BufferPool() = default; + // global + static std::mutex bufferPoolMutex; + // thread local + static thread_local bool isInitialized; + static thread_local std::vector > registeredBuffers[2]; + static thread_local long global_used_size; + static thread_local long global_free_size; + static thread_local std::shared_ptr directIoLib; + static thread_local int nextRingIndex; + static thread_local std::shared_ptr nextEmptyBufferPoolEntry[2]; static thread_local int colCount; - static thread_local std::map - nrBytes; - static thread_local bool isInitialized; - static thread_local std::map> - buffers[2]; - static std::shared_ptr directIoLib; + static thread_local int currBufferIdx; static thread_local int nextBufferIdx; + static thread_local std::map > buffersAllocated[2]; friend class DirectUringRandomAccessFile; + + static thread_local std::unordered_map> ringBufferMap[2]; + + + + static thread_local size_t thread_local_used_size[2]; // 线程已使用大小 + static thread_local int thread_local_buffer_count[2]; // 线程持有的缓冲区数量 }; #endif // DUCKDB_BUFFERPOOL_H diff --git a/cpp/pixels-common/include/physical/BufferPool/Bitmap.h b/cpp/pixels-common/include/physical/BufferPool/Bitmap.h new file mode 100644 index 0000000000..70d9a31790 --- /dev/null +++ b/cpp/pixels-common/include/physical/BufferPool/Bitmap.h @@ -0,0 +1,48 @@ +/* +/* @author whz +/* @create 7/30/25. +*/ + +#ifndef BITMAP_H +#define BITMAP_H +#include +#include "exception/InvalidArgumentException.h" +#include + +class Bitmap { +public: + + explicit Bitmap(size_t size) + : bits((size + 7) / 8, 0), num_bits(size) {} + + void set(size_t index) { + if (index >= num_bits) throw InvalidArgumentException("Bitmap::set: index out of range"); + bits[index / 8] |= (1 << (index % 8)); + } + + void clear(size_t index) { + if (index >= num_bits) throw InvalidArgumentException("Bitmap::clear: index out of range"); + bits[index / 8] &= ~(1 << (index % 8)); + } + + bool test(size_t index) const { + if (index >= num_bits) throw InvalidArgumentException("Bitmap::test: index out of range"); + return bits[index / 8] & (1 << (index % 8)); + } + + size_t size() const { return num_bits; } + + void print() const { + for (size_t i = 0; i < num_bits; ++i) { + std::cout << (test(i) ? '1' : '0'); + if ((i + 1) % 8 == 0) std::cout << ' '; + } + std::cout << '\n'; + } + +private: + std::vector bits; + size_t num_bits; +}; + +#endif //BITMAP_H diff --git a/cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h b/cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h new file mode 100644 index 0000000000..74536a7b6a --- /dev/null +++ b/cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h @@ -0,0 +1,46 @@ +/* +/* @author whz +/* @create 7/30/25. +*/ + +#ifndef BUFFERPOOLENTRY_H +#define BUFFERPOOLENTRY_H +#include +#include +#include +#include "physical/BufferPool/Bitmap.h" +class BufferPoolEntry { +public: +explicit BufferPoolEntry(size_t size,int slice_size,std::shared_ptr direct_lib,int offset,int ring_index); + size_t getSize() const; + std::shared_ptr getBitmap() const; + std::shared_ptr getBuffer() const; + bool isFull() const; + int getNextFreeIndex() const; + int setNextFreeIndex(int index); + ~BufferPoolEntry(); + uint64_t checkCol(uint32_t) const; + void addCol(uint32_t colId,uint64_t bytes); + bool isInUse() const; + void setInUse(bool in_use); + int getOffsetInBuffers() const; + void setOffsetInBuffers(int offset); + bool getIsRegistered() const; + void setIsRegistered(bool registered); + int getRingIndex() const; + void setRingIndex(int ring_index); + void reset(); +private: +size_t size_; +std::shared_ptr bitmap_; +std::shared_ptr buffer_; + bool is_full_; + int next_free_; + std::map nr_bytes_; + bool is_in_use_; + int offset_in_buffers_; + bool is_registered; + int ring_index; +}; + +#endif //BUFFERPOOLENTRY_H diff --git a/cpp/pixels-common/include/physical/Request.h b/cpp/pixels-common/include/physical/Request.h index eb8f909214..08bfaedb46 100644 --- a/cpp/pixels-common/include/physical/Request.h +++ b/cpp/pixels-common/include/physical/Request.h @@ -34,6 +34,7 @@ class Request uint64_t queryId; uint64_t start; uint64_t length; + int ring_index; Request(uint64_t queryId_, uint64_t start_, uint64_t length_, int64_t bufferId = -1); diff --git a/cpp/pixels-common/include/physical/RequestBatch.h b/cpp/pixels-common/include/physical/RequestBatch.h index 423fb74ff6..39d23b7e12 100644 --- a/cpp/pixels-common/include/physical/RequestBatch.h +++ b/cpp/pixels-common/include/physical/RequestBatch.h @@ -42,6 +42,8 @@ class RequestBatch void add(Request request); + Request& getRequest(int index); + int getSize(); std::vector getRequests(); diff --git a/cpp/pixels-common/include/physical/Scheduler.h b/cpp/pixels-common/include/physical/Scheduler.h index f07add0ed5..8da823445c 100644 --- a/cpp/pixels-common/include/physical/Scheduler.h +++ b/cpp/pixels-common/include/physical/Scheduler.h @@ -28,6 +28,7 @@ #include "physical/PhysicalReader.h" #include "physical/RequestBatch.h" #include "profiler/TimeProfiler.h" +#include class Scheduler { @@ -46,5 +47,6 @@ class Scheduler RequestBatch batch, std::vector > reuseBuffers, long queryId) = 0; + std::mutex mtx; }; #endif //PIXELS_SCHEDULER_H diff --git a/cpp/pixels-common/include/physical/io/PhysicalLocalReader.h b/cpp/pixels-common/include/physical/io/PhysicalLocalReader.h index 9ebe51a622..23a3e42f9a 100644 --- a/cpp/pixels-common/include/physical/io/PhysicalLocalReader.h +++ b/cpp/pixels-common/include/physical/io/PhysicalLocalReader.h @@ -26,52 +26,63 @@ #define PIXELS_READER_PHYSICALLOCALREADER_H #include "physical/PhysicalReader.h" -#include "physical/storage/LocalFS.h" #include "physical/natives/DirectRandomAccessFile.h" #include "physical/natives/DirectUringRandomAccessFile.h" -#include +#include "physical/storage/LocalFS.h" #include +#include +#include - -class PhysicalLocalReader : public PhysicalReader -{ +class PhysicalLocalReader : public PhysicalReader { public: - PhysicalLocalReader(std::shared_ptr storage, std::string path); + PhysicalLocalReader(std::shared_ptr storage, std::string path); - std::shared_ptr readFully(int length) override; + std::shared_ptr readFully(int length) override; - std::shared_ptr readFully(int length, std::shared_ptr bb) override; + std::shared_ptr readFully(int length, std::shared_ptr bb) override; - std::shared_ptr readAsync(int length, std::shared_ptr bb, int index); + std::shared_ptr readAsync(int length, std::shared_ptr bb, int index,int ring_index,int start_offset); - void readAsyncSubmit(uint32_t size); + void readAsyncSubmit(std::unordered_map sizes,std::unordered_set ring_index); - void readAsyncComplete(uint32_t size); + void readAsyncComplete(std::unordered_map sizes,std::unordered_set ring_index); - void readAsyncSubmitAndComplete(uint32_t size); + void readAsyncSubmitAndComplete(uint32_t size,std::unordered_set ring_index); - void close() override; + void close() override; - long getFileLength() override; + long getFileLength() override; - void seek(long desired) override; + void seek(long desired) override; - long readLong() override; + long readLong() override; - int readInt() override; + int readInt() override; - char readChar() override; + char readChar() override; - std::string getName() override; + std::string getName() override; -private: - std::shared_ptr local; - std::string path; - long id; - std::atomic numRequests; - std::atomic asyncNumRequests; - std::shared_ptr raf; + void addRingIndex(int ring_index); + + std::unordered_set& getRingIndexes(); + + std::unordered_map getRingIndexCountMap(); + void setRingIndexCountMap(std::unordered_map ringIndexCount); + +private: + std::shared_ptr local; + std::string path; + long id; + std::atomic numRequests; + std::atomic asyncNumRequests; + std::shared_ptr raf; + // usr for bufferpool + bool isBufferValid; + std::unordered_set> ringIndexes; + std::unordered_set ring_index_vector; + std::unordered_map ringIndexCountMap; }; #endif //PIXELS_READER_PHYSICALLOCALREADER_H diff --git a/cpp/pixels-common/include/physical/natives/ByteBuffer.h b/cpp/pixels-common/include/physical/natives/ByteBuffer.h index ab64114641..1ebd430a9c 100644 --- a/cpp/pixels-common/include/physical/natives/ByteBuffer.h +++ b/cpp/pixels-common/include/physical/natives/ByteBuffer.h @@ -43,8 +43,12 @@ class ByteBuffer ByteBuffer(ByteBuffer &bb, uint32_t startId, uint32_t length); + ByteBuffer(ByteBuffer &bb,uint32_t startId,uint32_t length,bool from_slice); + ~ByteBuffer(); + std::shared_ptr slice(uint32_t offset, uint32_t length); + void filp();// reset the readPosition uint32_t bytesRemaining(); // Number of uint8_ts from the current read position till the end of the buffer void clear(); // Clear our the vector and reset read and write positions @@ -167,6 +171,7 @@ class ByteBuffer std::string name; uint32_t rmark; bool fromOtherBB; + bool fromSlice; // Sometimes the buffer is allocated by malloc/poxis_memalign, in this case, we // should use free() to deallocate the buf bool allocated_by_new; diff --git a/cpp/pixels-common/include/physical/natives/DirectIoLib.h b/cpp/pixels-common/include/physical/natives/DirectIoLib.h index 2fec848b13..dafe33fa0c 100644 --- a/cpp/pixels-common/include/physical/natives/DirectIoLib.h +++ b/cpp/pixels-common/include/physical/natives/DirectIoLib.h @@ -67,6 +67,10 @@ class DirectIoLib long blockEnd(long value); + int getBlockSize() const { + return fsBlockSize; + }; + private: int fsBlockSize; long fsBlockNotMask; diff --git a/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h b/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h index ab5293aa27..43650e46ae 100644 --- a/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h +++ b/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h @@ -31,6 +31,9 @@ #include "exception/InvalidArgumentException.h" #include "DirectIoLib.h" #include "physical/BufferPool.h" +#include "unordered_set" +#include +#include "utils/MutexTracker.h" class DirectUringRandomAccessFile : public DirectRandomAccessFile { @@ -45,19 +48,37 @@ class DirectUringRandomAccessFile : public DirectRandomAccessFile static void Reset(); - std::shared_ptr readAsync(int length, std::shared_ptr buffer, int index); + static bool RegisterMoreBuffer(int index,std::vector> buffers); - void readAsyncSubmit(int size); + std::shared_ptr readAsync(int length, std::shared_ptr buffer, int index,int ring_index,int start_offset); - void readAsyncComplete(int size); + void readAsyncSubmit(std::unordered_map sizes,std::unordered_set ring_indexs); + + void readAsyncComplete(std::unordered_map sizes,std::unordered_set ring_indexs); + + void seekByIndex(long offset,int index); + + static struct io_uring* getRing(int index); ~DirectUringRandomAccessFile(); private: - static thread_local struct io_uring *ring; - static thread_local bool isRegistered; - static thread_local struct iovec *iovecs; - static thread_local uint32_t - iovecSize; + // global isRegistered + // static bool isRegistered; + // static MutexTracker g_mutex_tracker; + // static TrackedMutex g_mutex; + // static std::vector ring_vector; + // static std::vector iovecs_vector; + // static uint32_t iovecSize; + // static std::vector offsets_vector; + // thread_local + static std::mutex mutex_; + static thread_local bool isRegistered; + // static MutexTracker g_mutex_tracker; + // static TrackedMutex g_mutex; + static thread_local std::vector ring_vector; + static thread_local std::vector iovecs_vector; + static thread_local uint32_t iovecSize; + static thread_local std::vector offsets_vector; }; #endif // DUCKDB_DIRECTURINGRANDOMACCESSFILE_H diff --git a/cpp/pixels-common/include/utils/MutexTracker.h b/cpp/pixels-common/include/utils/MutexTracker.h new file mode 100644 index 0000000000..96dc47ed39 --- /dev/null +++ b/cpp/pixels-common/include/utils/MutexTracker.h @@ -0,0 +1,129 @@ +/* +/* @author whz +/* @create 8/23/25. +*/ + +#ifndef DUCKDB_MUTEX_H +#define DUCKDB_MUTEX_H +#include +#include +#include +#include +#include +#include +class MutexTracker { +private: + // 存储锁与持有者线程ID的映射 + std::unordered_map mutex_owners; + // 保护映射表的内部锁 + std::mutex internal_mutex; + +public: + // 获取锁的持有者线程ID,若未被持有则返回默认ID + std::thread::id get_owner(const std::mutex* mutex) { + std::lock_guard lock(internal_mutex); + auto it = mutex_owners.find(mutex); + if (it != mutex_owners.end()) { + return it->second; + } + return std::thread::id(); // 未被持有 + } + + // 记录锁被持有 + void lock_acquired(const std::mutex* mutex) { + std::lock_guard lock(internal_mutex); + auto current_thread = std::this_thread::get_id(); + + // 检查是否重复获取(防止递归锁问题) + auto it = mutex_owners.find(mutex); + if (it != mutex_owners.end()) { + if (it->second == current_thread) { + // 同一线程重复获取非递归锁,可能导致死锁 + throw std::runtime_error("同一线程重复获取非递归锁"); + } + } + + mutex_owners[mutex] = current_thread; + } + + // 记录锁被释放 + void lock_released(const std::mutex* mutex) { + std::lock_guard lock(internal_mutex); + mutex_owners.erase(mutex); + } + + // 打印锁的持有者信息 + void print_owner(const std::mutex* mutex, const std::string& mutex_name) { + std::thread::id owner = get_owner(mutex); + if (owner == std::thread::id()) { + std::cout << mutex_name << " 未被任何线程持有" << std::endl; + } else { + std::cout << mutex_name << " 被线程 " << std::hash{}(owner) + << " 持有" << std::endl; + } + } +}; + + +// 带跟踪功能的互斥锁包装器 +class TrackedMutex { +private: + std::mutex internal_mutex; + std::string name; + MutexTracker g_mutex_tracker; + +public: + TrackedMutex(const std::string& name,MutexTracker &g_mutex_tracker) : name(name) {} + + // 加锁并记录持有者 + void lock() { + internal_mutex.lock(); + g_mutex_tracker.lock_acquired(&internal_mutex); + } + + // 解锁并清除持有者记录 + void unlock() { + g_mutex_tracker.lock_released(&internal_mutex); + internal_mutex.unlock(); + } + + // 尝试加锁(成功则记录持有者) + bool try_lock() { + if (internal_mutex.try_lock()) { + g_mutex_tracker.lock_acquired(&internal_mutex); + return true; + } + return false; + } + + // 获取锁名称 + const std::string& get_name() const { return name; } + + // 获取内部互斥锁指针(用于跟踪) + const std::mutex* get_internal_mutex() const { + return &internal_mutex; + } +}; + +// 与TrackedMutex配合使用的RAII锁管理 +template +class TrackedLockGuard { +private: + Mutex& mutex; + +public: + TrackedLockGuard(Mutex& m) : mutex(m) { + mutex.lock(); + } + + ~TrackedLockGuard() { + mutex.unlock(); + } + + // 禁止拷贝 + TrackedLockGuard(const TrackedLockGuard&) = delete; + TrackedLockGuard& operator=(const TrackedLockGuard&) = delete; +}; + + +#endif // DUCKDB_MUTEX_H diff --git a/cpp/pixels-common/lib/physical/BufferPool.cpp b/cpp/pixels-common/lib/physical/BufferPool.cpp index 4dcc7bdeab..acd5519de5 100644 --- a/cpp/pixels-common/lib/physical/BufferPool.cpp +++ b/cpp/pixels-common/lib/physical/BufferPool.cpp @@ -23,30 +23,62 @@ * @create 2023-05-25 */ #include "physical/BufferPool.h" +#include +#include +#include +#include +#include +using TimePoint = std::chrono::high_resolution_clock::time_point; +using Duration = std::chrono::duration; + +TimePoint getCurrentTime() { + return std::chrono::high_resolution_clock::now(); +} + + +// global +std::string columnSizePath = ConfigFactory::Instance().getProperty("pixel.column.size.path"); +std::shared_ptr csvReader; +int sliceSize= std::stoi(ConfigFactory::Instance().getProperty("pixel.bufferpool.sliceSize")); +int bufferSize=std::stoi(ConfigFactory::Instance().getProperty("pixel.bufferpool.bufferpoolSize")); +const size_t SLICE_SIZE = sliceSize * 1024; +int bufferNum=std::stoi(ConfigFactory::Instance().getProperty("pixel.bufferpool.bufferNum")); +std::mutex BufferPool::bufferPoolMutex; + +// thread_local +thread_local bool BufferPool::isInitialized; +thread_local std::vector > BufferPool::registeredBuffers[2]; +thread_local long BufferPool::global_free_size=0; +thread_local long BufferPool::global_used_size = 0; +thread_local std::shared_ptr BufferPool::directIoLib; +thread_local int BufferPool::nextRingIndex=1; +thread_local std::shared_ptr BufferPool::nextEmptyBufferPoolEntry[2]={nullptr,nullptr}; +thread_local std::vector globalBuffers; thread_local int BufferPool::colCount = 0; -thread_local std::map -BufferPool::nrBytes; -thread_local bool BufferPool::isInitialized = false; -thread_local std::map> -BufferPool::buffers[2]; // The currBufferIdx is set to 1. When executing the first file, this value is 0 // since we call switch function first. thread_local int BufferPool::currBufferIdx = 1; thread_local int BufferPool::nextBufferIdx = 0; -std::shared_ptr BufferPool::directIoLib; + +thread_local std::map> BufferPool::buffersAllocated[2]; +thread_local std::unordered_map> BufferPool::ringBufferMap[2]; +thread_local size_t BufferPool::thread_local_used_size[2]={0,0}; +thread_local int BufferPool::thread_local_buffer_count[2]={0,0}; void BufferPool::Initialize(std::vector colIds, std::vector bytes, std::vector columnNames) { + assert(colIds.size() == bytes.size()); + // std::lock_guard lock(bufferPoolMutex); int fsBlockSize = std::stoi(ConfigFactory::Instance().getProperty("localfs.block.size")); - std::string columnSizePath = ConfigFactory::Instance().getProperty("pixel.column.size.path"); - std::shared_ptr csvReader; - if (!columnSizePath.empty()) - { - csvReader = std::make_shared(columnSizePath); - } + auto strToBool = [](const std::string& s) { + return s == "true" || s == "1" || s == "yes"; + }; + std::string configValue = ConfigFactory::Instance().getProperty("pixel.bufferpool.fixedsize"); + bool isFixedSize = strToBool(configValue); + // std::cout<<"是否初始化:"< colIds, std::vector (fsBlockSize); + BufferPool::InitializeBuffers(); for (int i = 0; i < colIds.size(); i++) { uint32_t colId = colIds.at(i); - std::string columnName = columnNames[colId]; - for (int idx = 0; idx < 2; idx++) - { - std::shared_ptr buffer; - if (columnSizePath.empty()) - { - buffer = BufferPool::directIoLib->allocateDirectBuffer(bytes.at(i) + EXTRA_POOL_SIZE); - } - else + if (isFixedSize) { + if (!columnSizePath.empty()) { - buffer = BufferPool::directIoLib->allocateDirectBuffer(csvReader->get(columnName)); + csvReader = std::make_shared(columnSizePath); } - - BufferPool::nrBytes[colId] = buffer->size(); - BufferPool::buffers[idx][colId] = buffer; } + auto byte=bytes.at(i); + BufferPool::ringBufferMap[currBufferIdx][colId]=std::make_shared(registeredBuffers[currBufferIdx][0],0,byte,0); + // BufferPool::ringBufferMap[nextBufferIdx][colId]=registeredBuffers[nextBufferIdx][0]; + // std::cout<<"Initialized columnID:"<(registeredBuffers[currBufferIdx][0],0,byte,0); + // BufferPool::ringBufferMap[nextBufferIdx][colId]=registeredBuffers[nextBufferIdx][0]; + // GetBuffer(colId,byte,columnNames.at(i)); + // std::cout<<"Not Initialized columnID:"<size()< buffer_pool_entry=std::make_shared(size_,sliceSize,directIoLib,idx,0); + registeredBuffers[idx].emplace_back(buffer_pool_entry); + buffer_pool_entry->setInUse(true); + global_free_size+=size_; + } +} + + +int64_t BufferPool::GetBufferId() { - return index + currBufferIdx * colCount; + return currBufferIdx; } -std::shared_ptr BufferPool::GetBuffer(uint32_t colId) +std::shared_ptr BufferPool::AllocateNewBuffer(std::shared_ptr currentBufferManagedEntry, uint32_t colId,uint64_t byte,std::string columnName) { + auto strToBool = [](const std::string& s) { + return s == "true" || s == "1" || s == "yes"; + }; + std::string configValue = ConfigFactory::Instance().getProperty("pixel.bufferpool.fixedsize"); + bool isFixedSize = strToBool(configValue); + if (isFixedSize) { + byte=csvReader->get(columnName); + // std::cout<<"colId: "<getBufferPoolEntry(); + std::shared_ptr original = currentBuffer->getBuffer(); + + // get offset + size_t offset = currentBuffer->getNextFreeIndex(); + // fisrt find anthor empty buffer + + if (offset + totalSize > original->size()) + { + + if (nextEmptyBufferPoolEntry[currBufferIdx]!=nullptr&& + nextEmptyBufferPoolEntry[currBufferIdx]->getNextFreeIndex()+totalSizegetSize()) { + // there are anthor regisitered Buffers + // std::cout<<"curBufID:"<size()<setInUse(false); + currentBuffer=BufferPool::AddNewBuffer(currentBuffer->getSize()); + std::vector> buffers; + buffers.emplace_back(currentBuffer->getBuffer()); + if (!::DirectUringRandomAccessFile::RegisterMoreBuffer(currentBuffer->getRingIndex(),buffers)) { + throw std::runtime_error("Failed to register more buffers"); + } + currentBuffer->setInUse(true); + currentBuffer->setIsRegistered(true); + } + offset=currentBuffer->getNextFreeIndex(); + // change the find bitmap + auto newBufferPoolManageEntry=std::make_shared(currentBuffer,currentBuffer->getRingIndex(),totalSize,offset); + BufferPool::ringBufferMap[currBufferIdx][colId]=newBufferPoolManageEntry; + newBufferPoolManageEntry->setStatus(BufferPoolManagedEntry::State::AllocatedAndInUse); + original=currentBuffer->getBuffer(); + } + // update bitmap (maybe costly) + // size_t startSlice = offset / SLICE_SIZE; + // for (size_t i = 0; i < sliceCount; ++i) { + // size_t sliceIndex = startSlice + i; + // if (currentBuffer->getBitmap()->test(sliceIndex)) { + // throw std::runtime_error("Buffer slice already used! Potential bitmap inconsistency."); + // } + // currentBuffer->getBitmap()->set(sliceIndex); + // } + + currentBuffer->setNextFreeIndex(offset + totalSize); + + std::shared_ptr sliced = original->slice(offset, totalSize); + currentBuffer->addCol(colId,sliced->size()); + BufferPool::buffersAllocated[currBufferIdx][colId] = sliced; + auto newBufferPoolManageEntry=BufferPool::ringBufferMap[currBufferIdx][colId]; + newBufferPoolManageEntry->setStatus(BufferPoolManagedEntry::State::AllocatedAndInUse); + newBufferPoolManageEntry->setCurrentSize(sliced->size()); + newBufferPoolManageEntry->setOffset(offset); + newBufferPoolManageEntry->setRingIndex(currentBuffer->getRingIndex()); + + // std::cout<<"buffer pointer:"<(sliced->getPointer())<< + // " length:"<size()<<" bufferOffsets:"<getNextFreeIndex()<<" buffersAllocated:"<size()< BufferPool::ReusePreviousBuffer(std::shared_ptr currentBufferManagedEntry,uint32_t colId,uint64_t byte,std::string columnName) { + auto currentBuffer=currentBufferManagedEntry->getBufferPoolEntry(); + // std::cout<<"resue previous buffer size:"<size() + // <<" colID:"<getCurrentSize():"<getCurrentSize()<<" byte:"< BufferPool::GetBuffer(uint32_t colId,uint64_t byte,std::string columnName) { - return BufferPool::buffers[currBufferIdx][colId]; + // another buffer + // std::lock_guard lock(bufferPoolMutex); + auto currentBufferManagedEntry = ringBufferMap[currBufferIdx][colId]; + // 分配情况 + // 情况一 未分配 检查entry的状态 + // 情况二 已经分配 + // 大小情况 + // 情况一 可以复用之前的buffer resuePrevious() + // 情况二 可以在同一个大buffer中划分出需要的空间 allocateBuffer + // 情况三 当前大buffer剩余内存不够用了 需要新的大buffer + if (currentBufferManagedEntry->getStatus()==BufferPoolManagedEntry::State::InitizaledNotAllocated) { + //未分配 + return AllocateNewBuffer(currentBufferManagedEntry,colId,byte,columnName); + } + else { + // 已经分配 + if (currentBufferManagedEntry->getCurrentSize()>=byte&¤tBufferManagedEntry->getCurrentSize()-directIoLib->getBlockSize()>=byte) { + // 复用之前的buffer + return ReusePreviousBuffer(currentBufferManagedEntry,colId,byte,columnName); + }else { + return AllocateNewBuffer(currentBufferManagedEntry,colId,byte,columnName); + } + } } void BufferPool::Reset() { - BufferPool::isInitialized = false; - BufferPool::nrBytes.clear(); + // BufferPool::isInitialized = false; + // std::lock_guard lock(bufferPoolMutex); for (int idx = 0; idx < 2; idx++) { - BufferPool::buffers[idx].clear(); + // BufferPool::currentBuffers[idx]->clear();; + BufferPool::buffersAllocated[idx].clear(); + BufferPool::ringBufferMap[idx].clear(); + // BufferPool::registeredBuffers[idx].clear(); + // thread_local_used_size[idx]=0; + // thread_local_buffer_count[idx]=0; + // global_free_size=0; + global_used_size=0; + for (auto bufferEntry: registeredBuffers[idx]) { + bufferEntry->reset(); + } } + + + + BufferPool::colCount = 0; } @@ -127,3 +296,24 @@ void BufferPool::Switch() } + +std::shared_ptr BufferPool::AddNewBuffer(size_t size) { + // std::cout<<"Adding new buffer"< buffer_pool_entry=std::make_shared(size,sliceSize,directIoLib,currBufferIdx,nextRingIndex++); + std::cout<<"申请新buffer:"<setInUse(true); + global_free_size+=size; + nextEmptyBufferPoolEntry[currBufferIdx]=buffer_pool_entry; + return buffer_pool_entry; +} + +int BufferPool::getRingIndex(uint32_t colId) { + return ringBufferMap[currBufferIdx][colId]->getBufferPoolEntry()->getRingIndex(); +} + diff --git a/cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp b/cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp new file mode 100644 index 0000000000..6970d2a034 --- /dev/null +++ b/cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp @@ -0,0 +1,7 @@ +/* +/* @author whz +/* @create 7/30/25. +*/ +#include "physical/BufferPool/BufferPoolEntry.h" +#include "physical/BufferPool/Bitmap.h" + diff --git a/cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp b/cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp new file mode 100644 index 0000000000..723c425e25 --- /dev/null +++ b/cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp @@ -0,0 +1,113 @@ +/* +/* @author whz +/* @create 7/30/25. +*/ +#include "physical/BufferPool/BufferPoolEntry.h" +#include "physical/BufferPool/Bitmap.h" + +BufferPoolEntry::BufferPoolEntry(size_t size, int slice_size, std::shared_ptr direct_lib,int offset,int ring_index) + : size_(size), is_full_(false), next_free_(0),is_in_use_(false),offset_in_buffers_(offset),is_registered(false),ring_index(ring_index) { + + if (size == 0 || slice_size <= 0) { + throw std::invalid_argument("Invalid buffer size or slice size"); + } + if (!direct_lib) { + throw std::invalid_argument("DirectIoLib pointer cannot be null"); + } + + const int slice_count = static_cast(size + slice_size-1/ slice_size); + bitmap_ = std::make_shared(slice_count); + + buffer_ = direct_lib->allocateDirectBuffer(size_); + if (!buffer_) { + throw std::runtime_error("Failed to allocate direct buffer"); + } +} + +size_t BufferPoolEntry::getSize() const { + return size_; +} + +std::shared_ptr BufferPoolEntry::getBitmap() const { + return bitmap_; +} + + +std::shared_ptr BufferPoolEntry::getBuffer() const { + return buffer_; +} + + +bool BufferPoolEntry::isFull() const { + return is_full_; +} + + +int BufferPoolEntry::getNextFreeIndex() const { + return next_free_; +} + + +int BufferPoolEntry::setNextFreeIndex(int index) { + const int old_index = next_free_; + next_free_ = index; + is_full_ = (index > size_); + if (is_full_) { + return -1; + // the buffer is full + } + return old_index; +} + +uint64_t BufferPoolEntry::checkCol(uint32_t col) const { + return nr_bytes_.find(col)==nr_bytes_.end() ? -1 : nr_bytes_.at(col); +} + +void BufferPoolEntry::addCol(uint32_t colId,uint64_t bytes) { + nr_bytes_[colId] =bytes; +} + +bool BufferPoolEntry::isInUse() const { + return is_in_use_; +} + +void BufferPoolEntry::setInUse(bool in_use) { + is_in_use_ = in_use; +} + +int BufferPoolEntry::getOffsetInBuffers() const { + return offset_in_buffers_; +} + +void BufferPoolEntry::setOffsetInBuffers(int offset) { + offset_in_buffers_=offset; +} + +bool BufferPoolEntry::getIsRegistered() const { + return is_registered; +} + +void BufferPoolEntry::setIsRegistered(bool registered) { + is_registered=registered; +} + +int BufferPoolEntry::getRingIndex() const { + return ring_index; +} + +void BufferPoolEntry::setRingIndex(int ring_index) { + ring_index=ring_index; +} + +void BufferPoolEntry::reset() { + is_full_ = false; + next_free_=0; + if (bitmap_) { + bitmap_.reset(); + } + nr_bytes_.clear(); + is_in_use_ = false; +// reset direct buffer? +} + +BufferPoolEntry::~BufferPoolEntry() = default; \ No newline at end of file diff --git a/cpp/pixels-common/lib/physical/Request.cpp b/cpp/pixels-common/lib/physical/Request.cpp index 6ef2ee7d79..e904fff5fd 100644 --- a/cpp/pixels-common/lib/physical/Request.cpp +++ b/cpp/pixels-common/lib/physical/Request.cpp @@ -31,6 +31,7 @@ Request::Request(uint64_t queryId_, uint64_t start_, uint64_t length_, int64_t b start = start_; length = length_; this->bufferId = bufferId; + ring_index=0; } int Request::hashCode() diff --git a/cpp/pixels-common/lib/physical/RequestBatch.cpp b/cpp/pixels-common/lib/physical/RequestBatch.cpp index 92db4c5efc..7ecd8de66d 100644 --- a/cpp/pixels-common/lib/physical/RequestBatch.cpp +++ b/cpp/pixels-common/lib/physical/RequestBatch.cpp @@ -66,3 +66,11 @@ void RequestBatch::add(Request request) requests.push_back(request); size++; } + + +Request& RequestBatch::getRequest(int index) { + if (index < 0 || index >= size) { + throw std::out_of_range("RequestBatch::getRequest: index out of range"); + } + return requests[index]; +} \ No newline at end of file diff --git a/cpp/pixels-common/lib/physical/io/PhysicalLocalReader.cpp b/cpp/pixels-common/lib/physical/io/PhysicalLocalReader.cpp index 4b364ef859..86e60e7489 100644 --- a/cpp/pixels-common/lib/physical/io/PhysicalLocalReader.cpp +++ b/cpp/pixels-common/lib/physical/io/PhysicalLocalReader.cpp @@ -104,14 +104,30 @@ std::string PhysicalLocalReader::getName() } return path.substr(path.find_last_of('/') + 1); } +void PhysicalLocalReader::addRingIndex(int ring_index) { + ring_index_vector.insert(ring_index); +} + +std::unordered_set& PhysicalLocalReader::getRingIndexes() { + return ring_index_vector; +} -std::shared_ptr PhysicalLocalReader::readAsync(int length, std::shared_ptr buffer, int index) +std::unordered_map PhysicalLocalReader::getRingIndexCountMap() { + return ringIndexCountMap; +} + +void PhysicalLocalReader::setRingIndexCountMap(std::unordered_map ringIndexCount) { + //first clear + ringIndexCountMap.clear(); + ringIndexCountMap=ringIndexCount; +} +std::shared_ptr PhysicalLocalReader::readAsync(int length, std::shared_ptr buffer, int index,int ring_index,int start_offset) { numRequests++; if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "iouring") { auto directRaf = std::static_pointer_cast(raf); - return directRaf->readAsync(length, std::move(buffer), index); + return directRaf->readAsync(length, std::move(buffer), index,ring_index,start_offset); } else if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "aio") { @@ -124,13 +140,13 @@ std::shared_ptr PhysicalLocalReader::readAsync(int length, std::sha } -void PhysicalLocalReader::readAsyncSubmit(uint32_t size) +void PhysicalLocalReader::readAsyncSubmit(std::unordered_map sizes,std::unordered_set ring_index) { numRequests++; if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "iouring") { auto directRaf = std::static_pointer_cast(raf); - directRaf->readAsyncSubmit(size); + directRaf->readAsyncSubmit(sizes,ring_index); } else if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "aio") { @@ -142,13 +158,13 @@ void PhysicalLocalReader::readAsyncSubmit(uint32_t size) } } -void PhysicalLocalReader::readAsyncComplete(uint32_t size) +void PhysicalLocalReader::readAsyncComplete(std::unordered_map sizes,std::unordered_set ring_index) { numRequests++; if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "iouring") { auto directRaf = std::static_pointer_cast(raf); - directRaf->readAsyncComplete(size); + directRaf->readAsyncComplete(sizes,ring_index); } else if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "aio") { @@ -160,15 +176,19 @@ void PhysicalLocalReader::readAsyncComplete(uint32_t size) } } -void PhysicalLocalReader::readAsyncSubmitAndComplete(uint32_t size) + +// not use? +void PhysicalLocalReader::readAsyncSubmitAndComplete(uint32_t size,std::unordered_set ring_index) { numRequests++; if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "iouring") { auto directRaf = std::static_pointer_cast(raf); - directRaf->readAsyncSubmit(size); + std::unordered_map sizes; + sizes[0]=0; + directRaf->readAsyncSubmit(sizes,ring_index); ::TimeProfiler::Instance().Start("async wait"); - directRaf->readAsyncComplete(size); + directRaf->readAsyncComplete(sizes,ring_index); ::TimeProfiler::Instance().End("async wait"); } else if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "aio") diff --git a/cpp/pixels-common/lib/physical/natives/ByteBuffer.cpp b/cpp/pixels-common/lib/physical/natives/ByteBuffer.cpp index bb929e2d4f..0013f0b31b 100644 --- a/cpp/pixels-common/lib/physical/natives/ByteBuffer.cpp +++ b/cpp/pixels-common/lib/physical/natives/ByteBuffer.cpp @@ -66,6 +66,26 @@ ByteBuffer::ByteBuffer(ByteBuffer &bb, uint32_t startId, uint32_t length) name = ""; fromOtherBB = true; allocated_by_new = true; + fromSlice=false; +} +ByteBuffer::ByteBuffer(ByteBuffer &bb, uint32_t startId, uint32_t length,bool from_slice) +{ + assert(startId >= 0 && startId + length <= bb.size() && length > 0); + buf = bb.getPointer() + startId; + bufSize = length; + resetPosition(); + name = ""; + fromOtherBB = true; + allocated_by_new = true; + fromSlice=true; +} + +std::shared_ptr ByteBuffer::slice(uint32_t offset, uint32_t length) { + if (offset + length > this->bufSize) { + throw std::runtime_error("Slice range out of bounds"); + } + + return std::make_shared(*this, offset, length,true); } /** diff --git a/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp b/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp index 5fdcb2b0ba..d4ff5707ef 100644 --- a/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp +++ b/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp @@ -46,8 +46,9 @@ std::shared_ptr DirectIoLib::read(int fd, long fileOffset, // the file will be read from blockStart(fileOffset), and the first fileDelta bytes should be ignored. long fileOffsetAligned = blockStart(fileOffset); long toRead = blockEnd(fileOffset + length) - blockStart(fileOffset); - if (pread(fd, directBuffer->getPointer(), toRead, fileOffsetAligned) == -1) - { + ssize_t ret = pread(fd, directBuffer->getPointer(), toRead, fileOffsetAligned); + if (ret == -1) { + perror("pread failed"); throw InvalidArgumentException("DirectIoLib::read: pread fail. "); } auto bb = std::make_shared(*directBuffer, diff --git a/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp b/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp index 70e22fbe37..22966390fe 100644 --- a/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp +++ b/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp @@ -24,11 +24,22 @@ */ #include "physical/natives/DirectUringRandomAccessFile.h" -thread_local struct io_uring *DirectUringRandomAccessFile::ring = nullptr; +#include + + +// global +std::mutex DirectUringRandomAccessFile::mutex_; +// thread local thread_local bool DirectUringRandomAccessFile::isRegistered = false; -thread_local struct iovec *DirectUringRandomAccessFile::iovecs = nullptr; -thread_local uint32_t -DirectUringRandomAccessFile::iovecSize = 0; +thread_local std::vector DirectUringRandomAccessFile::ring_vector; +thread_local std::vector DirectUringRandomAccessFile::iovecs_vector; +thread_local std::vector DirectUringRandomAccessFile::offsets_vector; +thread_local uint32_t DirectUringRandomAccessFile::iovecSize = 0; + + +// static thread_local int memset_count = 0; +// static thread_local std::chrono::nanoseconds total_time0(0); // 总耗时(纳秒) +// static thread_local std::chrono::nanoseconds total_time(0); // memest总耗时(纳秒) DirectUringRandomAccessFile::DirectUringRandomAccessFile(const std::string &file) : DirectRandomAccessFile(file) { @@ -37,16 +48,20 @@ DirectUringRandomAccessFile::DirectUringRandomAccessFile(const std::string &file void DirectUringRandomAccessFile::RegisterBufferFromPool(std::vector colIds) { - std::vector > tmpBuffers; - if (!isRegistered) - { - for (auto buffer: ::BufferPool::buffers) - { - for (auto colId: colIds) - { - tmpBuffers.emplace_back(buffer[colId]); + // std::cout << "函数:RegisterBufferFromPool 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; + // std::lock_guard lock(mutex_); + if (!isRegistered) { + + std::vector > tmpBuffers; + auto ring=DirectUringRandomAccessFile::getRing(0); + struct iovec* iovecs=nullptr; + for (auto buffers : ::BufferPool::registeredBuffers){ + for (auto buffer:buffers) { + buffer->setIsRegistered(true); + tmpBuffers.emplace_back(buffer->getBuffer()); } } + // auto start0 = std::chrono::high_resolution_clock::now(); iovecs = (iovec *) calloc(tmpBuffers.size(), sizeof(struct iovec)); iovecSize = tmpBuffers.size(); for (auto i = 0; i < tmpBuffers.size(); i++) @@ -54,22 +69,46 @@ void DirectUringRandomAccessFile::RegisterBufferFromPool(std::vector auto buffer = tmpBuffers.at(i); iovecs[i].iov_base = buffer->getPointer(); iovecs[i].iov_len = buffer->size(); - memset(iovecs[i].iov_base, 0, buffer->size()); + // auto start = std::chrono::high_resolution_clock::now(); + + // 目标代码 + // memset(iovecs[i].iov_base, 0, buffer->size()); + + // 记录结束时间,累加耗时 + // auto end = std::chrono::high_resolution_clock::now(); + // total_time += std::chrono::duration_cast(end - start); + // total_time0+=std::chrono::duration_cast(end-start0); + // 累加次数 + // memset_count++; } + // std::cout << "memset执行次数:" << memset_count << std::endl; + // 转换为易读单位(如微秒/毫秒) + // auto total_us0 = std::chrono::duration_cast(total_time0).count(); + // auto total_us = std::chrono::duration_cast(total_time).count(); + // std::cout << "总用时:" << total_us0 << " 微秒" << std::endl; + // std::cout << "memset用时:" << total_us << " 微秒" << std::endl; + iovecs_vector.emplace_back(iovecs); int ret = io_uring_register_buffers(ring, iovecs, iovecSize); + // std::cout<<"ret:"<> buffers) { + + // std::cout << "函数:RegisterBuffer 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; if (!isRegistered) { + auto ring=DirectUringRandomAccessFile::getRing(0); + struct iovec* iovecs=nullptr; iovecs = (iovec *) calloc(buffers.size(), sizeof(struct iovec)); iovecSize = buffers.size(); for (auto i = 0; i < buffers.size(); i++) @@ -79,6 +118,8 @@ void DirectUringRandomAccessFile::RegisterBuffer(std::vector size(); memset(iovecs[i].iov_base, 0, buffer->size()); } + ring_vector.emplace_back(ring); + iovecs_vector.emplace_back(iovecs); int ret = io_uring_register_buffers(ring, iovecs, iovecSize); if (ret != 0) { @@ -86,42 +127,86 @@ void DirectUringRandomAccessFile::RegisterBuffer(std::vector lock(mutex_); + // std::cout << "函数:Initialize 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; + getRing(0); + // std::cout << "函数:Initialze 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; + + // // initialize io_uring ring + // struct io_uring *ring=nullptr; + // if (getRing(0) == nullptr) + // { + // ring = new io_uring(); + // if (io_uring_queue_init(4096, ring, 0) < 0) + // { + // throw InvalidArgumentException("DirectRandomAccessFile: initialize io_uring fails."); + // } + // } + // ring_vector.emplace_back(ring); + // offsets_vector.emplace_back(0); } void DirectUringRandomAccessFile::Reset() { + // std::cout << "函数:Reset 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; // Important! Because sometimes ring is nullptr here. // For example, two threads A and B share the same global state. If A finish all files while B just starts, // B would execute Reset function from InitLocal. If we don't set this 'if' branch, ring would be double freed. - if (ring != nullptr) + for (auto ring :ring_vector) { + if (ring != nullptr) + { + // We don't use this function anymore since it slows down the speed + // if(io_uring_unregister_buffers(ring) != 0) { + // throw InvalidArgumentException("DirectUringRandomAccessFile::UnregisterBuffer: unregister buffer fails. "); + // } + io_uring_queue_exit(ring); + delete (ring); + ring = nullptr; + isRegistered = false; + } + } + ring_vector.clear(); + for (auto iovecs:iovecs_vector) { + if (iovecs != nullptr) + { + free(iovecs); + iovecs = nullptr; + } + } + iovecs_vector.clear(); + std::cout << "函数:Reset 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; +} + +bool DirectUringRandomAccessFile::RegisterMoreBuffer(int index,std::vector> buffers) { + // std::lock_guard lock(mutex_); + std::cout << "函数:RegisterMoreBuffer 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; + assert(isRegistered); + std::cout<<"现在有"<getPointer(); + iovecs[i].iov_len = buffer->size(); + // memset(iovecs[i].iov_base, 0, buffer->size()); } - if (iovecs != nullptr) + iovecs_vector.emplace_back(iovecs); + int ret = io_uring_register_buffers(ring, iovecs, iovecSize); + if (ret != 0) { - free(iovecs); - iovecs = nullptr; + throw InvalidArgumentException("DirectUringRandomAccessFile::RegisterMoreBuffer: register buffer fails. "); } + // std::cout << "函数:RegisterMoreBuffer 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; + return true; + } DirectUringRandomAccessFile::~DirectUringRandomAccessFile() @@ -130,22 +215,72 @@ DirectUringRandomAccessFile::~DirectUringRandomAccessFile() } std::shared_ptr -DirectUringRandomAccessFile::readAsync(int length, std::shared_ptr buffer, int index) +DirectUringRandomAccessFile::readAsync(int length, std::shared_ptr buffer, int index,int ring_index,int start_offset) { + // std::cout << "函数:readAsync 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; + // std::lock_guard lock(mutex_); + auto ring=DirectUringRandomAccessFile::getRing(ring_index); + auto offset = start_offset; + if (enableDirect) { struct io_uring_sqe *sqe = io_uring_get_sqe(ring); + if (!sqe) { + throw std::runtime_error("DirectUringRandomAccessFile::readAsync: failed to get SQE, submission queue is full"); + } // if(length > iovecs[index].iov_len) { // throw InvalidArgumentException("DirectUringRandomAccessFile::readAsync: the length is larger than buffer length."); // } // the file will be read from blockStart(fileOffset), and the first fileDelta bytes should be ignored. uint64_t fileOffsetAligned = directIoLib->blockStart(offset); uint64_t toRead = directIoLib->blockEnd(offset + length) - directIoLib->blockStart(offset); + // 检查缓冲区大小是否足够 + size_t block_size = directIoLib->getBlockSize(); // 假设存在获取块大小的方法 + size_t required_buffer_size = (offset - fileOffsetAligned) + length; + if (buffer->size() < required_buffer_size) { + std::stringstream ss; + std::cout << "DirectUringRandomAccessFile::readAsync: buffer size insufficient. " + << "Required: " << required_buffer_size << ", Actual: " << buffer->size() + << ", ring_index: " << ring_index << ", index: " << index + << "Required alignment: " << block_size + <<", length:"<(buffer->getPointer()); + if (buffer_addr % block_size != 0) { + std::stringstream ss; + ss << "DirectUringRandomAccessFile::readAsync: buffer misaligned. " + << "Required alignment: " << block_size << " bytes, " + << "Actual address: 0x" << std::hex << buffer_addr + << ", ring_index: " << ring_index << ", index: " << index + <<", length:"<getPointer(), toRead, fileOffsetAligned, index); + if (fd < 0) { + throw std::runtime_error("DirectUringRandomAccessFile::readAsync: invalid file descriptor"); + } + // if (buffer->size()(buffer->getPointer())<<" index:"<size()<<" startID:"<(*buffer, offset - fileOffsetAligned, length); - seek(offset + length); + seekByIndex(offset + length,ring_index); + // if (ring_index!=0) { + // // std::cout<<"notice uring File index:"<(buffer->getPointer())<<" start_offset"<(buffer->getPointer())<<" index:"<(bb->getPointer())<< + // " buffer size:"<size()<<" startID:"<(buffer->getPointer())<<" index:"<(bb->getPointer())< } } +void DirectUringRandomAccessFile::seekByIndex(long off,int index) { + // 检查索引有效性 + if (index < 0 || static_cast(index) >= offsets_vector.size()) { + std::stringstream ss; + ss << "DirectUringRandomAccessFile::seekByIndex: invalid index. " + << "Index: " << index << ", Vector size: " << offsets_vector.size(); + throw InvalidArgumentException(ss.str()); + } + // 检查偏移量有效性(假设偏移量不能为负) + if (off < 0) { + std::stringstream ss; + ss << "DirectUringRandomAccessFile::seekByIndex: invalid offset. " + << "Offset: " << off << ", Index: " << index; + throw InvalidArgumentException(ss.str()); + } + offsets_vector.at(index)=off; +} -void DirectUringRandomAccessFile::readAsyncSubmit(int size) + +void DirectUringRandomAccessFile::readAsyncSubmit(std::unordered_map sizes,std::unordered_set ring_index) { - int ret = io_uring_submit(ring); - if (ret != size) - { - throw InvalidArgumentException("DirectUringRandomAccessFile::readAsyncSubmit: submit fails"); + // std::lock_guard lock(mutex_); + // std::cout << "函数:readAsyncSubmit 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; + for (auto i:ring_index) { + auto ring=DirectUringRandomAccessFile::getRing(i); + int ret = io_uring_submit(ring); + // std::cout<<"线程:"< sizes,std::unordered_set ring_index) { + // std::lock_guard lock(mutex_); + // std::cout << "函数:readAsyncComplete 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; // Important! We cannot write the code as io_uring_wait_cqe_nr(ring, &cqe, iovecSize). // The reason is unclear, but some random bugs would happen. It takes me nearly a week to find this bug - struct io_uring_cqe *cqe; - for (int i = 0; i < size; i++) - { - if (io_uring_wait_cqe_nr(ring, &cqe, 1) != 0) + for (auto idx : ring_index) { // 重命名外层循环变量,避免与内层冲突 + struct io_uring_cqe *cqe; + auto ring = DirectUringRandomAccessFile::getRing(idx); + + // 为当前ring设置超时时间(根据实际场景调整,这里设为5秒) + struct __kernel_timespec timeout = { + .tv_sec = 1, // 秒 + .tv_nsec = 0 // 纳秒 + }; + + // 内层循环使用j作为变量,避免与外层idx冲突 + for (int j = 0; j < sizes[idx]; j++) // 注意:这里使用外层的idx作为sizes的键 { - throw InvalidArgumentException("DirectUringRandomAccessFile::readAsyncComplete: wait cqe fails"); + + if (io_uring_wait_cqe_nr(ring, &cqe, 1) != 0) + { + throw InvalidArgumentException("DirectUringRandomAccessFile::readAsyncComplete: wait cqe fails"); + } + io_uring_cqe_seen(ring, cqe); + // 使用带超时的等待函数,替代原有的io_uring_wait_cqe_nr + // int ret = io_uring_wait_cqe_timeout(ring, &cqe, &timeout); + // if (ret != 0) { + // if (ret == -ETIME) { + // // 超时错误:明确抛出超时异常,便于上层处理 + // throw InvalidArgumentException( + // "DirectUringRandomAccessFile::readAsyncComplete: " + // "wait cqe timeout for ring index " + std::to_string(idx) + + // " after " + std::to_string(timeout.tv_sec) + " seconds" + // ); + // } else if (ret == -EINTR) { + // // 信号中断:可根据需求选择重试或抛出异常 + // throw InvalidArgumentException( + // "DirectUringRandomAccessFile::readAsyncComplete: " + // "wait cqe interrupted by signal for ring index " + std::to_string(idx) + // ); + // } else { + // // 其他错误(如参数错误、ring已关闭等) + // throw InvalidArgumentException( + // "DirectUringRandomAccessFile::readAsyncComplete: " + // "wait cqe fails with error " + std::to_string(ret) + + // " for ring index " + std::to_string(idx) + // ); + // } + // } + // + // // 检查CQE结果是否正常(可选,根据业务需求) + // if (cqe->res < 0) { + // throw InvalidArgumentException( + // "DirectUringRandomAccessFile::readAsyncComplete: " + // "I/O operation failed with error " + std::to_string(cqe->res) + + // " for ring index " + std::to_string(idx) + // ); + // } + // + // io_uring_cqe_seen(ring, cqe); } - io_uring_cqe_seen(ring, cqe); } + // std::cout << "函数:readAsyncComplete 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; } +struct io_uring *DirectUringRandomAccessFile::getRing(int index) { + if (index>=ring_vector.size()) { + // need to add more rings + // initialize io_uring ring + struct io_uring *ring=nullptr; + if (ring == nullptr) + { + ring = new io_uring(); + if (io_uring_queue_init(4096, ring, 0) < 0) + { + throw InvalidArgumentException("DirectRandomAccessFile: initialize io_uring fails."); + } + } + ring_vector.emplace_back(ring); + offsets_vector.emplace_back(0); + } + return ring_vector[index]; +} diff --git a/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp b/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp index 4af3bfce9e..21314b571a 100644 --- a/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp +++ b/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp @@ -25,6 +25,7 @@ #include "physical/scheduler/NoopScheduler.h" #include "exception/InvalidArgumentException.h" #include "physical/io/PhysicalLocalReader.h" +#include Scheduler *NoopScheduler::instance = nullptr; @@ -48,6 +49,8 @@ std::vector > NoopScheduler::executeBatch(std::shared_ptr reader, RequestBatch batch, std::vector > reuseBuffers, long queryId) { + + std::lock_guard lock(mtx); // 进入临界区,自动加锁 auto requests = batch.getRequests(); std::vector > results; results.resize(batch.getSize()); @@ -55,13 +58,32 @@ NoopScheduler::executeBatch(std::shared_ptr reader, RequestBatc { // async read auto localReader = std::static_pointer_cast(reader); + std::unordered_set ring_index_set=localReader->getRingIndexes(); + std::unordered_map ringIndexCountMap; + for (int i = 0; i < batch.getSize(); i++) { Request request = requests[i]; - localReader->seek(request.start); - results.at(i) = localReader->readAsync(request.length, reuseBuffers.at(i), request.bufferId); + // localReader->seek(request.start); + if (request.length>reuseBuffers.at(i)->size()) { + throw InvalidArgumentException("说明出错的不是这个,需要关注之前的临界区\n"); + + } + results.at(i) = localReader->readAsync(request.length, reuseBuffers.at(i), request.bufferId,request.ring_index,request.start); + // if (request.ring_index !=0) { + // std::cout<<"notice readAsync"<size()<addRingIndex(request.ring_index); + } + ringIndexCountMap[request.ring_index]++; } - localReader->readAsyncSubmit(batch.getSize()); + localReader->readAsyncSubmit(ringIndexCountMap,ring_index_set); + localReader->setRingIndexCountMap(ringIndexCountMap); + } else { diff --git a/cpp/pixels-common/lib/utils/MutexTracker.cpp b/cpp/pixels-common/lib/utils/MutexTracker.cpp new file mode 100644 index 0000000000..6e26ec9fac --- /dev/null +++ b/cpp/pixels-common/lib/utils/MutexTracker.cpp @@ -0,0 +1,5 @@ +/* +/* @author whz +/* @create 8/23/25. +*/ +#include "utils/MutexTracker.h" \ No newline at end of file diff --git a/cpp/pixels-core/include/reader/PixelsReaderOption.h b/cpp/pixels-core/include/reader/PixelsReaderOption.h index a43200694b..9a051fd5ec 100644 --- a/cpp/pixels-core/include/reader/PixelsReaderOption.h +++ b/cpp/pixels-core/include/reader/PixelsReaderOption.h @@ -57,6 +57,8 @@ class PixelsReaderOption void setFilter(duckdb::TableFilterSet *filter); + void setRingIndex(int ringIndex); + duckdb::TableFilterSet *getFilter(); int getRGStart(); @@ -65,6 +67,8 @@ class PixelsReaderOption int getBatchSize() const; + int getRingIndex() const; + void setTolerantSchemaEvolution(bool t); bool isTolerantSchemaEvolution(); @@ -85,5 +89,6 @@ class PixelsReaderOption int batchSize; int rgStart; int rgLen; + int ring_index; }; #endif //PIXELS_PIXELSREADEROPTION_H diff --git a/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h b/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h index 190663f574..4ee9f2c99b 100644 --- a/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h +++ b/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h @@ -98,6 +98,8 @@ class PixelsRecordReaderImpl : public PixelsRecordReader void UpdateRowGroupInfo(); + + static std::mutex mutex_; std::shared_ptr physicalReader; pixels::proto::Footer footer; pixels::proto::PostScript postScript; @@ -105,6 +107,7 @@ class PixelsRecordReaderImpl : public PixelsRecordReader PixelsReaderOption option; duckdb::TableFilterSet *filter; long queryId; + int ring_index; int RGStart; int RGLen; bool everRead; diff --git a/cpp/pixels-core/include/vector/ColumnVector.h b/cpp/pixels-core/include/vector/ColumnVector.h index d2747311fc..f4f322ada4 100644 --- a/cpp/pixels-core/include/vector/ColumnVector.h +++ b/cpp/pixels-core/include/vector/ColumnVector.h @@ -85,6 +85,7 @@ class ColumnVector uint64_t *isValid; explicit ColumnVector(uint64_t len, bool encoding); + ~ColumnVector(); idx_t getCapacity() const; diff --git a/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp b/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp index dabcd702a9..f3c3784540 100644 --- a/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp +++ b/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp @@ -67,6 +67,15 @@ long PixelsReaderOption::getQueryId() return queryId; } +int PixelsReaderOption::getRingIndex() const { + return ring_index; +} + +void PixelsReaderOption::setRingIndex(int i) { + ring_index = i; +} + + void PixelsReaderOption::setRGRange(int start, int len) { rgStart = start; diff --git a/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp b/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp index 02f6fc1cf1..9d6322b9fc 100644 --- a/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp +++ b/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp @@ -25,7 +25,7 @@ #include "reader/PixelsRecordReaderImpl.h" #include "physical/io/PhysicalLocalReader.h" #include "profiler/CountProfiler.h" - +std::mutex PixelsRecordReaderImpl::mutex_; PixelsRecordReaderImpl::PixelsRecordReaderImpl(std::shared_ptr reader, const pixels::proto::PostScript &pixelsPostScript, const pixels::proto::Footer &pixelsFooter, @@ -37,7 +37,7 @@ PixelsRecordReaderImpl::PixelsRecordReaderImpl(std::shared_ptr postScript = pixelsPostScript; footerCache = pixelsFooterCache; option = opt; - // TODO: intialize all kinds of variables + // TODO: intialize all kinds of variable queryId = option.getQueryId(); RGStart = option.getRGStart(); RGLen = option.getRGLen(); @@ -361,7 +361,7 @@ void PixelsRecordReaderImpl::prepareRead() uint64_t footerOffset = rowGroupInformation.footeroffset(); uint64_t footerLength = rowGroupInformation.footerlength(); fis.push_back(i); - requestBatch.add(queryId, (int) footerOffset, (int) footerLength); + requestBatch.add(queryId, (int) footerOffset, (int) footerLength,ring_index); rowGroupFooterCacheHit.at(i) = false; } } @@ -401,7 +401,8 @@ void PixelsRecordReaderImpl::asyncReadComplete(int requestSize) if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "iouring") { auto localReader = std::static_pointer_cast(physicalReader); - localReader->readAsyncComplete(requestSize); + auto ringIndexCountMap=localReader->getRingIndexCountMap(); + localReader->readAsyncComplete(ringIndexCountMap,localReader->getRingIndexes()); asyncReadRequestNum -= requestSize; } else if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "aio") @@ -456,6 +457,7 @@ bool PixelsRecordReaderImpl::read() if (!diskChunks.empty()) { + // std::lock_guard lock(mutex_); RequestBatch requestBatch((int) diskChunks.size()); Scheduler *scheduler = SchedulerFactory::Instance()->getScheduler(); std::vector colIds; @@ -463,20 +465,48 @@ bool PixelsRecordReaderImpl::read() for (int i = 0; i < diskChunks.size(); i++) { ChunkId chunk = diskChunks.at(i); - requestBatch.add(queryId, chunk.offset, (int) chunk.length, ::BufferPool::GetBufferId(i)); + requestBatch.add(queryId, chunk.offset, (int) chunk.length, ::BufferPool::GetBufferId()); colIds.emplace_back(chunk.columnId); bytes.emplace_back(chunk.length); } - ::BufferPool::Initialize(colIds, bytes, fileSchema->getFieldNames()); + + // std::cout<<"开始 现在是BufferID:"<<::BufferPool::GetBufferId()<getFieldNames(); + ::BufferPool::Initialize(colIds, bytes, columnNames); + // std::cout<<"初始化bufferpool完毕?"<> originalByteBuffers; + // std::cout<<"初始化完毕?"< ring_col; for (int i = 0; i < colIds.size(); i++) { auto colId = colIds.at(i); - originalByteBuffers.emplace_back(::BufferPool::GetBuffer(colId)); + auto byte=bytes.at(i); + auto currentBufferEntry=::BufferPool::GetBuffer(colId,byte,columnNames[colId]); + originalByteBuffers.emplace_back(currentBufferEntry); + requestBatch.getRequest(i).ring_index=::BufferPool::getRingIndex(colId); + if (currentBufferEntry->size()-requestBatch.getRequest(i).length<=4096) { + std::cout<<"i:"<size()<< + " requestBatch.length"<(originalByteBuffers.at(i)->getPointer())<executeBatch( + // ::BufferPool::PrintStats(); + + auto byteBuffers = scheduler->executeBatch( physicalReader, requestBatch, originalByteBuffers, queryId); if(ConfigFactory::Instance().boolCheckProperty("localfs.enable.async.io") @@ -490,6 +520,10 @@ bool PixelsRecordReaderImpl::read() ChunkId chunk = diskChunks.at(index); std::shared_ptr bb = byteBuffers.at(index); uint32_t colId = chunk.columnId; + + // if (std::find(ring_col.begin(), ring_col.end(), colId) != ring_col.end()) + // std::cout<<"colId "<(bb->getPointer())< input, input->skipBytes(inputLength - sizeof(int)); int startsOffset = input->getInt(); input->resetReaderIndex(); + // std::cout<<"input getBuffer:"<(input->getPointer())<<" startOffset:" + // <(*input, 0, startsOffset); startsBuf = std::make_shared( *input, startsOffset, inputLength - sizeof(int) - startsOffset); nextStart = startsBuf->getInt(); // read out the first start offset, which is 0 + // assert(inputLength>0 && startsOffset>=0); + if (startsOffset>inputLength||startsOffset<=0) { + // std::cout<<"debug"<(&isValid), 64, ceil(1.0 * len / 64) * sizeof(uint64_t)); } +ColumnVector::~ColumnVector() +{ +} + void ColumnVector::close() { if (!closed) @@ -50,6 +54,10 @@ void ColumnVector::close() free(isValid); isValid = nullptr; } + if (isNull != nullptr) + { + isNull = nullptr; + } } } diff --git a/cpp/pixels-duckdb/PixelsScanFunction.cpp b/cpp/pixels-duckdb/PixelsScanFunction.cpp index c8aec8b7ef..869078154f 100644 --- a/cpp/pixels-duckdb/PixelsScanFunction.cpp +++ b/cpp/pixels-duckdb/PixelsScanFunction.cpp @@ -235,6 +235,10 @@ unique_ptr PixelsScanFunction::PixelsScanInitGlobal( result->max_threads = max_threads; + result->active_threads=max_threads; + + result->all_done=false; + result->batch_index = 0; result->filters = input.filters.get(); @@ -469,19 +473,23 @@ bool PixelsScanFunction::PixelsParallelStateNext(ClientContext &context, const P parallel_state.file_index.at(scan_data.deviceID) >= StorageInstance->getFileSum(scan_data.deviceID)) || scan_data.next_file_index >= StorageInstance->getFileSum(scan_data.deviceID)) { - ::BufferPool::Reset(); - // if async io is enabled, we need to unregister uring buffer - if (ConfigFactory::Instance().boolCheckProperty("localfs.enable.async.io")) + int remaining_threads = --parallel_state.active_threads; + if (remaining_threads==0&&!parallel_state.all_done) { + ::BufferPool::Reset(); + // if async io is enabled, we need to unregister uring buffer + if (ConfigFactory::Instance().boolCheckProperty("localfs.enable.async.io")) { - if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "iouring") + if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "iouring") { - ::DirectUringRandomAccessFile::Reset(); + ::DirectUringRandomAccessFile::Reset(); } else if (ConfigFactory::Instance().getProperty("localfs.async.lib") == "aio") { - throw InvalidArgumentException( - "PhysicalLocalReader::readAsync: We don't support aio for our async read yet."); + throw InvalidArgumentException( + "PhysicalLocalReader::readAsync: We don't support aio for our async read yet."); } } + parallel_state.all_done = true; // 标记已完成,避免重复调用 + } parallel_lock.unlock(); return false; } @@ -513,20 +521,20 @@ bool PixelsScanFunction::PixelsParallelStateNext(ClientContext &context, const P } if (scan_data.next_file_index < StorageInstance->getFileSum(scan_data.deviceID)) { - auto footerCache = std::make_shared(); - auto builder = std::make_shared(); - std::shared_ptr<::Storage> storage = StorageFactory::getInstance()->getStorage(::Storage::file); - scan_data.next_file_name = StorageInstance->getFileName(scan_data.deviceID, scan_data.next_file_index); - scan_data.nextReader = builder->setPath(scan_data.next_file_name) - ->setStorage(storage) - ->setPixelsFooterCache(footerCache) - ->build(); - - PixelsReaderOption option = GetPixelsReaderOption(scan_data, parallel_state); - scan_data.nextPixelsRecordReader = scan_data.nextReader->read(option); - auto nextPixelsRecordReader = std::static_pointer_cast( - scan_data.nextPixelsRecordReader); - nextPixelsRecordReader->read(); + auto footerCache = std::make_shared(); + auto builder = std::make_shared(); + std::shared_ptr<::Storage> storage = StorageFactory::getInstance()->getStorage(::Storage::file); + scan_data.next_file_name = StorageInstance->getFileName(scan_data.deviceID, scan_data.next_file_index); + scan_data.nextReader = builder->setPath(scan_data.next_file_name) + ->setStorage(storage) + ->setPixelsFooterCache(footerCache) + ->build(); + + PixelsReaderOption option = GetPixelsReaderOption(scan_data, parallel_state); + scan_data.nextPixelsRecordReader = scan_data.nextReader->read(option); + auto nextPixelsRecordReader = std::static_pointer_cast( + scan_data.nextPixelsRecordReader); + nextPixelsRecordReader->read(); } else { scan_data.nextReader = nullptr; diff --git a/cpp/pixels-duckdb/duckdb b/cpp/pixels-duckdb/duckdb index c3dc6d34c9..378f77c5c9 160000 --- a/cpp/pixels-duckdb/duckdb +++ b/cpp/pixels-duckdb/duckdb @@ -1 +1 @@ -Subproject commit c3dc6d34c905bc44f311bf670b1bbddef1c0c776 +Subproject commit 378f77c5c93f3e66453bbc9f829b2278527fbd56 From ad03785126569f61ac547fc17f979cd47414c67e Mon Sep 17 00:00:00 2001 From: whzruc Date: Mon, 15 Sep 2025 12:31:47 +0800 Subject: [PATCH 2/2] implementation of huge page Add io_uring mode selection (IOPOLL requires further code changes) update subgitmodule --- cpp/CMakeLists.txt | 2 +- cpp/include/PixelsReadBindData.hpp | 2 +- cpp/include/PixelsReadGlobalState.hpp | 4 +- cpp/include/PixelsScanFunction.hpp | 2 +- cpp/perf.sh | 74 ++++ .../include/physical/BufferPool.h | 256 ++++++------ .../include/physical/BufferPool/Bitmap.h | 94 +++-- .../physical/BufferPool/BufferPoolEntry.h | 95 +++-- .../include/physical/Scheduler.h | 1 - .../include/physical/natives/DirectIoLib.h | 35 +- .../natives/DirectUringRandomAccessFile.h | 43 +- .../include/utils/MutexTracker.h | 70 ++-- cpp/pixels-common/lib/physical/BufferPool.cpp | 357 +++++++++-------- .../lib/physical/BufferPool/Bitmap.cpp | 28 +- .../physical/BufferPool/BufferPoolEntry.cpp | 180 +++++---- cpp/pixels-common/lib/physical/Request.cpp | 25 +- .../lib/physical/SchedulerFactory.cpp | 6 +- .../lib/physical/natives/DirectIoLib.cpp | 110 +++++- .../natives/DirectRandomAccessFile.cpp | 4 +- .../natives/DirectUringRandomAccessFile.cpp | 370 +++++++----------- .../lib/physical/scheduler/NoopScheduler.cpp | 48 +-- cpp/pixels-common/lib/utils/MutexTracker.cpp | 26 +- .../include/reader/PixelsRecordReaderImpl.h | 1 - .../lib/reader/PixelsReaderOption.cpp | 1 - .../lib/reader/PixelsRecordReaderImpl.cpp | 10 - .../lib/reader/StringColumnReader.cpp | 7 - cpp/pixels-cpp.properties | 35 +- cpp/pixels-duckdb/PixelsScanFunction.cpp | 5 +- cpp/pixels-duckdb/duckdb | 2 +- cpp/pixels-duckdb/pixels_extension.cpp | 108 +++-- cpp/process_sqls.py | 254 ++++++++++++ 31 files changed, 1390 insertions(+), 865 deletions(-) create mode 100755 cpp/perf.sh create mode 100755 cpp/process_sqls.py diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 05196c99b6..e2a9f72867 100755 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -20,7 +20,7 @@ include_directories(${CMAKE_CURRENT_BINARY_DIR}) add_subdirectory(pixels-common) add_subdirectory(pixels-core) -#add_subdirectory(pixels-cli) +add_subdirectory(pixels-cli) add_subdirectory(third-party/googletest) add_subdirectory(tests) diff --git a/cpp/include/PixelsReadBindData.hpp b/cpp/include/PixelsReadBindData.hpp index 06de4b7969..14254c625f 100644 --- a/cpp/include/PixelsReadBindData.hpp +++ b/cpp/include/PixelsReadBindData.hpp @@ -42,7 +42,7 @@ namespace duckdb std::shared_ptr initialPixelsReader; std::shared_ptr fileSchema; vector files; - atomic curFileId; + atomic curFileId; }; } diff --git a/cpp/include/PixelsReadGlobalState.hpp b/cpp/include/PixelsReadGlobalState.hpp index 901fef255d..9747b2bddc 100644 --- a/cpp/include/PixelsReadGlobalState.hpp +++ b/cpp/include/PixelsReadGlobalState.hpp @@ -40,8 +40,8 @@ namespace duckdb { mutex lock; - atomic active_threads; // 活跃线程数 - atomic all_done; // 是否所有线程都已完成 + atomic active_threads; // Number of active threads + atomic all_done; // Whether all threads have completed //! The initial reader from the bind phase std::shared_ptr initialPixelsReader; diff --git a/cpp/include/PixelsScanFunction.hpp b/cpp/include/PixelsScanFunction.hpp index 297500a068..fb29fc6a1e 100644 --- a/cpp/include/PixelsScanFunction.hpp +++ b/cpp/include/PixelsScanFunction.hpp @@ -113,7 +113,7 @@ namespace duckdb PixelsScanInitLocal(ExecutionContext &context, TableFunctionInitInput &input, GlobalTableFunctionState *gstate_p); - static bool PixelsParallelStateNext(ClientContext &context, const PixelsReadBindData &bind_data, + static bool PixelsParallelStateNext(ClientContext &context, PixelsReadBindData &bind_data, PixelsReadLocalState &scan_data, PixelsReadGlobalState ¶llel_state, bool is_init_state = false); diff --git a/cpp/perf.sh b/cpp/perf.sh new file mode 100755 index 0000000000..ba1eea671f --- /dev/null +++ b/cpp/perf.sh @@ -0,0 +1,74 @@ +#!/bin/bash + +# Check number of arguments +if [ $# -ne 2 ]; then + echo "Usage: $0 " + echo "Example: $0 test_q01.sql query_1" + exit 1 +fi + +# Assign parameters +QUERY_FILE="$1" +OUTPUT_PREFIX="$2" + +# Record start time (nanoseconds) +START_TIME=$(date +%s%N) +PREVIOUS_TIME=$START_TIME + +# Function to display time duration +show_time() { + local start=$1 + local end=$2 + local stage=$3 + + # Calculate duration in milliseconds (integer arithmetic in bash) + local duration=$(( (end - start) / 1000000 )) + echo "Stage '$stage' duration: ${duration}ms" +} + +echo "Starting performance analysis..." + +# Run perf recording +echo "1. Running perf record..." +sudo -E perf record -F 1 --call-graph=dwarf -g ./build/release/duckdb < "$QUERY_FILE" +CURRENT_TIME=$(date +%s%N) +show_time $PREVIOUS_TIME $CURRENT_TIME "Running perf record" +PREVIOUS_TIME=$CURRENT_TIME + +# Generate perf script output +echo "2. Generating perf script output..." +sudo perf script -i perf.data > "${OUTPUT_PREFIX}.perf" +CURRENT_TIME=$(date +%s%N) +show_time $PREVIOUS_TIME $CURRENT_TIME "Generating perf script output" +PREVIOUS_TIME=$CURRENT_TIME + +# Collapse call stacks +echo "3. Collapsing call stacks..." +stackcollapse-perf.pl "${OUTPUT_PREFIX}.perf" > "${OUTPUT_PREFIX}.folded" +CURRENT_TIME=$(date +%s%N) +show_time $PREVIOUS_TIME $CURRENT_TIME "Collapsing call stacks" +PREVIOUS_TIME=$CURRENT_TIME + +# Generate flame graph +echo "4. Generating flame graph..." +flamegraph.pl "${OUTPUT_PREFIX}.folded" > "${OUTPUT_PREFIX}-cpu.svg" +CURRENT_TIME=$(date +%s%N) +show_time $PREVIOUS_TIME $CURRENT_TIME "Generating flame graph" +PREVIOUS_TIME=$CURRENT_TIME + +# Rename perf data file +echo "5. Renaming perf data file..." +mv perf.data "${OUTPUT_PREFIX}-perf.data" +CURRENT_TIME=$(date +%s%N) +show_time $PREVIOUS_TIME $CURRENT_TIME "Renaming files" +PREVIOUS_TIME=$CURRENT_TIME + +# Calculate total duration +TOTAL_DURATION=$(( (CURRENT_TIME - START_TIME) / 1000000 )) +echo "Total duration: ${TOTAL_DURATION}ms" + +echo "Operation completed. Generated files:" +echo "${OUTPUT_PREFIX}.perf" +echo "${OUTPUT_PREFIX}.folded" +echo "${OUTPUT_PREFIX}-cpu.svg" +echo "${OUTPUT_PREFIX}-perf.data" \ No newline at end of file diff --git a/cpp/pixels-common/include/physical/BufferPool.h b/cpp/pixels-common/include/physical/BufferPool.h index f07003efd8..2e5627dcb5 100644 --- a/cpp/pixels-common/include/physical/BufferPool.h +++ b/cpp/pixels-common/include/physical/BufferPool.h @@ -25,105 +25,111 @@ #ifndef DUCKDB_BUFFERPOOL_H #define DUCKDB_BUFFERPOOL_H -#include -#include +#include "exception/InvalidArgumentException.h" +#include "physical/BufferPool/Bitmap.h" +#include "physical/BufferPool/BufferPoolEntry.h" #include "physical/natives/ByteBuffer.h" -#include #include "physical/natives/DirectIoLib.h" -#include "exception/InvalidArgumentException.h" #include "utils/ColumnSizeCSVReader.h" +#include +#include #include -#include "physical/BufferPool/Bitmap.h" -#include "physical/BufferPool/BufferPoolEntry.h" +#include #include #include -#include -// when allocating buffer pool, we use the size of the first pxl file. Consider that -// the remaining pxl file has larger size than the first file, we allocate some extra -// size (10MB) to each column. +#include +// when allocating buffer pool, we use the size of the first pxl file. Consider +// that the remaining pxl file has larger size than the first file, we allocate +// some extra size (10MB) to each column. // TODO: how to evaluate the maximal pool size -#define EXTRA_POOL_SIZE 10*1024*1024 +#define EXTRA_POOL_SIZE 10 * 1024 * 1024 class DirectUringRandomAccessFile; + // This class is global class. The variable is shared by each thread class BufferPool { public: - // 嵌套子类,用于管理缓冲区池条目及其属性 - class BufferPoolManagedEntry { - public: - enum class State{ - InitizaledNotAllocated, - AllocatedAndInUse, - UselessButNotFree + class BufferPoolManagedEntry + { + public: + enum class State + { + InitizaledNotAllocated, + AllocatedAndInUse, + UselessButNotFree + }; + + private: + std::shared_ptr bufferPoolEntry; + int ring_index; + size_t current_size; + int offset; + State state; + + public: + BufferPoolManagedEntry(std::shared_ptr entry, int ringIdx, + size_t currSize, off_t off) + : bufferPoolEntry(std::move(entry)), ring_index(ringIdx), + current_size(currSize), offset(off), + state(State::InitizaledNotAllocated) + { + } + + std::shared_ptr getBufferPoolEntry() const + { + return bufferPoolEntry; + } + + int getRingIndex() const + { + return ring_index; + } + + void setRingIndex(int index) + { + ring_index = index; + } + + size_t getCurrentSize() const + { + return current_size; + } + + void setCurrentSize(size_t size) + { + current_size = size; + } + + int getOffset() const + { + return offset; + } + + void setOffset(int off) + { + offset = off; + } + + State getStatus() const + { + return state; + } + + void setStatus(State newStatus) + { + state = newStatus; + } }; - private: - std::shared_ptr bufferPoolEntry; // 指向缓冲区池条目的智能指针 - int ring_index; // 环形缓冲区索引 - size_t current_size; // 当前使用大小 - int offset; // 偏移量 - State state; - - - - public: - - BufferPoolManagedEntry(std::shared_ptr entry, - int ringIdx, - size_t currSize, - off_t off) - : bufferPoolEntry(std::move(entry)), - ring_index(ringIdx), - current_size(currSize), - offset(off) , - state(State::InitizaledNotAllocated){} - - std::shared_ptr getBufferPoolEntry() const { - return bufferPoolEntry; - } - - int getRingIndex() const { - return ring_index; - } - - void setRingIndex(int index) { - ring_index = index; - } - - size_t getCurrentSize() const { - return current_size; - } - - void setCurrentSize(size_t size) { - current_size = size; - } - - int getOffset() const { - return offset; - } - - void setOffset(int off) { - offset = off; - } - // 获取当前状态 - State getStatus() const { - return state; - } - - // 设置状态 - void setStatus(State newStatus) { - state = newStatus; - } - }; + static void Initialize(std::vector colIds, + std::vector bytes, + std::vector columnNames); - static void - Initialize(std::vector colIds, std::vector bytes, std::vector columnNames); + static void InitializeBuffers(); - static void - InitializeBuffers(); - - static std::shared_ptr GetBuffer(uint32_t colId,uint64_t byte,std::string columnName); + static std::shared_ptr GetBuffer(uint32_t colId, uint64_t byte, + std::string columnName); static int64_t GetBufferId(); @@ -135,52 +141,64 @@ class BufferPool static int getRingIndex(uint32_t colId); - static std::shared_ptr AllocateNewBuffer(std::shared_ptr currentBufferManagedEntry, uint32_t colId,uint64_t byte,std::string columnName); - - static std::shared_ptr ReusePreviousBuffer(std::shared_ptr currentBufferManagedEntry,uint32_t colId,uint64_t byte,std::string columnName); - - static void PrintStats() { - // 打印当前线程 ID - std::thread::id tid = std::this_thread::get_id(); - - printf("线程 %zu -> 全局缓冲区使用: %ld / %ld\n", - std::hash{}(tid), // 转换成整数便于阅读 - global_used_size, global_free_size); - - // 线程局部统计 - printf("线程 %zu -> Buffer0使用: %zu, 缓冲区数量: %d\n", - std::hash{}(tid), - thread_local_used_size[0], thread_local_buffer_count[0]); + static std::shared_ptr AllocateNewBuffer( + std::shared_ptr currentBufferManagedEntry, + uint32_t colId, uint64_t byte, std::string columnName); + + static std::shared_ptr ReusePreviousBuffer( + std::shared_ptr currentBufferManagedEntry, + uint32_t colId, uint64_t byte, std::string columnName); + + static void PrintStats() + { + // Get the ID of the current thread + std::thread::id tid = std::this_thread::get_id(); + + // Print global buffer usage: used size / free size + // Convert thread ID to integer for readability using hash + printf("Thread %zu -> Global buffer usage: %ld / %ld\n", + std::hash{}(tid), global_used_size, + global_free_size); + + // Print thread-local statistics for Buffer0 + printf("Thread %zu -> Buffer0 usage: %zu, Buffer count: %d\n", + std::hash{}(tid), thread_local_used_size[0], + thread_local_buffer_count[0]); + + // Print thread-local statistics for Buffer1 + printf("Thread %zu -> Buffer1 usage: %zu, Buffer count: %d\n", + std::hash{}(tid), thread_local_used_size[1], + thread_local_buffer_count[1]); + } - printf("线程 %zu -> Buffer1使用: %zu, 缓冲区数量: %d\n", - std::hash{}(tid), - thread_local_used_size[1], thread_local_buffer_count[1]); - } private: BufferPool() = default; - // global - static std::mutex bufferPoolMutex; - - // thread local - static thread_local bool isInitialized; - static thread_local std::vector > registeredBuffers[2]; - static thread_local long global_used_size; - static thread_local long global_free_size; - static thread_local std::shared_ptr directIoLib; - static thread_local int nextRingIndex; - static thread_local std::shared_ptr nextEmptyBufferPoolEntry[2]; + // global + static std::mutex bufferPoolMutex; + + // thread local + static thread_local bool isInitialized; + static thread_local std::vector> + registeredBuffers[2]; + static thread_local long global_used_size; + static thread_local long global_free_size; + static thread_local std::shared_ptr directIoLib; + static thread_local int nextRingIndex; + static thread_local std::shared_ptr + nextEmptyBufferPoolEntry[2]; static thread_local int colCount; static thread_local int currBufferIdx; static thread_local int nextBufferIdx; - static thread_local std::map > buffersAllocated[2]; + static thread_local std::map> + buffersAllocated[2]; friend class DirectUringRandomAccessFile; - static thread_local std::unordered_map> ringBufferMap[2]; - - + static thread_local std::unordered_map< + uint32_t, std::shared_ptr> + ringBufferMap[2]; - static thread_local size_t thread_local_used_size[2]; // 线程已使用大小 - static thread_local int thread_local_buffer_count[2]; // 线程持有的缓冲区数量 + static thread_local size_t thread_local_used_size[2]; + static thread_local int thread_local_buffer_count[2]; }; #endif // DUCKDB_BUFFERPOOL_H diff --git a/cpp/pixels-common/include/physical/BufferPool/Bitmap.h b/cpp/pixels-common/include/physical/BufferPool/Bitmap.h index 70d9a31790..ac67c547bc 100644 --- a/cpp/pixels-common/include/physical/BufferPool/Bitmap.h +++ b/cpp/pixels-common/include/physical/BufferPool/Bitmap.h @@ -1,48 +1,82 @@ /* -/* @author whz -/* @create 7/30/25. -*/ +* Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +/* + * @author whz + * @create 2025-07-30 + */ #ifndef BITMAP_H #define BITMAP_H -#include #include "exception/InvalidArgumentException.h" +#include +#include #include -class Bitmap { +class Bitmap +{ public: + explicit Bitmap(size_t size) : bits((size + 7) / 8, 0), num_bits(size) + { + } - explicit Bitmap(size_t size) - : bits((size + 7) / 8, 0), num_bits(size) {} - - void set(size_t index) { - if (index >= num_bits) throw InvalidArgumentException("Bitmap::set: index out of range"); - bits[index / 8] |= (1 << (index % 8)); - } + void set(size_t index) + { + if (index >= num_bits) + throw InvalidArgumentException("Bitmap::set: index out of range"); + bits[index / 8] |= (1 << (index % 8)); + } - void clear(size_t index) { - if (index >= num_bits) throw InvalidArgumentException("Bitmap::clear: index out of range"); - bits[index / 8] &= ~(1 << (index % 8)); - } + void clear(size_t index) + { + if (index >= num_bits) + throw InvalidArgumentException("Bitmap::clear: index out of range"); + bits[index / 8] &= ~(1 << (index % 8)); + } - bool test(size_t index) const { - if (index >= num_bits) throw InvalidArgumentException("Bitmap::test: index out of range"); - return bits[index / 8] & (1 << (index % 8)); - } + bool test(size_t index) const + { + if (index >= num_bits) + throw InvalidArgumentException("Bitmap::test: index out of range"); + return bits[index / 8] & (1 << (index % 8)); + } - size_t size() const { return num_bits; } + size_t size() const + { + return num_bits; + } - void print() const { - for (size_t i = 0; i < num_bits; ++i) { - std::cout << (test(i) ? '1' : '0'); - if ((i + 1) % 8 == 0) std::cout << ' '; + void print() const + { + for (size_t i = 0; i < num_bits; ++i) + { + std::cout << (test(i) ? '1' : '0'); + if ((i + 1) % 8 == 0) + std::cout << ' '; + } + std::cout << '\n'; } - std::cout << '\n'; - } private: - std::vector bits; - size_t num_bits; + std::vector bits; + size_t num_bits; }; -#endif //BITMAP_H +#endif // BITMAP_H diff --git a/cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h b/cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h index 74536a7b6a..5fde7f2fe9 100644 --- a/cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h +++ b/cpp/pixels-common/include/physical/BufferPool/BufferPoolEntry.h @@ -1,46 +1,71 @@ /* -/* @author whz -/* @create 7/30/25. -*/ +* Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +/* + * @author whz + * @create 2025-07-30 + */ #ifndef BUFFERPOOLENTRY_H #define BUFFERPOOLENTRY_H +#include "physical/BufferPool/Bitmap.h" #include #include #include -#include "physical/BufferPool/Bitmap.h" -class BufferPoolEntry { + +class BufferPoolEntry +{ public: -explicit BufferPoolEntry(size_t size,int slice_size,std::shared_ptr direct_lib,int offset,int ring_index); - size_t getSize() const; - std::shared_ptr getBitmap() const; - std::shared_ptr getBuffer() const; - bool isFull() const; - int getNextFreeIndex() const; - int setNextFreeIndex(int index); - ~BufferPoolEntry(); - uint64_t checkCol(uint32_t) const; - void addCol(uint32_t colId,uint64_t bytes); - bool isInUse() const; - void setInUse(bool in_use); - int getOffsetInBuffers() const; - void setOffsetInBuffers(int offset); - bool getIsRegistered() const; - void setIsRegistered(bool registered); - int getRingIndex() const; - void setRingIndex(int ring_index); - void reset(); + explicit BufferPoolEntry(size_t size, int slice_size, + std::shared_ptr direct_lib, int offset, + int ring_index); + size_t getSize() const; + std::shared_ptr getBitmap() const; + std::shared_ptr getBuffer() const; + bool isFull() const; + int getNextFreeIndex() const; + int setNextFreeIndex(int index); + ~BufferPoolEntry(); + uint64_t checkCol(uint32_t) const; + void addCol(uint32_t colId, uint64_t bytes); + bool isInUse() const; + void setInUse(bool in_use); + int getOffsetInBuffers() const; + void setOffsetInBuffers(int offset); + bool getIsRegistered() const; + void setIsRegistered(bool registered); + int getRingIndex() const; + void setRingIndex(int ring_index); + void reset(); + private: -size_t size_; -std::shared_ptr bitmap_; -std::shared_ptr buffer_; - bool is_full_; - int next_free_; - std::map nr_bytes_; - bool is_in_use_; - int offset_in_buffers_; - bool is_registered; - int ring_index; + size_t size_; + std::shared_ptr bitmap_; + std::shared_ptr buffer_; + bool is_full_; + int next_free_; + std::map nr_bytes_; + bool is_in_use_; + int offset_in_buffers_; + bool is_registered; + int ring_index; }; -#endif //BUFFERPOOLENTRY_H +#endif // BUFFERPOOLENTRY_H diff --git a/cpp/pixels-common/include/physical/Scheduler.h b/cpp/pixels-common/include/physical/Scheduler.h index 8da823445c..b6d3af6abe 100644 --- a/cpp/pixels-common/include/physical/Scheduler.h +++ b/cpp/pixels-common/include/physical/Scheduler.h @@ -47,6 +47,5 @@ class Scheduler RequestBatch batch, std::vector > reuseBuffers, long queryId) = 0; - std::mutex mtx; }; #endif //PIXELS_SCHEDULER_H diff --git a/cpp/pixels-common/include/physical/natives/DirectIoLib.h b/cpp/pixels-common/include/physical/natives/DirectIoLib.h index dafe33fa0c..1b1f4aeee6 100644 --- a/cpp/pixels-common/include/physical/natives/DirectIoLib.h +++ b/cpp/pixels-common/include/physical/natives/DirectIoLib.h @@ -27,30 +27,33 @@ /** * Mapping Linux I/O functions to native methods. - * Partially referenced the implementation of Jaydio (https://github.com/smacke/jaydio), - * which is implemented by Stephen Macke and licensed under Apache 2.0. - *

+ * Partially referenced the implementation of Jaydio(https://github.com/smacke/jaydio), + * which is implemented by Stephen Macke and + * licensed under Apache 2.0.

* Created at: 02/02/2023 * Author: Liangyong Yu */ -#include "utils/ConfigFactory.h" +#include "liburing.h" +#include "liburing/io_uring.h" #include "physical/natives/ByteBuffer.h" +#include "utils/ConfigFactory.h" +#include +#include #include +#include +#include +#include #include -#include +#include #include -#include "liburing.h" -#include "liburing/io_uring.h" - struct uringData { int idx; - ByteBuffer *bb; + ByteBuffer* bb; }; - class DirectIoLib { public: @@ -59,15 +62,21 @@ class DirectIoLib */ DirectIoLib(int fsBlockSize); - std::shared_ptr allocateDirectBuffer(long size); + std::shared_ptr allocateDirectBuffer(long size, + bool isSmallBuffer); + + int getToAllocate(int size); - std::shared_ptr read(int fd, long fileOffset, std::shared_ptr directBuffer, long length); + std::shared_ptr read(int fd, long fileOffset, + std::shared_ptr directBuffer, + long length); long blockStart(long value); long blockEnd(long value); - int getBlockSize() const { + int getBlockSize() const + { return fsBlockSize; }; diff --git a/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h b/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h index 43650e46ae..977ac2cd2a 100644 --- a/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h +++ b/cpp/pixels-common/include/physical/natives/DirectUringRandomAccessFile.h @@ -38,47 +38,40 @@ class DirectUringRandomAccessFile : public DirectRandomAccessFile { public: - explicit DirectUringRandomAccessFile(const std::string &file); + explicit DirectUringRandomAccessFile(const std::string& file); - static void RegisterBuffer(std::vector > buffers); + static void RegisterBuffer(std::vector> buffers); - static void RegisterBufferFromPool(std::vector colIds); + static void RegisterBufferFromPool(std::vector colIds); static void Initialize(); static void Reset(); - static bool RegisterMoreBuffer(int index,std::vector> buffers); + static bool RegisterMoreBuffer(int index, std::vector> buffers); - std::shared_ptr readAsync(int length, std::shared_ptr buffer, int index,int ring_index,int start_offset); + std::shared_ptr readAsync(int length, std::shared_ptr buffer, int index, int ring_index, + int start_offset); - void readAsyncSubmit(std::unordered_map sizes,std::unordered_set ring_indexs); + void readAsyncSubmit(std::unordered_map sizes, std::unordered_set ring_indexs); - void readAsyncComplete(std::unordered_map sizes,std::unordered_set ring_indexs); + void readAsyncComplete(std::unordered_map sizes, std::unordered_set ring_indexs); - void seekByIndex(long offset,int index); + void seekByIndex(long offset, int index); static struct io_uring* getRing(int index); ~DirectUringRandomAccessFile(); private: - // global isRegistered - // static bool isRegistered; - // static MutexTracker g_mutex_tracker; - // static TrackedMutex g_mutex; - // static std::vector ring_vector; - // static std::vector iovecs_vector; - // static uint32_t iovecSize; - // static std::vector offsets_vector; - // thread_local - static std::mutex mutex_; - static thread_local bool isRegistered; - // static MutexTracker g_mutex_tracker; - // static TrackedMutex g_mutex; - static thread_local std::vector ring_vector; - static thread_local std::vector iovecs_vector; - static thread_local uint32_t iovecSize; - static thread_local std::vector offsets_vector; + // thread_local + static std::mutex mutex_; + static thread_local bool isRegistered; + // static MutexTracker g_mutex_tracker; + // static TrackedMutex g_mutex; + static thread_local std::vector ring_vector; + static thread_local std::vector iovecs_vector; + static thread_local uint32_t iovecSize; + static thread_local std::vector offsets_vector; }; #endif // DUCKDB_DIRECTURINGRANDOMACCESSFILE_H diff --git a/cpp/pixels-common/include/utils/MutexTracker.h b/cpp/pixels-common/include/utils/MutexTracker.h index 96dc47ed39..e25ceb3a7c 100644 --- a/cpp/pixels-common/include/utils/MutexTracker.h +++ b/cpp/pixels-common/include/utils/MutexTracker.h @@ -1,7 +1,27 @@ /* -/* @author whz -/* @create 8/23/25. -*/ +* Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +/* + * @author whz + * @create 2025-08-23 + */ #ifndef DUCKDB_MUTEX_H #define DUCKDB_MUTEX_H @@ -13,59 +33,59 @@ #include class MutexTracker { private: - // 存储锁与持有者线程ID的映射 + // Stores the mapping between mutexes and their owning thread IDs std::unordered_map mutex_owners; - // 保护映射表的内部锁 + // Internal mutex to protect the mapping table std::mutex internal_mutex; public: - // 获取锁的持有者线程ID,若未被持有则返回默认ID + // Gets the thread ID of the mutex owner; returns default ID if not held std::thread::id get_owner(const std::mutex* mutex) { std::lock_guard lock(internal_mutex); auto it = mutex_owners.find(mutex); if (it != mutex_owners.end()) { return it->second; } - return std::thread::id(); // 未被持有 + return std::thread::id(); // Not held by any thread } - // 记录锁被持有 + // Records that a mutex has been acquired void lock_acquired(const std::mutex* mutex) { std::lock_guard lock(internal_mutex); auto current_thread = std::this_thread::get_id(); - // 检查是否重复获取(防止递归锁问题) + // Check for duplicate acquisition (to prevent recursive lock issues) auto it = mutex_owners.find(mutex); if (it != mutex_owners.end()) { if (it->second == current_thread) { - // 同一线程重复获取非递归锁,可能导致死锁 - throw std::runtime_error("同一线程重复获取非递归锁"); + // Same thread acquiring a non-recursive mutex repeatedly, may cause deadlock + throw std::runtime_error("Same thread acquiring non-recursive mutex repeatedly"); } } mutex_owners[mutex] = current_thread; } - // 记录锁被释放 + // Records that a mutex has been released void lock_released(const std::mutex* mutex) { std::lock_guard lock(internal_mutex); mutex_owners.erase(mutex); } - // 打印锁的持有者信息 + // Prints information about the mutex owner void print_owner(const std::mutex* mutex, const std::string& mutex_name) { std::thread::id owner = get_owner(mutex); if (owner == std::thread::id()) { - std::cout << mutex_name << " 未被任何线程持有" << std::endl; + std::cout << mutex_name << " is not held by any thread" << std::endl; } else { - std::cout << mutex_name << " 被线程 " << std::hash{}(owner) - << " 持有" << std::endl; + std::cout << mutex_name << " is held by thread " << std::hash{}(owner) + << std::endl; } } }; -// 带跟踪功能的互斥锁包装器 +// Mutex wrapper with tracking functionality class TrackedMutex { private: std::mutex internal_mutex; @@ -73,21 +93,21 @@ class TrackedMutex { MutexTracker g_mutex_tracker; public: - TrackedMutex(const std::string& name,MutexTracker &g_mutex_tracker) : name(name) {} + TrackedMutex(const std::string& name, MutexTracker& g_mutex_tracker) : name(name) {} - // 加锁并记录持有者 + // Locks the mutex and records the owner void lock() { internal_mutex.lock(); g_mutex_tracker.lock_acquired(&internal_mutex); } - // 解锁并清除持有者记录 + // Unlocks the mutex and clears the owner record void unlock() { g_mutex_tracker.lock_released(&internal_mutex); internal_mutex.unlock(); } - // 尝试加锁(成功则记录持有者) + // Tries to lock the mutex (records owner if successful) bool try_lock() { if (internal_mutex.try_lock()) { g_mutex_tracker.lock_acquired(&internal_mutex); @@ -96,16 +116,16 @@ class TrackedMutex { return false; } - // 获取锁名称 + // Gets the mutex name const std::string& get_name() const { return name; } - // 获取内部互斥锁指针(用于跟踪) + // Gets pointer to internal mutex (for tracking purposes) const std::mutex* get_internal_mutex() const { return &internal_mutex; } }; -// 与TrackedMutex配合使用的RAII锁管理 +// RAII lock manager for use with TrackedMutex template class TrackedLockGuard { private: @@ -120,7 +140,7 @@ class TrackedLockGuard { mutex.unlock(); } - // 禁止拷贝 + // Disable copying TrackedLockGuard(const TrackedLockGuard&) = delete; TrackedLockGuard& operator=(const TrackedLockGuard&) = delete; }; diff --git a/cpp/pixels-common/lib/physical/BufferPool.cpp b/cpp/pixels-common/lib/physical/BufferPool.cpp index acd5519de5..a3177886eb 100644 --- a/cpp/pixels-common/lib/physical/BufferPool.cpp +++ b/cpp/pixels-common/lib/physical/BufferPool.cpp @@ -32,28 +32,34 @@ using TimePoint = std::chrono::high_resolution_clock::time_point; using Duration = std::chrono::duration; -TimePoint getCurrentTime() { +TimePoint getCurrentTime() +{ return std::chrono::high_resolution_clock::now(); } - // global -std::string columnSizePath = ConfigFactory::Instance().getProperty("pixel.column.size.path"); -std::shared_ptr csvReader; -int sliceSize= std::stoi(ConfigFactory::Instance().getProperty("pixel.bufferpool.sliceSize")); -int bufferSize=std::stoi(ConfigFactory::Instance().getProperty("pixel.bufferpool.bufferpoolSize")); +std::string columnSizePath = + ConfigFactory::Instance().getProperty("pixel.column.size.path"); +std::shared_ptr csvReader; +int sliceSize = std::stoi( + ConfigFactory::Instance().getProperty("pixel.bufferpool.sliceSize")); +int bufferSize = std::stoi( + ConfigFactory::Instance().getProperty("pixel.bufferpool.bufferpoolSize")); const size_t SLICE_SIZE = sliceSize * 1024; -int bufferNum=std::stoi(ConfigFactory::Instance().getProperty("pixel.bufferpool.bufferNum")); -std::mutex BufferPool::bufferPoolMutex; +int bufferNum = std::stoi( + ConfigFactory::Instance().getProperty("pixel.bufferpool.bufferNum")); +std::mutex BufferPool::bufferPoolMutex; // thread_local thread_local bool BufferPool::isInitialized; -thread_local std::vector > BufferPool::registeredBuffers[2]; -thread_local long BufferPool::global_free_size=0; +thread_local std::vector> +BufferPool::registeredBuffers[2]; +thread_local long BufferPool::global_free_size = 0; thread_local long BufferPool::global_used_size = 0; -thread_local std::shared_ptr BufferPool::directIoLib; -thread_local int BufferPool::nextRingIndex=1; -thread_local std::shared_ptr BufferPool::nextEmptyBufferPoolEntry[2]={nullptr,nullptr}; +thread_local std::shared_ptr BufferPool::directIoLib; +thread_local int BufferPool::nextRingIndex = 1; +thread_local std::shared_ptr +BufferPool::nextEmptyBufferPoolEntry[2] = {nullptr, nullptr}; thread_local std::vector globalBuffers; thread_local int BufferPool::colCount = 0; // The currBufferIdx is set to 1. When executing the first file, this value is 0 @@ -61,24 +67,29 @@ thread_local int BufferPool::colCount = 0; thread_local int BufferPool::currBufferIdx = 1; thread_local int BufferPool::nextBufferIdx = 0; -thread_local std::map> BufferPool::buffersAllocated[2]; -thread_local std::unordered_map> BufferPool::ringBufferMap[2]; -thread_local size_t BufferPool::thread_local_used_size[2]={0,0}; -thread_local int BufferPool::thread_local_buffer_count[2]={0,0}; - -void BufferPool::Initialize(std::vector colIds, std::vector bytes, - std::vector columnNames) +thread_local std::map> +BufferPool::buffersAllocated[2]; +thread_local std::unordered_map< + uint32_t, std::shared_ptr> +BufferPool::ringBufferMap[2]; +thread_local size_t BufferPool::thread_local_used_size[2] = {0, 0}; +thread_local int BufferPool::thread_local_buffer_count[2] = {0, 0}; + +void BufferPool::Initialize(std::vector colIds, + std::vector bytes, + std::vector columnNames) { - assert(colIds.size() == bytes.size()); // std::lock_guard lock(bufferPoolMutex); - int fsBlockSize = std::stoi(ConfigFactory::Instance().getProperty("localfs.block.size")); - auto strToBool = [](const std::string& s) { + int fsBlockSize = + std::stoi(ConfigFactory::Instance().getProperty("localfs.block.size")); + auto strToBool = [](const std::string& s) + { return s == "true" || s == "1" || s == "yes"; }; - std::string configValue = ConfigFactory::Instance().getProperty("pixel.bufferpool.fixedsize"); + std::string configValue = + ConfigFactory::Instance().getProperty("pixel.bufferpool.fixedsize"); bool isFixedSize = strToBool(configValue); - // std::cout<<"是否初始化:"< colIds, std::vector (columnSizePath); } } - auto byte=bytes.at(i); - BufferPool::ringBufferMap[currBufferIdx][colId]=std::make_shared(registeredBuffers[currBufferIdx][0],0,byte,0); - // BufferPool::ringBufferMap[nextBufferIdx][colId]=registeredBuffers[nextBufferIdx][0]; - // std::cout<<"Initialized columnID:"<( + registeredBuffers[currBufferIdx][0], 0, byte, 0); } BufferPool::colCount = colIds.size(); BufferPool::isInitialized = true; - } else { @@ -116,150 +127,166 @@ void BufferPool::Initialize(std::vector colIds, std::vector (registeredBuffers[currBufferIdx][0],0,byte,0); - // BufferPool::ringBufferMap[nextBufferIdx][colId]=registeredBuffers[nextBufferIdx][0]; - // GetBuffer(colId,byte,columnNames.at(i)); - // std::cout<<"Not Initialized columnID:"<size()<( + registeredBuffers[currBufferIdx][0], 0, byte, 0); } } } - -void BufferPool::InitializeBuffers() { +void BufferPool::InitializeBuffers() +{ for (int idx = 0; idx < 2; idx++) { - const int size_=bufferSize+EXTRA_POOL_SIZE; - // Calculate the required space, allocate it in advance, and then register it. - std::shared_ptr buffer_pool_entry=std::make_shared(size_,sliceSize,directIoLib,idx,0); + const int size_ = bufferSize + EXTRA_POOL_SIZE; + // Calculate the required space, allocate it in advance, and then register + // it. + std::shared_ptr buffer_pool_entry = + std::make_shared(size_, sliceSize, directIoLib, idx, + 0); registeredBuffers[idx].emplace_back(buffer_pool_entry); buffer_pool_entry->setInUse(true); - global_free_size+=size_; + global_free_size += size_; } } - int64_t BufferPool::GetBufferId() { return currBufferIdx; } -std::shared_ptr BufferPool::AllocateNewBuffer(std::shared_ptr currentBufferManagedEntry, uint32_t colId,uint64_t byte,std::string columnName) { - auto strToBool = [](const std::string& s) { - return s == "true" || s == "1" || s == "yes"; - }; - std::string configValue = ConfigFactory::Instance().getProperty("pixel.bufferpool.fixedsize"); - bool isFixedSize = strToBool(configValue); - if (isFixedSize) { - byte=csvReader->get(columnName); - // std::cout<<"colId: "<getBufferPoolEntry(); - std::shared_ptr original = currentBuffer->getBuffer(); - - // get offset - size_t offset = currentBuffer->getNextFreeIndex(); - // fisrt find anthor empty buffer +std::shared_ptr BufferPool::AllocateNewBuffer( + std::shared_ptr currentBufferManagedEntry, + uint32_t colId, uint64_t byte, std::string columnName) +{ + auto strToBool = [](const std::string& s) + { + return s == "true" || s == "1" || s == "yes"; + }; + std::string configValue = + ConfigFactory::Instance().getProperty("pixel.bufferpool.fixedsize"); + bool isFixedSize = strToBool(configValue); + if (isFixedSize) + { + byte = csvReader->get(columnName); + } - if (offset + totalSize > original->size()) + size_t sliceCount = (byte + SLICE_SIZE - 1) / SLICE_SIZE + 1; + size_t totalSizeRaw = (sliceCount + 1) * SLICE_SIZE; + size_t totalSize = directIoLib->getToAllocate(totalSizeRaw); + // need to reallocate + // get origin registered ByteBuffer + auto currentBuffer = currentBufferManagedEntry->getBufferPoolEntry(); + std::shared_ptr original = currentBuffer->getBuffer(); + + // get offset + size_t offset = currentBuffer->getNextFreeIndex(); + // fisrt find anthor empty buffer + if (offset + totalSize > original->size()) + { + if (nextEmptyBufferPoolEntry[currBufferIdx] != nullptr && + nextEmptyBufferPoolEntry[currBufferIdx]->getNextFreeIndex() + + totalSize < + nextEmptyBufferPoolEntry[currBufferIdx]->getSize()) { - - if (nextEmptyBufferPoolEntry[currBufferIdx]!=nullptr&& - nextEmptyBufferPoolEntry[currBufferIdx]->getNextFreeIndex()+totalSizegetSize()) { // there are anthor regisitered Buffers - // std::cout<<"curBufID:"<size()<setInUse(false); - currentBuffer=BufferPool::AddNewBuffer(currentBuffer->getSize()); - std::vector> buffers; - buffers.emplace_back(currentBuffer->getBuffer()); - if (!::DirectUringRandomAccessFile::RegisterMoreBuffer(currentBuffer->getRingIndex(),buffers)) { - throw std::runtime_error("Failed to register more buffers"); - } - currentBuffer->setInUse(true); - currentBuffer->setIsRegistered(true); + // std::cout<<"curBufID:"<size()<setInUse(false); + currentBuffer = BufferPool::AddNewBuffer(currentBuffer->getSize()); + std::vector> buffers; + buffers.emplace_back(currentBuffer->getBuffer()); + if (!::DirectUringRandomAccessFile::RegisterMoreBuffer( + currentBuffer->getRingIndex(), buffers)) + { + throw std::runtime_error("Failed to register more buffers"); } - offset=currentBuffer->getNextFreeIndex(); - // change the find bitmap - auto newBufferPoolManageEntry=std::make_shared(currentBuffer,currentBuffer->getRingIndex(),totalSize,offset); - BufferPool::ringBufferMap[currBufferIdx][colId]=newBufferPoolManageEntry; - newBufferPoolManageEntry->setStatus(BufferPoolManagedEntry::State::AllocatedAndInUse); - original=currentBuffer->getBuffer(); + currentBuffer->setInUse(true); + currentBuffer->setIsRegistered(true); } - // update bitmap (maybe costly) - // size_t startSlice = offset / SLICE_SIZE; - // for (size_t i = 0; i < sliceCount; ++i) { - // size_t sliceIndex = startSlice + i; - // if (currentBuffer->getBitmap()->test(sliceIndex)) { - // throw std::runtime_error("Buffer slice already used! Potential bitmap inconsistency."); - // } - // currentBuffer->getBitmap()->set(sliceIndex); - // } - - currentBuffer->setNextFreeIndex(offset + totalSize); - - std::shared_ptr sliced = original->slice(offset, totalSize); - currentBuffer->addCol(colId,sliced->size()); - BufferPool::buffersAllocated[currBufferIdx][colId] = sliced; - auto newBufferPoolManageEntry=BufferPool::ringBufferMap[currBufferIdx][colId]; - newBufferPoolManageEntry->setStatus(BufferPoolManagedEntry::State::AllocatedAndInUse); - newBufferPoolManageEntry->setCurrentSize(sliced->size()); - newBufferPoolManageEntry->setOffset(offset); - newBufferPoolManageEntry->setRingIndex(currentBuffer->getRingIndex()); - - // std::cout<<"buffer pointer:"<(sliced->getPointer())<< - // " length:"<size()<<" bufferOffsets:"<getNextFreeIndex()<<" buffersAllocated:"<size()<getNextFreeIndex(); + auto newBufferPoolManageEntry = std::make_shared( + currentBuffer, currentBuffer->getRingIndex(), totalSize, offset); + BufferPool::ringBufferMap[currBufferIdx][colId] = newBufferPoolManageEntry; + newBufferPoolManageEntry->setStatus( + BufferPoolManagedEntry::State::AllocatedAndInUse); + original = currentBuffer->getBuffer(); + } -std::shared_ptr BufferPool::ReusePreviousBuffer(std::shared_ptr currentBufferManagedEntry,uint32_t colId,uint64_t byte,std::string columnName) { - auto currentBuffer=currentBufferManagedEntry->getBufferPoolEntry(); - // std::cout<<"resue previous buffer size:"<size() - // <<" colID:"<getCurrentSize():"<getCurrentSize()<<" byte:"<setNextFreeIndex(offset + totalSize); + + std::shared_ptr sliced = original->slice(offset, totalSize); + currentBuffer->addCol(colId, sliced->size()); + BufferPool::buffersAllocated[currBufferIdx][colId] = sliced; + auto newBufferPoolManageEntry = + BufferPool::ringBufferMap[currBufferIdx][colId]; + newBufferPoolManageEntry->setStatus( + BufferPoolManagedEntry::State::AllocatedAndInUse); + newBufferPoolManageEntry->setCurrentSize(sliced->size()); + newBufferPoolManageEntry->setOffset(offset); + newBufferPoolManageEntry->setRingIndex(currentBuffer->getRingIndex()); + + global_used_size += totalSize; + thread_local_used_size[currBufferIdx] += totalSize; + thread_local_buffer_count[currBufferIdx]++; + return sliced; } -std::shared_ptr BufferPool::GetBuffer(uint32_t colId,uint64_t byte,std::string columnName) +std::shared_ptr BufferPool::GetBuffer(uint32_t colId, uint64_t byte, + std::string columnName) { - // another buffer - // std::lock_guard lock(bufferPoolMutex); + // Retrieve the current buffer management entry and previously allocated + // buffer auto currentBufferManagedEntry = ringBufferMap[currBufferIdx][colId]; - // 分配情况 - // 情况一 未分配 检查entry的状态 - // 情况二 已经分配 - // 大小情况 - // 情况一 可以复用之前的buffer resuePrevious() - // 情况二 可以在同一个大buffer中划分出需要的空间 allocateBuffer - // 情况三 当前大buffer剩余内存不够用了 需要新的大buffer - if (currentBufferManagedEntry->getStatus()==BufferPoolManagedEntry::State::InitizaledNotAllocated) { - //未分配 - return AllocateNewBuffer(currentBufferManagedEntry,colId,byte,columnName); + auto previousBuffer = buffersAllocated[currBufferIdx][colId]; + + // Allocation scenarios: + // 1. Buffer not yet allocated - check the entry's state + // 2. Buffer already allocated + + // Size scenarios: + // 1. Can reuse the previous buffer (reusePrevious()) + // 2. Can allocate required space within the same large buffer + // (allocateBuffer) + // 3. Insufficient remaining space in current large buffer - need a new large + // buffer + + if (currentBufferManagedEntry->getStatus() == + BufferPoolManagedEntry::State::InitizaledNotAllocated) + { + // Buffer not allocated yet + return AllocateNewBuffer(currentBufferManagedEntry, colId, byte, + columnName); } - else { - // 已经分配 - if (currentBufferManagedEntry->getCurrentSize()>=byte&¤tBufferManagedEntry->getCurrentSize()-directIoLib->getBlockSize()>=byte) { - // 复用之前的buffer - return ReusePreviousBuffer(currentBufferManagedEntry,colId,byte,columnName); - }else { - return AllocateNewBuffer(currentBufferManagedEntry,colId,byte,columnName); + else + { + // Buffer already allocated + // Check if current buffer size is sufficient and leaves enough remaining + // space (after accounting for block size) + if (currentBufferManagedEntry->getCurrentSize() >= byte && + currentBufferManagedEntry->getCurrentSize() - + directIoLib->getBlockSize() >= + byte) + { + // Reuse the previous buffer + return previousBuffer; + } + else + { + // Need to allocate a new buffer + return AllocateNewBuffer(currentBufferManagedEntry, colId, byte, + columnName); } } } @@ -277,15 +304,13 @@ void BufferPool::Reset() // thread_local_used_size[idx]=0; // thread_local_buffer_count[idx]=0; // global_free_size=0; - global_used_size=0; - for (auto bufferEntry: registeredBuffers[idx]) { + global_used_size = 0; + for (auto bufferEntry : registeredBuffers[idx]) + { bufferEntry->reset(); } } - - - BufferPool::colCount = 0; } @@ -295,25 +320,27 @@ void BufferPool::Switch() nextBufferIdx = 1 - nextBufferIdx; } - - -std::shared_ptr BufferPool::AddNewBuffer(size_t size) { +std::shared_ptr BufferPool::AddNewBuffer(size_t size) +{ // std::cout<<"Adding new buffer"< buffer_pool_entry=std::make_shared(size,sliceSize,directIoLib,currBufferIdx,nextRingIndex++); - std::cout<<"申请新buffer:"< buffer_pool_entry = + std::make_shared(size, sliceSize, directIoLib, + currBufferIdx, nextRingIndex++); registeredBuffers[currBufferIdx].emplace_back(buffer_pool_entry); buffer_pool_entry->setInUse(true); - global_free_size+=size; - nextEmptyBufferPoolEntry[currBufferIdx]=buffer_pool_entry; + global_free_size += size; + nextEmptyBufferPoolEntry[currBufferIdx] = buffer_pool_entry; return buffer_pool_entry; } -int BufferPool::getRingIndex(uint32_t colId) { - return ringBufferMap[currBufferIdx][colId]->getBufferPoolEntry()->getRingIndex(); +int BufferPool::getRingIndex(uint32_t colId) +{ + return ringBufferMap[currBufferIdx][colId] + ->getBufferPoolEntry() + ->getRingIndex(); } - diff --git a/cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp b/cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp index 6970d2a034..8b0f09ef58 100644 --- a/cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp +++ b/cpp/pixels-common/lib/physical/BufferPool/Bitmap.cpp @@ -1,7 +1,27 @@ /* -/* @author whz -/* @create 7/30/25. -*/ +* Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + + +/* + * @author whz + * @create 2025-07-30 + */ #include "physical/BufferPool/BufferPoolEntry.h" #include "physical/BufferPool/Bitmap.h" - diff --git a/cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp b/cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp index 723c425e25..de5f7dbac7 100644 --- a/cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp +++ b/cpp/pixels-common/lib/physical/BufferPool/BufferPoolEntry.cpp @@ -1,113 +1,157 @@ /* -/* @author whz -/* @create 7/30/25. -*/ +* Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + + +/* + * @author whz + * @create 2025-07-30 + */ #include "physical/BufferPool/BufferPoolEntry.h" #include "physical/BufferPool/Bitmap.h" -BufferPoolEntry::BufferPoolEntry(size_t size, int slice_size, std::shared_ptr direct_lib,int offset,int ring_index) - : size_(size), is_full_(false), next_free_(0),is_in_use_(false),offset_in_buffers_(offset),is_registered(false),ring_index(ring_index) { +BufferPoolEntry::BufferPoolEntry(size_t size, int slice_size, std::shared_ptr direct_lib, int offset, + int ring_index) + : size_(size), is_full_(false), next_free_(0), is_in_use_(false), offset_in_buffers_(offset), is_registered(false), + ring_index(ring_index) +{ + if (size == 0 || slice_size <= 0) + { + throw std::invalid_argument("Invalid buffer size or slice size"); + } + if (!direct_lib) + { + throw std::invalid_argument("DirectIoLib pointer cannot be null"); + } - if (size == 0 || slice_size <= 0) { - throw std::invalid_argument("Invalid buffer size or slice size"); - } - if (!direct_lib) { - throw std::invalid_argument("DirectIoLib pointer cannot be null"); - } + const int slice_count = static_cast(size + slice_size - 1 / slice_size); + // bitmap_ = std::make_shared(slice_count); - const int slice_count = static_cast(size + slice_size-1/ slice_size); - bitmap_ = std::make_shared(slice_count); - - buffer_ = direct_lib->allocateDirectBuffer(size_); - if (!buffer_) { - throw std::runtime_error("Failed to allocate direct buffer"); - } + buffer_ = direct_lib->allocateDirectBuffer(size_, false); + memset(buffer_->getPointer(), 0, buffer_->size()); + if (!buffer_) + { + throw std::runtime_error("Failed to allocate direct buffer"); + } } -size_t BufferPoolEntry::getSize() const { - return size_; +size_t BufferPoolEntry::getSize() const +{ + return size_; } -std::shared_ptr BufferPoolEntry::getBitmap() const { - return bitmap_; -} +// std::shared_ptr BufferPoolEntry::getBitmap() const { +// return bitmap_; +// } -std::shared_ptr BufferPoolEntry::getBuffer() const { - return buffer_; +std::shared_ptr BufferPoolEntry::getBuffer() const +{ + return buffer_; } -bool BufferPoolEntry::isFull() const { - return is_full_; +bool BufferPoolEntry::isFull() const +{ + return is_full_; } -int BufferPoolEntry::getNextFreeIndex() const { - return next_free_; +int BufferPoolEntry::getNextFreeIndex() const +{ + return next_free_; } -int BufferPoolEntry::setNextFreeIndex(int index) { - const int old_index = next_free_; - next_free_ = index; - is_full_ = (index > size_); - if (is_full_) { - return -1; - // the buffer is full - } - return old_index; +int BufferPoolEntry::setNextFreeIndex(int index) +{ + const int old_index = next_free_; + next_free_ = index; + is_full_ = (index > size_); + if (is_full_) + { + return -1; + // the buffer is full + } + return old_index; } -uint64_t BufferPoolEntry::checkCol(uint32_t col) const { - return nr_bytes_.find(col)==nr_bytes_.end() ? -1 : nr_bytes_.at(col); +uint64_t BufferPoolEntry::checkCol(uint32_t col) const +{ + return nr_bytes_.find(col) == nr_bytes_.end() ? -1 : nr_bytes_.at(col); } -void BufferPoolEntry::addCol(uint32_t colId,uint64_t bytes) { - nr_bytes_[colId] =bytes; +void BufferPoolEntry::addCol(uint32_t colId, uint64_t bytes) +{ + nr_bytes_[colId] = bytes; } -bool BufferPoolEntry::isInUse() const { - return is_in_use_; +bool BufferPoolEntry::isInUse() const +{ + return is_in_use_; } -void BufferPoolEntry::setInUse(bool in_use) { - is_in_use_ = in_use; +void BufferPoolEntry::setInUse(bool in_use) +{ + is_in_use_ = in_use; } -int BufferPoolEntry::getOffsetInBuffers() const { - return offset_in_buffers_; +int BufferPoolEntry::getOffsetInBuffers() const +{ + return offset_in_buffers_; } -void BufferPoolEntry::setOffsetInBuffers(int offset) { - offset_in_buffers_=offset; +void BufferPoolEntry::setOffsetInBuffers(int offset) +{ + offset_in_buffers_ = offset; } -bool BufferPoolEntry::getIsRegistered() const { - return is_registered; +bool BufferPoolEntry::getIsRegistered() const +{ + return is_registered; } -void BufferPoolEntry::setIsRegistered(bool registered) { - is_registered=registered; +void BufferPoolEntry::setIsRegistered(bool registered) +{ + is_registered = registered; } -int BufferPoolEntry::getRingIndex() const { - return ring_index; +int BufferPoolEntry::getRingIndex() const +{ + return ring_index; } -void BufferPoolEntry::setRingIndex(int ring_index) { - ring_index=ring_index; +void BufferPoolEntry::setRingIndex(int ring_index) +{ + ring_index = ring_index; } -void BufferPoolEntry::reset() { - is_full_ = false; - next_free_=0; - if (bitmap_) { - bitmap_.reset(); - } - nr_bytes_.clear(); - is_in_use_ = false; -// reset direct buffer? +void BufferPoolEntry::reset() +{ + is_full_ = false; + next_free_ = 0; + // if (bitmap_) { + // bitmap_.reset(); + // } + nr_bytes_.clear(); + is_in_use_ = false; + // reset direct buffer? } -BufferPoolEntry::~BufferPoolEntry() = default; \ No newline at end of file +BufferPoolEntry::~BufferPoolEntry() = default; diff --git a/cpp/pixels-common/lib/physical/Request.cpp b/cpp/pixels-common/lib/physical/Request.cpp index e904fff5fd..7ba5903db1 100644 --- a/cpp/pixels-common/lib/physical/Request.cpp +++ b/cpp/pixels-common/lib/physical/Request.cpp @@ -24,22 +24,15 @@ */ #include "physical/Request.h" - -Request::Request(uint64_t queryId_, uint64_t start_, uint64_t length_, int64_t bufferId) -{ - queryId = queryId_; - start = start_; - length = length_; - this->bufferId = bufferId; - ring_index=0; +Request::Request(uint64_t queryId_, uint64_t start_, uint64_t length_, + int64_t bufferId) { + queryId = queryId_; + start = start_; + length = length_; + this->bufferId = bufferId; + ring_index = 0; } -int Request::hashCode() -{ - return (int) ((start << 32) >> 32); -} +int Request::hashCode() { return (int)((start << 32) >> 32); } -int Request::comparedTo(Request o) -{ - return start == o.start; -} +int Request::comparedTo(Request o) { return start == o.start; } diff --git a/cpp/pixels-common/lib/physical/SchedulerFactory.cpp b/cpp/pixels-common/lib/physical/SchedulerFactory.cpp index 77790890ba..2ee5f4025f 100644 --- a/cpp/pixels-common/lib/physical/SchedulerFactory.cpp +++ b/cpp/pixels-common/lib/physical/SchedulerFactory.cpp @@ -24,9 +24,9 @@ */ #include "physical/SchedulerFactory.h" -SchedulerFactory *SchedulerFactory::instance = nullptr; +SchedulerFactory* SchedulerFactory::instance = nullptr; -SchedulerFactory *SchedulerFactory::Instance() +SchedulerFactory* SchedulerFactory::Instance() { if (instance == nullptr) { @@ -35,7 +35,7 @@ SchedulerFactory *SchedulerFactory::Instance() return instance; } -Scheduler *SchedulerFactory::getScheduler() +Scheduler* SchedulerFactory::getScheduler() { return scheduler; } diff --git a/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp b/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp index d4ff5707ef..c5ffd5e88b 100644 --- a/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp +++ b/cpp/pixels-common/lib/physical/natives/DirectIoLib.cpp @@ -23,6 +23,10 @@ * @create 2023-04-19 */ #include "physical/natives/DirectIoLib.h" +#include // mmap, munmap, madvise +#include +#include +#include DirectIoLib::DirectIoLib(int fsBlockSize) @@ -31,15 +35,109 @@ DirectIoLib::DirectIoLib(int fsBlockSize) this->fsBlockNotMask = ~((long) fsBlockSize - 1); } -std::shared_ptr DirectIoLib::allocateDirectBuffer(long size) +template struct huge_page_allocator { + constexpr static std::size_t huge_page_size = 1 << 21; // 2 MiB + using value_type = T; + + huge_page_allocator() = default; + template + constexpr huge_page_allocator(const huge_page_allocator &) noexcept {} + + size_t round_to_huge_page_size(size_t n) { + return (((n - 1) / huge_page_size) + 1) * huge_page_size; + } + + T *allocate(std::size_t n) { + if (n > std::numeric_limits::max() / sizeof(T)) { + throw std::bad_alloc(); + } + auto p = static_cast(mmap( + nullptr, round_to_huge_page_size(n * sizeof(T)), PROT_READ | PROT_WRITE, + MAP_PRIVATE | MAP_ANONYMOUS | MAP_HUGETLB, -1, 0)); + if (p == MAP_FAILED) { + throw std::bad_alloc(); + } + return p; + } + + void deallocate(T *p, std::size_t n) { + munmap(p, round_to_huge_page_size(n * sizeof(T))); + } +}; + +std::shared_ptr DirectIoLib::allocateDirectBuffer(long size, bool isSmallBuffer) { - int toAllocate = blockEnd(size) + (size == 1 ? 0 : fsBlockSize); - uint8_t *directBufferPointer; - posix_memalign((void **) &directBufferPointer, fsBlockSize, toAllocate); - auto directBuffer = std::make_shared(directBufferPointer, toAllocate, false); - return directBuffer; + // Lambda function to convert string to boolean + auto strToBool = [](const std::string& s) { + return s == "true" || s == "1" || s == "yes"; + }; + + // Get configuration value for huge page buffer pool + std::string configValue = ConfigFactory::Instance().getProperty("pixel.bufferpool.hugepage"); + bool isHugePage = strToBool(configValue); + + if (isHugePage && !isSmallBuffer) { + // Allocate memory using huge page allocator + try { + // Calculate the size to allocate (considering block alignment) + int toAllocate = blockEnd(size) + (size == 1 ? 0 : fsBlockSize); + + // Create huge page allocator instance + huge_page_allocator allocator; + + // Allocate memory + uint8_t* directBufferPointer = allocator.allocate(toAllocate); + + // Create custom deleter to free memory using the allocator + auto deleter = [allocator, toAllocate](ByteBuffer* buf) mutable { + if (buf && buf->getBuffer()) { + // Deallocate memory using the allocator's deallocate method + allocator.deallocate(const_cast(buf->getBuffer()), toAllocate); + } + // delete buf; // Free the ByteBuffer object itself + }; + + // Create ByteBuffer (does not take over memory ownership, managed by shared_ptr's deleter) + ByteBuffer* buf = new ByteBuffer(directBufferPointer, toAllocate, false); + + // Return shared_ptr with custom deleter + return std::shared_ptr(buf, deleter); + } + catch (const std::bad_alloc& e) { + throw std::runtime_error("Huge page memory allocation failed: " + std::string(e.what()) + + ", error code: " + std::string(strerror(errno))); + } + } else { + // Normal memory allocation path + int toAllocate = blockEnd(size) + (size == 1 ? 0 : fsBlockSize); + uint8_t* directBufferPointer; + + // Allocate aligned memory using posix_memalign + if (posix_memalign((void**)&directBufferPointer, fsBlockSize, toAllocate) != 0) { + throw std::runtime_error("Normal memory allocation failed: " + std::string(strerror(errno))); + } + + // Create custom deleter to free memory + auto deleter = [](ByteBuffer* buf) { + if (buf && buf->getBuffer()) { + free(const_cast(buf->getBuffer())); + } + // delete buf; + }; + + // Create ByteBuffer and return shared_ptr + ByteBuffer* buf = new ByteBuffer(directBufferPointer, toAllocate, false); + return std::shared_ptr(buf, deleter); + } } +int DirectIoLib::getToAllocate(int size) { + return blockEnd(size) + (size == 1 ? 0 : fsBlockSize); +} + + + + std::shared_ptr DirectIoLib::read(int fd, long fileOffset, std::shared_ptr directBuffer, long length) { diff --git a/cpp/pixels-common/lib/physical/natives/DirectRandomAccessFile.cpp b/cpp/pixels-common/lib/physical/natives/DirectRandomAccessFile.cpp index 832b980019..786b47bdae 100644 --- a/cpp/pixels-common/lib/physical/natives/DirectRandomAccessFile.cpp +++ b/cpp/pixels-common/lib/physical/natives/DirectRandomAccessFile.cpp @@ -64,7 +64,7 @@ DirectRandomAccessFile::DirectRandomAccessFile(const std::string &file) directIoLib = std::make_shared(fsBlockSize); try { - smallDirectBuffer = directIoLib->allocateDirectBuffer(fsBlockSize); + smallDirectBuffer = directIoLib->allocateDirectBuffer(fsBlockSize,true); } catch (...) { @@ -91,7 +91,7 @@ std::shared_ptr DirectRandomAccessFile::readFully(int len) { if (enableDirect) { - auto directBuffer = directIoLib->allocateDirectBuffer(len); + auto directBuffer = directIoLib->allocateDirectBuffer(len,true); auto buffer = directIoLib->read(fd, offset, directBuffer, len); seek(offset + len); largeBuffers.emplace_back(directBuffer); diff --git a/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp b/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp index 22966390fe..8b53974fec 100644 --- a/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp +++ b/cpp/pixels-common/lib/physical/natives/DirectUringRandomAccessFile.cpp @@ -26,90 +26,69 @@ #include - // global std::mutex DirectUringRandomAccessFile::mutex_; // thread local thread_local bool DirectUringRandomAccessFile::isRegistered = false; -thread_local std::vector DirectUringRandomAccessFile::ring_vector; -thread_local std::vector DirectUringRandomAccessFile::iovecs_vector; -thread_local std::vector DirectUringRandomAccessFile::offsets_vector; -thread_local uint32_t DirectUringRandomAccessFile::iovecSize = 0; - - -// static thread_local int memset_count = 0; -// static thread_local std::chrono::nanoseconds total_time0(0); // 总耗时(纳秒) -// static thread_local std::chrono::nanoseconds total_time(0); // memest总耗时(纳秒) - -DirectUringRandomAccessFile::DirectUringRandomAccessFile(const std::string &file) : DirectRandomAccessFile(file) +thread_local std::vector +DirectUringRandomAccessFile::ring_vector; +thread_local std::vector +DirectUringRandomAccessFile::iovecs_vector; +thread_local std::vector DirectUringRandomAccessFile::offsets_vector; +thread_local uint32_t DirectUringRandomAccessFile::iovecSize = 0; + +DirectUringRandomAccessFile::DirectUringRandomAccessFile( + const std::string& file) + : DirectRandomAccessFile(file) { - } -void DirectUringRandomAccessFile::RegisterBufferFromPool(std::vector colIds) +void DirectUringRandomAccessFile::RegisterBufferFromPool( + std::vector colIds) { - // std::cout << "函数:RegisterBufferFromPool 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; - // std::lock_guard lock(mutex_); - if (!isRegistered) { - - std::vector > tmpBuffers; - auto ring=DirectUringRandomAccessFile::getRing(0); - struct iovec* iovecs=nullptr; - for (auto buffers : ::BufferPool::registeredBuffers){ - for (auto buffer:buffers) { + if (!isRegistered) + { + std::vector> tmpBuffers; + auto ring = DirectUringRandomAccessFile::getRing(0); + struct iovec* iovecs = nullptr; + for (auto buffers : ::BufferPool::registeredBuffers) + { + for (auto buffer : buffers) + { buffer->setIsRegistered(true); tmpBuffers.emplace_back(buffer->getBuffer()); } } - // auto start0 = std::chrono::high_resolution_clock::now(); - iovecs = (iovec *) calloc(tmpBuffers.size(), sizeof(struct iovec)); + iovecs = (iovec*)calloc(tmpBuffers.size(), sizeof(struct iovec)); iovecSize = tmpBuffers.size(); for (auto i = 0; i < tmpBuffers.size(); i++) { auto buffer = tmpBuffers.at(i); iovecs[i].iov_base = buffer->getPointer(); iovecs[i].iov_len = buffer->size(); - // auto start = std::chrono::high_resolution_clock::now(); - - // 目标代码 - // memset(iovecs[i].iov_base, 0, buffer->size()); - - // 记录结束时间,累加耗时 - // auto end = std::chrono::high_resolution_clock::now(); - // total_time += std::chrono::duration_cast(end - start); - // total_time0+=std::chrono::duration_cast(end-start0); - // 累加次数 - // memset_count++; } - // std::cout << "memset执行次数:" << memset_count << std::endl; - // 转换为易读单位(如微秒/毫秒) - // auto total_us0 = std::chrono::duration_cast(total_time0).count(); - // auto total_us = std::chrono::duration_cast(total_time).count(); - // std::cout << "总用时:" << total_us0 << " 微秒" << std::endl; - // std::cout << "memset用时:" << total_us << " 微秒" << std::endl; iovecs_vector.emplace_back(iovecs); int ret = io_uring_register_buffers(ring, iovecs, iovecSize); - // std::cout<<"ret:"<> buffers) +void DirectUringRandomAccessFile::RegisterBuffer( + std::vector> buffers) { - - // std::cout << "函数:RegisterBuffer 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; if (!isRegistered) { - auto ring=DirectUringRandomAccessFile::getRing(0); - struct iovec* iovecs=nullptr; - iovecs = (iovec *) calloc(buffers.size(), sizeof(struct iovec)); + auto ring = DirectUringRandomAccessFile::getRing(0); + struct iovec* iovecs = nullptr; + iovecs = (iovec*)calloc(buffers.size(), sizeof(struct iovec)); iovecSize = buffers.size(); for (auto i = 0; i < buffers.size(); i++) { @@ -123,46 +102,34 @@ void DirectUringRandomAccessFile::RegisterBuffer(std::vector lock(mutex_); - // std::cout << "函数:Initialize 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; getRing(0); - // std::cout << "函数:Initialze 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; - - // // initialize io_uring ring - // struct io_uring *ring=nullptr; - // if (getRing(0) == nullptr) - // { - // ring = new io_uring(); - // if (io_uring_queue_init(4096, ring, 0) < 0) - // { - // throw InvalidArgumentException("DirectRandomAccessFile: initialize io_uring fails."); - // } - // } - // ring_vector.emplace_back(ring); - // offsets_vector.emplace_back(0); } void DirectUringRandomAccessFile::Reset() { - // std::cout << "函数:Reset 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; // Important! Because sometimes ring is nullptr here. - // For example, two threads A and B share the same global state. If A finish all files while B just starts, - // B would execute Reset function from InitLocal. If we don't set this 'if' branch, ring would be double freed. - for (auto ring :ring_vector) { + // For example, two threads A and B share the same global state. If A finish + // all files while B just starts, B would execute Reset function from + // InitLocal. If we don't set this 'if' branch, ring would be double freed. + for (auto ring : ring_vector) + { if (ring != nullptr) { // We don't use this function anymore since it slows down the speed // if(io_uring_unregister_buffers(ring) != 0) { - // throw InvalidArgumentException("DirectUringRandomAccessFile::UnregisterBuffer: unregister buffer fails. "); + // throw + //InvalidArgumentException("DirectUringRandomAccessFile::UnregisterBuffer: + //unregister buffer fails. "); // } io_uring_queue_exit(ring); delete (ring); @@ -171,7 +138,8 @@ void DirectUringRandomAccessFile::Reset() } } ring_vector.clear(); - for (auto iovecs:iovecs_vector) { + for (auto iovecs : iovecs_vector) + { if (iovecs != nullptr) { free(iovecs); @@ -179,246 +147,188 @@ void DirectUringRandomAccessFile::Reset() } } iovecs_vector.clear(); - std::cout << "函数:Reset 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; } -bool DirectUringRandomAccessFile::RegisterMoreBuffer(int index,std::vector> buffers) { - // std::lock_guard lock(mutex_); - std::cout << "函数:RegisterMoreBuffer 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; +bool DirectUringRandomAccessFile::RegisterMoreBuffer( + int index, std::vector> buffers) +{ assert(isRegistered); - std::cout<<"现在有"<getPointer(); iovecs[i].iov_len = buffer->size(); - // memset(iovecs[i].iov_base, 0, buffer->size()); } iovecs_vector.emplace_back(iovecs); int ret = io_uring_register_buffers(ring, iovecs, iovecSize); if (ret != 0) { - throw InvalidArgumentException("DirectUringRandomAccessFile::RegisterMoreBuffer: register buffer fails. "); + throw InvalidArgumentException( + "DirectUringRandomAccessFile::RegisterMoreBuffer: register buffer " + "fails. "); } - // std::cout << "函数:RegisterMoreBuffer 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; return true; - } DirectUringRandomAccessFile::~DirectUringRandomAccessFile() { - } -std::shared_ptr -DirectUringRandomAccessFile::readAsync(int length, std::shared_ptr buffer, int index,int ring_index,int start_offset) +std::shared_ptr DirectUringRandomAccessFile::readAsync( + int length, std::shared_ptr buffer, int index, int ring_index, + int start_offset) { - // std::cout << "函数:readAsync 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; - // std::lock_guard lock(mutex_); - auto ring=DirectUringRandomAccessFile::getRing(ring_index); + auto ring = DirectUringRandomAccessFile::getRing(ring_index); auto offset = start_offset; if (enableDirect) { - struct io_uring_sqe *sqe = io_uring_get_sqe(ring); - if (!sqe) { - throw std::runtime_error("DirectUringRandomAccessFile::readAsync: failed to get SQE, submission queue is full"); + struct io_uring_sqe* sqe = io_uring_get_sqe(ring); + if (!sqe) + { + throw std::runtime_error("DirectUringRandomAccessFile::readAsync: failed " + "to get SQE, submission queue is full"); } -// if(length > iovecs[index].iov_len) { -// throw InvalidArgumentException("DirectUringRandomAccessFile::readAsync: the length is larger than buffer length."); -// } - // the file will be read from blockStart(fileOffset), and the first fileDelta bytes should be ignored. + // the file will be read from blockStart(fileOffset), and the first + // fileDelta bytes should be ignored. uint64_t fileOffsetAligned = directIoLib->blockStart(offset); - uint64_t toRead = directIoLib->blockEnd(offset + length) - directIoLib->blockStart(offset); - // 检查缓冲区大小是否足够 + uint64_t toRead = directIoLib->blockEnd(offset + length) - + directIoLib->blockStart(offset); size_t block_size = directIoLib->getBlockSize(); // 假设存在获取块大小的方法 size_t required_buffer_size = (offset - fileOffsetAligned) + length; - if (buffer->size() < required_buffer_size) { + if (buffer->size() < required_buffer_size) + { std::stringstream ss; - std::cout << "DirectUringRandomAccessFile::readAsync: buffer size insufficient. " - << "Required: " << required_buffer_size << ", Actual: " << buffer->size() - << ", ring_index: " << ring_index << ", index: " << index - << "Required alignment: " << block_size - <<", length:"<size() + << ", ring_index: " << ring_index << ", index: " << index + << "Required alignment: " << block_size << ", length:" << length + << std::endl;; throw InvalidArgumentException(ss.str()); } - // 检查缓冲区对齐是否符合直接I/O要求 - uintptr_t buffer_addr = reinterpret_cast(buffer->getPointer()); - if (buffer_addr % block_size != 0) { - std::stringstream ss; - ss << "DirectUringRandomAccessFile::readAsync: buffer misaligned. " - << "Required alignment: " << block_size << " bytes, " - << "Actual address: 0x" << std::hex << buffer_addr - << ", ring_index: " << ring_index << ", index: " << index - <<", length:"<getPointer(), toRead, fileOffsetAligned, index); - if (fd < 0) { - throw std::runtime_error("DirectUringRandomAccessFile::readAsync: invalid file descriptor"); + if (fd < 0) + { + throw std::runtime_error( + "DirectUringRandomAccessFile::readAsync: invalid file descriptor"); } - // if (buffer->size()(buffer->getPointer())<<" index:"<size()<<" startID:"<(*buffer, - offset - fileOffsetAligned, length); - seekByIndex(offset + length,ring_index); - // if (ring_index!=0) { - // // std::cout<<"notice uring File index:"<(buffer->getPointer())<<" start_offset"<(buffer->getPointer())<<" index:"<(bb->getPointer())<< - // " buffer size:"<size()<<" startID:"<(buffer->getPointer())<<" index:"<(bb->getPointer())<(*buffer, offset - fileOffsetAligned, + length); + seekByIndex(offset + length, ring_index); return bb; } else { - struct io_uring_sqe *sqe = io_uring_get_sqe(ring); -// if(length > iovecs[index].iov_len) { -// throw InvalidArgumentException("DirectUringRandomAccessFile::readAsync: the length is larger than buffer length."); -// } - io_uring_prep_read_fixed(sqe, fd, buffer->getPointer(), length, offset, index); + struct io_uring_sqe* sqe = io_uring_get_sqe(ring); + io_uring_prep_read_fixed(sqe, fd, buffer->getPointer(), length, offset, + index); seek(offset + length); auto result = std::make_shared(*buffer, 0, length); return result; } - } -void DirectUringRandomAccessFile::seekByIndex(long off,int index) { - // 检查索引有效性 - if (index < 0 || static_cast(index) >= offsets_vector.size()) { + +void DirectUringRandomAccessFile::seekByIndex(long off, int index) +{ + if (index < 0 || static_cast(index) >= offsets_vector.size()) + { std::stringstream ss; ss << "DirectUringRandomAccessFile::seekByIndex: invalid index. " - << "Index: " << index << ", Vector size: " << offsets_vector.size(); + << "Index: " << index << ", Vector size: " << offsets_vector.size(); throw InvalidArgumentException(ss.str()); } - // 检查偏移量有效性(假设偏移量不能为负) - if (off < 0) { + if (off < 0) + { std::stringstream ss; ss << "DirectUringRandomAccessFile::seekByIndex: invalid offset. " - << "Offset: " << off << ", Index: " << index; + << "Offset: " << off << ", Index: " << index; throw InvalidArgumentException(ss.str()); } - offsets_vector.at(index)=off; + offsets_vector.at(index) = off; } - - -void DirectUringRandomAccessFile::readAsyncSubmit(std::unordered_map sizes,std::unordered_set ring_index) +void DirectUringRandomAccessFile::readAsyncSubmit( + std::unordered_map sizes, + std::unordered_set ring_index) { - // std::lock_guard lock(mutex_); - // std::cout << "函数:readAsyncSubmit 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; - for (auto i:ring_index) { - auto ring=DirectUringRandomAccessFile::getRing(i); + for (auto i : ring_index) + { + auto ring = DirectUringRandomAccessFile::getRing(i); int ret = io_uring_submit(ring); - // std::cout<<"线程:"< sizes,std::unordered_set ring_index) +void DirectUringRandomAccessFile::readAsyncComplete( + std::unordered_map sizes, + std::unordered_set ring_index) { - // std::lock_guard lock(mutex_); - // std::cout << "函数:readAsyncComplete 线程 " << std::this_thread::get_id() << " 成功获取global_mutex" << std::endl; - // Important! We cannot write the code as io_uring_wait_cqe_nr(ring, &cqe, iovecSize). - // The reason is unclear, but some random bugs would happen. It takes me nearly a week to find this bug - for (auto idx : ring_index) { // 重命名外层循环变量,避免与内层冲突 - struct io_uring_cqe *cqe; + // Important! We cannot write the code as io_uring_wait_cqe_nr(ring, &cqe, + // iovecSize). The reason is unclear, but some random bugs would happen. It + // takes me nearly a week to find this bug + for (auto idx : ring_index) + { + // 重命名外层循环变量,避免与内层冲突 + struct io_uring_cqe* cqe; auto ring = DirectUringRandomAccessFile::getRing(idx); - - // 为当前ring设置超时时间(根据实际场景调整,这里设为5秒) - struct __kernel_timespec timeout = { - .tv_sec = 1, // 秒 - .tv_nsec = 0 // 纳秒 - }; - - // 内层循环使用j作为变量,避免与外层idx冲突 - for (int j = 0; j < sizes[idx]; j++) // 注意:这里使用外层的idx作为sizes的键 + for (int j = 0; j < sizes[idx]; j++) // 注意:这里使用外层的idx作为sizes的键 { - if (io_uring_wait_cqe_nr(ring, &cqe, 1) != 0) { - throw InvalidArgumentException("DirectUringRandomAccessFile::readAsyncComplete: wait cqe fails"); + throw InvalidArgumentException( + "DirectUringRandomAccessFile::readAsyncComplete: wait cqe fails"); } io_uring_cqe_seen(ring, cqe); - // 使用带超时的等待函数,替代原有的io_uring_wait_cqe_nr - // int ret = io_uring_wait_cqe_timeout(ring, &cqe, &timeout); - // if (ret != 0) { - // if (ret == -ETIME) { - // // 超时错误:明确抛出超时异常,便于上层处理 - // throw InvalidArgumentException( - // "DirectUringRandomAccessFile::readAsyncComplete: " - // "wait cqe timeout for ring index " + std::to_string(idx) + - // " after " + std::to_string(timeout.tv_sec) + " seconds" - // ); - // } else if (ret == -EINTR) { - // // 信号中断:可根据需求选择重试或抛出异常 - // throw InvalidArgumentException( - // "DirectUringRandomAccessFile::readAsyncComplete: " - // "wait cqe interrupted by signal for ring index " + std::to_string(idx) - // ); - // } else { - // // 其他错误(如参数错误、ring已关闭等) - // throw InvalidArgumentException( - // "DirectUringRandomAccessFile::readAsyncComplete: " - // "wait cqe fails with error " + std::to_string(ret) + - // " for ring index " + std::to_string(idx) - // ); - // } - // } - // - // // 检查CQE结果是否正常(可选,根据业务需求) - // if (cqe->res < 0) { - // throw InvalidArgumentException( - // "DirectUringRandomAccessFile::readAsyncComplete: " - // "I/O operation failed with error " + std::to_string(cqe->res) + - // " for ring index " + std::to_string(idx) - // ); - // } - // - // io_uring_cqe_seen(ring, cqe); } } - // std::cout << "函数:readAsyncComplete 线程 " << std::this_thread::get_id() << " 成功释放global_mutex" << std::endl; } -struct io_uring *DirectUringRandomAccessFile::getRing(int index) { - if (index>=ring_vector.size()) { +struct io_uring* DirectUringRandomAccessFile::getRing(int index) +{ + if (index >= ring_vector.size()) + { // need to add more rings // initialize io_uring ring - struct io_uring *ring=nullptr; + struct io_uring* ring = nullptr; + auto flag = std::stoi( + ConfigFactory::Instance().getProperty("pixels.io_uring.mode")); + // get io_uring flags for io mode + // 0 interrupt-dirven + // 1 IORING_SETUP_IOPOLL + // 2 IORING_SETUP_SQPOLL if (ring == nullptr) { ring = new io_uring(); - if (io_uring_queue_init(4096, ring, 0) < 0) + if (io_uring_queue_init(4096, ring, flag) < 0) { - throw InvalidArgumentException("DirectRandomAccessFile: initialize io_uring fails."); + throw InvalidArgumentException( + "DirectRandomAccessFile: initialize io_uring fails."); } } ring_vector.emplace_back(ring); offsets_vector.emplace_back(0); } - return ring_vector[index]; + return ring_vector[index]; } - diff --git a/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp b/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp index 21314b571a..c49856d268 100644 --- a/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp +++ b/cpp/pixels-common/lib/physical/scheduler/NoopScheduler.cpp @@ -27,9 +27,9 @@ #include "physical/io/PhysicalLocalReader.h" #include -Scheduler *NoopScheduler::instance = nullptr; +Scheduler* NoopScheduler::instance = nullptr; -Scheduler *NoopScheduler::Instance() +Scheduler* NoopScheduler::Instance() { if (instance == nullptr) { @@ -38,42 +38,40 @@ Scheduler *NoopScheduler::Instance() return instance; } -std::vector > NoopScheduler::executeBatch(std::shared_ptr reader, - RequestBatch batch, long queryId) +std::vector> +NoopScheduler::executeBatch(std::shared_ptr reader, + RequestBatch batch, long queryId) { return executeBatch(reader, batch, {}, queryId); } - -std::vector > -NoopScheduler::executeBatch(std::shared_ptr reader, RequestBatch batch, - std::vector > reuseBuffers, long queryId) +std::vector> NoopScheduler::executeBatch( + std::shared_ptr reader, RequestBatch batch, + std::vector> reuseBuffers, long queryId) { - - std::lock_guard lock(mtx); // 进入临界区,自动加锁 auto requests = batch.getRequests(); - std::vector > results; + std::vector> results; results.resize(batch.getSize()); - if (ConfigFactory::Instance().boolCheckProperty("localfs.enable.async.io") && reuseBuffers.size() > 0) + if (ConfigFactory::Instance().boolCheckProperty("localfs.enable.async.io") && + reuseBuffers.size() > 0) { // async read auto localReader = std::static_pointer_cast(reader); - std::unordered_set ring_index_set=localReader->getRingIndexes(); + std::unordered_set ring_index_set = localReader->getRingIndexes(); std::unordered_map ringIndexCountMap; for (int i = 0; i < batch.getSize(); i++) { Request request = requests[i]; - // localReader->seek(request.start); - if (request.length>reuseBuffers.at(i)->size()) { - throw InvalidArgumentException("说明出错的不是这个,需要关注之前的临界区\n"); - + if (request.length > reuseBuffers.at(i)->size()) + { + throw InvalidArgumentException( + "The error is not here; need to pay attention to the previous " + "critical section\n"); } - results.at(i) = localReader->readAsync(request.length, reuseBuffers.at(i), request.bufferId,request.ring_index,request.start); - // if (request.ring_index !=0) { - // std::cout<<"notice readAsync"<size()<readAsync(request.length, reuseBuffers.at(i), + request.bufferId, + request.ring_index, request.start); if (ring_index_set.find(request.ring_index) == ring_index_set.end()) { ring_index_set.insert(request.ring_index); @@ -81,9 +79,8 @@ NoopScheduler::executeBatch(std::shared_ptr reader, RequestBatc } ringIndexCountMap[request.ring_index]++; } - localReader->readAsyncSubmit(ringIndexCountMap,ring_index_set); + localReader->readAsyncSubmit(ringIndexCountMap, ring_index_set); localReader->setRingIndexCountMap(ringIndexCountMap); - } else { @@ -100,14 +97,11 @@ NoopScheduler::executeBatch(std::shared_ptr reader, RequestBatc { results.at(i) = reader->readFully(request.length); } - } } return results; - } - NoopScheduler::~NoopScheduler() { delete instance; diff --git a/cpp/pixels-common/lib/utils/MutexTracker.cpp b/cpp/pixels-common/lib/utils/MutexTracker.cpp index 6e26ec9fac..d4b006d9f6 100644 --- a/cpp/pixels-common/lib/utils/MutexTracker.cpp +++ b/cpp/pixels-common/lib/utils/MutexTracker.cpp @@ -1,5 +1,25 @@ /* -/* @author whz -/* @create 8/23/25. -*/ +* Copyright 2025 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ + +/* + * @author whz + * @create 2025-08-23 + */ #include "utils/MutexTracker.h" \ No newline at end of file diff --git a/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h b/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h index 4ee9f2c99b..c098482a58 100644 --- a/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h +++ b/cpp/pixels-core/include/reader/PixelsRecordReaderImpl.h @@ -98,7 +98,6 @@ class PixelsRecordReaderImpl : public PixelsRecordReader void UpdateRowGroupInfo(); - static std::mutex mutex_; std::shared_ptr physicalReader; pixels::proto::Footer footer; diff --git a/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp b/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp index f3c3784540..66b9b01da9 100644 --- a/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp +++ b/cpp/pixels-core/lib/reader/PixelsReaderOption.cpp @@ -75,7 +75,6 @@ void PixelsReaderOption::setRingIndex(int i) { ring_index = i; } - void PixelsReaderOption::setRGRange(int start, int len) { rgStart = start; diff --git a/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp b/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp index 9d6322b9fc..5c6f646938 100644 --- a/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp +++ b/cpp/pixels-core/lib/reader/PixelsRecordReaderImpl.cpp @@ -470,16 +470,11 @@ bool PixelsRecordReaderImpl::read() bytes.emplace_back(chunk.length); } - // std::cout<<"开始 现在是BufferID:"<<::BufferPool::GetBufferId()<getFieldNames(); ::BufferPool::Initialize(colIds, bytes, columnNames); - // std::cout<<"初始化bufferpool完毕?"<> originalByteBuffers; - // std::cout<<"初始化完毕?"< ring_col; for (int i = 0; i < colIds.size(); i++) { @@ -499,8 +494,6 @@ bool PixelsRecordReaderImpl::read() if (requestBatch.getRequest(i).ring_index !=0) { requestBatch.getRequest(i).bufferId=0; ring_col.emplace_back(i); - // std::cout<<"i:"<(originalByteBuffers.at(i)->getPointer())< bb = byteBuffers.at(index); uint32_t colId = chunk.columnId; - // if (std::find(ring_col.begin(), ring_col.end(), colId) != ring_col.end()) - // std::cout<<"colId "<(bb->getPointer())< input, input->skipBytes(inputLength - sizeof(int)); int startsOffset = input->getInt(); input->resetReaderIndex(); - // std::cout<<"input getBuffer:"<(input->getPointer())<<" startOffset:" - // <(*input, 0, startsOffset); startsBuf = std::make_shared( *input, startsOffset, inputLength - sizeof(int) - startsOffset); nextStart = startsBuf->getInt(); // read out the first start offset, which is 0 - // assert(inputLength>0 && startsOffset>=0); - if (startsOffset>inputLength||startsOffset<=0) { - // std::cout<<"debug"<increment(thisOutputChunkRows); } -bool PixelsScanFunction::PixelsParallelStateNext(ClientContext &context, const PixelsReadBindData &bind_data, +bool PixelsScanFunction::PixelsParallelStateNext(ClientContext &context, PixelsReadBindData &bind_data, PixelsReadLocalState &scan_data, PixelsReadGlobalState ¶llel_state, bool is_init_state) @@ -488,7 +488,7 @@ bool PixelsScanFunction::PixelsParallelStateNext(ClientContext &context, const P "PhysicalLocalReader::readAsync: We don't support aio for our async read yet."); } } - parallel_state.all_done = true; // 标记已完成,避免重复调用 + parallel_state.all_done = true; } parallel_lock.unlock(); return false; @@ -499,6 +499,7 @@ bool PixelsScanFunction::PixelsParallelStateNext(ClientContext &context, const P scan_data.next_file_index = parallel_state.file_index.at(scan_data.deviceID); scan_data.next_batch_index = StorageInstance->getBatchID(scan_data.deviceID, scan_data.next_file_index); scan_data.curr_file_name = scan_data.next_file_name; + bind_data.curFileId.fetch_add(1); parallel_state.file_index.at(scan_data.deviceID)++; parallel_lock.unlock(); // The below code uses global state but no race happens, so we don't need the lock anymore diff --git a/cpp/pixels-duckdb/duckdb b/cpp/pixels-duckdb/duckdb index 378f77c5c9..e0af5da3aa 160000 --- a/cpp/pixels-duckdb/duckdb +++ b/cpp/pixels-duckdb/duckdb @@ -1 +1 @@ -Subproject commit 378f77c5c93f3e66453bbc9f829b2278527fbd56 +Subproject commit e0af5da3aaddf16c7b54bc1acaba94d5c2bcdd73 diff --git a/cpp/pixels-duckdb/pixels_extension.cpp b/cpp/pixels-duckdb/pixels_extension.cpp index 2ab79edc5e..4b2c61f1cc 100644 --- a/cpp/pixels-duckdb/pixels_extension.cpp +++ b/cpp/pixels-duckdb/pixels_extension.cpp @@ -25,74 +25,66 @@ #define DUCKDB_EXTENSION_MAIN #include "pixels_extension.hpp" -#include "PixelsScanFunction.hpp" #include "PixelsReadBindData.hpp" +#include "PixelsScanFunction.hpp" #include "duckdb.hpp" #include "duckdb/common/exception.hpp" +#include "duckdb/common/optional_ptr.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/function/scalar_function.hpp" -#include "duckdb/common/optional_ptr.hpp" #include -namespace duckdb -{ - - // Pixels Scan Replacement for duckdb 1.1 - unique_ptr PixelsScanReplacement(ClientContext &context, ReplacementScanInput &input, - optional_ptr data) - { - auto table_name = ReplacementScan::GetFullPath(input); -// if(!ReplacementScan::CanReplace(table_name,{"pixels"})){ -// return nullptr; -// } - auto lower_name = StringUtil::Lower(table_name); - if (!StringUtil::EndsWith(lower_name, ".pxl") && !StringUtil::Contains(lower_name, ".pxl?")) - { - return nullptr; - } - auto table_function = make_uniq(); - vector > children; - children.push_back(make_uniq(Value(table_name))); - table_function->function = make_uniq("pixels_scan", std::move(children)); - if (!FileSystem::HasGlob(table_name)) - { - auto &fs = FileSystem::GetFileSystem(context); - table_function->alias = fs.ExtractBaseName(table_name); - } - return std::move(table_function); - } - - - void PixelsExtension::Load(ExtensionLoader &loader) { - auto &db_instance=loader.GetDatabaseInstance(); - Connection con(db_instance); - con.BeginTransaction(); - - auto &context=*con.context; - auto &catalog=Catalog::GetSystemCatalog(*con.context); - - auto scan_fun=PixelsScanFunction::GetFunctionSet(); - - CreateTableFunctionInfo cinfo(scan_fun); - cinfo.name = "pixels_scan"; - - catalog.CreateTableFunction(context, &cinfo); - con.Commit(); - - auto &config = DBConfig::GetConfig(db_instance); - config.replacement_scans.emplace_back(PixelsScanReplacement); - - } - - std::string PixelsExtension::Name() - { - return "pixels"; - } - +namespace duckdb { + +// Pixels Scan Replacement for duckdb 1.1 +unique_ptr +PixelsScanReplacement(ClientContext &context, ReplacementScanInput &input, + optional_ptr data) { + auto table_name = ReplacementScan::GetFullPath(input); + // if(!ReplacementScan::CanReplace(table_name,{"pixels"})){ + // return nullptr; + // } + auto lower_name = StringUtil::Lower(table_name); + if (!StringUtil::EndsWith(lower_name, ".pxl") && + !StringUtil::Contains(lower_name, ".pxl?")) { + return nullptr; + } + auto table_function = make_uniq(); + vector> children; + children.push_back(make_uniq(Value(table_name))); + table_function->function = + make_uniq("pixels_scan", std::move(children)); + if (!FileSystem::HasGlob(table_name)) { + auto &fs = FileSystem::GetFileSystem(context); + table_function->alias = fs.ExtractBaseName(table_name); + } + return std::move(table_function); +} + +void PixelsExtension::Load(ExtensionLoader &loader) { + auto &db_instance = loader.GetDatabaseInstance(); + Connection con(db_instance); + con.BeginTransaction(); + + auto &context = *con.context; + auto &catalog = Catalog::GetSystemCatalog(*con.context); + + auto scan_fun = PixelsScanFunction::GetFunctionSet(); + + CreateTableFunctionInfo cinfo(scan_fun); + cinfo.name = "pixels_scan"; + + catalog.CreateTableFunction(context, &cinfo); + con.Commit(); + + auto &config = DBConfig::GetConfig(db_instance); + config.replacement_scans.emplace_back(PixelsScanReplacement); +} + +std::string PixelsExtension::Name() { return "pixels"; } } // namespace duckdb #ifndef DUCKDB_EXTENSION_MAIN #error DUCKDB_EXTENSION_MAIN not defined #endif - diff --git a/cpp/process_sqls.py b/cpp/process_sqls.py new file mode 100755 index 0000000000..a3b9500f92 --- /dev/null +++ b/cpp/process_sqls.py @@ -0,0 +1,254 @@ +import os +import re +import subprocess +import csv +import time +import psutil # For checking residual processes +from typing import List +import argparse + +# -------------------------- 1. Configuration Parameters (overridable via CLI) -------------------------- +def parse_args(): + parser = argparse.ArgumentParser(description="DuckDB ClickBench Batch Test Script (Multi-column CSV, ensures resource release)") + parser.add_argument( + "--runs", + type=int, + default=3, + help="Number of runs per SQL file (default: 3)" + ) + parser.add_argument( + "--duckdb-bin", + type=str, + default="/home/whz/test/pixels/cpp/build/release/duckdb", + help="Path to duckdb executable" + ) + parser.add_argument( + "--sql-dir", + type=str, + default="/home/whz/test/pixels/cpp/pixels-duckdb/duckdb/benchmark/clickbench/queries", + help="Directory containing SQL files (only processes .sql files starting with 'q')" + ) + parser.add_argument( + "--output-csv", + type=str, + default="/home/whz/test/pixels/cpp/duckdb_benchmark_result.csv", + help="Path to output result CSV" + ) + parser.add_argument( + "--wait-after-run", + type=float, + default=2.0, + help="Seconds to wait after each run (ensures resource release, default: 2s)" + ) + return parser.parse_args() + +# SQL to create hits view (unchanged) +CREATE_VIEW_SQL = """ +CREATE VIEW hits AS SELECT * FROM pixels_scan([ + '/data/9a3-01/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-02/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-03/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-04/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-05/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-06/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-07/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-08/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-09/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-10/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-11/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-12/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-13/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-14/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-15/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-16/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-17/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-18/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-19/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-20/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-21/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-22/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-23/clickbench/pixels-e0/hits/v-0-ordered/*', + '/data/9a3-24/clickbench/pixels-e0/hits/v-0-ordered/*' +]); +""" + +# -------------------------- 2. Core Utility Functions (added resource release checks) -------------------------- +def get_sql_files(sql_dir: str) -> List[str]: + """Get sorted list of SQL files starting with 'q' in target directory""" + sql_files = [] + for filename in os.listdir(sql_dir): + if filename.endswith(".sql") and filename.startswith("q"): + sql_files.append(os.path.join(sql_dir, filename)) + sql_files.sort() + if not sql_files: + raise ValueError(f"No .sql files starting with 'q' found in {sql_dir}!") + return sql_files + +def extract_real_time(duckdb_output: str) -> float: + """Extract real execution time (in seconds) from duckdb output""" + pattern = r"Run Time \(s\): real (\d+\.\d+)" + match = re.search(pattern, duckdb_output, re.MULTILINE) + if not match: + raise ValueError(f"Failed to extract real time! Partial output:\n{duckdb_output[:500]}...") + return round(float(match.group(1)), 3) + +def kill_remaining_duckdb(duckdb_bin: str): + """Check and kill residual duckdb processes to prevent resource leaks""" + duckdb_name = os.path.basename(duckdb_bin) + for proc in psutil.process_iter(['name', 'cmdline']): + try: + # Match processes by name or command line containing duckdb path + if (proc.info['name'] == duckdb_name) or (duckdb_bin in ' '.join(proc.info['cmdline'] or [])): + print(f"⚠️ Found residual {duckdb_name} process (PID: {proc.pid}), killing...") + proc.terminate() + # Force kill if not terminated within 1 second + try: + proc.wait(timeout=1) + except psutil.TimeoutExpired: + proc.kill() + except (psutil.NoSuchProcess, psutil.AccessDenied): + continue + +def run_single_sql(duckdb_bin: str, sql_content: str, wait_after_run: float) -> float: + """Run SQL once, ensure process exits and resources are released before returning""" + duckdb_commands = f"{CREATE_VIEW_SQL.strip()}\nset threads=48;\n\n.timer on\n{sql_content.strip()}\n.exit" + process = None # Initialize process variable for exception handling + + try: + # Use stdin pipe for Python 2/3 compatibility (instead of input parameter) + process = subprocess.Popen( + [duckdb_bin], + stdin=subprocess.PIPE, + stdout=subprocess.PIPE, + stderr=subprocess.STDOUT + ) + + # Handle encoding differences between Python 2/3 + if isinstance(duckdb_commands, str): + input_data = duckdb_commands.encode("utf-8") # Python 3: string to bytes + else: + input_data = duckdb_commands # Python 2: use string directly + + # Pass input and wait for completion (timeout after 1 hour) + stdout, _ = process.communicate(input=input_data, timeout=360) + + # Decode output (Python 3 returns bytes, Python 2 returns string) + if isinstance(stdout, bytes): + output = stdout.decode("utf-8", errors="ignore") + else: + output = stdout + + # Check exit code + if process.returncode != 0: + raise RuntimeError( + f"duckdb execution failed (code {process.returncode}):\n{output[:1000]}..." + ) + + # Extract and return execution time + real_time = extract_real_time(output) + + # Wait to ensure resource release + time.sleep(wait_after_run) + + # Final check for residual processes + kill_remaining_duckdb(duckdb_bin) + + return real_time + + except subprocess.TimeoutExpired: + if process: + process.kill() + raise RuntimeError("duckdb execution timed out (exceeded 1 hour)") from None + finally: + # Ensure process is terminated + if process and process.poll() is None: + process.kill() + print("⚠️ Forcibly terminated non-exiting duckdb process") + +def init_csv(output_csv: str, runs: int): + """Initialize CSV file with headers""" + headers = ["SQL File Name"] + [f"Run {idx} Time (s)" for idx in range(1, runs + 1)] + with open(output_csv, "w", newline="", encoding="utf-8") as f: + writer = csv.DictWriter(f, fieldnames=headers) + writer.writeheader() + print(f"✅ Initialized multi-column CSV with headers: {','.join(headers)}") + +def write_single_row(output_csv: str, sql_filename: str, run_times: List[float], runs: int): + """Write single SQL file's results to CSV""" + row_data = {"SQL File Name": sql_filename} + for idx in range(1, runs + 1): + time_val = run_times[idx - 1] if (idx - 1) < len(run_times) else "" + row_data[f"Run {idx} Time (s)"] = time_val + with open(output_csv, "a", newline="", encoding="utf-8") as f: + writer = csv.DictWriter(f, fieldnames=row_data.keys()) + writer.writerow(row_data) + +# -------------------------- 3. Main Logic -------------------------- +def main(): + args = parse_args() + print("=" * 70) + print("DuckDB ClickBench Batch Test Script (Resource Release Ensured)") + print(f"Config: {args.runs} runs per SQL, {args.wait_after_run}s wait after each run") + print(f"DuckDB path: {args.duckdb_bin}") + print(f"SQL directory: {args.sql_dir}") + print(f"Output CSV: {args.output_csv}") + print("=" * 70) + + # Kill residual duckdb processes before starting + kill_remaining_duckdb(args.duckdb_bin) + + # Initialize result CSV + init_csv(args.output_csv, args.runs) + + # Get list of SQL files + try: + sql_files = get_sql_files(args.sql_dir) + print(f"\n✅ Found {len(sql_files)} eligible SQL files:") + for i, f in enumerate(sql_files, 1): + print(f" {i:2d}. {os.path.basename(f)}") + except ValueError as e: + print(f"\n❌ Error: {e}") + return + + # Process each SQL file + for sql_file in sql_files: + sql_filename = os.path.basename(sql_file).replace(".sql", "") + print(f"\n{'=' * 60}") + print(f"Processing: {sql_filename}.sql") + print(f"{'=' * 60}") + + # Read SQL content + try: + with open(sql_file, "r", encoding="utf-8") as f: + sql_content = f.read() + print(f"✅ Successfully read SQL file (content length: {len(sql_content)} chars)") + except Exception as e: + print(f"❌ Failed to read SQL file: {e}") + write_single_row(args.output_csv, sql_filename, [], args.runs) + continue + + # Run multiple times and record results + run_times = [] + for run_idx in range(1, args.runs + 1): + print(f"\n--- Run {run_idx:2d}/{args.runs} ---") + try: + real_time = run_single_sql(args.duckdb_bin, sql_content, args.wait_after_run) + run_times.append(real_time) + print(f"✅ Run successful, time: {real_time}s") + except (RuntimeError, ValueError) as e: + print(f"❌ Run failed: {e}") + continue + + # Write results to CSV + write_single_row(args.output_csv, sql_filename, run_times, args.runs) + print(f"\n✅ Written to CSV: {sql_filename}.sql → Valid runs: {len(run_times)}/{args.runs}") + + # Final check for residual processes after all tests + kill_remaining_duckdb(args.duckdb_bin) + print(f"\n{'=' * 70}") + print("All SQL files processed!") + print(f"Multi-column CSV: {args.output_csv}") + print("=" * 70) + +if __name__ == "__main__": + main() \ No newline at end of file