diff --git a/dwio/nimble/common/Buffer.h b/dwio/nimble/common/Buffer.h index 6408ca5..2e1d0f7 100644 --- a/dwio/nimble/common/Buffer.h +++ b/dwio/nimble/common/Buffer.h @@ -15,6 +15,8 @@ */ #pragma once +#include "dwio/nimble/common/Exceptions.h" +#include "folly/fibers/Semaphore.h" #include "velox/buffer/Buffer.h" #include "velox/common/memory/Memory.h" @@ -33,12 +35,11 @@ // and so on namespace facebook::nimble { +using MemoryPool = facebook::velox::memory::MemoryPool; -// Internally manages memory in chunks. Releases memory only upon destruction. +// Internally manages memory in chunks. releases memory when destroyed // Buffer is NOT threadsafe: external locking is required. class Buffer { - using MemoryPool = facebook::velox::memory::MemoryPool; - public: explicit Buffer( MemoryPool& memoryPool, @@ -52,7 +53,6 @@ class Buffer { // to, and guarantees for the lifetime of *this that that region will remain // valid. Does NOT guarantee that the region is initially 0'd. char* reserve(uint64_t bytes) { - std::scoped_lock l(mutex_); if (reserveEnd_ + bytes <= chunkEnd_) { pos_ = reserveEnd_; reserveEnd_ += bytes; @@ -98,11 +98,68 @@ class Buffer { char* reserveEnd_; std::vector chunks_; MemoryPool& memoryPool_; - // NOTE: this is temporary fix, to quickly enable parallel access to the - // buffer class. In the near future, we are going to templetize this class to - // produce a concurrent and a non-concurrent variants, and change the call - // sites to use each variant when needed. +}; + +// Manages a pool of buffers. Buffers are returned to the pool when released. +// maxPoolSize should be set to at least 90% of capacity for performance +class BufferPool { + public: + explicit BufferPool( + MemoryPool& memoryPool, + size_t maxPoolSize = std::thread::hardware_concurrency(), + uint64_t initialChunkSize = kMinChunkSize) + : defaultInitialChunkSize_{initialChunkSize}, + maxPoolSize(maxPoolSize), + semaphore_{maxPoolSize}, + memoryPool_{memoryPool} { + NIMBLE_CHECK(maxPoolSize > 0, "max pool size must be > 0") + pool_.reserve(maxPoolSize); + + for (size_t i = 0; i < maxPoolSize; ++i) { + addBuffer(newBuffer()); + } + } + + MemoryPool& getMemoryPool() { + return memoryPool_; + } + + // buffer back to the pool. + void addBuffer(std::unique_ptr buffer) { + std::scoped_lock lock(mutex_); + pool_.push_back(std::move(buffer)); + semaphore_.signal(); + } + + // Reserves a buffer from the pool. Adds a new buffer to the pool + // while there are buffers available + std::unique_ptr reserveBuffer() { + semaphore_.wait(); + + std::scoped_lock lock(mutex_); + auto buffer = std::move(pool_.back()); + pool_.pop_back(); + return buffer; + } + + // Returns estimated number of buffers in the pool + size_t size() { + return pool_.size(); + } + + private: + static const uint64_t kMinChunkSize = 1LL << 20; + const uint64_t defaultInitialChunkSize_ = kMinChunkSize; + const size_t maxPoolSize = std::thread::hardware_concurrency(); + std::mutex mutex_; + folly::fibers::Semaphore semaphore_; + std::vector> pool_; + MemoryPool& memoryPool_; + + std::unique_ptr newBuffer() { + return std::make_unique(memoryPool_, defaultInitialChunkSize_); + } }; } // namespace facebook::nimble diff --git a/dwio/nimble/common/tests/BufferPoolTests.cpp b/dwio/nimble/common/tests/BufferPoolTests.cpp new file mode 100644 index 0000000..a8a765a --- /dev/null +++ b/dwio/nimble/common/tests/BufferPoolTests.cpp @@ -0,0 +1,109 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include "dwio/nimble/common/Buffer.h" +#include "dwio/nimble/common/Exceptions.h" +#include "folly/executors/CPUThreadPoolExecutor.h" +#include "velox/common/memory/Memory.h" +#include "velox/dwio/common/ExecutorBarrier.h" + +namespace facebook::nimble::test { +using MemoryPool = velox::memory::MemoryPool; +using ExecutorBarrier = velox::dwio::common::ExecutorBarrier; + +class BufferPoolTest : public ::testing::Test { + protected: + static void SetUpTestCase() {} + + void SetUp() override { + memPool_ = velox::memory::deprecatedAddDefaultLeafMemoryPool(); + } + + std::shared_ptr memPool_; +}; + +TEST_F(BufferPoolTest, CreateBufferPool) { + auto bufferPool = BufferPool{*memPool_}; + EXPECT_EQ(bufferPool.size(), std::thread::hardware_concurrency()); +} + +TEST_F(BufferPoolTest, CreateBufferPoolBadMaxPool) { + try { + auto bufferPool = BufferPool{*memPool_, /* maxPoolSize */ 0}; + FAIL(); + } catch (const NimbleUserError& e) { + EXPECT_EQ(e.errorMessage(), "max pool size must be > 0"); + } +} + +TEST_F(BufferPoolTest, ReserveAddBuffer) { + auto bufferPool = BufferPool{*memPool_, /* maxPoolSize */ 10}; + auto buffer = bufferPool.reserveBuffer(); + EXPECT_EQ(bufferPool.size(), 9); + bufferPool.addBuffer(std::move(buffer)); + EXPECT_EQ(bufferPool.size(), 10); +} + +TEST_F(BufferPoolTest, EmptyFillBufferPool) { + size_t iterations = 10; + std::vector> buffers; + auto bufferPool = BufferPool{*memPool_, iterations}; + + for (auto i = 0; i < iterations; ++i) { + auto buffer = bufferPool.reserveBuffer(); + buffers.push_back(std::move(buffer)); + EXPECT_EQ(bufferPool.size(), iterations - i - 1); + } + EXPECT_EQ(bufferPool.size(), 0); + + for (auto i = 0; i < iterations; ++i) { + bufferPool.addBuffer(std::move(buffers.back())); + buffers.pop_back(); + EXPECT_EQ(bufferPool.size(), i + 1); + } +} + +TEST_F(BufferPoolTest, ParallelFillPool) { + for (auto parallelismFactor : {2U}) { + folly::CPUThreadPoolExecutor executor{parallelismFactor}; + ExecutorBarrier barrier{executor}; + std::vector> buffers; + auto bufferPool = BufferPool{*memPool_, parallelismFactor}; + EXPECT_EQ(bufferPool.size(), parallelismFactor); + + for (auto i = 0; i < parallelismFactor; ++i) { + barrier.add([&]() { + auto buffer = bufferPool.reserveBuffer(); + buffers.push_back(std::move(buffer)); + }); + } + + barrier.waitAll(); + EXPECT_EQ(bufferPool.size(), 0); + + for (auto i = 0; i < parallelismFactor; ++i) { + barrier.add([&]() { + bufferPool.addBuffer(std::move(buffers.back())); + buffers.pop_back(); + }); + } + + barrier.waitAll(); + EXPECT_EQ(bufferPool.size(), parallelismFactor); + } +} +} // namespace facebook::nimble::test diff --git a/dwio/nimble/velox/FieldWriter.cpp b/dwio/nimble/velox/FieldWriter.cpp index 419926c..f61fff8 100644 --- a/dwio/nimble/velox/FieldWriter.cpp +++ b/dwio/nimble/velox/FieldWriter.cpp @@ -290,9 +290,10 @@ class SimpleFieldWriter : public FieldWriter { void write(const velox::VectorPtr& vector, const OrderedRanges& ranges) override { auto size = ranges.size(); - auto& buffer = context_.stringBuffer(); auto& data = valuesStream_.mutableData(); - + auto bufferPtr = context_.bufferPool().reserveBuffer(); + auto bufferGuard = folly::makeGuard( + [&]() { context_.bufferPool().addBuffer(std::move(bufferPtr)); }); if (auto flat = vector->asFlatVector()) { valuesStream_.ensureNullsCapacity(flat->mayHaveNulls(), size); bool rangeCopied = false; @@ -332,6 +333,7 @@ class SimpleFieldWriter : public FieldWriter { valuesStream_.mutableNonNulls(), Flat{vector}, [&](SourceType value) { + auto& buffer = *bufferPtr; data.push_back( C::convert(value, buffer, valuesStream_.extraMemory())); }); @@ -349,6 +351,7 @@ class SimpleFieldWriter : public FieldWriter { valuesStream_.mutableNonNulls(), Decoded{decoded}, [&](SourceType value) { + auto& buffer = *bufferPtr; data.push_back( C::convert(value, buffer, valuesStream_.extraMemory())); }); diff --git a/dwio/nimble/velox/FieldWriter.h b/dwio/nimble/velox/FieldWriter.h index a9220b6..f90ff69 100644 --- a/dwio/nimble/velox/FieldWriter.h +++ b/dwio/nimble/velox/FieldWriter.h @@ -95,7 +95,8 @@ struct FieldWriterContext { velox::memory::MemoryPool& memoryPool, std::shared_ptr writeExecutor = nullptr, std::unique_ptr reclaimer = nullptr, - std::function vectorDecoderVisitor = []() {}) + std::function vectorDecoderVisitor = []() {}, + size_t maxPoolSize = std::thread::hardware_concurrency()) : bufferMemoryPool{memoryPool.addLeafChild( "field_writer_buffer", true, @@ -103,10 +104,10 @@ struct FieldWriterContext { writeExecutor{std::move(writeExecutor)}, inputBufferGrowthPolicy{ DefaultInputBufferGrowthPolicy::withDefaultRanges()}, + bufferPool_{ + std::make_unique(*bufferMemoryPool, maxPoolSize)}, decodingContextPool_{std::make_unique( - std::move(vectorDecoderVisitor))} { - resetStringBuffer(); - } + std::move(vectorDecoderVisitor))} {} std::shared_ptr bufferMemoryPool; std::shared_ptr writeExecutor; @@ -133,13 +134,8 @@ struct FieldWriterContext { return decodingContextPool_->addContext(std::move(context)); } - Buffer& stringBuffer() { - return *buffer_; - } - - // Reset writer context for use by next stripe. - void resetStringBuffer() { - buffer_ = std::make_unique(*bufferMemoryPool); + BufferPool& bufferPool() { + return *bufferPool_; } const std::vector>& streams() { @@ -169,7 +165,7 @@ struct FieldWriterContext { } private: - std::unique_ptr buffer_; + std::unique_ptr bufferPool_; std::unique_ptr decodingContextPool_; std::vector> streams_; }; diff --git a/dwio/nimble/velox/VeloxWriter.cpp b/dwio/nimble/velox/VeloxWriter.cpp index 194d079..159e955 100644 --- a/dwio/nimble/velox/VeloxWriter.cpp +++ b/dwio/nimble/velox/VeloxWriter.cpp @@ -68,7 +68,8 @@ class WriterContext : public FieldWriterContext { memoryPool, options.writeExecutor, options.reclaimerFactory(), - options.vectorDecoderVisitor + options.vectorDecoderVisitor, + options.maxPoolSize }, options{std::move(options)}, logger{this->options.metricsLogger} { @@ -620,9 +621,6 @@ void VeloxWriter::writeChunk(bool lastChunk) { LoggingScope scope{*context_->logger}; velox::CpuWallTimer veloxTimer{context_->stripeFlushTiming}; - if (!encodingBuffer_) { - encodingBuffer_ = std::make_unique(*encodingMemoryPool_); - } streams_.resize(context_->schemaBuilder.nodeCount()); // When writing null streams, we write the nulls as data, and the stream @@ -666,9 +664,13 @@ void VeloxWriter::writeChunk(bool lastChunk) { auto encode = [&](StreamData& streamData) { const auto offset = streamData.descriptor().offset(); - auto encoded = encodeStream(*context_, *encodingBuffer_, streamData); + auto bufferPtr = context_->bufferPool().reserveBuffer(); + auto bufferGuard = folly::makeGuard( + [&]() { context_->bufferPool().addBuffer(std::move(bufferPtr)); }); + auto& buffer = *bufferPtr; + auto encoded = encodeStream(*context_, buffer, streamData); if (!encoded.empty()) { - ChunkedStreamWriter chunkWriter{*encodingBuffer_}; + ChunkedStreamWriter chunkWriter{buffer}; NIMBLE_DASSERT(offset < streams_.size(), "Stream offset out of range."); auto& stream = streams_[offset]; for (auto& buffer : chunkWriter.encode(encoded)) { @@ -780,10 +782,6 @@ uint32_t VeloxWriter::writeStripe() { uint64_t startSize = writer_.size(); writer_.writeStripe(context_->rowsInStripe, std::move(streams_)); stripeSize = writer_.size() - startSize; - encodingBuffer_.reset(); - // TODO: once chunked string fields are supported, move string buffer - // reset to writeChunk() - context_->resetStringBuffer(); } NIMBLE_ASSERT( diff --git a/dwio/nimble/velox/VeloxWriter.h b/dwio/nimble/velox/VeloxWriter.h index adb1e5b..d67b2ca 100644 --- a/dwio/nimble/velox/VeloxWriter.h +++ b/dwio/nimble/velox/VeloxWriter.h @@ -77,7 +77,6 @@ class VeloxWriter { TabletWriter writer_; std::unique_ptr root_; - std::unique_ptr encodingBuffer_; std::vector streams_; std::exception_ptr lastException_; const velox::common::SpillConfig* const spillConfig_; diff --git a/dwio/nimble/velox/VeloxWriterOptions.h b/dwio/nimble/velox/VeloxWriterOptions.h index f4c0ecf..d9944f6 100644 --- a/dwio/nimble/velox/VeloxWriterOptions.h +++ b/dwio/nimble/velox/VeloxWriterOptions.h @@ -126,6 +126,7 @@ struct VeloxWriterOptions { const velox::common::SpillConfig* spillConfig{nullptr}; + size_t maxPoolSize = std::thread::hardware_concurrency(); // If provided, internal encoding operations will happen in parallel using // this executor. std::shared_ptr encodingExecutor; diff --git a/dwio/nimble/velox/tests/VeloxWriterTests.cpp b/dwio/nimble/velox/tests/VeloxWriterTests.cpp index bc54fa0..8ad1081 100644 --- a/dwio/nimble/velox/tests/VeloxWriterTests.cpp +++ b/dwio/nimble/velox/tests/VeloxWriterTests.cpp @@ -319,11 +319,15 @@ TEST_F(VeloxWriterTests, MemoryReclaimPath) { std::string file; auto writeFile = std::make_unique(&file); std::atomic_bool reclaimEntered = false; - nimble::VeloxWriterOptions writerOptions{.reclaimerFactory = [&]() { - auto reclaimer = std::make_unique(); - reclaimer->setEnterArbitrationFunc([&]() { reclaimEntered = true; }); - return reclaimer; - }}; + nimble::VeloxWriterOptions writerOptions{ + .reclaimerFactory = + [&]() { + auto reclaimer = std::make_unique(); + reclaimer->setEnterArbitrationFunc( + [&]() { reclaimEntered = true; }); + return reclaimer; + }, + .maxPoolSize = 2}; nimble::VeloxWriter writer( *writerPool, type, std::move(writeFile), std::move(writerOptions)); auto batches = generateBatches(type, 100, 4000, 20221110, *leafPool_);