From bfd04f746587a284c8282b896b8b0a78632af078 Mon Sep 17 00:00:00 2001 From: "Ma, Rong" Date: Tue, 5 Dec 2023 19:19:44 +0800 Subject: [PATCH 1/7] decouple ShuffleWriter and PartitionWriter --- .../backendsapi/velox/MetricsApiImpl.scala | 1 + cpp/core/CMakeLists.txt | 2 +- cpp/core/shuffle/LocalPartitionWriter.cc | 236 ++++++----- cpp/core/shuffle/LocalPartitionWriter.h | 40 +- cpp/core/shuffle/Options.h | 3 + cpp/core/shuffle/PartitionWriter.cc | 51 +++ cpp/core/shuffle/PartitionWriter.h | 53 ++- cpp/core/shuffle/PartitionWriterCreator.h | 4 +- cpp/core/shuffle/ShuffleReader.cc | 2 +- cpp/core/shuffle/ShuffleSchema.h | 2 +- cpp/core/shuffle/ShuffleWriter.cc | 19 - cpp/core/shuffle/ShuffleWriter.h | 86 ++-- cpp/core/shuffle/Utils.cc | 210 +++++++++- cpp/core/shuffle/Utils.h | 33 +- .../shuffle/rss/CelebornPartitionWriter.cc | 82 ++-- .../shuffle/rss/CelebornPartitionWriter.h | 28 +- cpp/core/shuffle/rss/RemotePartitionWriter.h | 3 +- cpp/velox/CMakeLists.txt | 5 +- cpp/velox/shuffle/VeloxShuffleReader.cc | 2 +- cpp/velox/shuffle/VeloxShuffleUtils.cc | 42 -- cpp/velox/shuffle/VeloxShuffleUtils.h | 36 -- cpp/velox/shuffle/VeloxShuffleWriter.cc | 369 +++--------------- cpp/velox/shuffle/VeloxShuffleWriter.h | 21 +- cpp/velox/tests/VeloxShuffleWriterTest.cc | 17 +- .../spark/sql/execution/utils/ExecUtil.scala | 1 + 25 files changed, 667 insertions(+), 681 deletions(-) create mode 100644 cpp/core/shuffle/PartitionWriter.cc delete mode 100644 cpp/velox/shuffle/VeloxShuffleUtils.cc delete mode 100644 cpp/velox/shuffle/VeloxShuffleUtils.h diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala index 309027806d3e..20437742df84 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala @@ -264,6 +264,7 @@ class MetricsApiImpl extends MetricsApi with Logging { sparkContext: SparkContext): Map[String, SQLMetric] = Map( "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions"), "bytesSpilled" -> SQLMetrics.createSizeMetric(sparkContext, "shuffle bytes spilled"), "splitBufferSize" -> SQLMetrics.createSizeMetric(sparkContext, "split buffer size total"), "splitTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "totaltime to split"), diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index fb7c714a3ef8..24b5b88a0eab 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -213,7 +213,7 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS utils/StringUtil.cc utils/ObjectStore.cc jni/JniError.cc - jni/JniCommon.cc) + jni/JniCommon.cc shuffle/PartitionWriter.cc) file(MAKE_DIRECTORY ${root_directory}/releases) add_library(gluten SHARED ${SPARK_COLUMNAR_PLUGIN_SRCS}) diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index 313be10653b8..2e21a3a33255 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -25,19 +25,16 @@ namespace gluten { -class LocalPartitionWriter::LocalEvictHandle : public EvictHandle { +class LocalPartitionWriter::LocalEvictor : public Evictor { public: - LocalEvictHandle( - uint32_t numPartitions, - const arrow::ipc::IpcWriteOptions& options, - const std::shared_ptr& spillInfo) - : numPartitions_(numPartitions), options_(options), spillInfo_(spillInfo) {} + LocalEvictor(uint32_t numPartitions, ShuffleWriterOptions* options, const std::shared_ptr& spillInfo) + : Evictor(options), numPartitions_(numPartitions), spillInfo_(spillInfo) {} - static std::shared_ptr create( + static arrow::Result> create( uint32_t numPartitions, - const arrow::ipc::IpcWriteOptions& options, + ShuffleWriterOptions* options, const std::shared_ptr& spillInfo, - bool flush); + Evictor::Type evictType); bool finished() { return finished_; @@ -45,22 +42,20 @@ class LocalPartitionWriter::LocalEvictHandle : public EvictHandle { virtual arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) = 0; + virtual Type evictType() = 0; + protected: uint32_t numPartitions_; - arrow::ipc::IpcWriteOptions options_; std::shared_ptr spillInfo_; std::shared_ptr os_; bool finished_{false}; }; -class CacheEvictHandle final : public LocalPartitionWriter::LocalEvictHandle { +class CacheEvictor final : public LocalPartitionWriter::LocalEvictor { public: - CacheEvictHandle( - uint32_t numPartitions, - const arrow::ipc::IpcWriteOptions& options, - const std::shared_ptr& spillInfo) - : LocalPartitionWriter::LocalEvictHandle(numPartitions, options, spillInfo) { + CacheEvictor(uint32_t numPartitions, ShuffleWriterOptions* options, const std::shared_ptr& spillInfo) + : LocalPartitionWriter::LocalEvictor(numPartitions, options, spillInfo) { partitionCachedPayload_.resize(numPartitions); } @@ -70,6 +65,7 @@ class CacheEvictHandle final : public LocalPartitionWriter::LocalEvictHandle { } arrow::Status finish() override { + ScopedTimer timer(evictTime_); if (!finished_) { ARROW_ASSIGN_OR_RAISE(os_, arrow::io::FileOutputStream::Open(spillInfo_->spilledFile, true)); int64_t start = 0; @@ -98,36 +94,42 @@ class CacheEvictHandle final : public LocalPartitionWriter::LocalEvictHandle { return arrow::Status::OK(); } + ScopedTimer timer(evictTime_); int32_t metadataLength = 0; // unused auto payloads = std::move(partitionCachedPayload_[partitionId]); // Clear cached batches before creating the payloads, to avoid spilling this partition. partitionCachedPayload_[partitionId].clear(); for (auto& payload : payloads) { - RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_, os, &metadataLength)); + RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, os, &metadataLength)); } return arrow::Status::OK(); } + Type evictType() override { + return Type::kCache; + } + private: std::vector>> partitionCachedPayload_; }; -class FlushOnSpillEvictHandle final : public LocalPartitionWriter::LocalEvictHandle { +class FlushOnSpillEvictor final : public LocalPartitionWriter::LocalEvictor { public: - FlushOnSpillEvictHandle( + FlushOnSpillEvictor( uint32_t numPartitions, - const arrow::ipc::IpcWriteOptions& options, + ShuffleWriterOptions* options, const std::shared_ptr& spillInfo) - : LocalPartitionWriter::LocalEvictHandle(numPartitions, options, spillInfo) {} + : LocalPartitionWriter::LocalEvictor(numPartitions, options, spillInfo) {} arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + ScopedTimer timer(evictTime_); if (!os_) { ARROW_ASSIGN_OR_RAISE(os_, arrow::io::FileOutputStream::Open(spillInfo_->spilledFile, true)); } int32_t metadataLength = 0; // unused. ARROW_ASSIGN_OR_RAISE(auto start, os_->Tell()); - RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_, os_.get(), &metadataLength)); + RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, os_.get(), &metadataLength)); ARROW_ASSIGN_OR_RAISE(auto end, os_->Tell()); DEBUG_OUT << "Spilled partition " << partitionId << " file start: " << start << ", file end: " << end << std::endl; spillInfo_->partitionSpillInfos.push_back({partitionId, end - start}); @@ -148,29 +150,36 @@ class FlushOnSpillEvictHandle final : public LocalPartitionWriter::LocalEvictHan arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) override { return arrow::Status::OK(); } + + Type evictType() override { + return Type::kFlush; + } }; -std::shared_ptr LocalPartitionWriter::LocalEvictHandle::create( +arrow::Result> LocalPartitionWriter::LocalEvictor::create( uint32_t numPartitions, - const arrow::ipc::IpcWriteOptions& options, + ShuffleWriterOptions* options, const std::shared_ptr& spillInfo, - bool flush) { - if (flush) { - return std::make_shared(numPartitions, options, spillInfo); - } else { - return std::make_shared(numPartitions, options, spillInfo); + Evictor::Type evictType) { + switch (evictType) { + case Evictor::Type::kFlush: + return std::make_unique(numPartitions, options, spillInfo); + case Type::kCache: + return std::make_unique(numPartitions, options, spillInfo); + default: + return arrow::Status::Invalid("Cannot create Evictor from type Evictor::Type::kStop."); } } std::string LocalPartitionWriter::nextSpilledFileDir() { auto spilledFileDir = getSpilledShuffleFileDir(configuredDirs_[dirSelection_], subDirSelection_[dirSelection_]); - subDirSelection_[dirSelection_] = (subDirSelection_[dirSelection_] + 1) % shuffleWriter_->options().num_sub_dirs; + subDirSelection_[dirSelection_] = (subDirSelection_[dirSelection_] + 1) % options_->num_sub_dirs; dirSelection_ = (dirSelection_ + 1) % configuredDirs_.size(); return spilledFileDir; } arrow::Status LocalPartitionWriter::setLocalDirs() { - configuredDirs_ = splitPaths(shuffleWriter_->options().local_dirs); + configuredDirs_ = splitPaths(options_->local_dirs); // Shuffle the configured local directories. This prevents each task from using the same directory for spilled files. std::random_device rd; std::default_random_engine engine(rd()); @@ -182,11 +191,10 @@ arrow::Status LocalPartitionWriter::setLocalDirs() { arrow::Status LocalPartitionWriter::openDataFile() { // open data file output stream std::shared_ptr fout; - ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(shuffleWriter_->options().data_file)); - if (shuffleWriter_->options().buffered_write) { + ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(options_->data_file)); + if (options_->buffered_write) { // Output stream buffer is neither partition buffer memory nor ipc memory. - ARROW_ASSIGN_OR_RAISE( - dataFileOs_, arrow::io::BufferedOutputStream::Create(16384, shuffleWriter_->options().memory_pool, fout)); + ARROW_ASSIGN_OR_RAISE(dataFileOs_, arrow::io::BufferedOutputStream::Create(16384, options_->memory_pool, fout)); } else { dataFileOs_ = fout; } @@ -202,16 +210,38 @@ arrow::Status LocalPartitionWriter::clearResource() { } arrow::Status LocalPartitionWriter::init() { + partitionLengths_.resize(numPartitions_, 0); + rawPartitionLengths_.resize(numPartitions_, 0); fs_ = std::make_shared(); RETURN_NOT_OK(setLocalDirs()); return arrow::Status::OK(); } -arrow::Status LocalPartitionWriter::stop() { - int64_t totalBytesEvicted = 0; - int64_t totalBytesWritten = 0; - auto numPartitions = shuffleWriter_->numPartitions(); +arrow::Status LocalPartitionWriter::mergeSpills(uint32_t partitionId) { + for (auto spill : spills_) { + // Read if partition exists in the spilled file and write to the final file. + if (spill->mergePos < spill->partitionSpillInfos.size() && + spill->partitionSpillInfos[spill->mergePos].partitionId == partitionId) { // A hit. + if (!spill->inputStream) { + // Open spilled file. + ARROW_ASSIGN_OR_RAISE( + spill->inputStream, arrow::io::MemoryMappedFile::Open(spill->spilledFile, arrow::io::FileMode::READ)); + // Add evict metrics. + ARROW_ASSIGN_OR_RAISE(auto spilledSize, spill->inputStream->GetSize()); + totalBytesEvicted_ += spilledSize; + } + auto spillInfo = spill->partitionSpillInfos[spill->mergePos]; + ARROW_ASSIGN_OR_RAISE(auto raw, spill->inputStream->Read(spillInfo.length)); + RETURN_NOT_OK(dataFileOs_->Write(raw)); + // Goto next partition in this spillInfo. + spill->mergePos++; + } + } + return arrow::Status::OK(); +} + +arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { // Open final file. // If options_.buffered_write is set, it will acquire 16KB memory that might trigger spill. RETURN_NOT_OK(openDataFile()); @@ -220,54 +250,38 @@ arrow::Status LocalPartitionWriter::stop() { writeTimer.start(); int64_t endInFinalFile = 0; + int32_t metadataLength = 0; + auto cachedBuffersIter = cachedPartitionBuffers_.begin(); // Iterator over pid. - for (auto pid = 0; pid < numPartitions; ++pid) { + for (auto pid = 0; pid < numPartitions_; ++pid) { // Record start offset. auto startInFinalFile = endInFinalFile; // Iterator over all spilled files. - for (auto spill : spills_) { - // Read if partition exists in the spilled file and write to the final file. - if (spill->mergePos < spill->partitionSpillInfos.size() && - spill->partitionSpillInfos[spill->mergePos].partitionId == pid) { // A hit. - if (!spill->inputStream) { - // Open spilled file. - ARROW_ASSIGN_OR_RAISE( - spill->inputStream, arrow::io::MemoryMappedFile::Open(spill->spilledFile, arrow::io::FileMode::READ)); - // Add evict metrics. - ARROW_ASSIGN_OR_RAISE(auto spilledSize, spill->inputStream->GetSize()); - totalBytesEvicted += spilledSize; - } - - auto spillInfo = spill->partitionSpillInfos[spill->mergePos]; - ARROW_ASSIGN_OR_RAISE(auto raw, spill->inputStream->Read(spillInfo.length)); - RETURN_NOT_OK(dataFileOs_->Write(raw)); - // Goto next partition in this spillInfo. - spill->mergePos++; - } - } + RETURN_NOT_OK(mergeSpills(pid)); // Write cached batches. - if (evictHandle_ && !evictHandle_->finished()) { - RETURN_NOT_OK(evictHandle_->flushCachedPayloads(pid, dataFileOs_.get())); + if (evictor_ && !evictor_->finished()) { + RETURN_NOT_OK(evictor_->flushCachedPayloads(pid, dataFileOs_.get())); } // Compress and write the last payload. // Stop the timer to prevent counting the compression time into write time. - writeTimer.stop(); - ARROW_ASSIGN_OR_RAISE(auto lastPayload, shuffleWriter_->createPayloadFromBuffer(pid, false)); - writeTimer.start(); - if (lastPayload) { - int32_t metadataLength = 0; // unused - RETURN_NOT_OK(arrow::ipc::WriteIpcPayload( - *lastPayload, shuffleWriter_->options().ipc_write_options, dataFileOs_.get(), &metadataLength)); + if (cachedBuffersIter != cachedPartitionBuffers_.end() && std::get<0>(*cachedBuffersIter) == pid) { + writeTimer.stop(); + ARROW_ASSIGN_OR_RAISE( + auto payload, + createPayloadFromBuffers(std::get<1>(*cachedBuffersIter), std::move(std::get<2>(*cachedBuffersIter)))); + writeTimer.start(); + RETURN_NOT_OK( + arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, dataFileOs_.get(), &metadataLength)); + cachedBuffersIter++; } ARROW_ASSIGN_OR_RAISE(endInFinalFile, dataFileOs_->Tell()); - if (endInFinalFile != startInFinalFile && shuffleWriter_->options().write_eos) { + if (endInFinalFile != startInFinalFile && options_->write_eos) { // Write EOS if any payload written. int64_t bytes; RETURN_NOT_OK(writeEos(dataFileOs_.get(), &bytes)); endInFinalFile += bytes; } - - shuffleWriter_->setPartitionLengths(pid, endInFinalFile - startInFinalFile); + partitionLengths_[pid] = endInFinalFile - startInFinalFile; } for (auto spill : spills_) { @@ -281,34 +295,36 @@ arrow::Status LocalPartitionWriter::stop() { } RETURN_NOT_OK(fs_->DeleteFile(spill->spilledFile)); } - - ARROW_ASSIGN_OR_RAISE(totalBytesWritten, dataFileOs_->Tell()); + ARROW_ASSIGN_OR_RAISE(totalBytesWritten_, dataFileOs_->Tell()); writeTimer.stop(); - - shuffleWriter_->setTotalWriteTime(writeTimer.realTimeUsed()); - shuffleWriter_->setTotalBytesEvicted(totalBytesEvicted); - shuffleWriter_->setTotalBytesWritten(totalBytesWritten); + writeTime_ = writeTimer.realTimeUsed(); // Close Final file, Clear buffered resources. RETURN_NOT_OK(clearResource()); - + // Populate metrics. + RETURN_NOT_OK(populateMetrics(metrics)); return arrow::Status::OK(); } -arrow::Status LocalPartitionWriter::requestNextEvict(bool flush) { +arrow::Status LocalPartitionWriter::requestEvict(Evictor::Type evictType) { + if (auto handle = getEvictHandle()) { + if (handle->evictType() == evictType) { + return arrow::Status::OK(); + } + } RETURN_NOT_OK(finishEvict()); + ARROW_ASSIGN_OR_RAISE(auto spilledFile, createTempShuffleFile(nextSpilledFileDir())); auto spillInfo = std::make_shared(spilledFile); spills_.push_back(spillInfo); - evictHandle_ = LocalEvictHandle::create( - shuffleWriter_->numPartitions(), shuffleWriter_->options().ipc_write_options, spillInfo, flush); + ARROW_ASSIGN_OR_RAISE(evictor_, LocalEvictor::create(numPartitions_, options_, spillInfo, evictType)); return arrow::Status::OK(); } -EvictHandle* LocalPartitionWriter::getEvictHandle() { - if (evictHandle_ && !evictHandle_->finished()) { - return evictHandle_.get(); +LocalPartitionWriter::LocalEvictor* LocalPartitionWriter::getEvictHandle() { + if (evictor_ && !evictor_->finished()) { + return evictor_.get(); } return nullptr; } @@ -316,6 +332,7 @@ EvictHandle* LocalPartitionWriter::getEvictHandle() { arrow::Status LocalPartitionWriter::finishEvict() { if (auto handle = getEvictHandle()) { RETURN_NOT_OK(handle->finish()); + evictTime_ += handle->getEvictTime(); // The spilled file should not be empty. However, defensively // discard the last SpillInfo to avoid iterating over invalid ones. auto lastSpillInfo = spills_.back(); @@ -327,11 +344,56 @@ arrow::Status LocalPartitionWriter::finishEvict() { return arrow::Status::OK(); } +arrow::Status LocalPartitionWriter::evict( + uint32_t partitionId, + uint32_t numRows, + std::vector> buffers, + Evictor::Type evictType) { + rawPartitionLengths_[partitionId] += getBufferSize(buffers); + if (evictType == Evictor::Type::kStop) { + cachedPartitionBuffers_.push_back({partitionId, numRows, std::move(buffers)}); + } else { + ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffers(numRows, std::move(buffers))); + RETURN_NOT_OK(requestEvict(evictType)); + RETURN_NOT_OK(getEvictHandle()->evict(partitionId, std::move(payload))); + } + return arrow::Status::OK(); +} + +arrow::Status LocalPartitionWriter::populateMetrics(ShuffleWriterMetrics* metrics) { + metrics->totalCompressTime += compressTime_; + metrics->totalEvictTime += evictTime_; + metrics->totalWriteTime += writeTime_; + metrics->totalBytesEvicted += totalBytesEvicted_; + metrics->totalBytesWritten += totalBytesWritten_; + metrics->partitionLengths = std::move(partitionLengths_); + metrics->rawPartitionLengths = std::move(rawPartitionLengths_); + return arrow::Status::OK(); +} + +arrow::Status LocalPartitionWriter::evictFixedSize(int64_t size, int64_t* actual) { + auto beforeShrink = options_->memory_pool->bytes_allocated(); + for (auto& item : cachedPartitionBuffers_) { + auto& buffers = std::get<2>(item); + for (auto& buffer : buffers) { + if (!buffer) { + continue; + } + if (auto parent = std::dynamic_pointer_cast(buffer->parent())) { + RETURN_NOT_OK(parent->Resize(buffer->size())); + } + } + } + *actual = beforeShrink - options_->memory_pool->bytes_allocated(); + return arrow::Status::OK(); +} + LocalPartitionWriterCreator::LocalPartitionWriterCreator() : PartitionWriterCreator() {} arrow::Result> LocalPartitionWriterCreator::make( - ShuffleWriter* shuffleWriter) { - auto partitionWriter = std::make_shared(shuffleWriter); + uint32_t numPartitions, + ShuffleWriterOptions* options) { + auto partitionWriter = std::make_shared(numPartitions, options); RETURN_NOT_OK(partitionWriter->init()); return partitionWriter; } diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index 27fa2c0c2c6f..96db7c824476 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -46,13 +46,16 @@ struct SpillInfo { class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { public: - explicit LocalPartitionWriter(ShuffleWriter* shuffleWriter) : PartitionWriter(shuffleWriter) {} + explicit LocalPartitionWriter(uint32_t numPartitions, ShuffleWriterOptions* options) + : PartitionWriter(numPartitions, options) {} arrow::Status init() override; - arrow::Status requestNextEvict(bool flush) override; - - EvictHandle* getEvictHandle() override; + arrow::Status evict( + uint32_t partitionId, + uint32_t numRows, + std::vector> buffers, + Evictor::Type evictType) override; arrow::Status finishEvict() override; @@ -75,22 +78,32 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { /// If spill is triggered by 2.c, cached payloads of the remaining unmerged partitions will be spilled. /// In both cases, if the cached payload size doesn't free enough memory, /// it will shrink partition buffers to free more memory. - arrow::Status stop() override; + arrow::Status stop(ShuffleWriterMetrics* metrics) override; + + arrow::Status evictFixedSize(int64_t size, int64_t* actual) override; - class LocalEvictHandle; + class LocalEvictor; private: + arrow::Status requestEvict(Evictor::Type evictType); + + LocalEvictor* getEvictHandle(); + arrow::Status setLocalDirs(); std::string nextSpilledFileDir(); arrow::Status openDataFile(); + arrow::Status mergeSpills(uint32_t partitionId); + arrow::Status clearResource(); - std::shared_ptr fs_{}; - std::shared_ptr evictHandle_; - std::vector> spills_; + arrow::Status populateMetrics(ShuffleWriterMetrics* metrics); + + std::shared_ptr fs_{nullptr}; + std::shared_ptr evictor_{nullptr}; + std::vector> spills_{}; // configured local dirs for spilled file int32_t dirSelection_ = 0; @@ -98,12 +111,19 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { std::vector configuredDirs_; std::shared_ptr dataFileOs_; + int64_t totalBytesEvicted_{0}; + int64_t totalBytesWritten_{0}; + std::vector partitionLengths_; + std::vector rawPartitionLengths_; + std::vector>>> cachedPartitionBuffers_; }; class LocalPartitionWriterCreator : public ShuffleWriter::PartitionWriterCreator { public: LocalPartitionWriterCreator(); - arrow::Result> make(ShuffleWriter* shuffleWriter) override; + arrow::Result> make( + uint32_t numPartitions, + ShuffleWriterOptions* options) override; }; } // namespace gluten diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h index 8c7a395c33f2..20f72f9f06ed 100644 --- a/cpp/core/shuffle/Options.h +++ b/cpp/core/shuffle/Options.h @@ -62,6 +62,9 @@ struct ShuffleWriterOptions { arrow::ipc::IpcWriteOptions ipc_write_options = arrow::ipc::IpcWriteOptions::Defaults(); + std::shared_ptr write_schema{nullptr}; + std::shared_ptr codec{nullptr}; + std::string data_file{}; std::string local_dirs{}; arrow::MemoryPool* memory_pool{}; diff --git a/cpp/core/shuffle/PartitionWriter.cc b/cpp/core/shuffle/PartitionWriter.cc new file mode 100644 index 000000000000..4b12511b8633 --- /dev/null +++ b/cpp/core/shuffle/PartitionWriter.cc @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 "shuffle/PartitionWriter.h" +#include "shuffle/Utils.h" + +namespace gluten { + +arrow::Result> gluten::ShuffleWriter::PartitionWriter::createPayloadFromBuffers( + uint32_t numRows, + std::vector> buffers) { + std::shared_ptr recordBatch; + if (options_->compression_type != arrow::Compression::UNCOMPRESSED) { + ARROW_ASSIGN_OR_RAISE( + recordBatch, + makeCompressedRecordBatch( + numRows, + std::move(buffers), + options_->write_schema, + options_->ipc_write_options.memory_pool, + options_->codec.get(), + options_->compression_threshold, + options_->compression_mode, + compressTime_)); + } else { + ARROW_ASSIGN_OR_RAISE( + recordBatch, + makeUncompressedRecordBatch( + numRows, std::move(buffers), options_->write_schema, options_->ipc_write_options.memory_pool)); + } + + auto payload = std::make_unique(); + RETURN_NOT_OK(arrow::ipc::GetRecordBatchPayload(*recordBatch, options_->ipc_write_options, payload.get())); + return payload; +} + +} // namespace gluten \ No newline at end of file diff --git a/cpp/core/shuffle/PartitionWriter.h b/cpp/core/shuffle/PartitionWriter.h index d87c20955c3c..9090df3d73fa 100644 --- a/cpp/core/shuffle/PartitionWriter.h +++ b/cpp/core/shuffle/PartitionWriter.h @@ -22,40 +22,61 @@ namespace gluten { -class EvictHandle { +class Evictor { public: - virtual ~EvictHandle() = default; + enum Type { kCache, kFlush, kStop }; + + Evictor(ShuffleWriterOptions* options) : options_(options) {} + + virtual ~Evictor() = default; virtual arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) = 0; virtual arrow::Status finish() = 0; + + int64_t getEvictTime() { + return evictTime_; + } + + protected: + ShuffleWriterOptions* options_; + + int64_t evictTime_{0}; }; -class ShuffleWriter::PartitionWriter { +class ShuffleWriter::PartitionWriter : public Evictable { public: - PartitionWriter(ShuffleWriter* shuffleWriter) : shuffleWriter_(shuffleWriter) {} + PartitionWriter(uint32_t numPartitions, ShuffleWriterOptions* options) + : numPartitions_(numPartitions), options_(options) {} + virtual ~PartitionWriter() = default; virtual arrow::Status init() = 0; - virtual arrow::Status stop() = 0; + virtual arrow::Status stop(ShuffleWriterMetrics* metrics) = 0; - /// Request next evict. The caller can use `requestNextEvict` to start a evict, and choose to call - /// `getEvictHandle()->evict()` immediately, or to call it latter somewhere else. - /// The caller can start new evict multiple times. Once it's called, the last `EvictHandle` - /// will be finished automatically. + /// Evict buffers for `partitionId` partition. /// \param flush Whether to flush the evicted data immediately. If it's false, /// the data can be cached first. - virtual arrow::Status requestNextEvict(bool flush) = 0; - - /// Get the current managed EvictHandle. Returns nullptr if the current EvictHandle was finished, - /// or requestNextEvict has not been called. - /// \return - virtual EvictHandle* getEvictHandle() = 0; + virtual arrow::Status evict( + uint32_t partitionId, + uint32_t numRows, + std::vector> buffers, + Evictor::Type evictType) = 0; virtual arrow::Status finishEvict() = 0; - ShuffleWriter* shuffleWriter_; + protected: + arrow::Result> createPayloadFromBuffers( + uint32_t numRows, + std::vector> buffers); + + uint32_t numPartitions_; + + ShuffleWriterOptions* options_; + int64_t compressTime_{0}; + int64_t evictTime_{0}; + int64_t writeTime_{0}; }; } // namespace gluten diff --git a/cpp/core/shuffle/PartitionWriterCreator.h b/cpp/core/shuffle/PartitionWriterCreator.h index f1cb5873fd74..49ce7fc33ac3 100644 --- a/cpp/core/shuffle/PartitionWriterCreator.h +++ b/cpp/core/shuffle/PartitionWriterCreator.h @@ -27,7 +27,9 @@ class ShuffleWriter::PartitionWriterCreator { PartitionWriterCreator() = default; virtual ~PartitionWriterCreator() = default; - virtual arrow::Result> make(ShuffleWriter* shuffleWriter) = 0; + virtual arrow::Result> make( + uint32_t numPartitions, + ShuffleWriterOptions* options) = 0; }; } // namespace gluten diff --git a/cpp/core/shuffle/ShuffleReader.cc b/cpp/core/shuffle/ShuffleReader.cc index b1b278f581f3..6b05dd64e5b8 100644 --- a/cpp/core/shuffle/ShuffleReader.cc +++ b/cpp/core/shuffle/ShuffleReader.cc @@ -36,7 +36,7 @@ class ShuffleReaderOutStream : public ColumnarBatchIterator { const std::function ipcTimeAccumulator) : options_(options), in_(in), ipcTimeAccumulator_(ipcTimeAccumulator) { if (options.compression_type != arrow::Compression::UNCOMPRESSED) { - writeSchema_ = toCompressWriteSchema(*schema); + writeSchema_ = toCompressWriteSchema(); } else { writeSchema_ = toWriteSchema(*schema); } diff --git a/cpp/core/shuffle/ShuffleSchema.h b/cpp/core/shuffle/ShuffleSchema.h index dd906fdc6cce..dbcd45154c92 100644 --- a/cpp/core/shuffle/ShuffleSchema.h +++ b/cpp/core/shuffle/ShuffleSchema.h @@ -49,7 +49,7 @@ inline std::shared_ptr toWriteSchema(arrow::Schema& schema) { return std::make_shared(fields); } -inline std::shared_ptr toCompressWriteSchema(arrow::Schema& schema) { +inline std::shared_ptr toCompressWriteSchema() { std::vector> fields; fields.emplace_back(std::make_shared("header", arrow::large_utf8())); fields.emplace_back(std::make_shared("lengthBuffer", arrow::large_utf8())); diff --git a/cpp/core/shuffle/ShuffleWriter.cc b/cpp/core/shuffle/ShuffleWriter.cc index e58869fdf548..5c42b94a4c85 100644 --- a/cpp/core/shuffle/ShuffleWriter.cc +++ b/cpp/core/shuffle/ShuffleWriter.cc @@ -30,23 +30,4 @@ namespace gluten { // by default, allocate 8M block, 2M page size #define SPLIT_BUFFER_SIZE 16 * 1024 * 1024 #endif - -std::shared_ptr ShuffleWriter::writeSchema() { - if (writeSchema_ != nullptr) { - return writeSchema_; - } - - writeSchema_ = toWriteSchema(*schema_); - return writeSchema_; -} - -std::shared_ptr ShuffleWriter::compressWriteSchema() { - if (compressWriteSchema_ != nullptr) { - return compressWriteSchema_; - } - - compressWriteSchema_ = toCompressWriteSchema(*schema_); - return compressWriteSchema_; -} - } // namespace gluten diff --git a/cpp/core/shuffle/ShuffleWriter.h b/cpp/core/shuffle/ShuffleWriter.h index df075a9984fe..91d6861086d7 100644 --- a/cpp/core/shuffle/ShuffleWriter.h +++ b/cpp/core/shuffle/ShuffleWriter.h @@ -31,6 +31,16 @@ namespace gluten { +struct ShuffleWriterMetrics { + int64_t totalBytesWritten{0}; + int64_t totalBytesEvicted{0}; + int64_t totalWriteTime{0}; + int64_t totalEvictTime{0}; + int64_t totalCompressTime{0}; + std::vector partitionLengths{}; + std::vector rawPartitionLengths{}; // Uncompressed size. +}; + class ShuffleMemoryPool : public arrow::MemoryPool { public: ShuffleMemoryPool(arrow::MemoryPool* pool) : pool_(pool) {} @@ -91,11 +101,7 @@ class ShuffleWriter : public Evictable { virtual arrow::Status split(std::shared_ptr cb, int64_t memLimit) = 0; - virtual arrow::Result> createPayloadFromBuffer( - uint32_t partitionId, - bool reuseBuffers) = 0; - - virtual arrow::Status evictPayload(uint32_t partitionId, std::unique_ptr payload) = 0; + virtual arrow::Status evictPartitionBuffers(uint32_t partitionId, bool reuseBuffers) = 0; virtual arrow::Status stop() = 0; @@ -107,66 +113,42 @@ class ShuffleWriter : public Evictable { return numPartitions_; } - int64_t totalBytesWritten() const { - return totalBytesWritten_; + int64_t partitionBufferSize() const { + return partitionBufferPool_->bytes_allocated(); } - int64_t totalBytesEvicted() const { - return totalBytesEvicted_; + int64_t totalBytesWritten() const { + return metrics_.totalBytesWritten; } - int64_t partitionBufferSize() const { - return partitionBufferPool_->bytes_allocated(); + int64_t totalBytesEvicted() const { + return metrics_.totalBytesEvicted; } int64_t totalWriteTime() const { - return totalWriteTime_; + return metrics_.totalWriteTime; } int64_t totalEvictTime() const { - return totalEvictTime_; + return metrics_.totalEvictTime; } int64_t totalCompressTime() const { - return totalCompressTime_; + return metrics_.totalCompressTime; } const std::vector& partitionLengths() const { - return partitionLengths_; + return metrics_.partitionLengths; } const std::vector& rawPartitionLengths() const { - return rawPartitionLengths_; + return metrics_.rawPartitionLengths; } ShuffleWriterOptions& options() { return options_; } - void setPartitionLengths(int32_t index, int64_t length) { - partitionLengths_[index] = length; - } - - void setRawPartitionLength(int32_t index, int64_t length) { - rawPartitionLengths_[index] = length; - } - - void setTotalWriteTime(int64_t totalWriteTime) { - totalWriteTime_ = totalWriteTime; - } - - void setTotalBytesWritten(int64_t totalBytesWritten) { - totalBytesWritten_ = totalBytesWritten; - } - - void setTotalEvictTime(int64_t totalEvictTime) { - totalEvictTime_ = totalEvictTime; - } - - void setTotalBytesEvicted(int64_t totalBytesEvicted) { - totalBytesEvicted_ = totalBytesEvicted; - } - virtual const uint64_t cachedPayloadSize() const = 0; class PartitionWriter; @@ -181,15 +163,12 @@ class ShuffleWriter : public Evictable { : numPartitions_(numPartitions), partitionWriterCreator_(std::move(partitionWriterCreator)), options_(std::move(options)), - partitionBufferPool_(std::make_shared(options_.memory_pool)), - codec_(createArrowIpcCodec(options_.compression_type, options_.codec_backend)) {} + partitionBufferPool_(std::make_shared(options_.memory_pool)) { + options_.codec = createArrowIpcCodec(options_.compression_type, options_.codec_backend); + } virtual ~ShuffleWriter() = default; - std::shared_ptr writeSchema(); - - std::shared_ptr compressWriteSchema(); - int32_t numPartitions_; std::shared_ptr partitionWriterCreator_; @@ -199,25 +178,14 @@ class ShuffleWriter : public Evictable { // The actual allocation is delegated to options_.memory_pool. std::shared_ptr partitionBufferPool_; - int64_t totalBytesWritten_ = 0; - int64_t totalBytesEvicted_ = 0; - int64_t totalWriteTime_ = 0; - int64_t totalEvictTime_ = 0; - int64_t totalCompressTime_ = 0; - - std::vector partitionLengths_; - std::vector rawPartitionLengths_; // Uncompressed size. - - std::unique_ptr codec_; - std::shared_ptr schema_; - std::shared_ptr writeSchema_; - std::shared_ptr compressWriteSchema_; // col partid std::vector>>> partitionBuffers_; std::shared_ptr partitioner_; + + ShuffleWriterMetrics metrics_{}; }; } // namespace gluten diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index 9255d679801c..f959e2a5cdfe 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -16,9 +16,7 @@ */ #include "shuffle/Utils.h" -#include "Options.h" -#include "utils/StringUtil.h" - +#include #include #include #include @@ -27,6 +25,194 @@ #include #include #include +#include "shuffle/Options.h" +#include "utils/Timer.h" + +namespace gluten { +namespace { +arrow::Result> makeNullBinaryArray( + std::shared_ptr type, + arrow::MemoryPool* pool) { + ARROW_ASSIGN_OR_RAISE(auto offsetBuffer, arrow::AllocateResizableBuffer(kSizeOfIpcOffsetBuffer << 1, pool)); + // set the first offset to 0, and set the value offset + uint8_t* offsetaddr = offsetBuffer->mutable_data(); + memset(offsetaddr, 0, kSizeOfIpcOffsetBuffer); + // second value offset 0 + memset(offsetaddr + kSizeOfIpcOffsetBuffer, 0, kSizeOfIpcOffsetBuffer); + // If it is not compressed array, null valueBuffer + // worked, but if compress, will core dump at buffer::size(), so replace by kNullBuffer + static std::shared_ptr kNullBuffer = std::make_shared(nullptr, 0); + return arrow::MakeArray(arrow::ArrayData::Make(type, 1, {nullptr, std::move(offsetBuffer), kNullBuffer})); +} + +arrow::Result> makeBinaryArray( + std::shared_ptr type, + std::shared_ptr valueBuffer, + arrow::MemoryPool* pool) { + if (valueBuffer == nullptr) { + return makeNullBinaryArray(type, pool); + } + + ARROW_ASSIGN_OR_RAISE(auto offsetBuffer, arrow::AllocateResizableBuffer(kSizeOfIpcOffsetBuffer << 1, pool)); + // set the first offset to 0, and set the value offset + uint8_t* offsetaddr = offsetBuffer->mutable_data(); + memset(offsetaddr, 0, kSizeOfIpcOffsetBuffer); + int64_t length = valueBuffer->size(); + memcpy(offsetaddr + kSizeOfIpcOffsetBuffer, reinterpret_cast(&length), kSizeOfIpcOffsetBuffer); + return arrow::MakeArray(arrow::ArrayData::Make(type, 1, {nullptr, std::move(offsetBuffer), valueBuffer})); +} + +// Length buffer layout |compressionMode|buffers.size()|buffer1 unCompressedLength|buffer1 compressedLength| buffer2... +arrow::Status getLengthBufferAndValueBufferOneByOne( + const std::vector>& buffers, + arrow::MemoryPool* pool, + arrow::util::Codec* codec, + std::shared_ptr& lengthBuffer, + std::shared_ptr& valueBuffer) { + ARROW_ASSIGN_OR_RAISE( + lengthBuffer, arrow::AllocateResizableBuffer((1 + 1 + buffers.size() * 2) * sizeof(int64_t), pool)); + auto lengthBufferPtr = (int64_t*)(lengthBuffer->mutable_data()); + // Write compression mode. + *lengthBufferPtr++ = CompressionMode::BUFFER; + // Write number of buffers. + *lengthBufferPtr++ = buffers.size(); + + int64_t compressedBufferMaxSize = getMaxCompressedBufferSize(buffers, codec); + ARROW_ASSIGN_OR_RAISE(valueBuffer, arrow::AllocateResizableBuffer(compressedBufferMaxSize, pool)); + int64_t compressValueOffset = 0; + for (auto& buffer : buffers) { + if (buffer != nullptr && buffer->size() != 0) { + int64_t actualLength; + int64_t maxLength = codec->MaxCompressedLen(buffer->size(), nullptr); + ARROW_ASSIGN_OR_RAISE( + actualLength, + codec->Compress( + buffer->size(), buffer->data(), maxLength, valueBuffer->mutable_data() + compressValueOffset)); + compressValueOffset += actualLength; + *lengthBufferPtr++ = buffer->size(); + *lengthBufferPtr++ = actualLength; + } else { + *lengthBufferPtr++ = 0; + *lengthBufferPtr++ = 0; + } + } + RETURN_NOT_OK(valueBuffer->Resize(compressValueOffset, /*shrink*/ true)); + return arrow::Status::OK(); +} + +// Length buffer layout |compressionMode|buffer unCompressedLength|buffer compressedLength|buffers.size()| buffer1 size +// | buffer2 size +arrow::Status getLengthBufferAndValueBufferStream( + const std::vector>& buffers, + arrow::MemoryPool* pool, + arrow::util::Codec* codec, + std::shared_ptr& lengthBuffer, + std::shared_ptr& compressedBuffer) { + ARROW_ASSIGN_OR_RAISE(lengthBuffer, arrow::AllocateResizableBuffer((1 + 3 + buffers.size()) * sizeof(int64_t), pool)); + auto originalBufferSize = getBufferSize(buffers); + + // because 64B align, uncompressedBuffer size maybe bigger than unCompressedBufferSize which is + // getBuffersSize(buffers), then cannot use this size + ARROW_ASSIGN_OR_RAISE(auto uncompressedBuffer, arrow::AllocateResizableBuffer(originalBufferSize, pool)); + int64_t uncompressedSize = uncompressedBuffer->size(); + + auto lengthBufferPtr = (int64_t*)(lengthBuffer->mutable_data()); + // First write metadata. + // Write compression mode. + *lengthBufferPtr++ = CompressionMode::ROWVECTOR; + // Store uncompressed size. + *lengthBufferPtr++ = uncompressedSize; // uncompressedLength + // Skip compressed size and update later. + auto compressedLengthPtr = lengthBufferPtr++; + // Store number of buffers. + *lengthBufferPtr++ = buffers.size(); + + int64_t compressValueOffset = 0; + for (auto& buffer : buffers) { + // Copy all buffers into one big buffer. + if (buffer != nullptr && buffer->size() != 0) { + *lengthBufferPtr++ = buffer->size(); + memcpy(uncompressedBuffer->mutable_data() + compressValueOffset, buffer->data(), buffer->size()); + compressValueOffset += buffer->size(); + } else { + *lengthBufferPtr++ = 0; + } + } + + // Compress the big buffer. + int64_t maxLength = codec->MaxCompressedLen(uncompressedSize, nullptr); + ARROW_ASSIGN_OR_RAISE(compressedBuffer, arrow::AllocateResizableBuffer(maxLength, pool)); + ARROW_ASSIGN_OR_RAISE( + int64_t actualLength, + codec->Compress(uncompressedSize, uncompressedBuffer->data(), maxLength, compressedBuffer->mutable_data())); + RETURN_NOT_OK(compressedBuffer->Resize(actualLength, /*shrink*/ true)); + + // Update compressed size. + *compressedLengthPtr = actualLength; + return arrow::Status::OK(); +} +} // namespace + +arrow::Result> makeCompressedRecordBatch( + uint32_t numRows, + const std::vector>& buffers, + const std::shared_ptr compressWriteSchema, + arrow::MemoryPool* pool, + arrow::util::Codec* codec, + int32_t bufferCompressThreshold, + CompressionMode compressionMode, + int64_t& compressionTime) { + ScopedTimer{compressionTime}; + std::vector> arrays; + // header col, numRows, compressionType + { + ARROW_ASSIGN_OR_RAISE(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); + memcpy(headerBuffer->mutable_data(), &numRows, sizeof(uint32_t)); + int32_t compressType = static_cast(codec->compression_type()); + memcpy(headerBuffer->mutable_data() + sizeof(uint32_t), &compressType, sizeof(int32_t)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE( + arrays.back(), makeBinaryArray(compressWriteSchema->field(0)->type(), std::move(headerBuffer), pool)); + } + std::shared_ptr lengthBuffer; + std::shared_ptr valueBuffer; + if (compressionMode == CompressionMode::BUFFER && numRows > bufferCompressThreshold) { + RETURN_NOT_OK(getLengthBufferAndValueBufferOneByOne(buffers, pool, codec, lengthBuffer, valueBuffer)); + } else { + RETURN_NOT_OK(getLengthBufferAndValueBufferStream(buffers, pool, codec, lengthBuffer, valueBuffer)); + } + + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(compressWriteSchema->field(1)->type(), lengthBuffer, pool)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(compressWriteSchema->field(2)->type(), valueBuffer, pool)); + return arrow::RecordBatch::Make(compressWriteSchema, 1, {arrays}); +} + +arrow::Result> makeUncompressedRecordBatch( + uint32_t numRows, + const std::vector>& buffers, + const std::shared_ptr writeSchema, + arrow::MemoryPool* pool) { + std::vector> arrays; + // header col, numRows, compressionType + { + ARROW_ASSIGN_OR_RAISE(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); + memcpy(headerBuffer->mutable_data(), &numRows, sizeof(uint32_t)); + int32_t compressType = static_cast(arrow::Compression::type::UNCOMPRESSED); + memcpy(headerBuffer->mutable_data() + sizeof(uint32_t), &compressType, sizeof(int32_t)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(writeSchema->field(0)->type(), std::move(headerBuffer), pool)); + } + + int32_t bufferNum = writeSchema->num_fields() - 1; + for (int32_t i = 0; i < bufferNum; i++) { + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(writeSchema->field(i + 1)->type(), buffers[i], pool)); + } + return arrow::RecordBatch::Make(writeSchema, 1, {arrays}); +} +} // namespace gluten std::string gluten::generateUuid() { boost::uuids::random_generator generator; @@ -115,17 +301,29 @@ arrow::Result>> gluten::toShuffleWr return shuffleWriterTypeId; } -int64_t gluten::getBufferSizes(const std::shared_ptr& array) { - return gluten::getBufferSizes(array->data()->buffers); +int64_t gluten::getBufferSize(const std::shared_ptr& array) { + return gluten::getBufferSize(array->data()->buffers); } -int64_t gluten::getBufferSizes(const std::vector>& buffers) { +int64_t gluten::getBufferSize(const std::vector>& buffers) { return std::accumulate( std::cbegin(buffers), std::cend(buffers), 0LL, [](int64_t sum, const std::shared_ptr& buf) { return buf == nullptr ? sum : sum + buf->size(); }); } +int64_t gluten::getMaxCompressedBufferSize( + const std::vector>& buffers, + arrow::util::Codec* codec) { + int64_t totalSize = 0; + for (auto& buffer : buffers) { + if (buffer != nullptr && buffer->size() != 0) { + totalSize += codec->MaxCompressedLen(buffer->size(), nullptr); + } + } + return totalSize; +} + arrow::Status gluten::writeEos(arrow::io::OutputStream* os, int64_t* bytes) { // write EOS static constexpr int32_t kIpcContinuationToken = -1; diff --git a/cpp/core/shuffle/Utils.h b/cpp/core/shuffle/Utils.h index 9c104d759828..fde8a7e0cc7c 100644 --- a/cpp/core/shuffle/Utils.h +++ b/cpp/core/shuffle/Utils.h @@ -25,10 +25,16 @@ #include #include #include +#include "utils/Compression.h" namespace gluten { -const std::string kGlutenSparkLocalDirs = "GLUTEN_SPARK_LOCAL_DIRS"; +using BinaryArrayLengthBufferType = uint32_t; +using IpcOffsetBufferType = arrow::LargeStringType::offset_type; + +static const size_t kSizeOfBinaryArrayLengthBuffer = sizeof(BinaryArrayLengthBufferType); +static const size_t kSizeOfIpcOffsetBuffer = sizeof(IpcOffsetBufferType); +static const std::string kGlutenSparkLocalDirs = "GLUTEN_SPARK_LOCAL_DIRS"; std::string generateUuid(); @@ -39,10 +45,31 @@ arrow::Result createTempShuffleFile(const std::string& dir); arrow::Result>> toShuffleWriterTypeId( const std::vector>& fields); -int64_t getBufferSizes(const std::shared_ptr& array); +int64_t getBufferSize(const std::shared_ptr& array); + +int64_t getBufferSize(const std::vector>& buffers); -int64_t getBufferSizes(const std::vector>& buffers); +int64_t getMaxCompressedBufferSize( + const std::vector>& buffers, + arrow::util::Codec* codec); arrow::Status writeEos(arrow::io::OutputStream* os, int64_t* bytes); +arrow::Result> makeCompressedRecordBatch( + uint32_t numRows, + const std::vector>& buffers, + const std::shared_ptr compressWriteSchema, + arrow::MemoryPool* pool, + arrow::util::Codec* codec, + int32_t bufferCompressThreshold, + CompressionMode compressionMode, + int64_t& compressionTime); + +// generate the new big one row several columns binary recordbatch +arrow::Result> makeUncompressedRecordBatch( + uint32_t numRows, + const std::vector>& buffers, + const std::shared_ptr writeSchema, + arrow::MemoryPool* pool); + } // namespace gluten diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc index e3ccbeec3161..d6d7cb28c644 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc @@ -16,24 +16,23 @@ */ #include "CelebornPartitionWriter.h" +#include "shuffle/Utils.h" +#include "utils/Timer.h" namespace gluten { -class CelebornEvictHandle final : public EvictHandle { +class CelebornEvictHandle final : public Evictor { public: - CelebornEvictHandle( - int64_t bufferSize, - const arrow::ipc::IpcWriteOptions& options, - arrow::MemoryPool* pool, - RssClient* client, - std::vector& bytesEvicted) - : bufferSize_(bufferSize), options_(options), pool_(pool), client_(client), bytesEvicted_(bytesEvicted) {} + CelebornEvictHandle(ShuffleWriterOptions* options, RssClient* client, std::vector& bytesEvicted) + : Evictor(options), client_(client), bytesEvicted_(bytesEvicted) {} arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { // Copy payload to arrow buffered os. - ARROW_ASSIGN_OR_RAISE(auto celebornBufferOs, arrow::io::BufferOutputStream::Create(bufferSize_, pool_)); + ARROW_ASSIGN_OR_RAISE( + auto celebornBufferOs, arrow::io::BufferOutputStream::Create(options_->buffer_size, options_->memory_pool)); int32_t metadataLength = 0; // unused - RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_, celebornBufferOs.get(), &metadataLength)); + RETURN_NOT_OK( + arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, celebornBufferOs.get(), &metadataLength)); payload = nullptr; // Invalidate payload immediately. // Push. @@ -48,56 +47,63 @@ class CelebornEvictHandle final : public EvictHandle { } private: - int64_t bufferSize_; - arrow::ipc::IpcWriteOptions options_; - arrow::MemoryPool* pool_; RssClient* client_; - std::vector& bytesEvicted_; + std::vector& bytesEvicted_; }; arrow::Status CelebornPartitionWriter::init() { - const auto& options = shuffleWriter_->options(); - bytesEvicted_.resize(shuffleWriter_->numPartitions(), 0); - evictHandle_ = std::make_shared( - options.buffer_size, options.ipc_write_options, options.memory_pool, celebornClient_.get(), bytesEvicted_); + bytesEvicted_.resize(numPartitions_, 0); + rawPartitionLengths_.resize(numPartitions_, 0); + evictHandle_ = std::make_shared(options_, celebornClient_.get(), bytesEvicted_); return arrow::Status::OK(); } -arrow::Status CelebornPartitionWriter::stop() { - // push data and collect metrics - for (auto pid = 0; pid < shuffleWriter_->numPartitions(); ++pid) { - ARROW_ASSIGN_OR_RAISE(auto payload, shuffleWriter_->createPayloadFromBuffer(pid, false)); - if (payload) { - RETURN_NOT_OK(evictHandle_->evict(pid, std::move(payload))); - } - shuffleWriter_->setPartitionLengths(pid, bytesEvicted_[pid]); - shuffleWriter_->setTotalBytesWritten(shuffleWriter_->totalBytesWritten() + bytesEvicted_[pid]); - } +arrow::Status CelebornPartitionWriter::stop(ShuffleWriterMetrics* metrics) { + // Push data and collect metrics. + auto totalBytesEvicted = std::accumulate(bytesEvicted_.begin(), bytesEvicted_.end(), 0); celebornClient_->stop(); + // Populate metrics. + metrics->totalCompressTime += compressTime_; + metrics->totalEvictTime += evictHandle_->getEvictTime(); + metrics->totalWriteTime += writeTime_; + metrics->totalBytesEvicted += totalBytesEvicted; + metrics->totalBytesWritten += totalBytesEvicted; + metrics->partitionLengths = std::move(bytesEvicted_); + metrics->rawPartitionLengths = std::move(rawPartitionLengths_); return arrow::Status::OK(); } -arrow::Status CelebornPartitionWriter::requestNextEvict(bool flush) { - return arrow::Status::OK(); +arrow::Status CelebornPartitionWriter::finishEvict() { + return evictHandle_->finish(); } -EvictHandle* CelebornPartitionWriter::getEvictHandle() { - return evictHandle_.get(); +arrow::Status CelebornPartitionWriter::evict( + uint32_t partitionId, + uint32_t numRows, + std::vector> buffers, + Evictor::Type evictType) { + rawPartitionLengths_[partitionId] += getBufferSize(buffers); + ScopedTimer timer(evictTime_); + ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffers(numRows, std::move(buffers))); + RETURN_NOT_OK(evictHandle_->evict(partitionId, std::move(payload))); + return arrow::Status::OK(); } -arrow::Status CelebornPartitionWriter::finishEvict() { - return evictHandle_->finish(); +arrow::Status CelebornPartitionWriter::evictFixedSize(int64_t size, int64_t* actual) { + *actual = 0; + return arrow::Status::OK(); } CelebornPartitionWriterCreator::CelebornPartitionWriterCreator(std::shared_ptr client) : PartitionWriterCreator(), client_(client) {} arrow::Result> CelebornPartitionWriterCreator::make( - ShuffleWriter* shuffleWriter) { - std::shared_ptr res(new CelebornPartitionWriter(shuffleWriter, client_)); - RETURN_NOT_OK(res->init()); - return res; + uint32_t numPartitions, + ShuffleWriterOptions* options) { + auto partitionWriter = std::make_shared(numPartitions, options, client_); + RETURN_NOT_OK(partitionWriter->init()); + return partitionWriter; } } // namespace gluten diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.h b/cpp/core/shuffle/rss/CelebornPartitionWriter.h index 3ca73d6d7af4..a3be4f46a17e 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.h +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.h @@ -29,34 +29,44 @@ namespace gluten { class CelebornPartitionWriter final : public RemotePartitionWriter { public: - CelebornPartitionWriter(ShuffleWriter* shuffleWriter, std::shared_ptr celebornClient) - : RemotePartitionWriter(shuffleWriter) { + CelebornPartitionWriter( + uint32_t numPartitions, + ShuffleWriterOptions* options, + std::shared_ptr celebornClient) + : RemotePartitionWriter(numPartitions, options) { celebornClient_ = celebornClient; } - arrow::Status requestNextEvict(bool flush /*unused*/) override; - - EvictHandle* getEvictHandle() override; + arrow::Status evict( + uint32_t partitionId, + uint32_t numRows, + std::vector> buffers, + Evictor::Type evictType /* unused */) override; arrow::Status finishEvict() override; arrow::Status init() override; - arrow::Status stop() override; + arrow::Status stop(ShuffleWriterMetrics* metrics) override; + + arrow::Status evictFixedSize(int64_t size, int64_t* actual) override; private: std::shared_ptr celebornClient_; - std::shared_ptr evictHandle_; + std::shared_ptr evictHandle_; - std::vector bytesEvicted_; + std::vector bytesEvicted_; + std::vector rawPartitionLengths_; }; class CelebornPartitionWriterCreator : public ShuffleWriter::PartitionWriterCreator { public: explicit CelebornPartitionWriterCreator(std::shared_ptr client); - arrow::Result> make(ShuffleWriter* shuffleWriter) override; + arrow::Result> make( + uint32_t numPartitions, + ShuffleWriterOptions* options) override; private: std::shared_ptr client_; diff --git a/cpp/core/shuffle/rss/RemotePartitionWriter.h b/cpp/core/shuffle/rss/RemotePartitionWriter.h index 8ca4cf1a87f7..76b1a0bc5ca7 100644 --- a/cpp/core/shuffle/rss/RemotePartitionWriter.h +++ b/cpp/core/shuffle/rss/RemotePartitionWriter.h @@ -23,7 +23,8 @@ namespace gluten { class RemotePartitionWriter : public ShuffleWriter::PartitionWriter { public: - explicit RemotePartitionWriter(ShuffleWriter* shuffleWriter) : PartitionWriter(shuffleWriter) {} + explicit RemotePartitionWriter(uint32_t numPartitions, ShuffleWriterOptions* options) + : PartitionWriter(numPartitions, options) {} }; } // namespace gluten diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 0d4a99dfd658..d776cc36c3e8 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -287,8 +287,7 @@ set(VELOX_SRCS operators/serializer/VeloxRowToColumnarConverter.cc operators/writer/VeloxParquetDatasource.cc shuffle/VeloxShuffleReader.cc - shuffle/VeloxShuffleUtils.cc - shuffle/VeloxShuffleWriter.cc + shuffle/VeloxShuffleWriter.cc substrait/SubstraitParser.cc substrait/SubstraitToVeloxExpr.cc substrait/SubstraitToVeloxPlan.cc @@ -303,7 +302,7 @@ set(VELOX_SRCS utils/VeloxArrowUtils.cc utils/ConfigExtractor.cc utils/Common.cc - ) + ) if(BUILD_TESTS OR BUILD_BENCHMARKS) list(APPEND VELOX_SRCS utils/tests/MemoryPoolUtils.cc) diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index dfff284ad67c..8aa8df34c663 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -19,8 +19,8 @@ #include -#include "VeloxShuffleUtils.h" #include "memory/VeloxColumnarBatch.h" +#include "shuffle/Utils.h" #include "utils/Common.h" #include "utils/Compression.h" #include "utils/VeloxArrowUtils.h" diff --git a/cpp/velox/shuffle/VeloxShuffleUtils.cc b/cpp/velox/shuffle/VeloxShuffleUtils.cc deleted file mode 100644 index d4a7fd317c38..000000000000 --- a/cpp/velox/shuffle/VeloxShuffleUtils.cc +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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 "shuffle/VeloxShuffleUtils.h" -#include -#include - -int64_t gluten::getMaxCompressedBufferSize( - const std::vector>& buffers, - arrow::util::Codec* codec) { - int64_t totalSize = 0; - for (auto& buffer : buffers) { - if (buffer != nullptr && buffer->size() != 0) { - totalSize += codec->MaxCompressedLen(buffer->size(), nullptr); - } - } - return totalSize; -} - -int64_t gluten::getBuffersSize(const std::vector>& buffers) { - int64_t totalSize = 0; - for (auto& buffer : buffers) { - if (buffer != nullptr) { - totalSize += buffer->size(); - } - } - return totalSize; -} diff --git a/cpp/velox/shuffle/VeloxShuffleUtils.h b/cpp/velox/shuffle/VeloxShuffleUtils.h deleted file mode 100644 index d679933ec34b..000000000000 --- a/cpp/velox/shuffle/VeloxShuffleUtils.h +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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. - */ - -#pragma once - -#include - -namespace gluten { - -using BinaryArrayLengthBufferType = uint32_t; -using IpcOffsetBufferType = arrow::LargeStringType::offset_type; - -static const size_t kSizeOfBinaryArrayLengthBuffer = sizeof(BinaryArrayLengthBufferType); -static const size_t kSizeOfIpcOffsetBuffer = sizeof(IpcOffsetBufferType); - -int64_t getBuffersSize(const std::vector>& buffers); - -int64_t getMaxCompressedBufferSize( - const std::vector>& buffers, - arrow::util::Codec* codec); - -} // namespace gluten diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.cc b/cpp/velox/shuffle/VeloxShuffleWriter.cc index 7955d24b1f40..dd99ec8eeac9 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxShuffleWriter.cc @@ -16,11 +16,12 @@ */ #include "VeloxShuffleWriter.h" -#include "VeloxShuffleUtils.h" #include "memory/ArrowMemory.h" #include "memory/VeloxColumnarBatch.h" #include "memory/VeloxMemoryManager.h" #include "shuffle/Partitioner.h" +#include "shuffle/ShuffleSchema.h" +#include "shuffle/Utils.h" #include "utils/Common.h" #include "utils/Compression.h" #include "utils/Timer.h" @@ -99,189 +100,6 @@ const int32_t* getFirstColumn(const facebook::velox::RowVector& rv) { return firstChild->asFlatVector()->rawValues(); } -arrow::Result> makeNullBinaryArray( - std::shared_ptr type, - arrow::MemoryPool* pool) { - ARROW_ASSIGN_OR_RAISE(auto offsetBuffer, arrow::AllocateResizableBuffer(kSizeOfIpcOffsetBuffer << 1, pool)); - // set the first offset to 0, and set the value offset - uint8_t* offsetaddr = offsetBuffer->mutable_data(); - memset(offsetaddr, 0, kSizeOfIpcOffsetBuffer); - // second value offset 0 - memset(offsetaddr + kSizeOfIpcOffsetBuffer, 0, kSizeOfIpcOffsetBuffer); - // If it is not compressed array, null valueBuffer - // worked, but if compress, will core dump at buffer::size(), so replace by kNullBuffer - static std::shared_ptr kNullBuffer = std::make_shared(nullptr, 0); - return arrow::MakeArray(arrow::ArrayData::Make(type, 1, {nullptr, std::move(offsetBuffer), kNullBuffer})); -} - -arrow::Result> makeBinaryArray( - std::shared_ptr type, - std::shared_ptr valueBuffer, - arrow::MemoryPool* pool) { - if (valueBuffer == nullptr) { - return makeNullBinaryArray(type, pool); - } - - ARROW_ASSIGN_OR_RAISE(auto offsetBuffer, arrow::AllocateResizableBuffer(kSizeOfIpcOffsetBuffer << 1, pool)); - // set the first offset to 0, and set the value offset - uint8_t* offsetaddr = offsetBuffer->mutable_data(); - memset(offsetaddr, 0, kSizeOfIpcOffsetBuffer); - int64_t length = valueBuffer->size(); - memcpy(offsetaddr + kSizeOfIpcOffsetBuffer, reinterpret_cast(&length), kSizeOfIpcOffsetBuffer); - return arrow::MakeArray(arrow::ArrayData::Make(type, 1, {nullptr, std::move(offsetBuffer), valueBuffer})); -} - -// Length buffer layout |compressionMode|buffers.size()|buffer1 unCompressedLength|buffer1 compressedLength| buffer2... -arrow::Status getLengthBufferAndValueBufferOneByOne( - const std::vector>& buffers, - arrow::MemoryPool* pool, - arrow::util::Codec* codec, - std::shared_ptr& lengthBuffer, - std::shared_ptr& valueBuffer) { - ARROW_ASSIGN_OR_RAISE( - lengthBuffer, arrow::AllocateResizableBuffer((1 + 1 + buffers.size() * 2) * sizeof(int64_t), pool)); - auto lengthBufferPtr = (int64_t*)(lengthBuffer->mutable_data()); - // Write compression mode. - *lengthBufferPtr++ = CompressionMode::BUFFER; - // Write number of buffers. - *lengthBufferPtr++ = buffers.size(); - - int64_t compressedBufferMaxSize = getMaxCompressedBufferSize(buffers, codec); - ARROW_ASSIGN_OR_RAISE(valueBuffer, arrow::AllocateResizableBuffer(compressedBufferMaxSize, pool)); - int64_t compressValueOffset = 0; - for (auto& buffer : buffers) { - if (buffer != nullptr && buffer->size() != 0) { - int64_t actualLength; - int64_t maxLength = codec->MaxCompressedLen(buffer->size(), nullptr); - ARROW_ASSIGN_OR_RAISE( - actualLength, - codec->Compress( - buffer->size(), buffer->data(), maxLength, valueBuffer->mutable_data() + compressValueOffset)); - compressValueOffset += actualLength; - *lengthBufferPtr++ = buffer->size(); - *lengthBufferPtr++ = actualLength; - } else { - *lengthBufferPtr++ = 0; - *lengthBufferPtr++ = 0; - } - } - RETURN_NOT_OK(valueBuffer->Resize(compressValueOffset, /*shrink*/ true)); - return arrow::Status::OK(); -} - -// Length buffer layout |compressionMode|buffer unCompressedLength|buffer compressedLength|buffers.size()| buffer1 size -// | buffer2 size -arrow::Status getLengthBufferAndValueBufferStream( - const std::vector>& buffers, - arrow::MemoryPool* pool, - arrow::util::Codec* codec, - std::shared_ptr& lengthBuffer, - std::shared_ptr& compressedBuffer) { - ARROW_ASSIGN_OR_RAISE(lengthBuffer, arrow::AllocateResizableBuffer((1 + 3 + buffers.size()) * sizeof(int64_t), pool)); - auto originalBufferSize = getBuffersSize(buffers); - - // because 64B align, uncompressedBuffer size maybe bigger than unCompressedBufferSize which is - // getBuffersSize(buffers), then cannot use this size - ARROW_ASSIGN_OR_RAISE(auto uncompressedBuffer, arrow::AllocateResizableBuffer(originalBufferSize, pool)); - int64_t uncompressedSize = uncompressedBuffer->size(); - - auto lengthBufferPtr = (int64_t*)(lengthBuffer->mutable_data()); - // First write metadata. - // Write compression mode. - *lengthBufferPtr++ = CompressionMode::ROWVECTOR; - // Store uncompressed size. - *lengthBufferPtr++ = uncompressedSize; // uncompressedLength - // Skip compressed size and update later. - auto compressedLengthPtr = lengthBufferPtr++; - // Store number of buffers. - *lengthBufferPtr++ = buffers.size(); - - int64_t compressValueOffset = 0; - for (auto& buffer : buffers) { - // Copy all buffers into one big buffer. - if (buffer != nullptr && buffer->size() != 0) { - *lengthBufferPtr++ = buffer->size(); - gluten::fastCopy(uncompressedBuffer->mutable_data() + compressValueOffset, buffer->data(), buffer->size()); - compressValueOffset += buffer->size(); - } else { - *lengthBufferPtr++ = 0; - } - } - - // Compress the big buffer. - int64_t maxLength = codec->MaxCompressedLen(uncompressedSize, nullptr); - ARROW_ASSIGN_OR_RAISE(compressedBuffer, arrow::AllocateResizableBuffer(maxLength, pool)); - ARROW_ASSIGN_OR_RAISE( - int64_t actualLength, - codec->Compress(uncompressedSize, uncompressedBuffer->data(), maxLength, compressedBuffer->mutable_data())); - RETURN_NOT_OK(compressedBuffer->Resize(actualLength, /*shrink*/ true)); - - // Update compressed size. - *compressedLengthPtr = actualLength; - return arrow::Status::OK(); -} - -arrow::Result> makeCompressedRecordBatch( - uint32_t numRows, - const std::vector>& buffers, - const std::shared_ptr compressWriteSchema, - arrow::MemoryPool* pool, - arrow::util::Codec* codec, - int32_t bufferCompressThreshold, - CompressionMode compressionMode, - int64_t& compressionTime) { - ScopedTimer{compressionTime}; - std::vector> arrays; - // header col, numRows, compressionType - { - ARROW_ASSIGN_OR_RAISE(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); - memcpy(headerBuffer->mutable_data(), &numRows, sizeof(uint32_t)); - int32_t compressType = static_cast(codec->compression_type()); - memcpy(headerBuffer->mutable_data() + sizeof(uint32_t), &compressType, sizeof(int32_t)); - arrays.emplace_back(); - ARROW_ASSIGN_OR_RAISE( - arrays.back(), makeBinaryArray(compressWriteSchema->field(0)->type(), std::move(headerBuffer), pool)); - } - std::shared_ptr lengthBuffer; - std::shared_ptr valueBuffer; - if (compressionMode == CompressionMode::BUFFER && numRows > bufferCompressThreshold) { - RETURN_NOT_OK(getLengthBufferAndValueBufferOneByOne(buffers, pool, codec, lengthBuffer, valueBuffer)); - } else { - RETURN_NOT_OK(getLengthBufferAndValueBufferStream(buffers, pool, codec, lengthBuffer, valueBuffer)); - } - - arrays.emplace_back(); - ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(compressWriteSchema->field(1)->type(), lengthBuffer, pool)); - arrays.emplace_back(); - ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(compressWriteSchema->field(2)->type(), valueBuffer, pool)); - return arrow::RecordBatch::Make(compressWriteSchema, 1, {arrays}); -} - -// generate the new big one row several columns binary recordbatch -arrow::Result> makeUncompressedRecordBatch( - uint32_t numRows, - const std::vector>& buffers, - const std::shared_ptr writeSchema, - arrow::MemoryPool* pool) { - std::vector> arrays; - // header col, numRows, compressionType - { - ARROW_ASSIGN_OR_RAISE(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); - memcpy(headerBuffer->mutable_data(), &numRows, sizeof(uint32_t)); - int32_t compressType = static_cast(arrow::Compression::type::UNCOMPRESSED); - memcpy(headerBuffer->mutable_data() + sizeof(uint32_t), &compressType, sizeof(int32_t)); - arrays.emplace_back(); - ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(writeSchema->field(0)->type(), std::move(headerBuffer), pool)); - } - - int32_t bufferNum = writeSchema->num_fields() - 1; - for (int32_t i = 0; i < bufferNum; i++) { - arrays.emplace_back(); - ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(writeSchema->field(i + 1)->type(), buffers[i], pool)); - } - return arrow::RecordBatch::Make(writeSchema, 1, {arrays}); -} - class EvictGuard { public: explicit EvictGuard(EvictState& evictState) : evictState_(evictState) { @@ -424,7 +242,7 @@ arrow::Status VeloxShuffleWriter::init() { // memory_pool should be assigned. VELOX_CHECK_NOT_NULL(options_.memory_pool); - ARROW_ASSIGN_OR_RAISE(partitionWriter_, partitionWriterCreator_->make(this)); + ARROW_ASSIGN_OR_RAISE(partitionWriter_, partitionWriterCreator_->make(numPartitions_, &options_)); ARROW_ASSIGN_OR_RAISE( partitioner_, Partitioner::make(options_.partitioning, numPartitions_, options_.start_partition_id)); @@ -438,9 +256,6 @@ arrow::Status VeloxShuffleWriter::init() { partitionBufferIdxBase_.resize(numPartitions_); - partitionLengths_.resize(numPartitions_); - rawPartitionLengths_.resize(numPartitions_); - return arrow::Status::OK(); } @@ -525,11 +340,7 @@ arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64 buffers.emplace_back(); ARROW_ASSIGN_OR_RAISE(buffers.back(), generateComplexTypeBuffers(rowVector)); } - - rawPartitionLengths_[0] += getBuffersSize(buffers); - ARROW_ASSIGN_OR_RAISE(auto rb, makeRecordBatch(rv.size(), buffers)); - ARROW_ASSIGN_OR_RAISE(auto payload, createPayload(*rb, false)); - RETURN_NOT_OK(evictPayload(0, std::move(payload))); + RETURN_NOT_OK(evictBuffers(0, rv.size(), std::move(buffers))); } else if (options_.partitioning == Partitioning::kRange) { auto compositeBatch = std::dynamic_pointer_cast(cb); VELOX_CHECK_NOT_NULL(compositeBatch); @@ -571,10 +382,17 @@ arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64 } arrow::Status VeloxShuffleWriter::stop() { + if (options_.partitioning != Partitioning::kSingle) { + for (auto pid = 0; pid < numPartitions_; ++pid) { + auto numRows = partitionBufferIdxBase_[pid]; + ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(pid, false)); + RETURN_NOT_OK(partitionWriter_->evict(pid, numRows, std::move(buffers), Evictor::Type::kStop)); + } + } { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingStop]); setSplitState(SplitState::kStop); - RETURN_NOT_OK(partitionWriter_->stop()); + RETURN_NOT_OK(partitionWriter_->stop(&metrics_)); partitionBuffers_.clear(); } @@ -971,7 +789,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel binaryBuf.valueOffset = valueOffset; } - return arrow::Status::OK(); } @@ -1029,6 +846,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel arrow::Status VeloxShuffleWriter::initColumnTypes(const facebook::velox::RowVector& rv) { schema_ = toArrowSchema(rv.type(), veloxPool_.get()); + options_.write_schema = options_.codec ? toCompressWriteSchema() : toWriteSchema(*schema_); for (size_t i = 0; i < rv.childrenSize(); ++i) { veloxColumnTypes_.push_back(rv.childAt(i)->type()); @@ -1221,58 +1039,39 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel return arrow::Status::OK(); } - arrow::Result> VeloxShuffleWriter::createPayloadFromBuffer( - uint32_t partitionId, bool reuseBuffers) { - ARROW_ASSIGN_OR_RAISE(auto rb, createArrowRecordBatchFromBuffer(partitionId, reuseBuffers)); - if (rb) { - ARROW_ASSIGN_OR_RAISE(auto payload, createPayload(*rb, reuseBuffers)); - return payload; + arrow::Status VeloxShuffleWriter::evictBuffers( + uint32_t partitionId, uint32_t numRows, std::vector> buffers) { + if (!buffers.empty()) { + RETURN_NOT_OK(partitionWriter_->evict(partitionId, numRows, std::move(buffers), Evictor::Type::kCache)); } - return nullptr; + return arrow::Status::OK(); } - arrow::Result> VeloxShuffleWriter::createPayload( - const arrow::RecordBatch& rb, bool reuseBuffers) { - auto payload = std::make_unique(); - // Extract numRows from header column - RETURN_NOT_OK(arrow::ipc::GetRecordBatchPayload(rb, options_.ipc_write_options, payload.get())); - if (codec_ == nullptr) { - // Without compression, we need to perform a manual copy of the original buffers - // so that we can reuse them for next split. - if (reuseBuffers) { - for (auto i = 0; i < payload->body_buffers.size(); ++i) { - auto& buffer = payload->body_buffers[i]; - if (buffer) { - ARROW_ASSIGN_OR_RAISE(auto copy, ::arrow::AllocateResizableBuffer(buffer->size(), payloadPool_.get())); - if (buffer->size() > 0) { - gluten::fastCopy(copy->mutable_data(), buffer->data(), static_cast(buffer->size())); - } - buffer = std::move(copy); - } - } - } + arrow::Status VeloxShuffleWriter::evictPartitionBuffers(uint32_t partitionId, bool reuseBuffers) { + auto numRows = partitionBufferIdxBase_[partitionId]; + if (numRows > 0) { + ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(partitionId, reuseBuffers)); + RETURN_NOT_OK(evictBuffers(partitionId, numRows, buffers)); } - return payload; + return arrow::Status::OK(); } - arrow::Result> VeloxShuffleWriter::createArrowRecordBatchFromBuffer( + arrow::Result>> VeloxShuffleWriter::assembleBuffers( uint32_t partitionId, bool reuseBuffers) { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingCreateRbFromBuffer]); if (partitionBufferIdxBase_[partitionId] == 0) { - return nullptr; + return std::vector>{}; } auto numRows = partitionBufferIdxBase_[partitionId]; - - // already filled auto fixedWidthIdx = 0; auto binaryIdx = 0; auto numFields = schema_->num_fields(); std::vector> arrays(numFields); std::vector> allBuffers; - // one column should have 2 buffers at least, string column has 3 column buffers + // One column should have 2 buffers at least, string column has 3 column buffers. allBuffers.reserve(fixedWidthColumnCount_ * 2 + binaryColumnIndices_.size() * 3); bool hasComplexType = false; for (int i = 0; i < numFields; ++i) { @@ -1301,11 +1100,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel if (reuseBuffers) { // Set the first value offset to 0. binaryBuf.valueOffset = 0; - } else { - // Reset all buffers. - partitionValidityAddrs_[fixedWidthColumnCount_ + binaryIdx][partitionId] = nullptr; - binaryBuf = BinaryBuf(); - partitionBuffers_[fixedWidthColumnCount_ + binaryIdx][partitionId].clear(); } binaryIdx++; break; @@ -1341,11 +1135,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel arrow::SliceBuffer(valueBuffer, 0, numRows * (arrow::bit_width(arrowColumnTypes_[i]->id()) >> 3)); } allBuffers.push_back(std::move(slicedValueBuffer)); - if (!reuseBuffers) { - partitionValidityAddrs_[fixedWidthIdx][partitionId] = nullptr; - partitionFixedWidthValueAddrs_[fixedWidthIdx][partitionId] = nullptr; - partitionBuffers_[fixedWidthIdx][partitionId].clear(); - } fixedWidthIdx++; break; } @@ -1369,31 +1158,25 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel arenas_[partitionId] = nullptr; } + partitionBufferIdxBase_[partitionId] = 0; if (!reuseBuffers) { - partition2BufferSize_[partitionId] = 0; + RETURN_NOT_OK(resetPartitionBuffer(partitionId)); } - partitionBufferIdxBase_[partitionId] = 0; - - rawPartitionLengths_[partitionId] += getBuffersSize(allBuffers); - return makeRecordBatch(numRows, allBuffers); - } - - arrow::Result> VeloxShuffleWriter::makeRecordBatch( - uint32_t numRows, const std::vector>& buffers) { - SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingMakeRB]); - if (codec_ == nullptr) { - return makeUncompressedRecordBatch(numRows, buffers, writeSchema(), payloadPool_.get()); - } else { - return makeCompressedRecordBatch( - numRows, - buffers, - compressWriteSchema(), - payloadPool_.get(), - codec_.get(), - options_.compression_threshold, - options_.compression_mode, - totalCompressTime_); + if (options_.ipc_write_options.codec == nullptr && reuseBuffers) { + // Without compression, we need to perform a manual copy of the original buffers + // so that we can reuse them for next split. + for (auto i = 0; i < allBuffers.size(); ++i) { + auto& buffer = allBuffers[i]; + if (buffer) { + ARROW_ASSIGN_OR_RAISE(auto copy, arrow::AllocateResizableBuffer(buffer->size(), payloadPool_.get())); + if (buffer->size() > 0) { + gluten::fastCopy(copy->mutable_data(), buffer->data(), static_cast(buffer->size())); + } + buffer = std::move(copy); + } + } } + return allBuffers; } arrow::Status VeloxShuffleWriter::evictFixedSize(int64_t size, int64_t * actual) { @@ -1405,7 +1188,8 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel int64_t reclaimed = 0; if (reclaimed < size && shrinkPartitionBuffersBeforeSpill()) { - ARROW_ASSIGN_OR_RAISE(auto shrunken, shrinkPartitionBuffers()); + int64_t shrunken = 0; + RETURN_NOT_OK(partitionWriter_->evictFixedSize(size - reclaimed, &shrunken)); reclaimed += shrunken; } if (reclaimed < size) { @@ -1431,12 +1215,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel return 0; } auto evicted = beforeEvict; - - { - ScopedTimer evictTime(totalEvictTime_); - RETURN_NOT_OK(partitionWriter_->finishEvict()); - } - + RETURN_NOT_OK(partitionWriter_->finishEvict()); if (auto afterEvict = cachedPayloadSize()) { // Evict can be triggered by compressing buffers. The cachedPayloadSize is not empty. evicted -= afterEvict; @@ -1558,19 +1337,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel return resizePartitionBuffer(partitionId, newSize, /*preserveData=*/true); } - arrow::Result VeloxShuffleWriter::shrinkPartitionBuffers() { - auto beforeShrink = partitionBufferPool_->bytes_allocated(); - if (beforeShrink == 0) { - return 0; - } - for (auto pid = 0; pid < numPartitions_; ++pid) { - RETURN_NOT_OK(shrinkPartitionBuffer(pid)); - } - auto shrunken = beforeShrink - partitionBufferPool_->bytes_allocated(); - VLOG(2) << shrunken << " bytes released from shrinking."; - return shrunken; - } - uint64_t VeloxShuffleWriter::valueBufferSizeForBinaryArray(uint32_t binaryIdx, int64_t newSize) { return (binaryArrayTotalSizeBytes_[binaryIdx] + totalInputNumRows_ - 1) / totalInputNumRows_ * newSize + 1024; } @@ -1630,24 +1396,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel return payloadPool_->bytes_allocated(); } - arrow::Status VeloxShuffleWriter::evictPartitionBuffer(uint32_t partitionId, uint32_t newSize, bool reuseBuffers) { - ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffer(partitionId, reuseBuffers)); - if (payload) { - RETURN_NOT_OK(evictPayload(partitionId, std::move(payload))); - } - return arrow::Status::OK(); - } - - arrow::Status VeloxShuffleWriter::evictPayload( - uint32_t partitionId, std::unique_ptr payload) { - ScopedTimer spillTime(totalEvictTime_); - if (!partitionWriter_->getEvictHandle()) { - RETURN_NOT_OK(partitionWriter_->requestNextEvict(false)); - } - RETURN_NOT_OK(partitionWriter_->getEvictHandle()->evict(partitionId, std::move(payload))); - return arrow::Status::OK(); - } - arrow::Result VeloxShuffleWriter::shrinkPartitionBuffersMinSize(int64_t size) { // Sort partition buffers by (partition2BufferSize_ - partitionBufferIdxBase_) std::vector> pidToSize; @@ -1689,29 +1437,16 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel pidToSize.emplace_back(pid, partition2BufferSize_[pid]); } if (!pidToSize.empty()) { - Timer spillTime; - spillTime.start(); - RETURN_NOT_OK(partitionWriter_->requestNextEvict(true)); - auto evictHandle = partitionWriter_->getEvictHandle(); - spillTime.stop(); - for (auto& item : pidToSize) { auto pid = item.first; - ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffer(pid, false)); - - spillTime.start(); - RETURN_NOT_OK(evictHandle->evict(pid, std::move(payload))); - spillTime.stop(); - + ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(pid, false)); + RETURN_NOT_OK(partitionWriter_->evict(pid, item.second, std::move(buffers), Evictor::Type::kFlush)); evicted = beforeEvict - partitionBufferPool_->bytes_allocated(); if (evicted >= size) { break; } } - spillTime.start(); RETURN_NOT_OK(partitionWriter_->finishEvict()); - spillTime.stop(); - totalEvictTime_ += spillTime.realTimeUsed(); } return evicted; } @@ -1758,7 +1493,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel } else if (beyondThreshold(pid, newSize)) { if (newSize <= partitionBufferIdxBase_[pid]) { // If the newSize is smaller, cache the buffered data and reuse and shrink the buffer. - RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, true)); + RETURN_NOT_OK(evictPartitionBuffers(pid, true)); RETURN_NOT_OK(resizePartitionBuffer(pid, newSize, /*preserveData=*/false)); } else { // If the newSize is larger, check if alreadyFilled + toBeFilled <= newSize @@ -1773,7 +1508,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel // If newSize <= allocated buffer size, reuse and shrink the buffer. // Else free and allocate new buffers. bool reuseBuffers = newSize <= partition2BufferSize_[pid]; - RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, reuseBuffers)); + RETURN_NOT_OK(evictPartitionBuffers(pid, reuseBuffers)); if (reuseBuffers) { RETURN_NOT_OK(resizePartitionBuffer(pid, newSize, /*preserveData=*/false)); } else { @@ -1786,11 +1521,11 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel // buffer. if (newSize > partition2BufferSize_[pid]) { // If the partition size after split is already larger than allocated buffer size, need reallocate. - RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, false)); + RETURN_NOT_OK(evictPartitionBuffers(pid, false)); RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize)); } else { // Partition size after split is smaller than buffer size. Reuse the buffers. - RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, true)); + RETURN_NOT_OK(evictPartitionBuffers(pid, true)); // Reset validity buffer for reallocate. RETURN_NOT_OK(resetValidityBuffer(pid)); } diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.h b/cpp/velox/shuffle/VeloxShuffleWriter.h index f257589fcd2d..13872fb4f408 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxShuffleWriter.h @@ -133,18 +133,14 @@ class VeloxShuffleWriter final : public ShuffleWriter { arrow::Status stop() override; - arrow::Status evictFixedSize(int64_t size, int64_t* actual) override; - - arrow::Result> createPayloadFromBuffer( - uint32_t partitionId, - bool reuseBuffers) override; + arrow::Status evictPartitionBuffers(uint32_t partitionId, bool reuseBuffers) override; - arrow::Status evictPayload(uint32_t partitionId, std::unique_ptr payload) override; + arrow::Status evictFixedSize(int64_t size, int64_t* actual) override; const uint64_t cachedPayloadSize() const override; int64_t rawPartitionBytes() const { - return std::accumulate(rawPartitionLengths_.begin(), rawPartitionLengths_.end(), 0LL); + return std::accumulate(metrics_.rawPartitionLengths.begin(), metrics_.rawPartitionLengths.end(), 0LL); } // for testing @@ -254,13 +250,10 @@ class VeloxShuffleWriter final : public ShuffleWriter { arrow::Status splitComplexType(const facebook::velox::RowVector& rv); - arrow::Status evictPartitionBuffer(uint32_t partitionId, uint32_t newSize, bool reuseBuffers); + arrow::Status + evictBuffers(uint32_t partitionId, uint32_t numRows, std::vector> buffers); - arrow::Result> createArrowRecordBatchFromBuffer( - uint32_t partitionId, - bool reuseBuffers); - - arrow::Result> createPayload(const arrow::RecordBatch& rb, bool reuseBuffers); + arrow::Result>> assembleBuffers(uint32_t partitionId, bool reuseBuffers); template arrow::Status splitFixedType(const uint8_t* srcAddr, const std::vector& dstAddrs) { @@ -293,8 +286,6 @@ class VeloxShuffleWriter final : public ShuffleWriter { arrow::Result shrinkPartitionBuffersMinSize(int64_t size); - arrow::Result shrinkPartitionBuffers(); - arrow::Result evictPartitionBuffersMinSize(int64_t size); arrow::Status shrinkPartitionBuffer(uint32_t partitionId); diff --git a/cpp/velox/tests/VeloxShuffleWriterTest.cc b/cpp/velox/tests/VeloxShuffleWriterTest.cc index b8853ed93552..2c475efa202b 100644 --- a/cpp/velox/tests/VeloxShuffleWriterTest.cc +++ b/cpp/velox/tests/VeloxShuffleWriterTest.cc @@ -82,13 +82,6 @@ TEST_P(SinglePartitioningShuffleWriter, single) { auto shuffleWriter = createShuffleWriter(); testShuffleWrite(*shuffleWriter, {inputVector1_, inputVector2_, inputVector1_}); } - // Test not compress small buffer. - { - shuffleWriterOptions_.compression_type = arrow::Compression::LZ4_FRAME; - shuffleWriterOptions_.compression_threshold = 1024; - auto shuffleWriter = createShuffleWriter(); - testShuffleWrite(*shuffleWriter, {inputVector1_, inputVector1_}); - } // Split null RowVector. { auto shuffleWriter = createShuffleWriter(); @@ -373,10 +366,7 @@ TEST_P(RoundRobinPartitioningShuffleWriter, spillVerifyResult) { // Clear buffers and evict payloads and cache. for (auto pid : {0, 1}) { - GLUTEN_ASSIGN_OR_THROW(auto payload, shuffleWriter->createPayloadFromBuffer(pid, true)); - if (payload) { - ASSERT_NOT_OK(shuffleWriter->evictPayload(pid, std::move(payload))); - } + ASSERT_NOT_OK(shuffleWriter->evictPartitionBuffers(pid, true)); } ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); @@ -484,10 +474,7 @@ TEST_F(VeloxShuffleWriterMemoryTest, kInit) { // Clear buffers then the size after shrink will be 0. for (auto pid = 0; pid < kDefaultShufflePartitions; ++pid) { - GLUTEN_ASSIGN_OR_THROW(auto payload, shuffleWriter->createPayloadFromBuffer(pid, true)); - if (payload) { - ASSERT_NOT_OK(shuffleWriter->evictPayload(pid, std::move(payload))); - } + ASSERT_NOT_OK(shuffleWriter->evictPartitionBuffers(pid, true)); } auto bufferSize = shuffleWriter->partitionBufferSize(); diff --git a/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala b/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala index 5f3db8bf8a27..0d0eae30e9aa 100644 --- a/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala +++ b/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala @@ -80,6 +80,7 @@ object ExecUtil { serializer: Serializer, writeMetrics: Map[String, SQLMetric], metrics: Map[String, SQLMetric]): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { + metrics("numPartitions").set(newPartitioning.numPartitions) // scalastyle:on argcount // only used for fallback range partitioning val rangePartitioner: Option[Partitioner] = newPartitioning match { From 983f8a5be82f90d4c9f1ef57a53e1d8397405676 Mon Sep 17 00:00:00 2001 From: "Ma, Rong" Date: Wed, 6 Dec 2023 12:59:28 +0800 Subject: [PATCH 2/7] fix coredump --- cpp/core/shuffle/LocalPartitionWriter.cc | 137 +++++++++--------- cpp/core/shuffle/LocalPartitionWriter.h | 4 +- cpp/core/shuffle/PartitionWriter.h | 2 +- .../shuffle/rss/CelebornPartitionWriter.cc | 10 +- .../shuffle/rss/CelebornPartitionWriter.h | 2 +- cpp/velox/shuffle/VeloxShuffleWriter.cc | 6 +- 6 files changed, 80 insertions(+), 81 deletions(-) diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index 2e21a3a33255..8fc24bf8a021 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -36,10 +36,6 @@ class LocalPartitionWriter::LocalEvictor : public Evictor { const std::shared_ptr& spillInfo, Evictor::Type evictType); - bool finished() { - return finished_; - }; - virtual arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) = 0; virtual Type evictType() = 0; @@ -47,70 +43,71 @@ class LocalPartitionWriter::LocalEvictor : public Evictor { protected: uint32_t numPartitions_; std::shared_ptr spillInfo_; - std::shared_ptr os_; - bool finished_{false}; }; class CacheEvictor final : public LocalPartitionWriter::LocalEvictor { public: CacheEvictor(uint32_t numPartitions, ShuffleWriterOptions* options, const std::shared_ptr& spillInfo) - : LocalPartitionWriter::LocalEvictor(numPartitions, options, spillInfo) { - partitionCachedPayload_.resize(numPartitions); - } + : LocalPartitionWriter::LocalEvictor(numPartitions, options, spillInfo) {} arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + if (partitionCachedPayload_.find(partitionId) == partitionCachedPayload_.end()) { + partitionCachedPayload_.emplace(partitionId, std::vector>{}); + } partitionCachedPayload_[partitionId].push_back(std::move(payload)); return arrow::Status::OK(); } arrow::Status finish() override { + if (partitionCachedPayload_.empty()) { + return arrow::Status::OK(); + } + ScopedTimer timer(evictTime_); - if (!finished_) { - ARROW_ASSIGN_OR_RAISE(os_, arrow::io::FileOutputStream::Open(spillInfo_->spilledFile, true)); - int64_t start = 0; - for (uint32_t pid = 0; pid < numPartitions_; ++pid) { - if (!partitionCachedPayload_[pid].empty()) { - RETURN_NOT_OK(flushCachedPayloads(pid, os_.get())); - ARROW_ASSIGN_OR_RAISE(auto end, os_->Tell()); - spillInfo_->partitionSpillInfos.push_back({pid, end - start}); - start = end; - } - } - partitionCachedPayload_.clear(); - ARROW_ASSIGN_OR_RAISE(auto written, os_->Tell()); - RETURN_NOT_OK(os_->Close()); - if (written > 0) { - spillInfo_->empty = false; + ARROW_ASSIGN_OR_RAISE(os_, arrow::io::FileOutputStream::Open(spillInfo_->spilledFile, true)); + int64_t start = 0; + for (uint32_t pid = 0; pid < numPartitions_; ++pid) { + if (partitionCachedPayload_.find(pid) != partitionCachedPayload_.end()) { + RETURN_NOT_OK(flushInternal(pid, os_.get())); + ARROW_ASSIGN_OR_RAISE(auto end, os_->Tell()); + spillInfo_->partitionSpillInfos.push_back({pid, end - start}); + start = end; } - - finished_ = true; } + ARROW_ASSIGN_OR_RAISE(auto written, os_->Tell()); + RETURN_NOT_OK(os_->Close()); + if (written > 0) { + spillInfo_->empty = false; + } + partitionCachedPayload_.clear(); return arrow::Status::OK(); } arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) override { - if (partitionCachedPayload_[partitionId].empty()) { - return arrow::Status::OK(); + if (partitionCachedPayload_.find(partitionId) != partitionCachedPayload_.end()) { + RETURN_NOT_OK(flushInternal(partitionId, os)); } + return arrow::Status::OK(); + } + + Type evictType() override { + return Type::kCache; + } + + private: + std::unordered_map>> partitionCachedPayload_; + arrow::Status flushInternal(uint32_t partitionId, arrow::io::OutputStream* os) { ScopedTimer timer(evictTime_); int32_t metadataLength = 0; // unused auto payloads = std::move(partitionCachedPayload_[partitionId]); - // Clear cached batches before creating the payloads, to avoid spilling this partition. - partitionCachedPayload_[partitionId].clear(); + partitionCachedPayload_.erase(partitionId); for (auto& payload : payloads) { RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, os, &metadataLength)); } return arrow::Status::OK(); } - - Type evictType() override { - return Type::kCache; - } - - private: - std::vector>> partitionCachedPayload_; }; class FlushOnSpillEvictor final : public LocalPartitionWriter::LocalEvictor { @@ -137,12 +134,9 @@ class FlushOnSpillEvictor final : public LocalPartitionWriter::LocalEvictor { } arrow::Status finish() override { - if (!finished_) { - if (os_) { - RETURN_NOT_OK(os_->Close()); - spillInfo_->empty = false; - } - finished_ = true; + if (os_ && !os_->closed()) { + RETURN_NOT_OK(os_->Close()); + spillInfo_->empty = false; } return arrow::Status::OK(); } @@ -206,6 +200,7 @@ arrow::Status LocalPartitionWriter::clearResource() { // When buffered_write = true, dataFileOs_->Close doesn't release underlying buffer. dataFileOs_.reset(); spills_.clear(); + cachedPartitionBuffers_.clear(); return arrow::Status::OK(); } @@ -242,6 +237,11 @@ arrow::Status LocalPartitionWriter::mergeSpills(uint32_t partitionId) { } arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { + if (stopped_) { + return arrow::Status::OK(); + } + stopped_ = true; + // Open final file. // If options_.buffered_write is set, it will acquire 16KB memory that might trigger spill. RETURN_NOT_OK(openDataFile()); @@ -250,8 +250,8 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { writeTimer.start(); int64_t endInFinalFile = 0; - int32_t metadataLength = 0; - auto cachedBuffersIter = cachedPartitionBuffers_.begin(); + int32_t metadataLength = 0; // Unused. + auto cachedPartitionBuffersIter = cachedPartitionBuffers_.begin(); // Iterator over pid. for (auto pid = 0; pid < numPartitions_; ++pid) { // Record start offset. @@ -259,20 +259,22 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { // Iterator over all spilled files. RETURN_NOT_OK(mergeSpills(pid)); // Write cached batches. - if (evictor_ && !evictor_->finished()) { + if (evictor_) { RETURN_NOT_OK(evictor_->flushCachedPayloads(pid, dataFileOs_.get())); } // Compress and write the last payload. // Stop the timer to prevent counting the compression time into write time. - if (cachedBuffersIter != cachedPartitionBuffers_.end() && std::get<0>(*cachedBuffersIter) == pid) { + if (cachedPartitionBuffersIter != cachedPartitionBuffers_.end() && + std::get<0>(*cachedPartitionBuffersIter) == pid) { writeTimer.stop(); ARROW_ASSIGN_OR_RAISE( auto payload, - createPayloadFromBuffers(std::get<1>(*cachedBuffersIter), std::move(std::get<2>(*cachedBuffersIter)))); + createPayloadFromBuffers( + std::get<1>(*cachedPartitionBuffersIter), std::move(std::get<2>(*cachedPartitionBuffersIter)))); writeTimer.start(); RETURN_NOT_OK( arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, dataFileOs_.get(), &metadataLength)); - cachedBuffersIter++; + cachedPartitionBuffersIter++; } ARROW_ASSIGN_OR_RAISE(endInFinalFile, dataFileOs_->Tell()); if (endInFinalFile != startInFinalFile && options_->write_eos) { @@ -283,6 +285,7 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { } partitionLengths_[pid] = endInFinalFile - startInFinalFile; } + RETURN_NOT_OK(finishEvict()); for (auto spill : spills_) { // Check if all spilled data are merged. @@ -295,23 +298,25 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { } RETURN_NOT_OK(fs_->DeleteFile(spill->spilledFile)); } - ARROW_ASSIGN_OR_RAISE(totalBytesWritten_, dataFileOs_->Tell()); + // Check if all partition buffers are merged. + ARROW_RETURN_IF( + cachedPartitionBuffersIter != cachedPartitionBuffers_.end(), + arrow::Status::Invalid("Not all partition buffers are merged.")); writeTimer.stop(); writeTime_ = writeTimer.realTimeUsed(); + ARROW_ASSIGN_OR_RAISE(totalBytesWritten_, dataFileOs_->Tell()); // Close Final file, Clear buffered resources. RETURN_NOT_OK(clearResource()); - // Populate metrics. + // Populate shuffle writer metrics. RETURN_NOT_OK(populateMetrics(metrics)); return arrow::Status::OK(); } arrow::Status LocalPartitionWriter::requestEvict(Evictor::Type evictType) { - if (auto handle = getEvictHandle()) { - if (handle->evictType() == evictType) { - return arrow::Status::OK(); - } + if (evictor_ && evictor_->evictType() == evictType) { + return arrow::Status::OK(); } RETURN_NOT_OK(finishEvict()); @@ -322,24 +327,16 @@ arrow::Status LocalPartitionWriter::requestEvict(Evictor::Type evictType) { return arrow::Status::OK(); } -LocalPartitionWriter::LocalEvictor* LocalPartitionWriter::getEvictHandle() { - if (evictor_ && !evictor_->finished()) { - return evictor_.get(); - } - return nullptr; -} - arrow::Status LocalPartitionWriter::finishEvict() { - if (auto handle = getEvictHandle()) { - RETURN_NOT_OK(handle->finish()); - evictTime_ += handle->getEvictTime(); - // The spilled file should not be empty. However, defensively - // discard the last SpillInfo to avoid iterating over invalid ones. + if (evictor_) { + RETURN_NOT_OK(evictor_->finish()); + evictTime_ += evictor_->getEvictTime(); auto lastSpillInfo = spills_.back(); if (lastSpillInfo->empty) { RETURN_NOT_OK(fs_->DeleteFile(lastSpillInfo->spilledFile)); spills_.pop_back(); } + evictor_ = nullptr; } return arrow::Status::OK(); } @@ -351,11 +348,11 @@ arrow::Status LocalPartitionWriter::evict( Evictor::Type evictType) { rawPartitionLengths_[partitionId] += getBufferSize(buffers); if (evictType == Evictor::Type::kStop) { - cachedPartitionBuffers_.push_back({partitionId, numRows, std::move(buffers)}); + cachedPartitionBuffers_.emplace_back(partitionId, numRows, std::move(buffers)); } else { ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffers(numRows, std::move(buffers))); RETURN_NOT_OK(requestEvict(evictType)); - RETURN_NOT_OK(getEvictHandle()->evict(partitionId, std::move(payload))); + RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); } return arrow::Status::OK(); } diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index 96db7c824476..d89f8d20cb7d 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -87,8 +87,6 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { private: arrow::Status requestEvict(Evictor::Type evictType); - LocalEvictor* getEvictHandle(); - arrow::Status setLocalDirs(); std::string nextSpilledFileDir(); @@ -101,6 +99,7 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { arrow::Status populateMetrics(ShuffleWriterMetrics* metrics); + bool stopped_{false}; std::shared_ptr fs_{nullptr}; std::shared_ptr evictor_{nullptr}; std::vector> spills_{}; @@ -115,6 +114,7 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { int64_t totalBytesWritten_{0}; std::vector partitionLengths_; std::vector rawPartitionLengths_; + // Partition id, num rows, partition buffers. std::vector>>> cachedPartitionBuffers_; }; diff --git a/cpp/core/shuffle/PartitionWriter.h b/cpp/core/shuffle/PartitionWriter.h index 9090df3d73fa..bc655d03ff84 100644 --- a/cpp/core/shuffle/PartitionWriter.h +++ b/cpp/core/shuffle/PartitionWriter.h @@ -72,8 +72,8 @@ class ShuffleWriter::PartitionWriter : public Evictable { std::vector> buffers); uint32_t numPartitions_; - ShuffleWriterOptions* options_; + int64_t compressTime_{0}; int64_t evictTime_{0}; int64_t writeTime_{0}; diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc index d6d7cb28c644..8444c2e0f817 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc @@ -55,17 +55,17 @@ class CelebornEvictHandle final : public Evictor { arrow::Status CelebornPartitionWriter::init() { bytesEvicted_.resize(numPartitions_, 0); rawPartitionLengths_.resize(numPartitions_, 0); - evictHandle_ = std::make_shared(options_, celebornClient_.get(), bytesEvicted_); + evictor_ = std::make_shared(options_, celebornClient_.get(), bytesEvicted_); return arrow::Status::OK(); } arrow::Status CelebornPartitionWriter::stop(ShuffleWriterMetrics* metrics) { // Push data and collect metrics. - auto totalBytesEvicted = std::accumulate(bytesEvicted_.begin(), bytesEvicted_.end(), 0); + auto totalBytesEvicted = std::accumulate(bytesEvicted_.begin(), bytesEvicted_.end(), 0LL); celebornClient_->stop(); // Populate metrics. metrics->totalCompressTime += compressTime_; - metrics->totalEvictTime += evictHandle_->getEvictTime(); + metrics->totalEvictTime += evictor_->getEvictTime(); metrics->totalWriteTime += writeTime_; metrics->totalBytesEvicted += totalBytesEvicted; metrics->totalBytesWritten += totalBytesEvicted; @@ -75,7 +75,7 @@ arrow::Status CelebornPartitionWriter::stop(ShuffleWriterMetrics* metrics) { } arrow::Status CelebornPartitionWriter::finishEvict() { - return evictHandle_->finish(); + return evictor_->finish(); } arrow::Status CelebornPartitionWriter::evict( @@ -86,7 +86,7 @@ arrow::Status CelebornPartitionWriter::evict( rawPartitionLengths_[partitionId] += getBufferSize(buffers); ScopedTimer timer(evictTime_); ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffers(numRows, std::move(buffers))); - RETURN_NOT_OK(evictHandle_->evict(partitionId, std::move(payload))); + RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); return arrow::Status::OK(); } diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.h b/cpp/core/shuffle/rss/CelebornPartitionWriter.h index a3be4f46a17e..10bfc21c19ee 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.h +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.h @@ -54,7 +54,7 @@ class CelebornPartitionWriter final : public RemotePartitionWriter { private: std::shared_ptr celebornClient_; - std::shared_ptr evictHandle_; + std::shared_ptr evictor_; std::vector bytesEvicted_; std::vector rawPartitionLengths_; diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.cc b/cpp/velox/shuffle/VeloxShuffleWriter.cc index dd99ec8eeac9..8e4acc8b8c58 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxShuffleWriter.cc @@ -385,8 +385,10 @@ arrow::Status VeloxShuffleWriter::stop() { if (options_.partitioning != Partitioning::kSingle) { for (auto pid = 0; pid < numPartitions_; ++pid) { auto numRows = partitionBufferIdxBase_[pid]; - ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(pid, false)); - RETURN_NOT_OK(partitionWriter_->evict(pid, numRows, std::move(buffers), Evictor::Type::kStop)); + if (numRows > 0) { + ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(pid, false)); + RETURN_NOT_OK(partitionWriter_->evict(pid, numRows, std::move(buffers), Evictor::Type::kStop)); + } } } { From e19f44b16836fb5af8b5fb5c82fc121b7a6735a9 Mon Sep 17 00:00:00 2001 From: "Ma, Rong" Date: Wed, 6 Dec 2023 20:44:07 +0800 Subject: [PATCH 3/7] codestyle --- cpp/core/CMakeLists.txt | 3 ++- cpp/velox/CMakeLists.txt | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 24b5b88a0eab..60e63dd4846d 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -205,6 +205,7 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS shuffle/Partitioning.cc shuffle/PartitionWriterCreator.cc shuffle/LocalPartitionWriter.cc + shuffle/PartitionWriter.cc shuffle/rss/RemotePartitionWriter.cc shuffle/rss/CelebornPartitionWriter.cc shuffle/Utils.cc @@ -213,7 +214,7 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS utils/StringUtil.cc utils/ObjectStore.cc jni/JniError.cc - jni/JniCommon.cc shuffle/PartitionWriter.cc) + jni/JniCommon.cc) file(MAKE_DIRECTORY ${root_directory}/releases) add_library(gluten SHARED ${SPARK_COLUMNAR_PLUGIN_SRCS}) diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index d776cc36c3e8..ff00c1090975 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -287,7 +287,7 @@ set(VELOX_SRCS operators/serializer/VeloxRowToColumnarConverter.cc operators/writer/VeloxParquetDatasource.cc shuffle/VeloxShuffleReader.cc - shuffle/VeloxShuffleWriter.cc + shuffle/VeloxShuffleWriter.cc substrait/SubstraitParser.cc substrait/SubstraitToVeloxExpr.cc substrait/SubstraitToVeloxPlan.cc @@ -302,7 +302,7 @@ set(VELOX_SRCS utils/VeloxArrowUtils.cc utils/ConfigExtractor.cc utils/Common.cc - ) + ) if(BUILD_TESTS OR BUILD_BENCHMARKS) list(APPEND VELOX_SRCS utils/tests/MemoryPoolUtils.cc) From a7c9e01fede3edc15eec8a856564326b6dd955fd Mon Sep 17 00:00:00 2001 From: "Ma, Rong" Date: Fri, 8 Dec 2023 22:28:45 +0800 Subject: [PATCH 4/7] fix metrics --- cpp/core/shuffle/LocalPartitionWriter.cc | 3 +-- cpp/core/shuffle/Utils.cc | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index 8fc24bf8a021..e92ab7719f16 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -213,7 +213,7 @@ arrow::Status LocalPartitionWriter::init() { } arrow::Status LocalPartitionWriter::mergeSpills(uint32_t partitionId) { - for (auto spill : spills_) { + for (const auto& spill : spills_) { // Read if partition exists in the spilled file and write to the final file. if (spill->mergePos < spill->partitionSpillInfos.size() && spill->partitionSpillInfos[spill->mergePos].partitionId == partitionId) { // A hit. @@ -285,7 +285,6 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { } partitionLengths_[pid] = endInFinalFile - startInFinalFile; } - RETURN_NOT_OK(finishEvict()); for (auto spill : spills_) { // Check if all spilled data are merged. diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index f959e2a5cdfe..9bef574b27f1 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -162,7 +162,7 @@ arrow::Result> makeCompressedRecordBatch( int32_t bufferCompressThreshold, CompressionMode compressionMode, int64_t& compressionTime) { - ScopedTimer{compressionTime}; + ScopedTimer timer{compressionTime}; std::vector> arrays; // header col, numRows, compressionType { From f5e3c80aad8dadce59529828998879d8eeacfa9c Mon Sep 17 00:00:00 2001 From: "Ma, Rong" Date: Fri, 8 Dec 2023 10:01:38 +0800 Subject: [PATCH 5/7] remove arrow ipc payload and partition writer creator --- cpp/core/CMakeLists.txt | 5 +- cpp/core/compute/Runtime.h | 4 +- cpp/core/jni/JniCommon.h | 5 +- cpp/core/jni/JniWrapper.cc | 61 +++--- cpp/core/shuffle/BlockPayload.cc | 196 ++++++++++++++++++ cpp/core/shuffle/BlockPayload.h | 154 ++++++++++++++ cpp/core/shuffle/LocalPartitionWriter.cc | 87 ++++---- cpp/core/shuffle/LocalPartitionWriter.h | 35 ++-- cpp/core/shuffle/Options.cc | 8 - cpp/core/shuffle/Options.h | 22 +- cpp/core/shuffle/PartitionWriter.cc | 51 ----- cpp/core/shuffle/PartitionWriter.h | 37 +++- cpp/core/shuffle/PartitionWriterCreator.cc | 20 -- cpp/core/shuffle/ShuffleMemoryPool.cc | 66 ++++++ ...ionWriterCreator.h => ShuffleMemoryPool.h} | 36 ++-- cpp/core/shuffle/ShuffleReader.cc | 71 ++----- cpp/core/shuffle/ShuffleReader.h | 28 ++- cpp/core/shuffle/ShuffleWriter.cc | 33 --- cpp/core/shuffle/ShuffleWriter.h | 97 ++------- cpp/core/shuffle/Utils.cc | 2 - cpp/core/shuffle/Utils.h | 3 + .../shuffle/rss/CelebornPartitionWriter.cc | 28 +-- .../shuffle/rss/CelebornPartitionWriter.h | 27 +-- cpp/core/shuffle/rss/RemotePartitionWriter.cc | 2 +- cpp/core/shuffle/rss/RemotePartitionWriter.h | 2 +- cpp/core/utils/Timer.h | 26 ++- cpp/velox/benchmarks/GenericBenchmark.cc | 52 +++-- cpp/velox/benchmarks/ShuffleSplitBenchmark.cc | 51 +++-- cpp/velox/benchmarks/common/BenchmarkUtils.cc | 23 +- cpp/velox/benchmarks/common/BenchmarkUtils.h | 5 +- cpp/velox/compute/VeloxBackend.cc | 3 - cpp/velox/compute/VeloxRuntime.cc | 13 +- cpp/velox/compute/VeloxRuntime.h | 4 +- cpp/velox/shuffle/VeloxShuffleReader.cc | 135 ++++++------ cpp/velox/shuffle/VeloxShuffleReader.h | 58 +++++- cpp/velox/shuffle/VeloxShuffleWriter.cc | 109 ++++------ cpp/velox/shuffle/VeloxShuffleWriter.h | 37 ++-- cpp/velox/tests/RuntimeTest.cc | 4 +- cpp/velox/tests/VeloxShuffleWriterTest.cc | 96 +++++---- .../utils/tests/VeloxShuffleWriterTestBase.h | 158 +++++++++----- .../vectorized/ColumnarBatchSerializer.scala | 11 +- 41 files changed, 1088 insertions(+), 777 deletions(-) create mode 100644 cpp/core/shuffle/BlockPayload.cc create mode 100644 cpp/core/shuffle/BlockPayload.h delete mode 100644 cpp/core/shuffle/PartitionWriter.cc delete mode 100644 cpp/core/shuffle/PartitionWriterCreator.cc create mode 100644 cpp/core/shuffle/ShuffleMemoryPool.cc rename cpp/core/shuffle/{PartitionWriterCreator.h => ShuffleMemoryPool.h} (54%) delete mode 100644 cpp/core/shuffle/ShuffleWriter.cc diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 60e63dd4846d..f19714a5006b 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -194,18 +194,17 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS memory/ArrowMemoryPool.cc memory/ColumnarBatch.cc operators/writer/ArrowWriter.cc + shuffle/BlockPayload.cc shuffle/Options.cc shuffle/ShuffleReader.cc - shuffle/ShuffleWriter.cc shuffle/Partitioner.cc shuffle/FallbackRangePartitioner.cc shuffle/HashPartitioner.cc shuffle/RoundRobinPartitioner.cc shuffle/SinglePartitioner.cc shuffle/Partitioning.cc - shuffle/PartitionWriterCreator.cc shuffle/LocalPartitionWriter.cc - shuffle/PartitionWriter.cc + shuffle/ShuffleMemoryPool.cc shuffle/rss/RemotePartitionWriter.cc shuffle/rss/CelebornPartitionWriter.cc shuffle/Utils.cc diff --git a/cpp/core/compute/Runtime.h b/cpp/core/compute/Runtime.h index f188e24203d1..5627091312d8 100644 --- a/cpp/core/compute/Runtime.h +++ b/cpp/core/compute/Runtime.h @@ -100,8 +100,8 @@ class Runtime : public std::enable_shared_from_this { virtual std::shared_ptr createShuffleWriter( int numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, + std::unique_ptr partitionWriter, + std::unique_ptr options, MemoryManager* memoryManager) = 0; virtual Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) = 0; diff --git a/cpp/core/jni/JniCommon.h b/cpp/core/jni/JniCommon.h index 265382aa2d20..662f4df2af1e 100644 --- a/cpp/core/jni/JniCommon.h +++ b/cpp/core/jni/JniCommon.h @@ -339,7 +339,7 @@ class CelebornClient : public RssClient { env->DeleteGlobalRef(array_); } - int32_t pushPartitionData(int32_t partitionId, char* bytes, int64_t size) { + int32_t pushPartitionData(int32_t partitionId, char* bytes, int64_t size) override { JNIEnv* env; if (vm_->GetEnv(reinterpret_cast(&env), jniVersion) != JNI_OK) { throw gluten::GlutenException("JNIEnv was not attached to current thread"); @@ -359,8 +359,9 @@ class CelebornClient : public RssClient { return static_cast(celebornBytesSize); } - void stop() {} + void stop() override {} + private: JavaVM* vm_; jobject javaCelebornShuffleWriter_; jmethodID javaCelebornPushPartitionData_; diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index e19d4a683ee4..b59c8e88f32f 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -31,13 +31,13 @@ #include "memory/AllocationListener.h" #include "operators/serializer/ColumnarBatchSerializer.h" #include "shuffle/LocalPartitionWriter.h" -#include "shuffle/PartitionWriterCreator.h" #include "shuffle/Partitioning.h" #include "shuffle/ShuffleReader.h" #include "shuffle/ShuffleWriter.h" #include "shuffle/Utils.h" #include "shuffle/rss/CelebornPartitionWriter.h" #include "utils/ArrowStatus.h" +#include "utils/StringUtil.h" using namespace gluten; @@ -793,25 +793,24 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper auto memoryManager = jniCastOrThrow(memoryManagerHandle); if (partitioningNameJstr == nullptr) { throw gluten::GlutenException(std::string("Short partitioning name can't be null")); - return kInvalidResourceHandle; } - auto shuffleWriterOptions = ShuffleWriterOptions::defaults(); + auto shuffleWriterOptions = std::make_unique(); auto partitioningName = jStringToCString(env, partitioningNameJstr); - shuffleWriterOptions.partitioning = gluten::toPartitioning(partitioningName); + shuffleWriterOptions->partitioning = gluten::toPartitioning(partitioningName); if (bufferSize > 0) { - shuffleWriterOptions.buffer_size = bufferSize; + shuffleWriterOptions->buffer_size = bufferSize; } - shuffleWriterOptions.compression_type = getCompressionType(env, codecJstr); + shuffleWriterOptions->compression_type = getCompressionType(env, codecJstr); if (codecJstr != NULL) { - shuffleWriterOptions.codec_backend = getCodecBackend(env, codecBackendJstr); - shuffleWriterOptions.compression_mode = getCompressionMode(env, compressionModeJstr); + shuffleWriterOptions->codec_backend = getCodecBackend(env, codecBackendJstr); + shuffleWriterOptions->compression_mode = getCompressionMode(env, compressionModeJstr); } - shuffleWriterOptions.memory_pool = memoryManager->getArrowMemoryPool(); + shuffleWriterOptions->memory_pool = memoryManager->getArrowMemoryPool(); jclass cls = env->FindClass("java/lang/Thread"); jmethodID mid = env->GetStaticMethodID(cls, "currentThread", "()Ljava/lang/Thread;"); @@ -823,21 +822,21 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper jmethodID midGetid = getMethodIdOrError(env, cls, "getId", "()J"); jlong sid = env->CallLongMethod(thread, midGetid); checkException(env); - shuffleWriterOptions.thread_id = (int64_t)sid; + shuffleWriterOptions->thread_id = (int64_t)sid; } - shuffleWriterOptions.task_attempt_id = (int64_t)taskAttemptId; - shuffleWriterOptions.start_partition_id = startPartitionId; - shuffleWriterOptions.compression_threshold = bufferCompressThreshold; + shuffleWriterOptions->task_attempt_id = (int64_t)taskAttemptId; + shuffleWriterOptions->start_partition_id = startPartitionId; + shuffleWriterOptions->compression_threshold = bufferCompressThreshold; auto partitionWriterTypeC = env->GetStringUTFChars(partitionWriterTypeJstr, JNI_FALSE); auto partitionWriterType = std::string(partitionWriterTypeC); env->ReleaseStringUTFChars(partitionWriterTypeJstr, partitionWriterTypeC); - std::shared_ptr partitionWriterCreator; + std::unique_ptr partitionWriter; if (partitionWriterType == "local") { - shuffleWriterOptions.partition_writer_type = kLocal; + shuffleWriterOptions->partition_writer_type = kLocal; if (dataFileJstr == NULL) { throw gluten::GlutenException(std::string("Shuffle DataFile can't be null")); } @@ -845,30 +844,31 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper throw gluten::GlutenException(std::string("Shuffle DataFile can't be null")); } - shuffleWriterOptions.write_eos = writeEOS; - shuffleWriterOptions.buffer_realloc_threshold = reallocThreshold; + shuffleWriterOptions->write_eos = writeEOS; + shuffleWriterOptions->buffer_realloc_threshold = reallocThreshold; if (numSubDirs > 0) { - shuffleWriterOptions.num_sub_dirs = numSubDirs; + shuffleWriterOptions->num_sub_dirs = numSubDirs; } auto dataFileC = env->GetStringUTFChars(dataFileJstr, JNI_FALSE); - shuffleWriterOptions.data_file = std::string(dataFileC); + auto dataFile = std::string(dataFileC); env->ReleaseStringUTFChars(dataFileJstr, dataFileC); - auto localDirs = env->GetStringUTFChars(localDirsJstr, JNI_FALSE); - shuffleWriterOptions.local_dirs = std::string(localDirs); - env->ReleaseStringUTFChars(localDirsJstr, localDirs); + auto localDirsC = env->GetStringUTFChars(localDirsJstr, JNI_FALSE); + auto configuredDirs = gluten::splitPaths(std::string(localDirsC)); + env->ReleaseStringUTFChars(localDirsJstr, localDirsC); - partitionWriterCreator = std::make_shared(); + partitionWriter = + std::make_unique(numPartitions, dataFile, configuredDirs, shuffleWriterOptions.get()); } else if (partitionWriterType == "celeborn") { - shuffleWriterOptions.partition_writer_type = PartitionWriterType::kCeleborn; + shuffleWriterOptions->partition_writer_type = PartitionWriterType::kCeleborn; jclass celebornPartitionPusherClass = createGlobalClassReferenceOrError(env, "Lorg/apache/spark/shuffle/CelebornPartitionPusher;"); jmethodID celebornPushPartitionDataMethod = getMethodIdOrError(env, celebornPartitionPusherClass, "pushPartitionData", "(I[BI)I"); if (pushBufferMaxSize > 0) { - shuffleWriterOptions.push_buffer_max_size = pushBufferMaxSize; + shuffleWriterOptions->push_buffer_max_size = pushBufferMaxSize; } JavaVM* vm; if (env->GetJavaVM(&vm) != JNI_OK) { @@ -876,13 +876,14 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper } std::shared_ptr celebornClient = std::make_shared(vm, partitionPusher, celebornPushPartitionDataMethod); - partitionWriterCreator = std::make_shared(std::move(celebornClient)); + partitionWriter = + std::make_unique(numPartitions, shuffleWriterOptions.get(), std::move(celebornClient)); } else { throw gluten::GlutenException("Unrecognizable partition writer type: " + partitionWriterType); } return ctx->objectStore()->save(ctx->createShuffleWriter( - numPartitions, std::move(partitionWriterCreator), std::move(shuffleWriterOptions), memoryManager)); + numPartitions, std::move(partitionWriter), std::move(shuffleWriterOptions), memoryManager)); JNI_METHOD_END(kInvalidResourceHandle) } @@ -1009,9 +1010,7 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleReaderJniWrapper auto memoryManager = jniCastOrThrow(memoryManagerHandle); auto pool = memoryManager->getArrowMemoryPool(); - ShuffleReaderOptions options = ShuffleReaderOptions::defaults(); - options.ipc_read_options.memory_pool = pool; - options.ipc_read_options.use_threads = false; + ShuffleReaderOptions options = ShuffleReaderOptions{}; options.compression_type = getCompressionType(env, compressionType); if (compressionType != nullptr) { options.codec_backend = getCodecBackend(env, compressionBackend); @@ -1049,7 +1048,7 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_ShuffleReaderJniWrapper_ auto reader = ctx->objectStore()->retrieve(shuffleReaderHandle); env->CallVoidMethod(metrics, shuffleReaderMetricsSetDecompressTime, reader->getDecompressTime()); env->CallVoidMethod(metrics, shuffleReaderMetricsSetIpcTime, reader->getIpcTime()); - env->CallVoidMethod(metrics, shuffleReaderMetricsSetDeserializeTime, reader->getDeserializeTime()); + env->CallVoidMethod(metrics, shuffleReaderMetricsSetDeserializeTime, reader->getArrowToVeloxTime()); checkException(env); JNI_METHOD_END() diff --git a/cpp/core/shuffle/BlockPayload.cc b/cpp/core/shuffle/BlockPayload.cc new file mode 100644 index 000000000000..92813b129dce --- /dev/null +++ b/cpp/core/shuffle/BlockPayload.cc @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 "shuffle/BlockPayload.h" + +namespace gluten { + +arrow::Result> BlockPayload::fromBuffers( + uint32_t numRows, + std::vector> buffers, + ShuffleWriterOptions* options, + arrow::MemoryPool* pool, + arrow::util::Codec* codec, + bool reuseBuffers) { + if (codec && numRows >= options->compression_threshold) { + // Compress. + // Compressed buffer layout: | buffer1 compressedLength | buffer1 uncompressedLength | buffer1 | ... + auto metadataLength = sizeof(int64_t) * 2 * buffers.size(); + int64_t totalCompressedLength = + std::accumulate(buffers.begin(), buffers.end(), 0LL, [&](auto sum, const auto& buffer) { + if (!buffer) { + return sum; + } + return sum + codec->MaxCompressedLen(buffer->size(), buffer->data()); + }); + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr compressed, + arrow::AllocateResizableBuffer(metadataLength + totalCompressedLength, pool)); + auto output = compressed->mutable_data(); + + // Compress buffers one by one. + for (auto& buffer : buffers) { + auto availableLength = compressed->size() - (output - compressed->data()); + RETURN_NOT_OK(compressBuffer(buffer, output, availableLength, codec)); + } + + int64_t actualLength = output - compressed->data(); + ARROW_RETURN_IF(actualLength < 0, arrow::Status::Invalid("Writing compressed buffer out of bound.")); + RETURN_NOT_OK(compressed->Resize(actualLength)); + return std::make_unique( + Type::kCompressed, numRows, std::vector>{compressed}); + } + if (reuseBuffers) { + // Copy. + std::vector> copies; + for (auto& buffer : buffers) { + if (!buffer) { + copies.push_back(nullptr); + continue; + } + ARROW_ASSIGN_OR_RAISE(auto copy, arrow::AllocateResizableBuffer(buffer->size(), pool)); + memcpy(copy->mutable_data(), buffer->data(), buffer->size()); + copies.push_back(std::move(copy)); + } + return std::make_unique(Type::kUncompressed, numRows, std::move(copies)); + } + return std::make_unique(Type::kUncompressed, numRows, std::move(buffers)); +} + +arrow::Status BlockPayload::serialize(arrow::io::OutputStream* outputStream) { + RETURN_NOT_OK(outputStream->Write(&type_, sizeof(Type))); + RETURN_NOT_OK(outputStream->Write(&numRows_, sizeof(uint32_t))); + if (type_ == Type::kUncompressed) { + for (auto& buffer : buffers_) { + if (!buffer) { + RETURN_NOT_OK(outputStream->Write(&kNullBuffer, sizeof(int64_t))); + continue; + } + int64_t bufferSize = buffer->size(); + RETURN_NOT_OK(outputStream->Write(&bufferSize, sizeof(int64_t))); + RETURN_NOT_OK(outputStream->Write(std::move(buffer))); + } + } else { + RETURN_NOT_OK(outputStream->Write(std::move(buffers_[0]))); + } + buffers_.clear(); + return arrow::Status::OK(); +} + +arrow::Result>> BlockPayload::deserialize( + arrow::io::InputStream* inputStream, + const std::shared_ptr& schema, + const std::shared_ptr& codec, + arrow::MemoryPool* pool, + uint32_t& numRows) { + static const std::vector> kEmptyBuffers{}; + ARROW_ASSIGN_OR_RAISE(auto typeAndRows, readTypeAndRows(inputStream)); + if (typeAndRows.first == kIpcContinuationToken && typeAndRows.second == kZeroLength) { + numRows = 0; + return kEmptyBuffers; + } + numRows = typeAndRows.second; + auto fields = schema->fields(); + + auto isCompressionEnabled = typeAndRows.first == Type::kUncompressed || codec == nullptr; + auto readBuffer = [&]() { + if (isCompressionEnabled) { + return readUncompressedBuffer(inputStream); + } else { + return readCompressedBuffer(inputStream, codec, pool); + } + }; + + bool hasComplexDataType = false; + std::vector> buffers; + for (const auto& field : fields) { + auto fieldType = field->type()->id(); + switch (fieldType) { + case arrow::BinaryType::type_id: + case arrow::StringType::type_id: { + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readBuffer()); + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readBuffer()); + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readBuffer()); + break; + } + case arrow::StructType::type_id: + case arrow::MapType::type_id: + case arrow::ListType::type_id: { + hasComplexDataType = true; + } break; + default: { + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readBuffer()); + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readBuffer()); + break; + } + } + } + if (hasComplexDataType) { + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readBuffer()); + } + return buffers; +} + +arrow::Result> BlockPayload::readTypeAndRows(arrow::io::InputStream* inputStream) { + int32_t type; + uint32_t numRows; + RETURN_NOT_OK(inputStream->Read(sizeof(Type), &type)); + RETURN_NOT_OK(inputStream->Read(sizeof(uint32_t), &numRows)); + return std::make_pair(type, numRows); +} + +arrow::Result> BlockPayload::readUncompressedBuffer( + arrow::io::InputStream* inputStream) { + int64_t bufferLength; + RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &bufferLength)); + if (bufferLength == kNullBuffer) { + return nullptr; + } + ARROW_ASSIGN_OR_RAISE(auto buffer, inputStream->Read(bufferLength)); + return buffer; +} + +arrow::Result> BlockPayload::readCompressedBuffer( + arrow::io::InputStream* inputStream, + const std::shared_ptr& codec, + arrow::MemoryPool* pool) { + int64_t compressedLength; + int64_t uncompressedLength; + RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &compressedLength)); + RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &uncompressedLength)); + if (compressedLength == kNullBuffer) { + return nullptr; + } + if (compressedLength == kUncompressedBuffer) { + ARROW_ASSIGN_OR_RAISE(auto uncompressed, arrow::AllocateBuffer(uncompressedLength, pool)); + RETURN_NOT_OK(inputStream->Read(uncompressedLength, const_cast(uncompressed->data()))); + return uncompressed; + } + ARROW_ASSIGN_OR_RAISE(auto compressed, arrow::AllocateBuffer(compressedLength, pool)); + RETURN_NOT_OK(inputStream->Read(compressedLength, const_cast(compressed->data()))); + ARROW_ASSIGN_OR_RAISE(auto output, arrow::AllocateBuffer(uncompressedLength, pool)); + RETURN_NOT_OK(codec->Decompress( + compressedLength, compressed->data(), uncompressedLength, const_cast(output->data()))); + return output; +} +} // namespace gluten diff --git a/cpp/core/shuffle/BlockPayload.h b/cpp/core/shuffle/BlockPayload.h new file mode 100644 index 000000000000..572b7a3b0d1e --- /dev/null +++ b/cpp/core/shuffle/BlockPayload.h @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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. + */ +#pragma once + +#include + +#include +#include "shuffle/Options.h" +#include "shuffle/PartitionWriter.h" +#include "shuffle/Utils.h" + +namespace gluten { +// A block represents data to be cached in-memory or spilled. +// Can be compressed or uncompressed. + +namespace { + +static constexpr int64_t kNullBuffer = -1; +static constexpr int64_t kUncompressedBuffer = -2; + +template +void write(uint8_t** dst, T data) { + auto ptr = reinterpret_cast(*dst); + *ptr = data; + *dst += sizeof(T); +} + +template +T* advance(uint8_t** dst) { + auto ptr = reinterpret_cast(*dst); + *dst += sizeof(T); + return ptr; +} + +arrow::Status compressBuffer( + std::shared_ptr& buffer, + uint8_t*& output, + int64_t outputLength, + arrow::util::Codec* codec) { + if (!buffer) { + write(&output, kNullBuffer); + write(&output, kNullBuffer); + return arrow::Status::OK(); + } + auto* compressedLengthPtr = advance(&output); + write(&output, static_cast(buffer->size())); + ARROW_ASSIGN_OR_RAISE(auto compressedLength, codec->Compress(buffer->size(), buffer->data(), outputLength, output)); + if (compressedLength > buffer->size()) { + // Write uncompressed buffer. + memcpy(output, buffer->data(), buffer->size()); + output += buffer->size(); + *compressedLengthPtr = kUncompressedBuffer; + } else { + output += compressedLength; + *compressedLengthPtr = static_cast(compressedLength); + } + // Release buffer after compression. + buffer = nullptr; + return arrow::Status::OK(); +} + +} // namespace + +class BlockPayload : public Payload { + public: + enum Type : int32_t { kCompressed, kUncompressed }; + + BlockPayload(BlockPayload::Type type, uint32_t numRows, std::vector> buffers) + : type_(type), numRows_(numRows), buffers_(std::move(buffers)) {} + + static arrow::Result> fromBuffers( + uint32_t numRows, + std::vector> buffers, + ShuffleWriterOptions* options, + arrow::MemoryPool* pool, + arrow::util::Codec* codec, + bool reuseBuffers); + + arrow::Status serialize(arrow::io::OutputStream* outputStream) override; + + static arrow::Result>> deserialize( + arrow::io::InputStream* inputStream, + const std::shared_ptr& schema, + const std::shared_ptr& codec, + arrow::MemoryPool* pool, + uint32_t& numRows); + + static arrow::Result> readTypeAndRows(arrow::io::InputStream* inputStream); + + static arrow::Result> readUncompressedBuffer(arrow::io::InputStream* inputStream); + + static arrow::Result> readCompressedBuffer( + arrow::io::InputStream* inputStream, + const std::shared_ptr& codec, + arrow::MemoryPool* pool); + + static arrow::Status mergeCompressed( + arrow::io::InputStream* inputStream, + arrow::io::OutputStream* outputStream, + uint32_t numRows, + int64_t totalLength) { + static const Type kType = Type::kUncompressed; + RETURN_NOT_OK(outputStream->Write(&kType, sizeof(Type))); + RETURN_NOT_OK(outputStream->Write(&numRows, sizeof(uint32_t))); + ARROW_ASSIGN_OR_RAISE(auto buffer, inputStream->Read(totalLength)); + RETURN_NOT_OK(outputStream->Write(buffer)); + return arrow::Status::OK(); + } + + static arrow::Status mergeUncompressed(arrow::io::InputStream* inputStream, arrow::ResizableBuffer* output) { + ARROW_ASSIGN_OR_RAISE(auto input, readUncompressedBuffer(inputStream)); + auto data = output->mutable_data() + output->size(); + auto newSize = output->size() + input->size(); + RETURN_NOT_OK(output->Resize(newSize)); + memcpy(data, input->data(), input->size()); + return arrow::Status::OK(); + } + + static arrow::Status compressAndWrite( + std::shared_ptr buffer, + arrow::io::OutputStream* outputStream, + arrow::util::Codec* codec, + ShuffleMemoryPool* pool) { + auto maxCompressedLength = codec->MaxCompressedLen(buffer->size(), buffer->data()); + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr compressed, + arrow::AllocateResizableBuffer(sizeof(int64_t) * 2 + maxCompressedLength, pool)); + auto output = compressed->mutable_data(); + RETURN_NOT_OK(compressBuffer(buffer, output, maxCompressedLength, codec)); + RETURN_NOT_OK(outputStream->Write(compressed->data(), output - compressed->data())); + return arrow::Status::OK(); + } + + private: + Type type_; + uint32_t numRows_; + std::vector> buffers_; +}; + +} // namespace gluten \ No newline at end of file diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index e92ab7719f16..bdfa00e9bac2 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -15,12 +15,13 @@ * limitations under the License. */ -#include "shuffle/LocalPartitionWriter.h" #include #include + +#include "shuffle/BlockPayload.h" +#include "shuffle/LocalPartitionWriter.h" #include "shuffle/Utils.h" #include "utils/DebugOut.h" -#include "utils/StringUtil.h" #include "utils/Timer.h" namespace gluten { @@ -51,9 +52,9 @@ class CacheEvictor final : public LocalPartitionWriter::LocalEvictor { CacheEvictor(uint32_t numPartitions, ShuffleWriterOptions* options, const std::shared_ptr& spillInfo) : LocalPartitionWriter::LocalEvictor(numPartitions, options, spillInfo) {} - arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { if (partitionCachedPayload_.find(partitionId) == partitionCachedPayload_.end()) { - partitionCachedPayload_.emplace(partitionId, std::vector>{}); + partitionCachedPayload_.emplace(partitionId, std::vector>{}); } partitionCachedPayload_[partitionId].push_back(std::move(payload)); return arrow::Status::OK(); @@ -96,15 +97,14 @@ class CacheEvictor final : public LocalPartitionWriter::LocalEvictor { } private: - std::unordered_map>> partitionCachedPayload_; + std::unordered_map>> partitionCachedPayload_; arrow::Status flushInternal(uint32_t partitionId, arrow::io::OutputStream* os) { ScopedTimer timer(evictTime_); - int32_t metadataLength = 0; // unused auto payloads = std::move(partitionCachedPayload_[partitionId]); partitionCachedPayload_.erase(partitionId); for (auto& payload : payloads) { - RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, os, &metadataLength)); + RETURN_NOT_OK(payload->serialize(os)); } return arrow::Status::OK(); } @@ -118,15 +118,14 @@ class FlushOnSpillEvictor final : public LocalPartitionWriter::LocalEvictor { const std::shared_ptr& spillInfo) : LocalPartitionWriter::LocalEvictor(numPartitions, options, spillInfo) {} - arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { ScopedTimer timer(evictTime_); if (!os_) { ARROW_ASSIGN_OR_RAISE(os_, arrow::io::FileOutputStream::Open(spillInfo_->spilledFile, true)); } - int32_t metadataLength = 0; // unused. ARROW_ASSIGN_OR_RAISE(auto start, os_->Tell()); - RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, os_.get(), &metadataLength)); + RETURN_NOT_OK(payload->serialize(os_.get())); ARROW_ASSIGN_OR_RAISE(auto end, os_->Tell()); DEBUG_OUT << "Spilled partition " << partitionId << " file start: " << start << ", file end: " << end << std::endl; spillInfo_->partitionSpillInfos.push_back({partitionId, end - start}); @@ -165,27 +164,26 @@ arrow::Result> LocalPartitio } } +LocalPartitionWriter::LocalPartitionWriter( + uint32_t numPartitions, + const std::string& dataFile, + const std::vector& localDirs, + ShuffleWriterOptions* options) + : PartitionWriter(numPartitions, options), dataFile_(dataFile), localDirs_(localDirs) { + init(); +} + std::string LocalPartitionWriter::nextSpilledFileDir() { - auto spilledFileDir = getSpilledShuffleFileDir(configuredDirs_[dirSelection_], subDirSelection_[dirSelection_]); + auto spilledFileDir = getSpilledShuffleFileDir(localDirs_[dirSelection_], subDirSelection_[dirSelection_]); subDirSelection_[dirSelection_] = (subDirSelection_[dirSelection_] + 1) % options_->num_sub_dirs; - dirSelection_ = (dirSelection_ + 1) % configuredDirs_.size(); + dirSelection_ = (dirSelection_ + 1) % localDirs_.size(); return spilledFileDir; } -arrow::Status LocalPartitionWriter::setLocalDirs() { - configuredDirs_ = splitPaths(options_->local_dirs); - // Shuffle the configured local directories. This prevents each task from using the same directory for spilled files. - std::random_device rd; - std::default_random_engine engine(rd()); - std::shuffle(configuredDirs_.begin(), configuredDirs_.end(), engine); - subDirSelection_.assign(configuredDirs_.size(), 0); - return arrow::Status::OK(); -} - arrow::Status LocalPartitionWriter::openDataFile() { // open data file output stream std::shared_ptr fout; - ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(options_->data_file)); + ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(dataFile_)); if (options_->buffered_write) { // Output stream buffer is neither partition buffer memory nor ipc memory. ARROW_ASSIGN_OR_RAISE(dataFileOs_, arrow::io::BufferedOutputStream::Create(16384, options_->memory_pool, fout)); @@ -204,12 +202,16 @@ arrow::Status LocalPartitionWriter::clearResource() { return arrow::Status::OK(); } -arrow::Status LocalPartitionWriter::init() { +void LocalPartitionWriter::init() { partitionLengths_.resize(numPartitions_, 0); rawPartitionLengths_.resize(numPartitions_, 0); fs_ = std::make_shared(); - RETURN_NOT_OK(setLocalDirs()); - return arrow::Status::OK(); + + // Shuffle the configured local directories. This prevents each task from using the same directory for spilled files. + std::random_device rd; + std::default_random_engine engine(rd()); + std::shuffle(localDirs_.begin(), localDirs_.end(), engine); + subDirSelection_.assign(localDirs_.size(), 0); } arrow::Status LocalPartitionWriter::mergeSpills(uint32_t partitionId) { @@ -250,7 +252,6 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { writeTimer.start(); int64_t endInFinalFile = 0; - int32_t metadataLength = 0; // Unused. auto cachedPartitionBuffersIter = cachedPartitionBuffers_.begin(); // Iterator over pid. for (auto pid = 0; pid < numPartitions_; ++pid) { @@ -269,11 +270,15 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { writeTimer.stop(); ARROW_ASSIGN_OR_RAISE( auto payload, - createPayloadFromBuffers( - std::get<1>(*cachedPartitionBuffersIter), std::move(std::get<2>(*cachedPartitionBuffersIter)))); + BlockPayload::fromBuffers( + std::get<1>(*cachedPartitionBuffersIter), + std::move(std::get<2>(*cachedPartitionBuffersIter)), + options_, + payloadPool_.get(), + codec_ ? codec_.get() : nullptr, + false)); writeTimer.start(); - RETURN_NOT_OK( - arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, dataFileOs_.get(), &metadataLength)); + RETURN_NOT_OK(payload->serialize(dataFileOs_.get())); cachedPartitionBuffersIter++; } ARROW_ASSIGN_OR_RAISE(endInFinalFile, dataFileOs_->Tell()); @@ -344,12 +349,21 @@ arrow::Status LocalPartitionWriter::evict( uint32_t partitionId, uint32_t numRows, std::vector> buffers, + bool reuseBuffers, Evictor::Type evictType) { rawPartitionLengths_[partitionId] += getBufferSize(buffers); if (evictType == Evictor::Type::kStop) { cachedPartitionBuffers_.emplace_back(partitionId, numRows, std::move(buffers)); } else { - ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffers(numRows, std::move(buffers))); + ARROW_ASSIGN_OR_RAISE( + auto payload, + BlockPayload::fromBuffers( + numRows, + std::move(buffers), + options_, + payloadPool_.get(), + (codec_ && evictType == Evictor::kCache) ? codec_.get() : nullptr, + reuseBuffers)); RETURN_NOT_OK(requestEvict(evictType)); RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); } @@ -384,13 +398,4 @@ arrow::Status LocalPartitionWriter::evictFixedSize(int64_t size, int64_t* actual return arrow::Status::OK(); } -LocalPartitionWriterCreator::LocalPartitionWriterCreator() : PartitionWriterCreator() {} - -arrow::Result> LocalPartitionWriterCreator::make( - uint32_t numPartitions, - ShuffleWriterOptions* options) { - auto partitionWriter = std::make_shared(numPartitions, options); - RETURN_NOT_OK(partitionWriter->init()); - return partitionWriter; -} } // namespace gluten diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index d89f8d20cb7d..d30f2a21b479 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -22,8 +22,6 @@ #include "shuffle/PartitionWriter.h" #include "shuffle/ShuffleWriter.h" - -#include "PartitionWriterCreator.h" #include "utils/macros.h" namespace gluten { @@ -44,17 +42,19 @@ struct SpillInfo { SpillInfo(std::string spilledFile) : spilledFile(spilledFile) {} }; -class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { +class LocalPartitionWriter : public PartitionWriter { public: - explicit LocalPartitionWriter(uint32_t numPartitions, ShuffleWriterOptions* options) - : PartitionWriter(numPartitions, options) {} - - arrow::Status init() override; + explicit LocalPartitionWriter( + uint32_t numPartitions, + const std::string& dataFile, + const std::vector& localDirs, + ShuffleWriterOptions* options); arrow::Status evict( uint32_t partitionId, uint32_t numRows, std::vector> buffers, + bool reuseBuffers, Evictor::Type evictType) override; arrow::Status finishEvict() override; @@ -85,9 +85,9 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { class LocalEvictor; private: - arrow::Status requestEvict(Evictor::Type evictType); + void init(); - arrow::Status setLocalDirs(); + arrow::Status requestEvict(Evictor::Type evictType); std::string nextSpilledFileDir(); @@ -99,17 +99,19 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { arrow::Status populateMetrics(ShuffleWriterMetrics* metrics); + std::string dataFile_; + std::vector localDirs_; + bool stopped_{false}; std::shared_ptr fs_{nullptr}; std::shared_ptr evictor_{nullptr}; std::vector> spills_{}; // configured local dirs for spilled file - int32_t dirSelection_ = 0; + int32_t dirSelection_{0}; std::vector subDirSelection_; - std::vector configuredDirs_; - std::shared_ptr dataFileOs_; + int64_t totalBytesEvicted_{0}; int64_t totalBytesWritten_{0}; std::vector partitionLengths_; @@ -117,13 +119,4 @@ class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { // Partition id, num rows, partition buffers. std::vector>>> cachedPartitionBuffers_; }; - -class LocalPartitionWriterCreator : public ShuffleWriter::PartitionWriterCreator { - public: - LocalPartitionWriterCreator(); - - arrow::Result> make( - uint32_t numPartitions, - ShuffleWriterOptions* options) override; -}; } // namespace gluten diff --git a/cpp/core/shuffle/Options.cc b/cpp/core/shuffle/Options.cc index 25fc22ea0b14..8e05a10d6859 100644 --- a/cpp/core/shuffle/Options.cc +++ b/cpp/core/shuffle/Options.cc @@ -16,11 +16,3 @@ */ #include "shuffle/Options.h" - -gluten::ShuffleReaderOptions gluten::ShuffleReaderOptions::defaults() { - return {}; -} - -gluten::ShuffleWriterOptions gluten::ShuffleWriterOptions::defaults() { - return {}; -} diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h index 20f72f9f06ed..8b54f0415670 100644 --- a/cpp/core/shuffle/Options.h +++ b/cpp/core/shuffle/Options.h @@ -34,11 +34,8 @@ static constexpr bool kWriteEos = true; enum PartitionWriterType { kLocal, kCeleborn }; struct ShuffleReaderOptions { - arrow::ipc::IpcReadOptions ipc_read_options = arrow::ipc::IpcReadOptions::Defaults(); arrow::Compression::type compression_type = arrow::Compression::type::LZ4_FRAME; CodecBackend codec_backend = CodecBackend::NONE; - - static ShuffleReaderOptions defaults(); }; struct ShuffleWriterOptions { @@ -59,17 +56,16 @@ struct ShuffleWriterOptions { int64_t thread_id = -1; int64_t task_attempt_id = -1; int32_t start_partition_id = 0; - - arrow::ipc::IpcWriteOptions ipc_write_options = arrow::ipc::IpcWriteOptions::Defaults(); - - std::shared_ptr write_schema{nullptr}; - std::shared_ptr codec{nullptr}; - - std::string data_file{}; - std::string local_dirs{}; arrow::MemoryPool* memory_pool{}; - - static ShuffleWriterOptions defaults(); }; +struct ShuffleWriterMetrics { + int64_t totalBytesWritten{0}; + int64_t totalBytesEvicted{0}; + int64_t totalWriteTime{0}; + int64_t totalEvictTime{0}; + int64_t totalCompressTime{0}; + std::vector partitionLengths{}; + std::vector rawPartitionLengths{}; // Uncompressed size. +}; } // namespace gluten diff --git a/cpp/core/shuffle/PartitionWriter.cc b/cpp/core/shuffle/PartitionWriter.cc deleted file mode 100644 index 4b12511b8633..000000000000 --- a/cpp/core/shuffle/PartitionWriter.cc +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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 "shuffle/PartitionWriter.h" -#include "shuffle/Utils.h" - -namespace gluten { - -arrow::Result> gluten::ShuffleWriter::PartitionWriter::createPayloadFromBuffers( - uint32_t numRows, - std::vector> buffers) { - std::shared_ptr recordBatch; - if (options_->compression_type != arrow::Compression::UNCOMPRESSED) { - ARROW_ASSIGN_OR_RAISE( - recordBatch, - makeCompressedRecordBatch( - numRows, - std::move(buffers), - options_->write_schema, - options_->ipc_write_options.memory_pool, - options_->codec.get(), - options_->compression_threshold, - options_->compression_mode, - compressTime_)); - } else { - ARROW_ASSIGN_OR_RAISE( - recordBatch, - makeUncompressedRecordBatch( - numRows, std::move(buffers), options_->write_schema, options_->ipc_write_options.memory_pool)); - } - - auto payload = std::make_unique(); - RETURN_NOT_OK(arrow::ipc::GetRecordBatchPayload(*recordBatch, options_->ipc_write_options, payload.get())); - return payload; -} - -} // namespace gluten \ No newline at end of file diff --git a/cpp/core/shuffle/PartitionWriter.h b/cpp/core/shuffle/PartitionWriter.h index bc655d03ff84..81507e78d689 100644 --- a/cpp/core/shuffle/PartitionWriter.h +++ b/cpp/core/shuffle/PartitionWriter.h @@ -17,11 +17,19 @@ #pragma once +#include "ShuffleMemoryPool.h" +#include "memory/Evictable.h" #include "shuffle/Options.h" -#include "shuffle/ShuffleWriter.h" namespace gluten { +class Payload { + public: + virtual ~Payload() = default; + + virtual arrow::Status serialize(arrow::io::OutputStream* outputStream) = 0; +}; + class Evictor { public: enum Type { kCache, kFlush, kStop }; @@ -30,7 +38,7 @@ class Evictor { virtual ~Evictor() = default; - virtual arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) = 0; + virtual arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) = 0; virtual arrow::Status finish() = 0; @@ -44,15 +52,16 @@ class Evictor { int64_t evictTime_{0}; }; -class ShuffleWriter::PartitionWriter : public Evictable { +class PartitionWriter : public Evictable { public: PartitionWriter(uint32_t numPartitions, ShuffleWriterOptions* options) - : numPartitions_(numPartitions), options_(options) {} + : numPartitions_(numPartitions), options_(options) { + payloadPool_ = std::make_unique(options_->memory_pool); + codec_ = createArrowIpcCodec(options_->compression_type, options_->codec_backend); + } virtual ~PartitionWriter() = default; - virtual arrow::Status init() = 0; - virtual arrow::Status stop(ShuffleWriterMetrics* metrics) = 0; /// Evict buffers for `partitionId` partition. @@ -62,21 +71,27 @@ class ShuffleWriter::PartitionWriter : public Evictable { uint32_t partitionId, uint32_t numRows, std::vector> buffers, + bool reuseBuffers, Evictor::Type evictType) = 0; virtual arrow::Status finishEvict() = 0; - protected: - arrow::Result> createPayloadFromBuffers( - uint32_t numRows, - std::vector> buffers); + uint64_t cachedPayloadSize() { + return payloadPool_->bytes_allocated(); + } + protected: uint32_t numPartitions_; ShuffleWriterOptions* options_; + // Memory Pool used to track memory allocation of partition payloads. + // The actual allocation is delegated to options_.memory_pool. + std::unique_ptr payloadPool_; + + std::unique_ptr codec_; + int64_t compressTime_{0}; int64_t evictTime_{0}; int64_t writeTime_{0}; }; - } // namespace gluten diff --git a/cpp/core/shuffle/PartitionWriterCreator.cc b/cpp/core/shuffle/PartitionWriterCreator.cc deleted file mode 100644 index 58b886ac254d..000000000000 --- a/cpp/core/shuffle/PartitionWriterCreator.cc +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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 "PartitionWriterCreator.h" - -namespace gluten {} // namespace gluten diff --git a/cpp/core/shuffle/ShuffleMemoryPool.cc b/cpp/core/shuffle/ShuffleMemoryPool.cc new file mode 100644 index 000000000000..25bbaae78969 --- /dev/null +++ b/cpp/core/shuffle/ShuffleMemoryPool.cc @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 "shuffle/ShuffleMemoryPool.h" + +namespace gluten { +gluten::ShuffleMemoryPool::ShuffleMemoryPool(arrow::MemoryPool* pool) : pool_(pool) {} + +arrow::Status ShuffleMemoryPool::Allocate(int64_t size, int64_t alignment, uint8_t** out) { + auto before = pool_->bytes_allocated(); + auto status = pool_->Allocate(size, alignment, out); + if (status.ok()) { + bytesAllocated_ += (pool_->bytes_allocated() - before); + } + return status; +} + +arrow::Status ShuffleMemoryPool::Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) { + auto before = pool_->bytes_allocated(); + auto status = pool_->Reallocate(old_size, new_size, alignment, ptr); + if (status.ok()) { + bytesAllocated_ += (pool_->bytes_allocated() - before); + } + return status; +} + +void ShuffleMemoryPool::Free(uint8_t* buffer, int64_t size, int64_t alignment) { + auto before = pool_->bytes_allocated(); + pool_->Free(buffer, size, alignment); + bytesAllocated_ += (pool_->bytes_allocated() - before); +} + +int64_t ShuffleMemoryPool::bytes_allocated() const { + return bytesAllocated_; +} + +int64_t ShuffleMemoryPool::max_memory() const { + return pool_->max_memory(); +} + +std::string ShuffleMemoryPool::backend_name() const { + return pool_->backend_name(); +} + +int64_t ShuffleMemoryPool::total_bytes_allocated() const { + return pool_->total_bytes_allocated(); +} + +int64_t ShuffleMemoryPool::num_allocations() const { + throw pool_->num_allocations(); +} +} // namespace gluten diff --git a/cpp/core/shuffle/PartitionWriterCreator.h b/cpp/core/shuffle/ShuffleMemoryPool.h similarity index 54% rename from cpp/core/shuffle/PartitionWriterCreator.h rename to cpp/core/shuffle/ShuffleMemoryPool.h index 49ce7fc33ac3..64755e689dbe 100644 --- a/cpp/core/shuffle/PartitionWriterCreator.h +++ b/cpp/core/shuffle/ShuffleMemoryPool.h @@ -15,21 +15,33 @@ * limitations under the License. */ -#pragma once +#include -#include "shuffle/PartitionWriter.h" -#include "shuffle/ShuffleWriter.h" +#pragma once namespace gluten { - -class ShuffleWriter::PartitionWriterCreator { +class ShuffleMemoryPool : public arrow::MemoryPool { public: - PartitionWriterCreator() = default; - virtual ~PartitionWriterCreator() = default; + ShuffleMemoryPool(arrow::MemoryPool* pool); - virtual arrow::Result> make( - uint32_t numPartitions, - ShuffleWriterOptions* options) = 0; -}; + arrow::Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override; + + arrow::Status Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) override; + + void Free(uint8_t* buffer, int64_t size, int64_t alignment) override; -} // namespace gluten + int64_t bytes_allocated() const override; + + int64_t max_memory() const override; + + std::string backend_name() const override; + + int64_t total_bytes_allocated() const override; + + int64_t num_allocations() const override; + + private: + arrow::MemoryPool* pool_; + uint64_t bytesAllocated_ = 0; +}; +} // namespace gluten \ No newline at end of file diff --git a/cpp/core/shuffle/ShuffleReader.cc b/cpp/core/shuffle/ShuffleReader.cc index 6b05dd64e5b8..f67f6aeae6ca 100644 --- a/cpp/core/shuffle/ShuffleReader.cc +++ b/cpp/core/shuffle/ShuffleReader.cc @@ -24,65 +24,12 @@ #include "ShuffleSchema.h" -namespace { -using namespace gluten; - -class ShuffleReaderOutStream : public ColumnarBatchIterator { - public: - ShuffleReaderOutStream( - const ShuffleReaderOptions& options, - const std::shared_ptr& schema, - const std::shared_ptr& in, - const std::function ipcTimeAccumulator) - : options_(options), in_(in), ipcTimeAccumulator_(ipcTimeAccumulator) { - if (options.compression_type != arrow::Compression::UNCOMPRESSED) { - writeSchema_ = toCompressWriteSchema(); - } else { - writeSchema_ = toWriteSchema(*schema); - } - } - - std::shared_ptr next() override { - std::shared_ptr arrowBatch; - std::unique_ptr messageToRead; - - int64_t ipcTime = 0; - TIME_NANO_START(ipcTime); - - GLUTEN_ASSIGN_OR_THROW(messageToRead, arrow::ipc::ReadMessage(in_.get())) - if (messageToRead == nullptr) { - return nullptr; - } - - GLUTEN_ASSIGN_OR_THROW( - arrowBatch, arrow::ipc::ReadRecordBatch(*messageToRead, writeSchema_, nullptr, options_.ipc_read_options)) - - TIME_NANO_END(ipcTime); - ipcTimeAccumulator_(ipcTime); - - std::shared_ptr glutenBatch = std::make_shared(arrowBatch); - return glutenBatch; - } - - private: - ShuffleReaderOptions options_; - std::shared_ptr in_; - std::function ipcTimeAccumulator_; - std::shared_ptr writeSchema_; -}; -} // namespace - namespace gluten { -ShuffleReader::ShuffleReader( - std::shared_ptr schema, - ShuffleReaderOptions options, - arrow::MemoryPool* pool) - : pool_(pool), options_(std::move(options)), schema_(schema) {} +ShuffleReader::ShuffleReader(std::unique_ptr factory) : factory_(std::move(factory)) {} std::shared_ptr ShuffleReader::readStream(std::shared_ptr in) { - return std::make_shared(std::make_unique( - options_, schema_, in, [this](int64_t ipcTime) { this->ipcTime_ += ipcTime; })); + return std::make_shared(factory_->createDeserializer(in)); } arrow::Status ShuffleReader::close() { @@ -90,7 +37,19 @@ arrow::Status ShuffleReader::close() { } arrow::MemoryPool* ShuffleReader::getPool() const { - return pool_; + return factory_->getPool(); +} + +int64_t ShuffleReader::getDecompressTime() const { + return factory_->getDecompressTime(); +} + +int64_t ShuffleReader::getIpcTime() const { + return ipcTime_; +} + +int64_t ShuffleReader::getArrowToVeloxTime() const { + return factory_->getArrowToVeloxTime(); } } // namespace gluten diff --git a/cpp/core/shuffle/ShuffleReader.h b/cpp/core/shuffle/ShuffleReader.h index 676211afb4ff..818ca6a5bd5e 100644 --- a/cpp/core/shuffle/ShuffleReader.h +++ b/cpp/core/shuffle/ShuffleReader.h @@ -28,9 +28,22 @@ namespace gluten { +class DeserializerFactory { + public: + virtual ~DeserializerFactory() = default; + + virtual std::unique_ptr createDeserializer(std::shared_ptr in) = 0; + + virtual arrow::MemoryPool* getPool() = 0; + + virtual int64_t getDecompressTime() = 0; + + virtual int64_t getArrowToVeloxTime() = 0; +}; + class ShuffleReader { public: - explicit ShuffleReader(std::shared_ptr schema, ShuffleReaderOptions options, arrow::MemoryPool* pool); + explicit ShuffleReader(std::unique_ptr factory); virtual ~ShuffleReader() = default; @@ -39,17 +52,11 @@ class ShuffleReader { arrow::Status close(); - int64_t getDecompressTime() const { - return decompressTime_; - } + int64_t getDecompressTime() const; - int64_t getIpcTime() const { - return ipcTime_; - } + int64_t getIpcTime() const; - int64_t getDeserializeTime() const { - return deserializeTime_; - } + int64_t getArrowToVeloxTime() const; arrow::MemoryPool* getPool() const; @@ -63,6 +70,7 @@ class ShuffleReader { private: std::shared_ptr schema_; + std::unique_ptr factory_; }; } // namespace gluten diff --git a/cpp/core/shuffle/ShuffleWriter.cc b/cpp/core/shuffle/ShuffleWriter.cc deleted file mode 100644 index 5c42b94a4c85..000000000000 --- a/cpp/core/shuffle/ShuffleWriter.cc +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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 "ShuffleWriter.h" - -#include - -#include "ShuffleSchema.h" -#include "utils/macros.h" - -#include "PartitionWriterCreator.h" - -namespace gluten { - -#ifndef SPLIT_BUFFER_SIZE -// by default, allocate 8M block, 2M page size -#define SPLIT_BUFFER_SIZE 16 * 1024 * 1024 -#endif -} // namespace gluten diff --git a/cpp/core/shuffle/ShuffleWriter.h b/cpp/core/shuffle/ShuffleWriter.h index 91d6861086d7..a826042ccc68 100644 --- a/cpp/core/shuffle/ShuffleWriter.h +++ b/cpp/core/shuffle/ShuffleWriter.h @@ -25,76 +25,14 @@ #include "memory/ColumnarBatch.h" #include "memory/Evictable.h" #include "shuffle/Options.h" +#include "shuffle/PartitionWriter.h" #include "shuffle/Partitioner.h" #include "shuffle/Partitioning.h" +#include "shuffle/ShuffleMemoryPool.h" #include "utils/Compression.h" namespace gluten { -struct ShuffleWriterMetrics { - int64_t totalBytesWritten{0}; - int64_t totalBytesEvicted{0}; - int64_t totalWriteTime{0}; - int64_t totalEvictTime{0}; - int64_t totalCompressTime{0}; - std::vector partitionLengths{}; - std::vector rawPartitionLengths{}; // Uncompressed size. -}; - -class ShuffleMemoryPool : public arrow::MemoryPool { - public: - ShuffleMemoryPool(arrow::MemoryPool* pool) : pool_(pool) {} - - arrow::MemoryPool* delegated() { - return pool_; - } - - arrow::Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override { - auto status = pool_->Allocate(size, alignment, out); - if (status.ok()) { - bytesAllocated_ += size; - } - return status; - } - - arrow::Status Reallocate(int64_t old_size, int64_t new_size, int64_t alignment, uint8_t** ptr) override { - auto status = pool_->Reallocate(old_size, new_size, alignment, ptr); - if (status.ok()) { - bytesAllocated_ += (new_size - old_size); - } - return status; - } - - void Free(uint8_t* buffer, int64_t size, int64_t alignment) override { - pool_->Free(buffer, size, alignment); - bytesAllocated_ -= size; - } - - int64_t bytes_allocated() const override { - return bytesAllocated_; - } - - int64_t max_memory() const override { - return pool_->max_memory(); - } - - std::string backend_name() const override { - return pool_->backend_name(); - } - - int64_t total_bytes_allocated() const override { - return pool_->total_bytes_allocated(); - } - - int64_t num_allocations() const override { - throw pool_->num_allocations(); - } - - private: - arrow::MemoryPool* pool_; - uint64_t bytesAllocated_ = 0; -}; - class ShuffleWriter : public Evictable { public: static constexpr int64_t kMinMemLimit = 128LL * 1024 * 1024; @@ -105,10 +43,6 @@ class ShuffleWriter : public Evictable { virtual arrow::Status stop() = 0; - virtual std::shared_ptr& schema() { - return schema_; - } - int32_t numPartitions() const { return numPartitions_; } @@ -145,42 +79,37 @@ class ShuffleWriter : public Evictable { return metrics_.rawPartitionLengths; } - ShuffleWriterOptions& options() { - return options_; + ShuffleWriterOptions* options() { + return options_.get(); } virtual const uint64_t cachedPayloadSize() const = 0; - class PartitionWriter; - - class PartitionWriterCreator; - protected: ShuffleWriter( int32_t numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options) + std::unique_ptr partitionWriter, + std::unique_ptr options) : numPartitions_(numPartitions), - partitionWriterCreator_(std::move(partitionWriterCreator)), options_(std::move(options)), - partitionBufferPool_(std::make_shared(options_.memory_pool)) { - options_.codec = createArrowIpcCodec(options_.compression_type, options_.codec_backend); - } + partitionBufferPool_(std::make_unique(options_->memory_pool)), + partitionWriter_(std::move(partitionWriter)) {} virtual ~ShuffleWriter() = default; int32_t numPartitions_; - std::shared_ptr partitionWriterCreator_; + std::unique_ptr options_; - ShuffleWriterOptions options_; // Memory Pool used to track memory usage of partition buffers. // The actual allocation is delegated to options_.memory_pool. - std::shared_ptr partitionBufferPool_; + std::unique_ptr partitionBufferPool_; + + std::unique_ptr partitionWriter_; std::shared_ptr schema_; - // col partid + // Column index, partition id, buffers. std::vector>>> partitionBuffers_; std::shared_ptr partitioner_; diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index 9bef574b27f1..17cac38c8227 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -326,8 +326,6 @@ int64_t gluten::getMaxCompressedBufferSize( arrow::Status gluten::writeEos(arrow::io::OutputStream* os, int64_t* bytes) { // write EOS - static constexpr int32_t kIpcContinuationToken = -1; - static constexpr int32_t kZeroLength = 0; static const int64_t kSizeOfEos = sizeof(kIpcContinuationToken) + sizeof(kZeroLength); RETURN_NOT_OK(os->Write(&kIpcContinuationToken, sizeof(kIpcContinuationToken))); RETURN_NOT_OK(os->Write(&kZeroLength, sizeof(kZeroLength))); diff --git a/cpp/core/shuffle/Utils.h b/cpp/core/shuffle/Utils.h index fde8a7e0cc7c..588f1d7015aa 100644 --- a/cpp/core/shuffle/Utils.h +++ b/cpp/core/shuffle/Utils.h @@ -32,6 +32,9 @@ namespace gluten { using BinaryArrayLengthBufferType = uint32_t; using IpcOffsetBufferType = arrow::LargeStringType::offset_type; +static constexpr int32_t kIpcContinuationToken = -1; +static constexpr uint32_t kZeroLength = 0; + static const size_t kSizeOfBinaryArrayLengthBuffer = sizeof(BinaryArrayLengthBufferType); static const size_t kSizeOfIpcOffsetBuffer = sizeof(IpcOffsetBufferType); static const std::string kGlutenSparkLocalDirs = "GLUTEN_SPARK_LOCAL_DIRS"; diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc index 8444c2e0f817..33cbf93c8886 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc @@ -16,6 +16,7 @@ */ #include "CelebornPartitionWriter.h" +#include "shuffle/BlockPayload.h" #include "shuffle/Utils.h" #include "utils/Timer.h" @@ -26,13 +27,11 @@ class CelebornEvictHandle final : public Evictor { CelebornEvictHandle(ShuffleWriterOptions* options, RssClient* client, std::vector& bytesEvicted) : Evictor(options), client_(client), bytesEvicted_(bytesEvicted) {} - arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { // Copy payload to arrow buffered os. ARROW_ASSIGN_OR_RAISE( auto celebornBufferOs, arrow::io::BufferOutputStream::Create(options_->buffer_size, options_->memory_pool)); - int32_t metadataLength = 0; // unused - RETURN_NOT_OK( - arrow::ipc::WriteIpcPayload(*payload, options_->ipc_write_options, celebornBufferOs.get(), &metadataLength)); + RETURN_NOT_OK(payload->serialize(celebornBufferOs.get())); payload = nullptr; // Invalidate payload immediately. // Push. @@ -52,11 +51,10 @@ class CelebornEvictHandle final : public Evictor { std::vector& bytesEvicted_; }; -arrow::Status CelebornPartitionWriter::init() { +void CelebornPartitionWriter::init() { bytesEvicted_.resize(numPartitions_, 0); rawPartitionLengths_.resize(numPartitions_, 0); evictor_ = std::make_shared(options_, celebornClient_.get(), bytesEvicted_); - return arrow::Status::OK(); } arrow::Status CelebornPartitionWriter::stop(ShuffleWriterMetrics* metrics) { @@ -82,10 +80,14 @@ arrow::Status CelebornPartitionWriter::evict( uint32_t partitionId, uint32_t numRows, std::vector> buffers, + bool reuseBuffers, Evictor::Type evictType) { rawPartitionLengths_[partitionId] += getBufferSize(buffers); ScopedTimer timer(evictTime_); - ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffers(numRows, std::move(buffers))); + ARROW_ASSIGN_OR_RAISE( + auto payload, + BlockPayload::fromBuffers( + numRows, std::move(buffers), options_, payloadPool_.get(), codec_ ? codec_.get() : nullptr, false)); RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); return arrow::Status::OK(); } @@ -94,16 +96,4 @@ arrow::Status CelebornPartitionWriter::evictFixedSize(int64_t size, int64_t* act *actual = 0; return arrow::Status::OK(); } - -CelebornPartitionWriterCreator::CelebornPartitionWriterCreator(std::shared_ptr client) - : PartitionWriterCreator(), client_(client) {} - -arrow::Result> CelebornPartitionWriterCreator::make( - uint32_t numPartitions, - ShuffleWriterOptions* options) { - auto partitionWriter = std::make_shared(numPartitions, options, client_); - RETURN_NOT_OK(partitionWriter->init()); - return partitionWriter; -} - } // namespace gluten diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.h b/cpp/core/shuffle/rss/CelebornPartitionWriter.h index 10bfc21c19ee..4e7830111daa 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.h +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.h @@ -20,9 +20,7 @@ #include #include "shuffle/rss/RemotePartitionWriter.h" - -#include "jni/JniCommon.h" -#include "shuffle/PartitionWriterCreator.h" +#include "shuffle/rss/RssClient.h" #include "utils/macros.h" namespace gluten { @@ -33,43 +31,30 @@ class CelebornPartitionWriter final : public RemotePartitionWriter { uint32_t numPartitions, ShuffleWriterOptions* options, std::shared_ptr celebornClient) - : RemotePartitionWriter(numPartitions, options) { - celebornClient_ = celebornClient; + : RemotePartitionWriter(numPartitions, options), celebornClient_(celebornClient) { + init(); } arrow::Status evict( uint32_t partitionId, uint32_t numRows, std::vector> buffers, + bool reuseBuffers, Evictor::Type evictType /* unused */) override; arrow::Status finishEvict() override; - arrow::Status init() override; - arrow::Status stop(ShuffleWriterMetrics* metrics) override; arrow::Status evictFixedSize(int64_t size, int64_t* actual) override; private: - std::shared_ptr celebornClient_; + void init(); + std::shared_ptr celebornClient_; std::shared_ptr evictor_; std::vector bytesEvicted_; std::vector rawPartitionLengths_; }; - -class CelebornPartitionWriterCreator : public ShuffleWriter::PartitionWriterCreator { - public: - explicit CelebornPartitionWriterCreator(std::shared_ptr client); - - arrow::Result> make( - uint32_t numPartitions, - ShuffleWriterOptions* options) override; - - private: - std::shared_ptr client_; -}; - } // namespace gluten diff --git a/cpp/core/shuffle/rss/RemotePartitionWriter.cc b/cpp/core/shuffle/rss/RemotePartitionWriter.cc index 6fa6feddd582..9993956b6472 100644 --- a/cpp/core/shuffle/rss/RemotePartitionWriter.cc +++ b/cpp/core/shuffle/rss/RemotePartitionWriter.cc @@ -15,6 +15,6 @@ * limitations under the License. */ -#include "shuffle/rss/RemotePartitionWriter.h" +#include "RemotePartitionWriter.h" namespace gluten {} // namespace gluten diff --git a/cpp/core/shuffle/rss/RemotePartitionWriter.h b/cpp/core/shuffle/rss/RemotePartitionWriter.h index 76b1a0bc5ca7..18e0414aa324 100644 --- a/cpp/core/shuffle/rss/RemotePartitionWriter.h +++ b/cpp/core/shuffle/rss/RemotePartitionWriter.h @@ -21,7 +21,7 @@ namespace gluten { -class RemotePartitionWriter : public ShuffleWriter::PartitionWriter { +class RemotePartitionWriter : public PartitionWriter { public: explicit RemotePartitionWriter(uint32_t numPartitions, ShuffleWriterOptions* options) : PartitionWriter(numPartitions, options) {} diff --git a/cpp/core/utils/Timer.h b/cpp/core/utils/Timer.h index 69e1babe00ef..a0a8f2b563e6 100644 --- a/cpp/core/utils/Timer.h +++ b/cpp/core/utils/Timer.h @@ -40,6 +40,11 @@ class Timer { std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime_).count(); } + void reset() { + running_ = false; + realTimeUsed_ = 0; + } + bool running() const { return running_; } @@ -60,15 +65,30 @@ class Timer { class ScopedTimer : public Timer { public: explicit ScopedTimer(int64_t& toAdd) : Timer(), toAdd_(toAdd) { - Timer::start(); + startInternal(); } ~ScopedTimer() { - Timer::stop(); - toAdd_ += realTimeUsed(); + stopInternal(); + } + + void switchTo(int64_t& toAdd) { + stopInternal(); + toAdd_ = toAdd; + startInternal(); } private: int64_t& toAdd_; + + void stopInternal() { + Timer::stop(); + toAdd_ += realTimeUsed(); + } + + void startInternal() { + Timer::reset(); + Timer::start(); + } }; } // namespace gluten diff --git a/cpp/velox/benchmarks/GenericBenchmark.cc b/cpp/velox/benchmarks/GenericBenchmark.cc index c3736134afac..5e6373b5cb7d 100644 --- a/cpp/velox/benchmarks/GenericBenchmark.cc +++ b/cpp/velox/benchmarks/GenericBenchmark.cc @@ -54,49 +54,41 @@ struct WriterMetrics { int64_t compressTime; }; -std::shared_ptr createShuffleWriter(VeloxMemoryManager* memoryManager) { - std::shared_ptr partitionWriterCreator = - std::make_shared(); - - auto options = ShuffleWriterOptions::defaults(); - options.memory_pool = memoryManager->getArrowMemoryPool(); - options.partitioning = gluten::toPartitioning(FLAGS_partitioning); +std::shared_ptr createShuffleWriter( + VeloxMemoryManager* memoryManager, + const std::string& dataFile, + const std::vector& localDirs) { + auto options = std::make_unique(); + options->memory_pool = memoryManager->getArrowMemoryPool(); + options->partitioning = gluten::toPartitioning(FLAGS_partitioning); if (FLAGS_zstd) { - options.codec_backend = CodecBackend::NONE; - options.compression_type = arrow::Compression::ZSTD; + options->codec_backend = CodecBackend::NONE; + options->compression_type = arrow::Compression::ZSTD; } else if (FLAGS_qat_gzip) { - options.codec_backend = CodecBackend::QAT; - options.compression_type = arrow::Compression::GZIP; + options->codec_backend = CodecBackend::QAT; + options->compression_type = arrow::Compression::GZIP; } else if (FLAGS_qat_zstd) { - options.codec_backend = CodecBackend::QAT; - options.compression_type = arrow::Compression::ZSTD; + options->codec_backend = CodecBackend::QAT; + options->compression_type = arrow::Compression::ZSTD; } else if (FLAGS_iaa_gzip) { - options.codec_backend = CodecBackend::IAA; - options.compression_type = arrow::Compression::GZIP; + options->codec_backend = CodecBackend::IAA; + options->compression_type = arrow::Compression::GZIP; } - GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(options)); + std::unique_ptr partitionWriter = + std::make_unique(FLAGS_shuffle_partitions, dataFile, localDirs, options.get()); GLUTEN_ASSIGN_OR_THROW( auto shuffleWriter, VeloxShuffleWriter::create( FLAGS_shuffle_partitions, - std::move(partitionWriterCreator), + std::move(partitionWriter), std::move(options), memoryManager->getLeafMemoryPool())); return shuffleWriter; } -void cleanup(const std::shared_ptr& shuffleWriter) { - auto dataFile = std::filesystem::path(shuffleWriter->dataFile()); - const auto& parentDir = dataFile.parent_path(); - std::filesystem::remove(dataFile); - if (std::filesystem::is_empty(parentDir)) { - std::filesystem::remove(parentDir); - } -} - void populateWriterMetrics( const std::shared_ptr& shuffleWriter, int64_t shuffleWriteTime, @@ -153,7 +145,11 @@ auto BM_Generic = [](::benchmark::State& state, if (FLAGS_with_shuffle) { int64_t shuffleWriteTime; TIME_NANO_START(shuffleWriteTime); - const auto& shuffleWriter = createShuffleWriter(memoryManager.get()); + std::string dataFile; + std::vector localDirs; + bool isFromEnv; + GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(dataFile, localDirs, isFromEnv)); + const auto& shuffleWriter = createShuffleWriter(memoryManager.get(), dataFile, localDirs); while (resultIter->hasNext()) { GLUTEN_THROW_NOT_OK(shuffleWriter->split(resultIter->next(), ShuffleWriter::kMinMemLimit)); } @@ -161,7 +157,7 @@ auto BM_Generic = [](::benchmark::State& state, TIME_NANO_END(shuffleWriteTime); populateWriterMetrics(shuffleWriter, shuffleWriteTime, writerMetrics); // Cleanup shuffle outputs - cleanup(shuffleWriter); + cleanupShuffleOutput(dataFile, localDirs, isFromEnv); } else { // May write the output into file. ArrowSchema cSchema; diff --git a/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc b/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc index c4aec9650e1b..040833f8e308 100644 --- a/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc +++ b/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc @@ -110,14 +110,14 @@ class BenchmarkShuffleSplit { std::shared_ptr pool = defaultArrowMemoryPool(); - std::shared_ptr partitionWriterCreator = - std::make_shared(); - - auto options = ShuffleWriterOptions::defaults(); - options.buffer_size = kPartitionBufferSize; - options.memory_pool = pool.get(); - options.partitioning = Partitioning::kRoundRobin; - GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(options)); + auto options = std::make_unique(); + options->buffer_size = kPartitionBufferSize; + options->memory_pool = pool.get(); + options->partitioning = Partitioning::kRoundRobin; + std::string dataFile; + std::vector localDirs; + bool isFromEnv; + GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(dataFile, localDirs, isFromEnv)); std::shared_ptr shuffleWriter; int64_t elapseRead = 0; @@ -133,14 +133,14 @@ class BenchmarkShuffleSplit { numRows, splitTime, FLAGS_partitions, - partitionWriterCreator, - options, + std::move(options), + dataFile, + localDirs, state); auto endTime = std::chrono::steady_clock::now(); auto totalTime = (endTime - startTime).count(); - auto fs = std::make_shared(); - GLUTEN_THROW_NOT_OK(fs->DeleteFile(shuffleWriter->dataFile())); + cleanupShuffleOutput(dataFile, localDirs, isFromEnv); state.SetBytesProcessed(int64_t(shuffleWriter->rawPartitionBytes())); @@ -201,8 +201,9 @@ class BenchmarkShuffleSplit { int64_t& numRows, int64_t& splitTime, const int numPartitions, - std::shared_ptr partitionWriterCreator, - ShuffleWriterOptions options, + std::unique_ptr options, + const std::string& dataFile, + const std::vector& localDirs, benchmark::State& state) {} protected: @@ -225,8 +226,9 @@ class BenchmarkShuffleSplitCacheScanBenchmark : public BenchmarkShuffleSplit { int64_t& numRows, int64_t& splitTime, const int numPartitions, - std::shared_ptr partitionWriterCreator, - ShuffleWriterOptions options, + std::unique_ptr options, + const std::string& dataFile, + const std::vector& localDirs, benchmark::State& state) { std::vector localColumnIndices; // local_column_indices.push_back(0); @@ -256,10 +258,12 @@ class BenchmarkShuffleSplitCacheScanBenchmark : public BenchmarkShuffleSplit { if (state.thread_index() == 0) std::cout << localSchema->ToString() << std::endl; - auto pool = options.memory_pool; + auto pool = options->memory_pool; + auto partitionWriter = std::make_unique(numPartitions, dataFile, localDirs, options.get()); GLUTEN_ASSIGN_OR_THROW( shuffleWriter, - VeloxShuffleWriter::create(numPartitions, partitionWriterCreator, options, defaultLeafVeloxMemoryPool())); + VeloxShuffleWriter::create( + numPartitions, std::move(partitionWriter), std::move(options), defaultLeafVeloxMemoryPool())); std::shared_ptr recordBatch; @@ -311,23 +315,26 @@ class BenchmarkShuffleSplitIterateScanBenchmark : public BenchmarkShuffleSplit { int64_t& numRows, int64_t& splitTime, const int numPartitions, - std::shared_ptr partitionWriterCreator, - ShuffleWriterOptions options, + std::unique_ptr options, + const std::string& dataFile, + const std::vector& localDirs, benchmark::State& state) { if (state.thread_index() == 0) std::cout << schema_->ToString() << std::endl; + auto pool = options->memory_pool; + auto partitionWriter = std::make_unique(numPartitions, dataFile, localDirs, options.get()); GLUTEN_ASSIGN_OR_THROW( shuffleWriter, VeloxShuffleWriter::create( - numPartitions, std::move(partitionWriterCreator), std::move(options), defaultLeafVeloxMemoryPool())); + numPartitions, std::move(partitionWriter), std::move(options), defaultLeafVeloxMemoryPool())); std::shared_ptr recordBatch; std::unique_ptr<::parquet::arrow::FileReader> parquetReader; std::shared_ptr recordBatchReader; GLUTEN_THROW_NOT_OK(::parquet::arrow::FileReader::Make( - options.memory_pool, ::parquet::ParquetFileReader::Open(file_), properties_, &parquetReader)); + pool, ::parquet::ParquetFileReader::Open(file_), properties_, &parquetReader)); for (auto _ : state) { std::vector> batches; diff --git a/cpp/velox/benchmarks/common/BenchmarkUtils.cc b/cpp/velox/benchmarks/common/BenchmarkUtils.cc index 00dc804389fe..6a7d73c82290 100644 --- a/cpp/velox/benchmarks/common/BenchmarkUtils.cc +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.cc @@ -153,26 +153,37 @@ void setCpu(uint32_t cpuindex) { } } -arrow::Status setLocalDirsAndDataFileFromEnv(gluten::ShuffleWriterOptions& options) { +arrow::Status +setLocalDirsAndDataFileFromEnv(std::string& dataFile, std::vector& localDirs, bool& isFromEnv) { auto joinedDirsC = std::getenv(gluten::kGlutenSparkLocalDirs.c_str()); if (joinedDirsC != nullptr && strcmp(joinedDirsC, "") > 0) { + isFromEnv = true; // Set local dirs. auto joinedDirs = std::string(joinedDirsC); - options.local_dirs = joinedDirs; // Split local dirs and use thread id to choose one directory for data file. - auto localDirs = gluten::splitPaths(joinedDirs); + localDirs = gluten::splitPaths(joinedDirs); size_t id = std::hash{}(std::this_thread::get_id()) % localDirs.size(); - ARROW_ASSIGN_OR_RAISE(options.data_file, gluten::createTempShuffleFile(localDirs[id])); + ARROW_ASSIGN_OR_RAISE(dataFile, gluten::createTempShuffleFile(localDirs[id])); } else { + isFromEnv = false; // Otherwise create 1 temp dir and data file. static const std::string kBenchmarkDirsPrefix = "columnar-shuffle-benchmark-"; { // Because tmpDir will be deleted in the dtor, allow it to be deleted upon exiting the block and then recreate it // in createTempShuffleFile. ARROW_ASSIGN_OR_RAISE(auto tmpDir, arrow::internal::TemporaryDir::Make(kBenchmarkDirsPrefix)) - options.local_dirs = tmpDir->path().ToString(); + localDirs.push_back(tmpDir->path().ToString()); } - ARROW_ASSIGN_OR_RAISE(options.data_file, gluten::createTempShuffleFile(options.local_dirs)); + ARROW_ASSIGN_OR_RAISE(dataFile, gluten::createTempShuffleFile(localDirs.back())); } return arrow::Status::OK(); } + +void cleanupShuffleOutput(const std::string& dataFile, const std::vector& localDirs, bool isFromEnv) { + std::filesystem::remove(dataFile); + for (auto& localDir : localDirs) { + if (std::filesystem::is_empty(localDir)) { + std::filesystem::remove(localDir); + } + } +} diff --git a/cpp/velox/benchmarks/common/BenchmarkUtils.h b/cpp/velox/benchmarks/common/BenchmarkUtils.h index 6fb0524159a9..0632f968644c 100644 --- a/cpp/velox/benchmarks/common/BenchmarkUtils.h +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.h @@ -111,4 +111,7 @@ bool endsWith(const std::string& data, const std::string& suffix); void setCpu(uint32_t cpuindex); -arrow::Status setLocalDirsAndDataFileFromEnv(gluten::ShuffleWriterOptions& options); +arrow::Status +setLocalDirsAndDataFileFromEnv(std::string& dataFile, std::vector& localDirs, bool& isFromEnv); + +void cleanupShuffleOutput(const std::string& dataFile, const std::vector& localDirs, bool isFromEnv); diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index 9dcbd9b49416..82abd7acb2ed 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -153,9 +153,6 @@ void VeloxBackend::init(const std::unordered_map& conf // Set backtrace_allocation gluten::backtrace_allocation = veloxcfg->get(kBacktraceAllocation, false); - // Set veloxShuffleReaderPrintFlag - gluten::veloxShuffleReaderPrintFlag = veloxcfg->get(kVeloxShuffleReaderPrintFlag, false); - // Setup and register. velox::filesystems::registerLocalFileSystem(); initJolFilesystem(veloxcfg); diff --git a/cpp/velox/compute/VeloxRuntime.cc b/cpp/velox/compute/VeloxRuntime.cc index a25a74cb3268..d63af52f848c 100644 --- a/cpp/velox/compute/VeloxRuntime.cc +++ b/cpp/velox/compute/VeloxRuntime.cc @@ -28,6 +28,7 @@ #include "shuffle/VeloxShuffleReader.h" #include "shuffle/VeloxShuffleWriter.h" #include "utils/ConfigExtractor.h" +#include "utils/VeloxArrowUtils.h" using namespace facebook; @@ -147,13 +148,13 @@ std::shared_ptr VeloxRuntime::createRow2ColumnarConverte std::shared_ptr VeloxRuntime::createShuffleWriter( int numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, + std::unique_ptr partitionWriter, + std::unique_ptr options, MemoryManager* memoryManager) { auto ctxPool = getLeafVeloxPool(memoryManager); GLUTEN_ASSIGN_OR_THROW( auto shuffle_writer, - VeloxShuffleWriter::create(numPartitions, std::move(partitionWriterCreator), std::move(options), ctxPool)); + VeloxShuffleWriter::create(numPartitions, std::move(partitionWriter), std::move(options), ctxPool)); return shuffle_writer; } @@ -174,8 +175,12 @@ std::shared_ptr VeloxRuntime::createShuffleReader( ShuffleReaderOptions options, arrow::MemoryPool* pool, MemoryManager* memoryManager) { + auto rowType = facebook::velox::asRowType(gluten::fromArrowSchema(schema)); + auto codec = gluten::createArrowIpcCodec(options.compression_type, options.codec_backend); auto ctxVeloxPool = getLeafVeloxPool(memoryManager); - return std::make_shared(schema, options, pool, ctxVeloxPool); + auto deserializerFactory = std::make_unique( + schema, std::move(codec), rowType, pool, ctxVeloxPool); + return std::make_shared(std::move(deserializerFactory)); } std::unique_ptr VeloxRuntime::createColumnarBatchSerializer( diff --git a/cpp/velox/compute/VeloxRuntime.h b/cpp/velox/compute/VeloxRuntime.h index 2d5d727624f2..bd2291db5153 100644 --- a/cpp/velox/compute/VeloxRuntime.h +++ b/cpp/velox/compute/VeloxRuntime.h @@ -82,8 +82,8 @@ class VeloxRuntime final : public Runtime { std::shared_ptr createShuffleWriter( int numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, + std::unique_ptr partitionWriter, + std::unique_ptr options, MemoryManager* memoryManager) override; Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) override { diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index 8aa8df34c663..f3a7aba9e66d 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -18,11 +18,14 @@ #include "VeloxShuffleReader.h" #include +#include #include "memory/VeloxColumnarBatch.h" +#include "shuffle/BlockPayload.h" #include "shuffle/Utils.h" #include "utils/Common.h" #include "utils/Compression.h" +#include "utils/Timer.h" #include "utils/VeloxArrowUtils.h" #include "utils/macros.h" #include "velox/serializers/PrestoSerializer.h" @@ -37,8 +40,6 @@ using namespace facebook::velox; namespace gluten { -bool veloxShuffleReaderPrintFlag = false; - namespace { struct BufferViewReleaser { @@ -366,54 +367,6 @@ RowVectorPtr readRowVector( return rv; } -class VeloxShuffleReaderOutStream : public ColumnarBatchIterator { - public: - VeloxShuffleReaderOutStream( - arrow::MemoryPool* pool, - const std::shared_ptr& veloxPool, - const ShuffleReaderOptions& options, - const RowTypePtr& rowType, - const std::function decompressionTimeAccumulator, - const std::function deserializeTimeAccumulator, - ResultIterator& in) - : pool_(pool), - veloxPool_(veloxPool), - options_(options), - rowType_(rowType), - decompressionTimeAccumulator_(decompressionTimeAccumulator), - deserializeTimeAccumulator_(deserializeTimeAccumulator), - in_(std::move(in)) {} - - std::shared_ptr next() override { - if (!in_.hasNext()) { - return nullptr; - } - auto batch = in_.next(); - auto rb = std::dynamic_pointer_cast(batch)->getRecordBatch(); - - int64_t decompressTime = 0LL; - int64_t deserializeTime = 0LL; - - auto vp = - readRowVector(*rb, rowType_, options_.codec_backend, decompressTime, deserializeTime, pool_, veloxPool_.get()); - - decompressionTimeAccumulator_(decompressTime); - deserializeTimeAccumulator_(deserializeTime); - return std::make_shared(vp); - } - - private: - arrow::MemoryPool* pool_; - std::shared_ptr veloxPool_; - ShuffleReaderOptions options_; - facebook::velox::RowTypePtr rowType_; - - std::function decompressionTimeAccumulator_; - std::function deserializeTimeAccumulator_; - - ResultIterator in_; -}; - std::string getCodecBackend(CodecBackend type) { if (type == CodecBackend::QAT) { return "QAT"; @@ -440,31 +393,69 @@ std::string getCompressionType(arrow::Compression::type type) { } // namespace -VeloxShuffleReader::VeloxShuffleReader( - std::shared_ptr schema, - ShuffleReaderOptions options, - arrow::MemoryPool* pool, - std::shared_ptr veloxPool) - : ShuffleReader(schema, options, pool), veloxPool_(std::move(veloxPool)) { - rowType_ = asRowType(gluten::fromArrowSchema(schema)); - if (gluten::veloxShuffleReaderPrintFlag) { - std::ostringstream oss; - oss << "VeloxShuffleReader create, compression_type:" << getCompressionType(options.compression_type); - oss << " codec_backend:" << getCodecBackend(options.codec_backend); - LOG(INFO) << oss.str(); +VeloxShuffleReader::VeloxShuffleReader(std::unique_ptr factory) + : ShuffleReader(std::move(factory)) {} + +VeloxColumnarBatchDeserializer::VeloxColumnarBatchDeserializer( + const std::shared_ptr& in, + const std::shared_ptr& schema, + const std::shared_ptr& codec, + const facebook::velox::RowTypePtr& rowType, + arrow::MemoryPool* memoryPool, + facebook::velox::memory::MemoryPool* veloxPool, + int64_t& arrowToVeloxTime, + int64_t& decompressTime) + : in_(in), + schema_(schema), + codec_(codec), + rowType_(rowType), + memoryPool_(memoryPool), + veloxPool_(veloxPool), + arrowToVeloxTime_(arrowToVeloxTime), + decompressTime_(decompressTime) {} + +std::shared_ptr VeloxColumnarBatchDeserializer::next() { + ScopedTimer timer(decompressTime_); + uint32_t numRows; + GLUTEN_ASSIGN_OR_THROW( + auto arrowBuffers, BlockPayload::deserialize(in_.get(), schema_, codec_, memoryPool_, numRows)); + if (numRows == 0) { + // Reach EOS. + return nullptr; } + timer.switchTo(arrowToVeloxTime_); + std::vector veloxBuffers; + veloxBuffers.reserve(arrowBuffers.size()); + for (auto& buffer : arrowBuffers) { + veloxBuffers.push_back(convertToVeloxBuffer(buffer)); + } + auto rowVector = deserialize(rowType_, numRows, veloxBuffers, veloxPool_); + return std::make_shared(rowVector); } -std::shared_ptr VeloxShuffleReader::readStream(std::shared_ptr in) { - auto wrappedIn = ShuffleReader::readStream(in); - return std::make_shared(std::make_unique( - pool_, - veloxPool_, - options_, - rowType_, - [this](int64_t decompressionTime) { this->decompressTime_ += decompressionTime; }, - [this](int64_t deserializeTime) { this->deserializeTime_ += deserializeTime; }, - *wrappedIn)); +VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( + const std::shared_ptr& schema, + const std::shared_ptr& codec, + const RowTypePtr& rowType, + arrow::MemoryPool* memoryPool, + std::shared_ptr veloxPool) + : schema_(schema), codec_(codec), rowType_(rowType), memoryPool_(memoryPool), veloxPool_(veloxPool) {} + +std::unique_ptr VeloxColumnarBatchDeserializerFactory::createDeserializer( + std::shared_ptr in) { + return std::make_unique( + std::move(in), schema_, codec_, rowType_, memoryPool_, veloxPool_.get(), arrowToVeloxTime_, decompressTime_); } +arrow::MemoryPool* VeloxColumnarBatchDeserializerFactory::getPool() { + return memoryPool_; +} + +int64_t VeloxColumnarBatchDeserializerFactory::getDecompressTime() { + return decompressTime_; +} + +int64_t VeloxColumnarBatchDeserializerFactory::getArrowToVeloxTime() { + return arrowToVeloxTime_; +} } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index df1236970db6..1d7c3bc441d6 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -17,27 +17,69 @@ #pragma once +#include "shuffle/BlockPayload.h" #include "shuffle/ShuffleReader.h" #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" namespace gluten { -class VeloxShuffleReader final : public ShuffleReader { +class VeloxColumnarBatchDeserializer final : public ColumnarBatchIterator { + public: + VeloxColumnarBatchDeserializer( + const std::shared_ptr& in, + const std::shared_ptr& schema, + const std::shared_ptr& codec, + const facebook::velox::RowTypePtr& rowType, + arrow::MemoryPool* memoryPool, + facebook::velox::memory::MemoryPool* veloxPool, + int64_t& arrowToVeloxTime, + int64_t& decompressTime); + + std::shared_ptr next(); + + private: + std::shared_ptr in_; + std::shared_ptr schema_; + std::shared_ptr codec_; + facebook::velox::RowTypePtr rowType_; + arrow::MemoryPool* memoryPool_; + facebook::velox::memory::MemoryPool* veloxPool_; + + int64_t& arrowToVeloxTime_; + int64_t& decompressTime_; +}; + +class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { public: - VeloxShuffleReader( - std::shared_ptr schema, - ShuffleReaderOptions options, - arrow::MemoryPool* pool, + VeloxColumnarBatchDeserializerFactory( + const std::shared_ptr& schema, + const std::shared_ptr& codec, + const facebook::velox::RowTypePtr& rowType, + arrow::MemoryPool* memoryPool, std::shared_ptr veloxPool); - std::shared_ptr readStream(std::shared_ptr in) override; + std::unique_ptr createDeserializer(std::shared_ptr in) override; + + arrow::MemoryPool* getPool() override; + + int64_t getDecompressTime() override; + + int64_t getArrowToVeloxTime() override; private: + std::shared_ptr schema_; + std::shared_ptr codec_; facebook::velox::RowTypePtr rowType_; + arrow::MemoryPool* memoryPool_; std::shared_ptr veloxPool_; -}; -extern bool veloxShuffleReaderPrintFlag; + int64_t arrowToVeloxTime_{0}; + int64_t decompressTime_{0}; +}; +class VeloxShuffleReader final : public ShuffleReader { + public: + VeloxShuffleReader(std::unique_ptr factory); +}; } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.cc b/cpp/velox/shuffle/VeloxShuffleWriter.cc index 8e4acc8b8c58..e0a3595ace7f 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxShuffleWriter.cc @@ -19,12 +19,9 @@ #include "memory/ArrowMemory.h" #include "memory/VeloxColumnarBatch.h" #include "memory/VeloxMemoryManager.h" -#include "shuffle/Partitioner.h" #include "shuffle/ShuffleSchema.h" #include "shuffle/Utils.h" #include "utils/Common.h" -#include "utils/Compression.h" -#include "utils/Timer.h" #include "utils/VeloxArrowUtils.h" #include "utils/macros.h" #include "velox/buffer/Buffer.h" @@ -199,56 +196,49 @@ arrow::Status collectFlatVectorBuffer( arrow::Result> VeloxShuffleWriter::create( uint32_t numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, + std::unique_ptr partitionWriter, + std::unique_ptr options, std::shared_ptr veloxPool) { #if VELOX_SHUFFLE_WRITER_LOG_FLAG std::ostringstream oss; oss << "Velox shuffle writer created,"; oss << " partitionNum:" << numPartitions; - oss << " partitionWriterCreator:" << typeid(*partitionWriterCreator.get()).name(); - oss << " partitioning:" << options.partitioning; - oss << " buffer_size:" << options.buffer_size; - oss << " compression_type:" << (int)options.compression_type; - oss << " codec_backend:" << (int)options.codec_backend; - oss << " compression_mode:" << (int)options.compression_mode; - oss << " buffered_write:" << options.buffered_write; - oss << " write_eos:" << options.write_eos; - oss << " partition_writer_type:" << options.partition_writer_type; - oss << " thread_id:" << options.thread_id; + oss << " partitionWriterCreator:" << typeid(*partitionWriter.get()).name(); + oss << " partitioning:" << options->partitioning; + oss << " buffer_size:" << options->buffer_size; + oss << " compression_mode:" << (int)options->compression_mode; + oss << " buffered_write:" << options->buffered_write; + oss << " write_eos:" << options->write_eos; + oss << " partition_writer_type:" << options->partition_writer_type; + oss << " thread_id:" << options->thread_id; LOG(INFO) << oss.str(); #endif std::shared_ptr res( - new VeloxShuffleWriter(numPartitions, partitionWriterCreator, options, veloxPool)); + new VeloxShuffleWriter(numPartitions, std::move(partitionWriter), std::move(options), veloxPool)); RETURN_NOT_OK(res->init()); return res; -} +} // namespace gluten arrow::Status VeloxShuffleWriter::init() { - RETURN_NOT_OK(initIpcWriteOptions()); - #if defined(__x86_64__) supportAvx512_ = __builtin_cpu_supports("avx512bw"); #else supportAvx512_ = false; #endif - // partition number should be less than 64k + // Partition number should be less than 64k. VELOX_CHECK_LE(numPartitions_, 64 * 1024); - - // split record batch size should be less than 32k - VELOX_CHECK_LE(options_.buffer_size, 32 * 1024); - + // Split record batch size should be less than 32k. + VELOX_CHECK_LE(options_->buffer_size, 32 * 1024); // memory_pool should be assigned. - VELOX_CHECK_NOT_NULL(options_.memory_pool); + VELOX_CHECK_NOT_NULL(options_->memory_pool); - ARROW_ASSIGN_OR_RAISE(partitionWriter_, partitionWriterCreator_->make(numPartitions_, &options_)); ARROW_ASSIGN_OR_RAISE( - partitioner_, Partitioner::make(options_.partitioning, numPartitions_, options_.start_partition_id)); + partitioner_, Partitioner::make(options_->partitioning, numPartitions_, options_->start_partition_id)); // pre-allocated buffer size for each partition, unit is row count // when partitioner is SinglePart, partial variables don`t need init - if (options_.partitioning != Partitioning::kSingle) { + if (options_->partitioning != Partitioning::kSingle) { partition2RowCount_.resize(numPartitions_); partition2BufferSize_.resize(numPartitions_); partition2RowOffset_.resize(numPartitions_ + 1); @@ -259,12 +249,6 @@ arrow::Status VeloxShuffleWriter::init() { return arrow::Status::OK(); } -arrow::Status VeloxShuffleWriter::initIpcWriteOptions() { - options_.ipc_write_options.memory_pool = payloadPool_.get(); - options_.ipc_write_options.use_threads = false; - return arrow::Status::OK(); -} - arrow::Status VeloxShuffleWriter::initPartitions() { auto simpleColumnCount = simpleColumnIndices_.size(); @@ -300,7 +284,7 @@ arrow::Result> VeloxShuffleWriter::generateComple auto serializedSize = serializer->maxSerializedSize(); auto flushBuffer = complexTypeFlushBuffer_[0]; if (flushBuffer == nullptr) { - ARROW_ASSIGN_OR_RAISE(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, payloadPool_.get())); + ARROW_ASSIGN_OR_RAISE(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, partitionBufferPool_.get())); } else if (serializedSize > flushBuffer->capacity()) { RETURN_NOT_OK(flushBuffer->Reserve(serializedSize)); } @@ -314,7 +298,7 @@ arrow::Result> VeloxShuffleWriter::generateComple } arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64_t memLimit) { - if (options_.partitioning == Partitioning::kSingle) { + if (options_->partitioning == Partitioning::kSingle) { auto veloxColumnBatch = VeloxColumnarBatch::from(veloxPool_.get(), cb); VELOX_CHECK_NOT_NULL(veloxColumnBatch); auto& rv = *veloxColumnBatch->getFlattenedRowVector(); @@ -324,7 +308,7 @@ arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64 for (auto& child : rv.children()) { if (child->encoding() == facebook::velox::VectorEncoding::Simple::FLAT) { auto status = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH_ALL( - collectFlatVectorBuffer, child->typeKind(), child.get(), buffers, payloadPool_.get()); + collectFlatVectorBuffer, child->typeKind(), child.get(), buffers, partitionBufferPool_.get()); RETURN_NOT_OK(status); } else { complexChildren.emplace_back(child); @@ -340,8 +324,8 @@ arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64 buffers.emplace_back(); ARROW_ASSIGN_OR_RAISE(buffers.back(), generateComplexTypeBuffers(rowVector)); } - RETURN_NOT_OK(evictBuffers(0, rv.size(), std::move(buffers))); - } else if (options_.partitioning == Partitioning::kRange) { + RETURN_NOT_OK(evictBuffers(0, rv.size(), std::move(buffers), false)); + } else if (options_->partitioning == Partitioning::kRange) { auto compositeBatch = std::dynamic_pointer_cast(cb); VELOX_CHECK_NOT_NULL(compositeBatch); auto batches = compositeBatch->getBatches(); @@ -382,12 +366,12 @@ arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64 } arrow::Status VeloxShuffleWriter::stop() { - if (options_.partitioning != Partitioning::kSingle) { + if (options_->partitioning != Partitioning::kSingle) { for (auto pid = 0; pid < numPartitions_; ++pid) { auto numRows = partitionBufferIdxBase_[pid]; if (numRows > 0) { ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(pid, false)); - RETURN_NOT_OK(partitionWriter_->evict(pid, numRows, std::move(buffers), Evictor::Type::kStop)); + RETURN_NOT_OK(partitionWriter_->evict(pid, numRows, std::move(buffers), false, Evictor::Type::kStop)); } } } @@ -848,8 +832,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel arrow::Status VeloxShuffleWriter::initColumnTypes(const facebook::velox::RowVector& rv) { schema_ = toArrowSchema(rv.type(), veloxPool_.get()); - options_.write_schema = options_.codec ? toCompressWriteSchema() : toWriteSchema(*schema_); - for (size_t i = 0; i < rv.childrenSize(); ++i) { veloxColumnTypes_.push_back(rv.childAt(i)->type()); } @@ -911,8 +893,8 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel inline bool VeloxShuffleWriter::beyondThreshold(uint32_t partitionId, uint64_t newSize) { auto currentBufferSize = partition2BufferSize_[partitionId]; - return newSize > (1 + options_.buffer_realloc_threshold) * currentBufferSize || - newSize < (1 - options_.buffer_realloc_threshold) * currentBufferSize; + return newSize > (1 + options_->buffer_realloc_threshold) * currentBufferSize || + newSize < (1 - options_->buffer_realloc_threshold) * currentBufferSize; } void VeloxShuffleWriter::calculateSimpleColumnBytes() { @@ -959,8 +941,8 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel } uint64_t preAllocRowCnt = - memLimit > 0 && bytesPerRow > 0 ? memLimit / bytesPerRow / numPartitions_ >> 2 : options_.buffer_size; - preAllocRowCnt = std::min(preAllocRowCnt, (uint64_t)options_.buffer_size); + memLimit > 0 && bytesPerRow > 0 ? memLimit / bytesPerRow / numPartitions_ >> 2 : options_->buffer_size; + preAllocRowCnt = std::min(preAllocRowCnt, (uint64_t)options_->buffer_size); VS_PRINTLF(preAllocRowCnt); @@ -1042,9 +1024,10 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel } arrow::Status VeloxShuffleWriter::evictBuffers( - uint32_t partitionId, uint32_t numRows, std::vector> buffers) { + uint32_t partitionId, uint32_t numRows, std::vector> buffers, bool reuseBuffers) { if (!buffers.empty()) { - RETURN_NOT_OK(partitionWriter_->evict(partitionId, numRows, std::move(buffers), Evictor::Type::kCache)); + RETURN_NOT_OK( + partitionWriter_->evict(partitionId, numRows, std::move(buffers), reuseBuffers, Evictor::Type::kCache)); } return arrow::Status::OK(); } @@ -1053,7 +1036,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel auto numRows = partitionBufferIdxBase_[partitionId]; if (numRows > 0) { ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(partitionId, reuseBuffers)); - RETURN_NOT_OK(evictBuffers(partitionId, numRows, buffers)); + RETURN_NOT_OK(evictBuffers(partitionId, numRows, buffers, reuseBuffers)); } return arrow::Status::OK(); } @@ -1146,7 +1129,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel auto flushBuffer = complexTypeFlushBuffer_[partitionId]; auto serializedSize = complexTypeData_[partitionId]->maxSerializedSize(); if (flushBuffer == nullptr) { - ARROW_ASSIGN_OR_RAISE(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, payloadPool_.get())); + ARROW_ASSIGN_OR_RAISE(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, partitionBufferPool_.get())); } else if (serializedSize > flushBuffer->capacity()) { RETURN_NOT_OK(flushBuffer->Reserve(serializedSize)); } @@ -1164,20 +1147,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel if (!reuseBuffers) { RETURN_NOT_OK(resetPartitionBuffer(partitionId)); } - if (options_.ipc_write_options.codec == nullptr && reuseBuffers) { - // Without compression, we need to perform a manual copy of the original buffers - // so that we can reuse them for next split. - for (auto i = 0; i < allBuffers.size(); ++i) { - auto& buffer = allBuffers[i]; - if (buffer) { - ARROW_ASSIGN_OR_RAISE(auto copy, arrow::AllocateResizableBuffer(buffer->size(), payloadPool_.get())); - if (buffer->size() > 0) { - gluten::fastCopy(copy->mutable_data(), buffer->data(), static_cast(buffer->size())); - } - buffer = std::move(copy); - } - } - } return allBuffers; } @@ -1395,7 +1364,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel } const uint64_t VeloxShuffleWriter::cachedPayloadSize() const { - return payloadPool_->bytes_allocated(); + return partitionWriter_->cachedPayloadSize(); } arrow::Result VeloxShuffleWriter::shrinkPartitionBuffersMinSize(int64_t size) { @@ -1442,7 +1411,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel for (auto& item : pidToSize) { auto pid = item.first; ARROW_ASSIGN_OR_RAISE(auto buffers, assembleBuffers(pid, false)); - RETURN_NOT_OK(partitionWriter_->evict(pid, item.second, std::move(buffers), Evictor::Type::kFlush)); + RETURN_NOT_OK(partitionWriter_->evict(pid, item.second, std::move(buffers), false, Evictor::Type::kFlush)); evicted = beforeEvict - partitionBufferPool_->bytes_allocated(); if (evicted >= size) { break; @@ -1457,7 +1426,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel // If OOM happens during stop(), the reclaim order is shrink->spill, // because the partition buffers will be freed soon. // SinglePartitioning doesn't maintain partition buffers. - return options_.partitioning != Partitioning::kSingle && splitState_ == SplitState::kStop; + return options_->partitioning != Partitioning::kSingle && splitState_ == SplitState::kStop; } bool VeloxShuffleWriter::shrinkPartitionBuffersAfterSpill() const { @@ -1465,14 +1434,14 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel // Or during SplitState::kInit, it is triggered by other operators. // The reclaim order is spill->shrink, because the partition buffers can be reused. // SinglePartitioning doesn't maintain partition buffers. - return options_.partitioning != Partitioning::kSingle && + return options_->partitioning != Partitioning::kSingle && (splitState_ == SplitState::kSplit || splitState_ == SplitState::kInit); } bool VeloxShuffleWriter::evictPartitionBuffersAfterSpill() const { // If OOM triggered by other operators, the splitState_ is SplitState::kInit. // The last resort is to evict the partition buffers to reclaim more space. - return options_.partitioning != Partitioning::kSingle && splitState_ == SplitState::kInit; + return options_->partitioning != Partitioning::kSingle && splitState_ == SplitState::kInit; } arrow::Result VeloxShuffleWriter::partitionBufferSizeAfterShrink(uint32_t partitionId) const { diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.h b/cpp/velox/shuffle/VeloxShuffleWriter.h index 13872fb4f408..0b25ada13592 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxShuffleWriter.h @@ -37,7 +37,7 @@ #include #include "memory/VeloxMemoryManager.h" -#include "shuffle/PartitionWriterCreator.h" +#include "shuffle/PartitionWriter.h" #include "shuffle/Partitioner.h" #include "shuffle/ShuffleWriter.h" #include "shuffle/Utils.h" @@ -125,8 +125,8 @@ class VeloxShuffleWriter final : public ShuffleWriter { static arrow::Result> create( uint32_t numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, + std::unique_ptr partitionWriter, + std::unique_ptr options, std::shared_ptr veloxPool); arrow::Status split(std::shared_ptr cb, int64_t memLimit) override; @@ -143,11 +143,6 @@ class VeloxShuffleWriter final : public ShuffleWriter { return std::accumulate(metrics_.rawPartitionLengths.begin(), metrics_.rawPartitionLengths.end(), 0LL); } - // for testing - const std::string& dataFile() const { - return options_.data_file; - } - // for debugging void printColumnsInfo() const { VS_PRINT_FUNCTION_SPLIT_LINE(); @@ -195,22 +190,18 @@ class VeloxShuffleWriter final : public ShuffleWriter { VS_PRINT_CONTAINER(input_has_null_); } - protected: + private: VeloxShuffleWriter( uint32_t numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, + std::unique_ptr partitionWriter, + std::unique_ptr options, std::shared_ptr veloxPool) - : ShuffleWriter(numPartitions, partitionWriterCreator, options), - payloadPool_(std::make_shared(options_.memory_pool)), - veloxPool_(std::move(veloxPool)) { + : ShuffleWriter(numPartitions, std::move(partitionWriter), std::move(options)), veloxPool_(std::move(veloxPool)) { arenas_.resize(numPartitions); } arrow::Status init(); - arrow::Status initIpcWriteOptions(); - arrow::Status initPartitions(); arrow::Status initColumnTypes(const facebook::velox::RowVector& rv); @@ -250,8 +241,11 @@ class VeloxShuffleWriter final : public ShuffleWriter { arrow::Status splitComplexType(const facebook::velox::RowVector& rv); - arrow::Status - evictBuffers(uint32_t partitionId, uint32_t numRows, std::vector> buffers); + arrow::Status evictBuffers( + uint32_t partitionId, + uint32_t numRows, + std::vector> buffers, + bool reuseBuffers); arrow::Result>> assembleBuffers(uint32_t partitionId, bool reuseBuffers); @@ -313,13 +307,6 @@ class VeloxShuffleWriter final : public ShuffleWriter { arrow::Result partitionBufferSizeAfterShrink(uint32_t partitionId) const; - protected: - // Memory Pool used to track memory allocation of Arrow IPC payloads. - // The actual allocation is delegated to options_.memory_pool. - std::shared_ptr payloadPool_; - - std::shared_ptr partitionWriter_; - SplitState splitState_{kInit}; EvictState evictState_{kEvictable}; diff --git a/cpp/velox/tests/RuntimeTest.cc b/cpp/velox/tests/RuntimeTest.cc index c6af63b8e0c2..035ecce806b8 100644 --- a/cpp/velox/tests/RuntimeTest.cc +++ b/cpp/velox/tests/RuntimeTest.cc @@ -55,8 +55,8 @@ class DummyRuntime final : public Runtime { } std::shared_ptr createShuffleWriter( int numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, + std::unique_ptr partitionWriter, + std::unique_ptr options, MemoryManager* memoryManager) override { throw GlutenException("Not yet implemented"); } diff --git a/cpp/velox/tests/VeloxShuffleWriterTest.cc b/cpp/velox/tests/VeloxShuffleWriterTest.cc index 2c475efa202b..3e149ada4d89 100644 --- a/cpp/velox/tests/VeloxShuffleWriterTest.cc +++ b/cpp/velox/tests/VeloxShuffleWriterTest.cc @@ -18,8 +18,6 @@ #include #include -#include - #include "shuffle/LocalPartitionWriter.h" #include "shuffle/VeloxShuffleWriter.h" #include "shuffle/rss/CelebornPartitionWriter.h" @@ -74,16 +72,19 @@ static const auto kShuffleWriteTestParams = createShuffleTestParams(); TEST_P(SinglePartitioningShuffleWriter, single) { // Split 1 RowVector. { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); testShuffleWrite(*shuffleWriter, {inputVector1_}); } // Split > 1 RowVector. { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); testShuffleWrite(*shuffleWriter, {inputVector1_, inputVector2_, inputVector1_}); } // Split null RowVector. { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); auto vector = makeRowVector({ makeNullableFlatVector({std::nullopt}), @@ -93,6 +94,7 @@ TEST_P(SinglePartitioningShuffleWriter, single) { } // Other types. { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); auto vector = makeRowVector({ makeNullableFlatVector({std::nullopt, 1}), @@ -118,6 +120,7 @@ TEST_P(SinglePartitioningShuffleWriter, single) { } TEST_P(HashPartitioningShuffleWriter, hashPart1Vector) { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); auto vector = makeRowVector({ makeFlatVector({1, 2, 1, 2}), @@ -176,6 +179,7 @@ TEST_P(HashPartitioningShuffleWriter, hashPart1Vector) { } TEST_P(HashPartitioningShuffleWriter, hashPart1VectorComplexType) { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); std::vector children = { makeNullableFlatVector({std::nullopt, 1}), @@ -224,6 +228,7 @@ TEST_P(HashPartitioningShuffleWriter, hashPart1VectorComplexType) { } TEST_P(HashPartitioningShuffleWriter, hashPart3Vectors) { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); auto block1Pid1 = takeRows(inputVector1_, {0, 5, 6, 7, 9}); @@ -241,6 +246,7 @@ TEST_P(HashPartitioningShuffleWriter, hashPart3Vectors) { } TEST_P(RangePartitioningShuffleWriter, rangePartition) { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); auto block1Pid1 = takeRows(inputVector1_, {0, 2, 4, 6, 8}); @@ -258,6 +264,7 @@ TEST_P(RangePartitioningShuffleWriter, rangePartition) { } TEST_P(RoundRobinPartitioningShuffleWriter, roundRobin) { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); auto block1Pid1 = takeRows(inputVector1_, {0, 2, 4, 6, 8}); @@ -275,7 +282,8 @@ TEST_P(RoundRobinPartitioningShuffleWriter, roundRobin) { } TEST_P(RoundRobinPartitioningShuffleWriter, preAllocForceRealloc) { - shuffleWriterOptions_.buffer_realloc_threshold = 0; // Force re-alloc on buffer size changed. + ASSERT_NOT_OK(initShuffleWriterOptions()); + shuffleWriterOptions_->buffer_realloc_threshold = 0; // Force re-alloc on buffer size changed. auto shuffleWriter = createShuffleWriter(); // First spilt no null. @@ -320,7 +328,7 @@ TEST_P(RoundRobinPartitioningShuffleWriter, preAllocForceRealloc) { // Evict cached data first. ASSERT_NOT_OK(shuffleWriter->evictFixedSize(shuffleWriter->cachedPayloadSize(), &evicted)); // Set a large buffer size. - shuffleWriter->options().buffer_size = 100; + shuffleWriter->options()->buffer_size = 100; ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); // No data got evicted so the cached size is 0. ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); @@ -329,7 +337,8 @@ TEST_P(RoundRobinPartitioningShuffleWriter, preAllocForceRealloc) { } TEST_P(RoundRobinPartitioningShuffleWriter, preAllocForceReuse) { - shuffleWriterOptions_.buffer_realloc_threshold = 100; // Force re-alloc on buffer size changed. + ASSERT_NOT_OK(initShuffleWriterOptions()); + shuffleWriterOptions_->buffer_realloc_threshold = 100; // Force re-alloc on buffer size changed. auto shuffleWriter = createShuffleWriter(); // First spilt no null. @@ -360,6 +369,7 @@ TEST_P(RoundRobinPartitioningShuffleWriter, preAllocForceReuse) { } TEST_P(RoundRobinPartitioningShuffleWriter, spillVerifyResult) { + ASSERT_NOT_OK(initShuffleWriterOptions()); auto shuffleWriter = createShuffleWriter(); ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); @@ -397,9 +407,10 @@ TEST_P(RoundRobinPartitioningShuffleWriter, spillVerifyResult) { } TEST_F(VeloxShuffleWriterMemoryTest, memoryLeak) { + ASSERT_NOT_OK(initShuffleWriterOptions()); std::shared_ptr pool = std::make_shared(); - shuffleWriterOptions_.memory_pool = pool.get(); - shuffleWriterOptions_.buffer_size = 4; + shuffleWriterOptions_->memory_pool = pool.get(); + shuffleWriterOptions_->buffer_size = 4; auto shuffleWriter = createShuffleWriter(); @@ -415,9 +426,10 @@ TEST_F(VeloxShuffleWriterMemoryTest, memoryLeak) { } TEST_F(VeloxShuffleWriterMemoryTest, spillFailWithOutOfMemory) { + ASSERT_NOT_OK(initShuffleWriterOptions()); std::shared_ptr pool = std::make_shared(0); - shuffleWriterOptions_.memory_pool = pool.get(); - shuffleWriterOptions_.buffer_size = 4; + shuffleWriterOptions_->memory_pool = pool.get(); + shuffleWriterOptions_->buffer_size = 4; auto shuffleWriter = createShuffleWriter(); @@ -428,7 +440,8 @@ TEST_F(VeloxShuffleWriterMemoryTest, spillFailWithOutOfMemory) { } TEST_F(VeloxShuffleWriterMemoryTest, kInit) { - shuffleWriterOptions_.buffer_size = 4; + ASSERT_NOT_OK(initShuffleWriterOptions()); + shuffleWriterOptions_->buffer_size = 4; auto shuffleWriter = createShuffleWriter(); // Test spill all partition buffers. @@ -497,8 +510,9 @@ TEST_F(VeloxShuffleWriterMemoryTest, kInit) { } TEST_F(VeloxShuffleWriterMemoryTest, kInitSingle) { - shuffleWriterOptions_.partitioning = Partitioning::kSingle; - shuffleWriterOptions_.buffer_size = 4; + ASSERT_NOT_OK(initShuffleWriterOptions()); + shuffleWriterOptions_->partitioning = Partitioning::kSingle; + shuffleWriterOptions_->buffer_size = 4; auto shuffleWriter = createShuffleWriter(); { @@ -518,9 +532,10 @@ TEST_F(VeloxShuffleWriterMemoryTest, kInitSingle) { } TEST_F(VeloxShuffleWriterMemoryTest, kSplit) { - shuffleWriterOptions_.buffer_size = 4; - auto pool = SelfEvictedMemoryPool(shuffleWriterOptions_.memory_pool); - shuffleWriterOptions_.memory_pool = &pool; + ASSERT_NOT_OK(initShuffleWriterOptions()); + shuffleWriterOptions_->buffer_size = 4; + auto pool = SelfEvictedMemoryPool(shuffleWriterOptions_->memory_pool); + shuffleWriterOptions_->memory_pool = &pool; auto shuffleWriter = createShuffleWriter(); pool.setEvictable(shuffleWriter.get()); @@ -540,9 +555,11 @@ TEST_F(VeloxShuffleWriterMemoryTest, kSplit) { } TEST_F(VeloxShuffleWriterMemoryTest, kSplitSingle) { - shuffleWriterOptions_.partitioning = Partitioning::kSingle; - auto pool = SelfEvictedMemoryPool(shuffleWriterOptions_.memory_pool); - shuffleWriterOptions_.memory_pool = &pool; + ASSERT_NOT_OK(initShuffleWriterOptions()); + shuffleWriterOptions_->partitioning = Partitioning::kSingle; + auto pool = SelfEvictedMemoryPool(shuffleWriterOptions_->memory_pool, false); + shuffleWriterOptions_->memory_pool = &pool; + auto shuffleWriter = createShuffleWriter(); pool.setEvictable(shuffleWriter.get()); @@ -551,18 +568,19 @@ TEST_F(VeloxShuffleWriterMemoryTest, kSplitSingle) { // Trigger spill for the next split. ASSERT_TRUE(pool.checkEvict( - shuffleWriter->cachedPayloadSize() * 2, [&] { ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); })); + shuffleWriter->cachedPayloadSize(), [&] { ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); })); ASSERT_NOT_OK(shuffleWriter->stop()); } TEST_F(VeloxShuffleWriterMemoryTest, kStop) { - auto delegated = shuffleWriterOptions_.memory_pool; for (const auto partitioning : {Partitioning::kSingle, Partitioning::kRoundRobin}) { - shuffleWriterOptions_.partitioning = partitioning; - shuffleWriterOptions_.buffer_size = 4; - auto pool = SelfEvictedMemoryPool(delegated); - shuffleWriterOptions_.memory_pool = &pool; + ASSERT_NOT_OK(initShuffleWriterOptions()); + auto delegated = shuffleWriterOptions_->memory_pool; + shuffleWriterOptions_->partitioning = partitioning; + shuffleWriterOptions_->buffer_size = 4; + auto pool = SelfEvictedMemoryPool(delegated, false); + shuffleWriterOptions_->memory_pool = &pool; auto shuffleWriter = createShuffleWriter(); pool.setEvictable(shuffleWriter.get()); @@ -579,11 +597,12 @@ TEST_F(VeloxShuffleWriterMemoryTest, kStop) { } } -TEST_F(VeloxShuffleWriterMemoryTest, kUnevictable) { - auto delegated = shuffleWriterOptions_.memory_pool; - shuffleWriterOptions_.buffer_size = 4; +TEST_F(VeloxShuffleWriterMemoryTest, evictPartitionBuffers) { + ASSERT_NOT_OK(initShuffleWriterOptions()); + auto delegated = shuffleWriterOptions_->memory_pool; + shuffleWriterOptions_->buffer_size = 4; auto pool = SelfEvictedMemoryPool(delegated); - shuffleWriterOptions_.memory_pool = &pool; + shuffleWriterOptions_->memory_pool = &pool; auto shuffleWriter = createShuffleWriter(); pool.setEvictable(shuffleWriter.get()); @@ -597,17 +616,17 @@ TEST_F(VeloxShuffleWriterMemoryTest, kUnevictable) { ASSERT_GT(shuffleWriter->partitionBufferSize(), 0); // Set limited capacity. pool.setCapacity(0); - // Evict again. Because no cached payload to evict, it will try to compress and evict partition buffers. - // Throws OOM during allocating compression buffers. - auto status = shuffleWriter->evictFixedSize(shuffleWriter->partitionBufferSize(), &evicted); - ASSERT_TRUE(status.IsOutOfMemory()); + // Evict again. Because no cached payload to evict, it will try to evict all partition buffers. + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(shuffleWriter->partitionBufferSize(), &evicted)); + ASSERT_EQ(shuffleWriter->partitionBufferSize(), 0); } TEST_F(VeloxShuffleWriterMemoryTest, kUnevictableSingle) { - auto delegated = shuffleWriterOptions_.memory_pool; - shuffleWriterOptions_.partitioning = Partitioning::kSingle; + ASSERT_NOT_OK(initShuffleWriterOptions()); + auto delegated = shuffleWriterOptions_->memory_pool; + shuffleWriterOptions_->partitioning = Partitioning::kSingle; auto pool = SelfEvictedMemoryPool(delegated); - shuffleWriterOptions_.memory_pool = &pool; + shuffleWriterOptions_->memory_pool = &pool; auto shuffleWriter = createShuffleWriter(); pool.setEvictable(shuffleWriter.get()); @@ -626,10 +645,11 @@ TEST_F(VeloxShuffleWriterMemoryTest, kUnevictableSingle) { } TEST_F(VeloxShuffleWriterMemoryTest, resizeBinaryBufferTriggerSpill) { - shuffleWriterOptions_.buffer_realloc_threshold = 1; - auto delegated = shuffleWriterOptions_.memory_pool; + ASSERT_NOT_OK(initShuffleWriterOptions()); + shuffleWriterOptions_->buffer_realloc_threshold = 1; + auto delegated = shuffleWriterOptions_->memory_pool; auto pool = SelfEvictedMemoryPool(delegated, false); - shuffleWriterOptions_.memory_pool = &pool; + shuffleWriterOptions_->memory_pool = &pool; auto shuffleWriter = createShuffleWriter(); pool.setEvictable(shuffleWriter.get()); diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h index 812eb319c760..ef1cc6a1d884 100644 --- a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -45,6 +45,19 @@ std::string makeString(uint32_t length) { } return res; } + +std::unique_ptr createPartitionWriter( + PartitionWriterType partitionWriterType, + uint32_t numPartitions, + const std::string& dataFile, + const std::vector& localDirs, + ShuffleWriterOptions* options) { + if (partitionWriterType == PartitionWriterType::kCeleborn) { + auto rssClient = std::make_unique(dataFile); + return std::make_unique(numPartitions, options, std::move(rssClient)); + } + return std::make_unique(numPartitions, dataFile, localDirs, options); +} } // namespace struct ShuffleTestParams { @@ -61,13 +74,17 @@ struct ShuffleTestParams { }; class VeloxShuffleWriterTestBase : public facebook::velox::test::VectorTestBase { + public: + virtual arrow::Status initShuffleWriterOptions() { + shuffleWriterOptions_ = std::make_unique(); + shuffleWriterOptions_->compression_threshold = 0; + shuffleWriterOptions_->memory_pool = defaultArrowMemoryPool().get(); + RETURN_NOT_OK(setLocalDirsAndDataFile()); + return arrow::Status::OK(); + } + protected: void setUp() { - shuffleWriterOptions_ = ShuffleWriterOptions::defaults(); - shuffleWriterOptions_.compression_threshold = 0; - shuffleWriterOptions_.memory_pool = defaultArrowMemoryPool().get(); - GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFile()); - // Set up test data. children1_ = { makeNullableFlatVector({1, 2, 3, std::nullopt, 4, std::nullopt, 5, 6, std::nullopt, 7}), @@ -168,31 +185,29 @@ class VeloxShuffleWriterTestBase : public facebook::velox::test::VectorTestBase // Create multiple local dirs and join with comma. arrow::Status setLocalDirsAndDataFile() { - auto& localDirs = shuffleWriterOptions_.local_dirs; static const std::string kTestLocalDirsPrefix = "columnar-shuffle-test-"; // Create first tmp dir and create data file. // To prevent tmpDirs from being deleted in the dtor, we need to store them. tmpDirs_.emplace_back(); ARROW_ASSIGN_OR_RAISE(tmpDirs_.back(), arrow::internal::TemporaryDir::Make(kTestLocalDirsPrefix)) - ARROW_ASSIGN_OR_RAISE(shuffleWriterOptions_.data_file, createTempShuffleFile(tmpDirs_.back()->path().ToString())); - localDirs += tmpDirs_.back()->path().ToString(); - localDirs.push_back(','); + ARROW_ASSIGN_OR_RAISE(dataFile_, createTempShuffleFile(tmpDirs_.back()->path().ToString())); + localDirs_.push_back(tmpDirs_.back()->path().ToString()); // Create second tmp dir. tmpDirs_.emplace_back(); ARROW_ASSIGN_OR_RAISE(tmpDirs_.back(), arrow::internal::TemporaryDir::Make(kTestLocalDirsPrefix)) - localDirs += tmpDirs_.back()->path().ToString(); + localDirs_.push_back(tmpDirs_.back()->path().ToString()); return arrow::Status::OK(); } virtual std::shared_ptr createShuffleWriter() = 0; - ShuffleWriterOptions shuffleWriterOptions_; - - std::shared_ptr partitionWriterCreator_; + std::unique_ptr shuffleWriterOptions_; std::vector> tmpDirs_; + std::string dataFile_; + std::vector localDirs_; std::vector children1_; std::vector children2_; @@ -210,20 +225,22 @@ class VeloxShuffleWriterTestBase : public facebook::velox::test::VectorTestBase }; class VeloxShuffleWriterTest : public ::testing::TestWithParam, public VeloxShuffleWriterTestBase { - protected: - virtual void SetUp() override { - VeloxShuffleWriterTestBase::setUp(); + public: + arrow::Status initShuffleWriterOptions() override { + RETURN_NOT_OK(VeloxShuffleWriterTestBase::initShuffleWriterOptions()); ShuffleTestParams params = GetParam(); if (params.partition_writer_type == PartitionWriterType::kCeleborn) { - partitionWriterCreator_ = std::make_shared( - std::make_shared(shuffleWriterOptions_.data_file)); - shuffleWriterOptions_.partition_writer_type = kCeleborn; - } else { - partitionWriterCreator_ = std::make_shared(); + shuffleWriterOptions_->partition_writer_type = kCeleborn; } - shuffleWriterOptions_.compression_type = params.compression_type; - shuffleWriterOptions_.compression_mode = params.compression_mode; + shuffleWriterOptions_->compression_type = params.compression_type; + shuffleWriterOptions_->compression_mode = params.compression_mode; + return arrow::Status::OK(); + } + + protected: + virtual void SetUp() override { + VeloxShuffleWriterTestBase::setUp(); } void TearDown() override { @@ -247,10 +264,17 @@ class VeloxShuffleWriterTest : public ::testing::TestWithParam schema, std::vector& vectors) { + void getRowVectors( + ShuffleWriterOptions* writerOptions, + std::shared_ptr schema, + std::vector& vectors) { ShuffleReaderOptions options; - options.compression_type = shuffleWriterOptions_.compression_type; - auto reader = std::make_shared(schema, options, defaultArrowMemoryPool().get(), pool_); + options.compression_type = writerOptions->compression_type; + auto codec = createArrowIpcCodec(options.compression_type, CodecBackend::NONE); + auto rowType = facebook::velox::asRowType(gluten::fromArrowSchema(schema)); + auto deserializerFactory = std::make_unique( + schema, std::move(codec), rowType, defaultArrowMemoryPool().get(), pool_); + auto reader = std::make_shared(std::move(deserializerFactory)); auto iter = reader->readStream(file_); while (iter->hasNext()) { auto vector = std::dynamic_pointer_cast(iter->next())->getRowVector(); @@ -266,20 +290,18 @@ class SinglePartitioningShuffleWriter : public VeloxShuffleWriterTest { void testShuffleWrite(VeloxShuffleWriter& shuffleWriter, std::vector vectors) { for (auto& vector : vectors) { ASSERT_NOT_OK(splitRowVector(shuffleWriter, vector)); - // No partition buffers for single partitioner. - ASSERT_EQ(shuffleWriter.partitionBufferSize(), 0); } ASSERT_NOT_OK(shuffleWriter.stop()); // verify data file - checkFileExists(shuffleWriter.dataFile()); + checkFileExists(dataFile_); // verify output temporary files const auto& lengths = shuffleWriter.partitionLengths(); ASSERT_EQ(lengths.size(), 1); auto schema = getArrowSchema(vectors[0]); std::vector deserializedVectors; - setReadableFile(shuffleWriter.dataFile()); - getRowVectors(schema, deserializedVectors); + setReadableFile(dataFile_); + getRowVectors(shuffleWriter.options(), schema, deserializedVectors); ASSERT_EQ(deserializedVectors.size(), vectors.size()); for (int32_t i = 0; i < deserializedVectors.size(); i++) { @@ -288,10 +310,19 @@ class SinglePartitioningShuffleWriter : public VeloxShuffleWriterTest { } std::shared_ptr createShuffleWriter() override { - shuffleWriterOptions_.buffer_size = 10; - shuffleWriterOptions_.partitioning = Partitioning::kSingle; + shuffleWriterOptions_->buffer_size = 10; + shuffleWriterOptions_->partitioning = Partitioning::kSingle; + static const uint32_t kNumPartitions = 1; + auto partitionWriter = createPartitionWriter( + shuffleWriterOptions_->partition_writer_type, + kNumPartitions, + dataFile_, + localDirs_, + shuffleWriterOptions_.get()); GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + auto shuffleWriter, + VeloxShuffleWriter::create( + kNumPartitions, std::move(partitionWriter), std::move(shuffleWriterOptions_), pool_)); return shuffleWriter; } }; @@ -305,20 +336,20 @@ class MultiplePartitioningShuffleWriter : public VeloxShuffleWriterTest { std::vector> expectedVectors) { /* blockId = pid, rowVector in block */ ASSERT_NOT_OK(shuffleWriter.stop()); // verify data file - checkFileExists(shuffleWriter.dataFile()); + checkFileExists(dataFile_); // verify output temporary files const auto& lengths = shuffleWriter.partitionLengths(); ASSERT_EQ(lengths.size(), expectPartitionLength); int64_t lengthSum = std::accumulate(lengths.begin(), lengths.end(), 0); auto schema = toArrowSchema(dataType, pool()); - setReadableFile(shuffleWriter.dataFile()); + setReadableFile(dataFile_); ASSERT_EQ(*file_->GetSize(), lengthSum); for (int32_t i = 0; i < expectPartitionLength; i++) { if (expectedVectors[i].size() == 0) { ASSERT_EQ(lengths[i], 0); } else { std::vector deserializedVectors; - getRowVectors(schema, deserializedVectors); + getRowVectors(shuffleWriter.options(), schema, deserializedVectors); if (i != 0) { ASSERT_NOT_OK(file_->Advance(lengths[i - 1])); } @@ -355,10 +386,19 @@ class HashPartitioningShuffleWriter : public MultiplePartitioningShuffleWriter { } std::shared_ptr createShuffleWriter() override { - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.partitioning = Partitioning::kHash; + shuffleWriterOptions_->buffer_size = 4; + shuffleWriterOptions_->partitioning = Partitioning::kHash; + static const uint32_t kNumPartitions = 2; + auto partitionWriter = createPartitionWriter( + shuffleWriterOptions_->partition_writer_type, + kNumPartitions, + dataFile_, + localDirs_, + shuffleWriterOptions_.get()); GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + auto shuffleWriter, + VeloxShuffleWriter::create( + kNumPartitions, std::move(partitionWriter), std::move(shuffleWriterOptions_), pool_)); return shuffleWriter; } @@ -385,10 +425,19 @@ class RangePartitioningShuffleWriter : public MultiplePartitioningShuffleWriter } std::shared_ptr createShuffleWriter() override { - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.partitioning = Partitioning::kRange; + shuffleWriterOptions_->buffer_size = 4; + shuffleWriterOptions_->partitioning = Partitioning::kRange; + static const uint32_t kNumPartitions = 2; + auto partitionWriter = createPartitionWriter( + shuffleWriterOptions_->partition_writer_type, + kNumPartitions, + dataFile_, + localDirs_, + shuffleWriterOptions_.get()); GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + auto shuffleWriter, + VeloxShuffleWriter::create( + kNumPartitions, std::move(partitionWriter), std::move(shuffleWriterOptions_), pool_)); return shuffleWriter; } @@ -411,9 +460,18 @@ class RangePartitioningShuffleWriter : public MultiplePartitioningShuffleWriter class RoundRobinPartitioningShuffleWriter : public MultiplePartitioningShuffleWriter { protected: std::shared_ptr createShuffleWriter() override { - shuffleWriterOptions_.buffer_size = 4; + shuffleWriterOptions_->buffer_size = 4; + static const uint32_t kNumPartitions = 2; + auto partitionWriter = createPartitionWriter( + shuffleWriterOptions_->partition_writer_type, + kNumPartitions, + dataFile_, + localDirs_, + shuffleWriterOptions_.get()); GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + auto shuffleWriter, + VeloxShuffleWriter::create( + kNumPartitions, std::move(partitionWriter), std::move(shuffleWriterOptions_), pool_)); return shuffleWriter; } }; @@ -422,14 +480,18 @@ class VeloxShuffleWriterMemoryTest : public VeloxShuffleWriterTestBase, public t protected: void SetUp() override { VeloxShuffleWriterTestBase::setUp(); - // Use LocalPartitionWriter to test OOM and spill. - partitionWriterCreator_ = std::make_shared(); } std::shared_ptr createShuffleWriter(uint32_t numPartitions) { + auto partitionWriter = createPartitionWriter( + shuffleWriterOptions_->partition_writer_type, + numPartitions, + dataFile_, + localDirs_, + shuffleWriterOptions_.get()); GLUTEN_ASSIGN_OR_THROW( auto shuffleWriter, - VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + VeloxShuffleWriter::create(numPartitions, std::move(partitionWriter), std::move(shuffleWriterOptions_), pool_)); return shuffleWriter; } diff --git a/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala b/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala index 563e143d9c24..171c39a9f9d8 100644 --- a/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala +++ b/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala @@ -145,9 +145,14 @@ private class ColumnarBatchSerializerInstance( // E.g. A Velox limit operator may suddenly drop the input stream after emitting enough // rows. In the case DeserializationStream#close() will not be called. Spark doesn't // call close() either. So we should handle the case especially. - TaskResources.addRecycler(s"ShuffleReaderDeserializationStream_${wrappedOut.getId}", 50) { - this.close() - } + // + // FIXME: Below code can cause heavy GC, especially when AQE is on with + // large number of coalesced partitions. + // Assuming 8000 shuffle write tasks, and 100 reducer tasks, 8 tasks per executor. + // There will be ~ 8000 / 100 * 8000 * 8 = 5120000 objects. + // TaskResources.addRecycler(s"ShuffleReaderDeserializationStream_${wrappedOut.getId}", 50) { + // this.close() + // } override def asIterator: Iterator[Any] = { // This method is never called by shuffle code. From 30c5e8749e6d7bbb5f1670baf3ce7e82c04193bb Mon Sep 17 00:00:00 2001 From: "Ma, Rong" Date: Wed, 13 Dec 2023 16:01:21 +0800 Subject: [PATCH 6/7] cache partition buffer as payload in the last evict --- cpp/core/shuffle/BlockPayload.cc | 40 +++++++- cpp/core/shuffle/BlockPayload.h | 23 ++++- cpp/core/shuffle/LocalPartitionWriter.cc | 81 +++++------------ cpp/core/shuffle/LocalPartitionWriter.h | 4 - cpp/core/shuffle/PartitionWriter.h | 2 +- cpp/core/shuffle/Utils.cc | 5 + cpp/core/shuffle/Utils.h | 2 + .../shuffle/rss/CelebornPartitionWriter.cc | 9 +- .../shuffle/rss/CelebornPartitionWriter.h | 2 - cpp/velox/shuffle/VeloxShuffleWriter.cc | 91 +++++++++++-------- cpp/velox/shuffle/VeloxShuffleWriter.h | 2 + 11 files changed, 150 insertions(+), 111 deletions(-) diff --git a/cpp/core/shuffle/BlockPayload.cc b/cpp/core/shuffle/BlockPayload.cc index 92813b129dce..7fa1047d545c 100644 --- a/cpp/core/shuffle/BlockPayload.cc +++ b/cpp/core/shuffle/BlockPayload.cc @@ -17,16 +17,19 @@ #include "shuffle/BlockPayload.h" +namespace { +static const gluten::BlockPayload::Type kCompressedType = gluten::BlockPayload::kCompressed; +} namespace gluten { arrow::Result> BlockPayload::fromBuffers( + BlockPayload::Type payloadType, uint32_t numRows, std::vector> buffers, - ShuffleWriterOptions* options, arrow::MemoryPool* pool, arrow::util::Codec* codec, bool reuseBuffers) { - if (codec && numRows >= options->compression_threshold) { + if (payloadType == BlockPayload::Type::kCompressed) { // Compress. // Compressed buffer layout: | buffer1 compressedLength | buffer1 uncompressedLength | buffer1 | ... auto metadataLength = sizeof(int64_t) * 2 * buffers.size(); @@ -68,6 +71,9 @@ arrow::Result> BlockPayload::fromBuffers( } return std::make_unique(Type::kUncompressed, numRows, std::move(copies)); } + if (payloadType == Type::kToBeCompressed) { + return std::make_unique(Type::kUncompressed, numRows, std::move(buffers), pool, codec); + } return std::make_unique(Type::kUncompressed, numRows, std::move(buffers)); } @@ -193,4 +199,34 @@ arrow::Result> BlockPayload::readCompressedBuffer compressedLength, compressed->data(), uncompressedLength, const_cast(output->data()))); return output; } + +arrow::Status CompressibleBlockPayload::serialize(arrow::io::OutputStream* outputStream) { + RETURN_NOT_OK(outputStream->Write(&kCompressedType, sizeof(Type))); + RETURN_NOT_OK(outputStream->Write(&numRows_, sizeof(uint32_t))); + auto metadataLength = sizeof(int64_t) * 2 * buffers_.size(); + int64_t totalCompressedLength = + std::accumulate(buffers_.begin(), buffers_.end(), 0LL, [&](auto sum, const auto& buffer) { + if (!buffer) { + return sum; + } + return sum + codec_->MaxCompressedLen(buffer->size(), buffer->data()); + }); + ARROW_ASSIGN_OR_RAISE( + std::shared_ptr compressed, + arrow::AllocateResizableBuffer(metadataLength + totalCompressedLength, pool_)); + auto output = compressed->mutable_data(); + + // Compress buffers one by one. + for (auto& buffer : buffers_) { + auto availableLength = compressed->size() - (output - compressed->data()); + RETURN_NOT_OK(compressBuffer(buffer, output, availableLength, codec_)); + } + + int64_t actualLength = output - compressed->data(); + ARROW_RETURN_IF(actualLength < 0, arrow::Status::Invalid("Writing compressed buffer out of bound.")); + RETURN_NOT_OK(compressed->Resize(actualLength)); + + RETURN_NOT_OK(outputStream->Write(std::move(compressed))); + return arrow::Status::OK(); +} } // namespace gluten diff --git a/cpp/core/shuffle/BlockPayload.h b/cpp/core/shuffle/BlockPayload.h index 572b7a3b0d1e..8240a492d711 100644 --- a/cpp/core/shuffle/BlockPayload.h +++ b/cpp/core/shuffle/BlockPayload.h @@ -77,15 +77,15 @@ arrow::Status compressBuffer( class BlockPayload : public Payload { public: - enum Type : int32_t { kCompressed, kUncompressed }; + enum Type : int32_t { kCompressed, kUncompressed, kToBeCompressed }; BlockPayload(BlockPayload::Type type, uint32_t numRows, std::vector> buffers) : type_(type), numRows_(numRows), buffers_(std::move(buffers)) {} static arrow::Result> fromBuffers( + BlockPayload::Type payloadType, uint32_t numRows, std::vector> buffers, - ShuffleWriterOptions* options, arrow::MemoryPool* pool, arrow::util::Codec* codec, bool reuseBuffers); @@ -145,10 +145,27 @@ class BlockPayload : public Payload { return arrow::Status::OK(); } - private: + protected: Type type_; uint32_t numRows_; std::vector> buffers_; }; +class CompressibleBlockPayload : public BlockPayload { + public: + CompressibleBlockPayload( + BlockPayload::Type type, + uint32_t numRows, + std::vector> buffers, + arrow::MemoryPool* pool, + arrow::util::Codec* codec) + : BlockPayload(type, numRows, std::move(buffers)), pool_(pool), codec_(codec) {} + + arrow::Status serialize(arrow::io::OutputStream* outputStream); + + private: + arrow::MemoryPool* pool_; + arrow::util::Codec* codec_; +}; + } // namespace gluten \ No newline at end of file diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index bdfa00e9bac2..3ce72aeaa261 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -103,8 +103,15 @@ class CacheEvictor final : public LocalPartitionWriter::LocalEvictor { ScopedTimer timer(evictTime_); auto payloads = std::move(partitionCachedPayload_[partitionId]); partitionCachedPayload_.erase(partitionId); + int32_t numPayloads = 0; + + ARROW_ASSIGN_OR_RAISE(auto startInFinalFile, os->Tell()); for (auto& payload : payloads) { RETURN_NOT_OK(payload->serialize(os)); + ARROW_ASSIGN_OR_RAISE(auto spillPos, os->Tell()); + DEBUG_OUT << "Partition " << partitionId << " cached payload " << numPayloads++ << " of bytes " + << spillPos - startInFinalFile << std::endl; + startInFinalFile = spillPos; } return arrow::Status::OK(); } @@ -198,7 +205,6 @@ arrow::Status LocalPartitionWriter::clearResource() { // When buffered_write = true, dataFileOs_->Close doesn't release underlying buffer. dataFileOs_.reset(); spills_.clear(); - cachedPartitionBuffers_.clear(); return arrow::Status::OK(); } @@ -252,35 +258,20 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { writeTimer.start(); int64_t endInFinalFile = 0; - auto cachedPartitionBuffersIter = cachedPartitionBuffers_.begin(); // Iterator over pid. for (auto pid = 0; pid < numPartitions_; ++pid) { // Record start offset. auto startInFinalFile = endInFinalFile; // Iterator over all spilled files. RETURN_NOT_OK(mergeSpills(pid)); +#ifdef GLUTEN_PRINT_DEBUG + ARROW_ASSIGN_OR_RAISE(auto spillPos, dataFileOs_->Tell()); + DEBUG_OUT << "Partition " << pid << " spilled from file of bytes " << spillPos - startInFinalFile << std::endl; +#endif // Write cached batches. if (evictor_) { RETURN_NOT_OK(evictor_->flushCachedPayloads(pid, dataFileOs_.get())); } - // Compress and write the last payload. - // Stop the timer to prevent counting the compression time into write time. - if (cachedPartitionBuffersIter != cachedPartitionBuffers_.end() && - std::get<0>(*cachedPartitionBuffersIter) == pid) { - writeTimer.stop(); - ARROW_ASSIGN_OR_RAISE( - auto payload, - BlockPayload::fromBuffers( - std::get<1>(*cachedPartitionBuffersIter), - std::move(std::get<2>(*cachedPartitionBuffersIter)), - options_, - payloadPool_.get(), - codec_ ? codec_.get() : nullptr, - false)); - writeTimer.start(); - RETURN_NOT_OK(payload->serialize(dataFileOs_.get())); - cachedPartitionBuffersIter++; - } ARROW_ASSIGN_OR_RAISE(endInFinalFile, dataFileOs_->Tell()); if (endInFinalFile != startInFinalFile && options_->write_eos) { // Write EOS if any payload written. @@ -302,10 +293,6 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { } RETURN_NOT_OK(fs_->DeleteFile(spill->spilledFile)); } - // Check if all partition buffers are merged. - ARROW_RETURN_IF( - cachedPartitionBuffersIter != cachedPartitionBuffers_.end(), - arrow::Status::Invalid("Not all partition buffers are merged.")); writeTimer.stop(); writeTime_ = writeTimer.realTimeUsed(); @@ -352,21 +339,21 @@ arrow::Status LocalPartitionWriter::evict( bool reuseBuffers, Evictor::Type evictType) { rawPartitionLengths_[partitionId] += getBufferSize(buffers); - if (evictType == Evictor::Type::kStop) { - cachedPartitionBuffers_.emplace_back(partitionId, numRows, std::move(buffers)); - } else { - ARROW_ASSIGN_OR_RAISE( - auto payload, - BlockPayload::fromBuffers( - numRows, - std::move(buffers), - options_, - payloadPool_.get(), - (codec_ && evictType == Evictor::kCache) ? codec_.get() : nullptr, - reuseBuffers)); - RETURN_NOT_OK(requestEvict(evictType)); - RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); + auto payloadType = (codec_ && evictType != Evictor::kFlush && numRows >= options_->compression_threshold) + ? BlockPayload::Type::kCompressed + : BlockPayload::Type::kUncompressed; + if (evictType == Evictor::kStop) { + evictType = Evictor::kCache; + if (payloadType == BlockPayload::Type::kCompressed) { + payloadType = BlockPayload::Type::kToBeCompressed; + } } + ARROW_ASSIGN_OR_RAISE( + auto payload, + BlockPayload::fromBuffers( + payloadType, numRows, std::move(buffers), payloadPool_.get(), codec_ ? codec_.get() : nullptr, reuseBuffers)); + RETURN_NOT_OK(requestEvict(evictType)); + RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); return arrow::Status::OK(); } @@ -380,22 +367,4 @@ arrow::Status LocalPartitionWriter::populateMetrics(ShuffleWriterMetrics* metric metrics->rawPartitionLengths = std::move(rawPartitionLengths_); return arrow::Status::OK(); } - -arrow::Status LocalPartitionWriter::evictFixedSize(int64_t size, int64_t* actual) { - auto beforeShrink = options_->memory_pool->bytes_allocated(); - for (auto& item : cachedPartitionBuffers_) { - auto& buffers = std::get<2>(item); - for (auto& buffer : buffers) { - if (!buffer) { - continue; - } - if (auto parent = std::dynamic_pointer_cast(buffer->parent())) { - RETURN_NOT_OK(parent->Resize(buffer->size())); - } - } - } - *actual = beforeShrink - options_->memory_pool->bytes_allocated(); - return arrow::Status::OK(); -} - } // namespace gluten diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index d30f2a21b479..78dd32454eb8 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -80,8 +80,6 @@ class LocalPartitionWriter : public PartitionWriter { /// it will shrink partition buffers to free more memory. arrow::Status stop(ShuffleWriterMetrics* metrics) override; - arrow::Status evictFixedSize(int64_t size, int64_t* actual) override; - class LocalEvictor; private: @@ -116,7 +114,5 @@ class LocalPartitionWriter : public PartitionWriter { int64_t totalBytesWritten_{0}; std::vector partitionLengths_; std::vector rawPartitionLengths_; - // Partition id, num rows, partition buffers. - std::vector>>> cachedPartitionBuffers_; }; } // namespace gluten diff --git a/cpp/core/shuffle/PartitionWriter.h b/cpp/core/shuffle/PartitionWriter.h index 81507e78d689..e349a1932118 100644 --- a/cpp/core/shuffle/PartitionWriter.h +++ b/cpp/core/shuffle/PartitionWriter.h @@ -52,7 +52,7 @@ class Evictor { int64_t evictTime_{0}; }; -class PartitionWriter : public Evictable { +class PartitionWriter { public: PartitionWriter(uint32_t numPartitions, ShuffleWriterOptions* options) : numPartitions_(numPartitions), options_(options) { diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index 17cac38c8227..9903503bd0e2 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -332,3 +332,8 @@ arrow::Status gluten::writeEos(arrow::io::OutputStream* os, int64_t* bytes) { *bytes = kSizeOfEos; return arrow::Status::OK(); } + +std::shared_ptr gluten::zeroLengthNullBuffer() { + static std::shared_ptr kNullBuffer = std::make_shared(nullptr, 0); + return kNullBuffer; +} diff --git a/cpp/core/shuffle/Utils.h b/cpp/core/shuffle/Utils.h index 588f1d7015aa..4d157c6e7658 100644 --- a/cpp/core/shuffle/Utils.h +++ b/cpp/core/shuffle/Utils.h @@ -75,4 +75,6 @@ arrow::Result> makeUncompressedRecordBatch( const std::shared_ptr writeSchema, arrow::MemoryPool* pool); +std::shared_ptr zeroLengthNullBuffer(); + } // namespace gluten diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc index 33cbf93c8886..1f60f2fbd078 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc @@ -84,16 +84,13 @@ arrow::Status CelebornPartitionWriter::evict( Evictor::Type evictType) { rawPartitionLengths_[partitionId] += getBufferSize(buffers); ScopedTimer timer(evictTime_); + auto payloadType = (codec_ && numRows >= options_->compression_threshold) ? BlockPayload::Type::kCompressed + : BlockPayload::Type::kUncompressed; ARROW_ASSIGN_OR_RAISE( auto payload, BlockPayload::fromBuffers( - numRows, std::move(buffers), options_, payloadPool_.get(), codec_ ? codec_.get() : nullptr, false)); + payloadType, numRows, std::move(buffers), payloadPool_.get(), codec_ ? codec_.get() : nullptr, false)); RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); return arrow::Status::OK(); } - -arrow::Status CelebornPartitionWriter::evictFixedSize(int64_t size, int64_t* actual) { - *actual = 0; - return arrow::Status::OK(); -} } // namespace gluten diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.h b/cpp/core/shuffle/rss/CelebornPartitionWriter.h index 4e7830111daa..21d154313efd 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.h +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.h @@ -46,8 +46,6 @@ class CelebornPartitionWriter final : public RemotePartitionWriter { arrow::Status stop(ShuffleWriterMetrics* metrics) override; - arrow::Status evictFixedSize(int64_t size, int64_t* actual) override; - private: void init(); diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.cc b/cpp/velox/shuffle/VeloxShuffleWriter.cc index e0a3595ace7f..d4351aac5f46 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxShuffleWriter.cc @@ -856,6 +856,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel complexColumnIndices_.push_back(i); complexNames.emplace_back(veloxColumnTypes_[i]->name()); complexChildrens.emplace_back(veloxColumnTypes_[i]); + hasComplexType_ = true; } break; default: simpleColumnIndices_.push_back(i); @@ -1057,8 +1058,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel std::vector> arrays(numFields); std::vector> allBuffers; // One column should have 2 buffers at least, string column has 3 column buffers. - allBuffers.reserve(fixedWidthColumnCount_ * 2 + binaryColumnIndices_.size() * 3); - bool hasComplexType = false; + allBuffers.reserve(fixedWidthColumnCount_ * 2 + binaryColumnIndices_.size() * 3 + hasComplexType_); for (int i = 0; i < numFields; ++i) { switch (arrowColumnTypes_[i]->id()) { case arrow::BinaryType::type_id: @@ -1067,20 +1067,41 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel auto& binaryBuf = partitionBinaryAddrs_[binaryIdx][partitionId]; // validity buffer if (buffers[kValidityBufferIndex] != nullptr) { - allBuffers.push_back( - arrow::SliceBuffer(buffers[kValidityBufferIndex], 0, arrow::bit_util::BytesForBits(numRows))); + auto validityBufferSize = arrow::bit_util::BytesForBits(numRows); + if (reuseBuffers) { + allBuffers.push_back( + arrow::SliceBuffer(buffers[kValidityBufferIndex], 0, arrow::bit_util::BytesForBits(numRows))); + } else { + RETURN_NOT_OK(buffers[kValidityBufferIndex]->Resize(validityBufferSize, true)); + allBuffers.push_back(std::move(buffers[kValidityBufferIndex])); + } } else { allBuffers.push_back(nullptr); } - // offset buffer + // Length buffer. + auto lengthBufferSize = numRows * kSizeOfBinaryArrayLengthBuffer; ARROW_RETURN_IF( !buffers[kBinaryLengthBufferIndex], arrow::Status::Invalid("Offset buffer of binary array is null.")); - allBuffers.push_back( - arrow::SliceBuffer(buffers[kBinaryLengthBufferIndex], 0, numRows * kSizeOfBinaryArrayLengthBuffer)); + if (reuseBuffers) { + allBuffers.push_back(arrow::SliceBuffer(buffers[kBinaryLengthBufferIndex], 0, lengthBufferSize)); + } else { + RETURN_NOT_OK(buffers[kBinaryLengthBufferIndex]->Resize(lengthBufferSize, true)); + allBuffers.push_back(std::move(buffers[kBinaryLengthBufferIndex])); + } + + // Value buffer. + auto valueBufferSize = binaryBuf.valueOffset; ARROW_RETURN_IF( !buffers[kBinaryValueBufferIndex], arrow::Status::Invalid("Value buffer of binary array is null.")); - // value buffer - allBuffers.push_back(arrow::SliceBuffer(buffers[kBinaryValueBufferIndex], 0, binaryBuf.valueOffset)); + if (reuseBuffers) { + allBuffers.push_back(arrow::SliceBuffer(buffers[kBinaryValueBufferIndex], 0, valueBufferSize)); + } else if (valueBufferSize > 0) { + RETURN_NOT_OK(buffers[kBinaryValueBufferIndex]->Resize(valueBufferSize, true)); + allBuffers.push_back(std::move(buffers[kBinaryValueBufferIndex])); + } else { + // Binary value buffer size can be 0, in which case cannot be resized. + allBuffers.push_back(zeroLengthNullBuffer()); + } if (reuseBuffers) { // Set the first value offset to 0. @@ -1091,41 +1112,49 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel } case arrow::StructType::type_id: case arrow::MapType::type_id: - case arrow::ListType::type_id: { - hasComplexType = true; - } break; + case arrow::ListType::type_id: + break; default: { auto& buffers = partitionBuffers_[fixedWidthIdx][partitionId]; // validity buffer if (buffers[kValidityBufferIndex] != nullptr) { - allBuffers.push_back( - arrow::SliceBuffer(buffers[kValidityBufferIndex], 0, arrow::bit_util::BytesForBits(numRows))); + auto validityBufferSize = arrow::bit_util::BytesForBits(numRows); + if (reuseBuffers) { + allBuffers.push_back( + arrow::SliceBuffer(buffers[kValidityBufferIndex], 0, arrow::bit_util::BytesForBits(numRows))); + } else { + RETURN_NOT_OK(buffers[kValidityBufferIndex]->Resize(validityBufferSize, true)); + allBuffers.push_back(std::move(buffers[kValidityBufferIndex])); + } } else { allBuffers.push_back(nullptr); } - // value buffer + // Value buffer. + uint64_t valueBufferSize = 0; auto& valueBuffer = buffers[kFixedWidthValueBufferIndex]; ARROW_RETURN_IF(!valueBuffer, arrow::Status::Invalid("Value buffer of fixed-width array is null.")); - std::shared_ptr slicedValueBuffer; if (arrowColumnTypes_[i]->id() == arrow::BooleanType::type_id) { - slicedValueBuffer = arrow::SliceBuffer(valueBuffer, 0, arrow::bit_util::BytesForBits(numRows)); + valueBufferSize = arrow::bit_util::BytesForBits(numRows); } else if (veloxColumnTypes_[i]->isShortDecimal()) { - slicedValueBuffer = - arrow::SliceBuffer(valueBuffer, 0, numRows * (arrow::bit_width(arrow::Int64Type::type_id) >> 3)); + valueBufferSize = numRows * (arrow::bit_width(arrow::Int64Type::type_id) >> 3); } else if (veloxColumnTypes_[i]->kind() == facebook::velox::TypeKind::TIMESTAMP) { - slicedValueBuffer = arrow::SliceBuffer( - valueBuffer, 0, facebook::velox::BaseVector::byteSize(numRows)); + valueBufferSize = facebook::velox::BaseVector::byteSize(numRows); + } else { + valueBufferSize = numRows * (arrow::bit_width(arrowColumnTypes_[i]->id()) >> 3); + } + if (reuseBuffers) { + auto slicedValueBuffer = arrow::SliceBuffer(valueBuffer, 0, valueBufferSize); + allBuffers.push_back(std::move(slicedValueBuffer)); } else { - slicedValueBuffer = - arrow::SliceBuffer(valueBuffer, 0, numRows * (arrow::bit_width(arrowColumnTypes_[i]->id()) >> 3)); + RETURN_NOT_OK(buffers[kFixedWidthValueBufferIndex]->Resize(valueBufferSize, true)); + allBuffers.push_back(std::move(buffers[kFixedWidthValueBufferIndex])); } - allBuffers.push_back(std::move(slicedValueBuffer)); fixedWidthIdx++; break; } } } - if (hasComplexType && complexTypeData_[partitionId] != nullptr) { + if (hasComplexType_ && complexTypeData_[partitionId] != nullptr) { auto flushBuffer = complexTypeFlushBuffer_[partitionId]; auto serializedSize = complexTypeData_[partitionId]->maxSerializedSize(); if (flushBuffer == nullptr) { @@ -1158,11 +1187,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel EvictGuard evictGuard{evictState_}; int64_t reclaimed = 0; - if (reclaimed < size && shrinkPartitionBuffersBeforeSpill()) { - int64_t shrunken = 0; - RETURN_NOT_OK(partitionWriter_->evictFixedSize(size - reclaimed, &shrunken)); - reclaimed += shrunken; - } if (reclaimed < size) { ARROW_ASSIGN_OR_RAISE(auto cached, evictCachedPayload()); reclaimed += cached; @@ -1422,13 +1446,6 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::vel return evicted; } - bool VeloxShuffleWriter::shrinkPartitionBuffersBeforeSpill() const { - // If OOM happens during stop(), the reclaim order is shrink->spill, - // because the partition buffers will be freed soon. - // SinglePartitioning doesn't maintain partition buffers. - return options_->partitioning != Partitioning::kSingle && splitState_ == SplitState::kStop; - } - bool VeloxShuffleWriter::shrinkPartitionBuffersAfterSpill() const { // If OOM happens during SplitState::kSplit, it is triggered by binary buffers resize. // Or during SplitState::kInit, it is triggered by other operators. diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.h b/cpp/velox/shuffle/VeloxShuffleWriter.h index 0b25ada13592..2488bcb2bad8 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxShuffleWriter.h @@ -315,6 +315,8 @@ class VeloxShuffleWriter final : public ShuffleWriter { bool supportAvx512_ = false; + bool hasComplexType_ = false; + // store arrow column types std::vector> arrowColumnTypes_; From 0788cbc97bfa0fc84bd42e313969b1870f4a925d Mon Sep 17 00:00:00 2001 From: "Ma, Rong" Date: Thu, 14 Dec 2023 00:35:04 +0800 Subject: [PATCH 7/7] separate last payload --- cpp/core/shuffle/LocalPartitionWriter.cc | 25 +++++++++++++++++------- cpp/core/shuffle/LocalPartitionWriter.h | 1 + 2 files changed, 19 insertions(+), 7 deletions(-) diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index 3ce72aeaa261..5ef803859a66 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -272,6 +272,10 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { if (evictor_) { RETURN_NOT_OK(evictor_->flushCachedPayloads(pid, dataFileOs_.get())); } + // Compress and write partition buffers. + if (partitionBufferEvictor_) { + RETURN_NOT_OK(partitionBufferEvictor_->flushCachedPayloads(pid, dataFileOs_.get())); + } ARROW_ASSIGN_OR_RAISE(endInFinalFile, dataFileOs_->Tell()); if (endInFinalFile != startInFinalFile && options_->write_eos) { // Write EOS if any payload written. @@ -342,18 +346,25 @@ arrow::Status LocalPartitionWriter::evict( auto payloadType = (codec_ && evictType != Evictor::kFlush && numRows >= options_->compression_threshold) ? BlockPayload::Type::kCompressed : BlockPayload::Type::kUncompressed; - if (evictType == Evictor::kStop) { - evictType = Evictor::kCache; - if (payloadType == BlockPayload::Type::kCompressed) { - payloadType = BlockPayload::Type::kToBeCompressed; - } + if (evictType == Evictor::kStop && payloadType == BlockPayload::Type::kCompressed) { + payloadType = BlockPayload::Type::kToBeCompressed; } ARROW_ASSIGN_OR_RAISE( auto payload, BlockPayload::fromBuffers( payloadType, numRows, std::move(buffers), payloadPool_.get(), codec_ ? codec_.get() : nullptr, reuseBuffers)); - RETURN_NOT_OK(requestEvict(evictType)); - RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); + if (evictType == Evictor::kStop) { + // Create an empty SpillInfo because the last partition buffers won't be spilled. + if (!partitionBufferEvictor_) { + auto spillInfo = std::make_shared(""); + ARROW_ASSIGN_OR_RAISE( + partitionBufferEvictor_, LocalEvictor::create(numPartitions_, options_, spillInfo, Evictor::Type::kCache)); + } + RETURN_NOT_OK(partitionBufferEvictor_->evict(partitionId, std::move(payload))); + } else { + RETURN_NOT_OK(requestEvict(evictType)); + RETURN_NOT_OK(evictor_->evict(partitionId, std::move(payload))); + } return arrow::Status::OK(); } diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index 78dd32454eb8..698b239b77a5 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -103,6 +103,7 @@ class LocalPartitionWriter : public PartitionWriter { bool stopped_{false}; std::shared_ptr fs_{nullptr}; std::shared_ptr evictor_{nullptr}; + std::shared_ptr partitionBufferEvictor_{nullptr}; std::vector> spills_{}; // configured local dirs for spilled file