diff --git a/velox/benchmarks/basic/CMakeLists.txt b/velox/benchmarks/basic/CMakeLists.txt index 45a57b0944fe2..1b668dbbeec5a 100644 --- a/velox/benchmarks/basic/CMakeLists.txt +++ b/velox/benchmarks/basic/CMakeLists.txt @@ -99,4 +99,5 @@ target_link_libraries( ${velox_benchmark_deps} velox_vector_test_lib velox_functions_spark - velox_functions_prestosql) + velox_functions_prestosql + velox_row_fast) diff --git a/velox/common/memory/MemoryPool.h b/velox/common/memory/MemoryPool.h index 506c592bcc4f2..3a84037d9e398 100644 --- a/velox/common/memory/MemoryPool.h +++ b/velox/common/memory/MemoryPool.h @@ -16,17 +16,14 @@ #pragma once -#include #include #include #include -#include #include #include "velox/common/base/BitUtil.h" #include "velox/common/base/Exceptions.h" #include "velox/common/base/Portability.h" -#include "velox/common/future/VeloxPromise.h" #include "velox/common/memory/Allocation.h" #include "velox/common/memory/MemoryAllocator.h" #include "velox/common/memory/MemoryArbitrator.h" diff --git a/velox/core/PlanNode.cpp b/velox/core/PlanNode.cpp index 609ad7183513a..cdb724f1ef0f0 100644 --- a/velox/core/PlanNode.cpp +++ b/velox/core/PlanNode.cpp @@ -221,6 +221,10 @@ void addSortingKeys( stream << sortingKeys[i]->name() << " " << sortingOrders[i].toString(); } } + +void addVectorSerdeKind(VectorSerde::Kind kind, std::stringstream& stream) { + stream << VectorSerde::kindName(kind); +} } // namespace bool AggregationNode::canSpill(const QueryConfig& queryConfig) const { @@ -804,20 +808,23 @@ const std::vector& ExchangeNode::sources() const { return kEmptySources; } -void ExchangeNode::addDetails(std::stringstream& /* stream */) const { - // Nothing to add. +void ExchangeNode::addDetails(std::stringstream& stream) const { + addVectorSerdeKind(serdeKind_, stream); } folly::dynamic ExchangeNode::serialize() const { auto obj = PlanNode::serialize(); obj["outputType"] = ExchangeNode::outputType()->serialize(); + obj["serdeKind"] = VectorSerde::kindName(serdeKind_); return obj; } // static PlanNodePtr ExchangeNode::create(const folly::dynamic& obj, void* context) { - auto outputType = deserializeRowType(obj["outputType"]); - return std::make_shared(deserializePlanNodeId(obj), outputType); + return std::make_shared( + deserializePlanNodeId(obj), + deserializeRowType(obj["outputType"]), + VectorSerde::kindByName(obj["serdeKind"].asString())); } UnnestNode::UnnestNode( @@ -1945,8 +1952,20 @@ PlanNodePtr TableWriteMergeNode::create( id, outputType, aggregationNode, source); } +MergeExchangeNode::MergeExchangeNode( + const PlanNodeId& id, + const RowTypePtr& type, + const std::vector& sortingKeys, + const std::vector& sortingOrders, + VectorSerde::Kind serdeKind) + : ExchangeNode(id, type, serdeKind), + sortingKeys_(sortingKeys), + sortingOrders_(sortingOrders) {} + void MergeExchangeNode::addDetails(std::stringstream& stream) const { addSortingKeys(sortingKeys_, sortingOrders_, stream); + stream << ", "; + addVectorSerdeKind(serdeKind(), stream); } folly::dynamic MergeExchangeNode::serialize() const { @@ -1954,6 +1973,7 @@ folly::dynamic MergeExchangeNode::serialize() const { obj["outputType"] = ExchangeNode::outputType()->serialize(); obj["sortingKeys"] = ISerializable::serialize(sortingKeys_); obj["sortingOrders"] = serializeSortingOrders(sortingOrders_); + obj["serdeKind"] = VectorSerde::kindName(serdeKind()); return obj; } @@ -1961,11 +1981,16 @@ folly::dynamic MergeExchangeNode::serialize() const { PlanNodePtr MergeExchangeNode::create( const folly::dynamic& obj, void* context) { - auto outputType = deserializeRowType(obj["outputType"]); - auto sortingKeys = deserializeFields(obj["sortingKeys"], context); - auto sortingOrders = deserializeSortingOrders(obj["sortingOrders"]); + const auto outputType = deserializeRowType(obj["outputType"]); + const auto sortingKeys = deserializeFields(obj["sortingKeys"], context); + const auto sortingOrders = deserializeSortingOrders(obj["sortingOrders"]); + const auto serdeKind = VectorSerde::kindByName(obj["serdeKind"].asString()); return std::make_shared( - deserializePlanNodeId(obj), outputType, sortingKeys, sortingOrders); + deserializePlanNodeId(obj), + outputType, + sortingKeys, + sortingOrders, + serdeKind); } void LocalPartitionNode::addDetails(std::stringstream& stream) const { @@ -2024,6 +2049,97 @@ LocalPartitionNode::Type LocalPartitionNode::typeFromName( return it->second; } +PartitionedOutputNode::PartitionedOutputNode( + const PlanNodeId& id, + Kind kind, + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + PartitionFunctionSpecPtr partitionFunctionSpec, + RowTypePtr outputType, + VectorSerde::Kind serdeKind, + PlanNodePtr source) + : PlanNode(id), + kind_(kind), + sources_{{std::move(source)}}, + keys_(keys), + numPartitions_(numPartitions), + replicateNullsAndAny_(replicateNullsAndAny), + partitionFunctionSpec_(std::move(partitionFunctionSpec)), + serdeKind_(serdeKind), + outputType_(std::move(outputType)) { + VELOX_USER_CHECK_GT(numPartitions_, 0); + if (numPartitions_ == 1) { + VELOX_USER_CHECK( + keys_.empty(), + "Non-empty partitioning keys require more than one partition"); + } + if (!isPartitioned()) { + VELOX_USER_CHECK( + keys_.empty(), + "{} partitioning doesn't allow for partitioning keys", + kindString(kind_)); + } +} + +// static +std::shared_ptr PartitionedOutputNode::broadcast( + const PlanNodeId& id, + int numPartitions, + RowTypePtr outputType, + VectorSerde::Kind serdeKind, + PlanNodePtr source) { + std::vector noKeys; + return std::make_shared( + id, + Kind::kBroadcast, + noKeys, + numPartitions, + false, + std::make_shared(), + std::move(outputType), + serdeKind, + std::move(source)); +} + +// static +std::shared_ptr PartitionedOutputNode::arbitrary( + const PlanNodeId& id, + RowTypePtr outputType, + VectorSerde::Kind serdeKind, + PlanNodePtr source) { + std::vector noKeys; + return std::make_shared( + id, + Kind::kArbitrary, + noKeys, + 1, + false, + std::make_shared(), + std::move(outputType), + serdeKind, + std::move(source)); +} + +// static +std::shared_ptr PartitionedOutputNode::single( + const PlanNodeId& id, + RowTypePtr outputType, + VectorSerde::Kind serdeKind, + PlanNodePtr source) { + std::vector noKeys; + return std::make_shared( + id, + Kind::kPartitioned, + noKeys, + 1, + false, + std::make_shared(), + std::move(outputType), + serdeKind, + std::move(source)); +} + void EnforceSingleRowNode::addDetails(std::stringstream& /* stream */) const { // Nothing to add. } @@ -2091,6 +2207,9 @@ void PartitionedOutputNode::addDetails(std::stringstream& stream) const { if (replicateNullsAndAny_) { stream << " replicate nulls and any"; } + + stream << " "; + addVectorSerdeKind(serdeKind_, stream); } folly::dynamic PartitionedOutputNode::serialize() const { @@ -2100,6 +2219,7 @@ folly::dynamic PartitionedOutputNode::serialize() const { obj["keys"] = ISerializable::serialize(keys_); obj["replicateNullsAndAny"] = replicateNullsAndAny_; obj["partitionFunctionSpec"] = partitionFunctionSpec_->serialize(); + obj["serdeKind"] = VectorSerde::kindName(serdeKind_); obj["outputType"] = outputType_->serialize(); return obj; } @@ -2117,6 +2237,7 @@ PlanNodePtr PartitionedOutputNode::create( ISerializable::deserialize( obj["partitionFunctionSpec"], context), deserializeRowType(obj["outputType"]), + VectorSerde::kindByName(obj["serdeKind"].asString()), deserializeSingleSource(obj, context)); } diff --git a/velox/core/PlanNode.h b/velox/core/PlanNode.h index c893a834fda8e..3d239445f756e 100644 --- a/velox/core/PlanNode.h +++ b/velox/core/PlanNode.h @@ -22,6 +22,7 @@ #include "velox/connectors/Connector.h" #include "velox/core/Expressions.h" #include "velox/core/QueryConfig.h" +#include "velox/vector/VectorStream.h" struct ArrowArrayStream; @@ -1012,8 +1013,16 @@ class GroupIdNode : public PlanNode { class ExchangeNode : public PlanNode { public: +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY ExchangeNode(const PlanNodeId& id, RowTypePtr type) - : PlanNode(id), outputType_(type) {} + : ExchangeNode(id, std::move(type), VectorSerde::Kind::kPresto) {} +#endif + + ExchangeNode( + const PlanNodeId& id, + RowTypePtr type, + VectorSerde::Kind serdeKind) + : PlanNode(id), outputType_(type), serdeKind_(serdeKind) {} const RowTypePtr& outputType() const override { return outputType_; @@ -1033,6 +1042,10 @@ class ExchangeNode : public PlanNode { return "Exchange"; } + VectorSerde::Kind serdeKind() const { + return serdeKind_; + } + folly::dynamic serialize() const override; static PlanNodePtr create(const folly::dynamic& obj, void* context); @@ -1040,19 +1053,32 @@ class ExchangeNode : public PlanNode { private: void addDetails(std::stringstream& stream) const override; - RowTypePtr outputType_; + const RowTypePtr outputType_; + const VectorSerde::Kind serdeKind_; }; class MergeExchangeNode : public ExchangeNode { public: - explicit MergeExchangeNode( +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY + MergeExchangeNode( const PlanNodeId& id, const RowTypePtr& type, const std::vector& sortingKeys, const std::vector& sortingOrders) - : ExchangeNode(id, type), - sortingKeys_(sortingKeys), - sortingOrders_(sortingOrders) {} + : MergeExchangeNode( + id, + type, + sortingKeys, + sortingOrders, + VectorSerde::Kind::kPresto) {} +#endif + + MergeExchangeNode( + const PlanNodeId& id, + const RowTypePtr& type, + const std::vector& sortingKeys, + const std::vector& sortingOrders, + VectorSerde::Kind serdeKind); const std::vector& sortingKeys() const { return sortingKeys_; @@ -1271,6 +1297,7 @@ class PartitionedOutputNode : public PlanNode { static std::string kindString(Kind kind); static Kind stringToKind(const std::string& str); +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY PartitionedOutputNode( const PlanNodeId& id, Kind kind, @@ -1280,72 +1307,84 @@ class PartitionedOutputNode : public PlanNode { PartitionFunctionSpecPtr partitionFunctionSpec, RowTypePtr outputType, PlanNodePtr source) - : PlanNode(id), - kind_(kind), - sources_{{std::move(source)}}, - keys_(keys), - numPartitions_(numPartitions), - replicateNullsAndAny_(replicateNullsAndAny), - partitionFunctionSpec_(std::move(partitionFunctionSpec)), - outputType_(std::move(outputType)) { - VELOX_USER_CHECK_GT(numPartitions, 0); - if (numPartitions_ == 1) { - VELOX_USER_CHECK( - keys_.empty(), - "Non-empty partitioning keys require more than one partition"); - } - if (!isPartitioned()) { - VELOX_USER_CHECK( - keys_.empty(), - "{} partitioning doesn't allow for partitioning keys", - kindString(kind_)); - } - } + : PartitionedOutputNode( + id, + kind, + keys, + numPartitions, + replicateNullsAndAny, + std::move(partitionFunctionSpec), + std::move(outputType), + VectorSerde::Kind::kPresto, + std::move(source)) {} +#endif + + PartitionedOutputNode( + const PlanNodeId& id, + Kind kind, + const std::vector& keys, + int numPartitions, + bool replicateNullsAndAny, + PartitionFunctionSpecPtr partitionFunctionSpec, + RowTypePtr outputType, + VectorSerde::Kind serdeKind, + PlanNodePtr source); +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY static std::shared_ptr broadcast( const PlanNodeId& id, int numPartitions, RowTypePtr outputType, PlanNodePtr source) { - std::vector noKeys; - return std::make_shared( + return broadcast( id, - Kind::kBroadcast, - noKeys, numPartitions, - false, - std::make_shared(), std::move(outputType), + VectorSerde::Kind::kPresto, std::move(source)); } +#endif + + static std::shared_ptr broadcast( + const PlanNodeId& id, + int numPartitions, + RowTypePtr outputType, + VectorSerde::Kind serdeKind, + PlanNodePtr source); +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY static std::shared_ptr arbitrary(const PlanNodeId& id, RowTypePtr outputType, PlanNodePtr source) { - std::vector noKeys; - return std::make_shared( + return arbitrary( id, - Kind::kArbitrary, - noKeys, - 1, - false, - std::make_shared(), std::move(outputType), + VectorSerde::Kind::kPresto, std::move(source)); } +#endif + + static std::shared_ptr arbitrary( + const PlanNodeId& id, + RowTypePtr outputType, + VectorSerde::Kind serdeKind, + PlanNodePtr source); +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY static std::shared_ptr single(const PlanNodeId& id, RowTypePtr outputType, PlanNodePtr source) { - std::vector noKeys; - return std::make_shared( + return single( id, - Kind::kPartitioned, - noKeys, - 1, - false, - std::make_shared(), std::move(outputType), + VectorSerde::Kind::kPresto, std::move(source)); } +#endif + + static std::shared_ptr single( + const PlanNodeId& id, + RowTypePtr outputType, + VectorSerde::Kind VectorSerde, + PlanNodePtr source); const RowTypePtr& outputType() const override { return outputType_; @@ -1383,6 +1422,10 @@ class PartitionedOutputNode : public PlanNode { return kind_; } + VectorSerde::Kind serdeKind() const { + return serdeKind_; + } + /// Returns true if an arbitrary row and all rows with null keys must be /// replicated to all destinations. This is used to ensure correct results /// for anti-join which requires all nodes to know whether combined build @@ -1416,6 +1459,7 @@ class PartitionedOutputNode : public PlanNode { const int numPartitions_; const bool replicateNullsAndAny_; const PartitionFunctionSpecPtr partitionFunctionSpec_; + const VectorSerde::Kind serdeKind_; const RowTypePtr outputType_; }; diff --git a/velox/docs/monitoring/stats.rst b/velox/docs/monitoring/stats.rst index fbabecbf52692..4958587e7bcfd 100644 --- a/velox/docs/monitoring/stats.rst +++ b/velox/docs/monitoring/stats.rst @@ -162,3 +162,20 @@ These stats are reported by operators that support spilling. * - spillDeserializationWallNanos - nanos - The time spent on deserializing rows read from spilled files. + +Shuffle +-------- +These stats are reported by shuffle operators. + +.. list-table:: + :widths: 50 25 50 + :header-rows: 1 + + * - Stats + - Unit + - Description + * - shuffleSerdeKind + - + - Indicates the vector serde kind used by an operator for shuffle with 1 + for Presto, 2 for CompactRow, 3 for UnsafeRow. It is reported by Exchange, + MergeExchange and PartitionedOutput operators for now. diff --git a/velox/exec/Exchange.cpp b/velox/exec/Exchange.cpp index 7551764f06dde..758957ac2a7c3 100644 --- a/velox/exec/Exchange.cpp +++ b/velox/exec/Exchange.cpp @@ -14,6 +14,7 @@ * limitations under the License. */ #include "velox/exec/Exchange.h" + #include "velox/exec/Task.h" namespace facebook::velox::exec { @@ -74,12 +75,9 @@ BlockingReason Exchange::isBlocked(ContinueFuture* future) { getSplits(&splitFuture_); } - const auto maxBytes = getSerde()->supportsAppendInDeserialize() - ? preferredOutputBatchBytes_ - : 1; - ContinueFuture dataFuture; - currentPages_ = exchangeClient_->next(maxBytes, &atEnd_, &dataFuture); + currentPages_ = + exchangeClient_->next(preferredOutputBatchBytes_, &atEnd_, &dataFuture); if (!currentPages_.empty() || atEnd_) { if (atEnd_ && noMoreSplits_) { const auto numSplits = stats_.rlock()->numSplits; @@ -116,23 +114,51 @@ RowVectorPtr Exchange::getOutput() { uint64_t rawInputBytes{0}; vector_size_t resultOffset = 0; - for (const auto& page : currentPages_) { - rawInputBytes += page->size(); - - auto inputStream = page->prepareStreamForDeserialize(); - - while (!inputStream->atEnd()) { - getSerde()->deserialize( - inputStream.get(), - pool(), - outputType_, - &result_, - resultOffset, - &options_); - resultOffset = result_->size(); + if (getSerde()->supportsAppendInDeserialize()) { + for (const auto& page : currentPages_) { + rawInputBytes += page->size(); + + auto inputStream = page->prepareStreamForDeserialize(); + + while (!inputStream->atEnd()) { + getSerde()->deserialize( + inputStream.get(), + pool(), + outputType_, + &result_, + resultOffset, + &options_); + resultOffset = result_->size(); + } } + } else { + VELOX_CHECK( + getSerde()->kind() == VectorSerde::Kind::kCompactRow || + getSerde()->kind() == VectorSerde::Kind::kUnsafeRow); + + std::unique_ptr mergedBufs; + for (const auto& page : currentPages_) { + rawInputBytes += page->size(); + if (mergedBufs == nullptr) { + mergedBufs = page->getIOBuf()->clone(); + } else { + mergedBufs->appendToChain(page->getIOBuf()->clone()); + } + } + VELOX_CHECK_NOT_NULL(mergedBufs); + auto mergedPages = std::make_unique(std::move(mergedBufs)); + auto inputStream = mergedPages->prepareStreamForDeserialize(); + getSerde()->deserialize( + inputStream.get(), + pool(), + outputType_, + &result_, + resultOffset, + &options_); + // We expect the row-wise deserialization to consume all the input into one + // output vector. + VELOX_CHECK(inputStream->atEnd()); } - currentPages_.clear(); { @@ -154,6 +180,9 @@ void Exchange::close() { exchangeClient_->close(); } exchangeClient_ = nullptr; + stats_.wlock()->addRuntimeStat( + Operator::kShuffleSerdeKind, + RuntimeCounter(static_cast(serdeKind_))); } void Exchange::recordExchangeClientStats() { @@ -182,7 +211,7 @@ void Exchange::recordExchangeClientStats() { } VectorSerde* Exchange::getSerde() { - return getVectorSerde(); + return getNamedVectorSerde(serdeKind_); } } // namespace facebook::velox::exec diff --git a/velox/exec/Exchange.h b/velox/exec/Exchange.h index dea346773355a..0aa69fa9f9509 100644 --- a/velox/exec/Exchange.h +++ b/velox/exec/Exchange.h @@ -15,8 +15,8 @@ */ #pragma once -#include #include + #include "velox/exec/ExchangeClient.h" #include "velox/exec/Operator.h" #include "velox/exec/OutputBufferManager.h" @@ -51,6 +51,7 @@ class Exchange : public SourceOperator { operatorType), preferredOutputBatchBytes_{ driverCtx->queryConfig().preferredOutputBatchBytes()}, + serdeKind_(exchangeNode->serdeKind()), processSplits_{operatorCtx_->driverCtx()->driverId == 0}, exchangeClient_{std::move(exchangeClient)} { options_.compressionKind = @@ -94,11 +95,16 @@ class Exchange : public SourceOperator { const uint64_t preferredOutputBatchBytes_; + const VectorSerde::Kind serdeKind_; + /// True if this operator is responsible for fetching splits from the Task and /// passing these to ExchangeClient. const bool processSplits_; + bool noMoreSplits_ = false; + std::shared_ptr exchangeClient_; + /// A future received from Task::getSplitOrFuture(). It will be complete when /// there are more splits available or no-more-splits signal has arrived. ContinueFuture splitFuture_{ContinueFuture::makeEmpty()}; @@ -106,7 +112,6 @@ class Exchange : public SourceOperator { // Reusable result vector. RowVectorPtr result_; - std::shared_ptr exchangeClient_; std::vector> currentPages_; bool atEnd_{false}; std::default_random_engine rng_{std::random_device{}()}; diff --git a/velox/exec/Merge.cpp b/velox/exec/Merge.cpp index dcca6d28e2c4f..e41c2aa84d300 100644 --- a/velox/exec/Merge.cpp +++ b/velox/exec/Merge.cpp @@ -304,7 +304,8 @@ MergeExchange::MergeExchange( mergeExchangeNode->sortingKeys(), mergeExchangeNode->sortingOrders(), mergeExchangeNode->id(), - "MergeExchange") {} + "MergeExchange"), + serde_(getNamedVectorSerde(mergeExchangeNode->serdeKind())) {} BlockingReason MergeExchange::addMergeSources(ContinueFuture* future) { if (operatorCtx_->driverCtx()->driverId != 0) { @@ -364,4 +365,10 @@ BlockingReason MergeExchange::addMergeSources(ContinueFuture* future) { } } +void MergeExchange::close() { + Operator::close(); + stats_.wlock()->addRuntimeStat( + Operator::kShuffleSerdeKind, + RuntimeCounter(static_cast(serde_->kind()))); +} } // namespace facebook::velox::exec diff --git a/velox/exec/Merge.h b/velox/exec/Merge.h index b33780613fb74..e8aa3749dd4d7 100644 --- a/velox/exec/Merge.h +++ b/velox/exec/Merge.h @@ -191,10 +191,17 @@ class MergeExchange : public Merge { DriverCtx* driverCtx, const std::shared_ptr& orderByNode); + VectorSerde* serde() const { + return serde_; + } + + void close() override; + protected: BlockingReason addMergeSources(ContinueFuture* future) override; private: + VectorSerde* const serde_; bool noMoreSplits_ = false; // Task Ids from all the splits we took to process so far. std::vector remoteSourceTaskIds_; diff --git a/velox/exec/MergeSource.cpp b/velox/exec/MergeSource.cpp index 3c888d8e1bf36..c5e77e956a146 100644 --- a/velox/exec/MergeSource.cpp +++ b/velox/exec/MergeSource.cpp @@ -163,6 +163,7 @@ class MergeExchangeSource : public MergeSource { inputStream_.get(), mergeExchange_->pool(), mergeExchange_->outputType(), + mergeExchange_->serde(), &data); auto lockedStats = mergeExchange_->stats().wlock(); diff --git a/velox/exec/Operator.h b/velox/exec/Operator.h index 2138fa47e3ccd..6d0fa34c3af85 100644 --- a/velox/exec/Operator.h +++ b/velox/exec/Operator.h @@ -360,6 +360,10 @@ class Operator : public BaseRuntimeStatWriter { static inline const std::string kSpillDeserializationTime{ "spillDeserializationWallNanos"}; + /// The vector serde kind used by an operator for shuffle. The recorded + /// runtime stats value is the corresponding enum value. + static inline const std::string kShuffleSerdeKind{"shuffleSerdeKind"}; + /// 'operatorId' is the initial index of the 'this' in the Driver's list of /// Operators. This is used as in index into OperatorStats arrays in the Task. /// 'planNodeId' is a query-level unique identifier of the PlanNode to which diff --git a/velox/exec/OperatorTraceReader.cpp b/velox/exec/OperatorTraceReader.cpp index 90e68af715d2e..e6376090d8075 100644 --- a/velox/exec/OperatorTraceReader.cpp +++ b/velox/exec/OperatorTraceReader.cpp @@ -30,6 +30,7 @@ OperatorTraceInputReader::OperatorTraceInputReader( fs_(filesystems::getFileSystem(traceDir_, nullptr)), dataType_(std::move(dataType)), pool_(pool), + serde_(getNamedVectorSerde(VectorSerde::Kind::kPresto)), inputStream_(getInputStream()) { VELOX_CHECK_NOT_NULL(dataType_); } @@ -45,7 +46,7 @@ bool OperatorTraceInputReader::read(RowVectorPtr& batch) const { } VectorStreamGroup::read( - inputStream_.get(), pool_, dataType_, &batch, &readOptions_); + inputStream_.get(), pool_, dataType_, serde_, &batch, &readOptions_); return true; } diff --git a/velox/exec/OperatorTraceReader.h b/velox/exec/OperatorTraceReader.h index 2236d73fe2f7c..6888d5a600b80 100644 --- a/velox/exec/OperatorTraceReader.h +++ b/velox/exec/OperatorTraceReader.h @@ -47,6 +47,7 @@ class OperatorTraceInputReader { const std::shared_ptr fs_; const RowTypePtr dataType_; memory::MemoryPool* const pool_; + VectorSerde* const serde_; const std::unique_ptr inputStream_; }; diff --git a/velox/exec/OperatorTraceWriter.cpp b/velox/exec/OperatorTraceWriter.cpp index 75f2a5609c01b..38987226ccd81 100644 --- a/velox/exec/OperatorTraceWriter.cpp +++ b/velox/exec/OperatorTraceWriter.cpp @@ -34,6 +34,7 @@ OperatorTraceWriter::OperatorTraceWriter( traceDir_(std::move(traceDir)), fs_(filesystems::getFileSystem(traceDir_, nullptr)), pool_(pool), + serde_(getNamedVectorSerde(VectorSerde::Kind::kPresto)), updateAndCheckTraceLimitCB_(std::move(updateAndCheckTraceLimitCB)) { traceFile_ = fs_->openFileForWrite(getOpTraceInputFilePath(traceDir_)); VELOX_CHECK_NOT_NULL(traceFile_); @@ -48,7 +49,7 @@ void OperatorTraceWriter::write(const RowVectorPtr& rows) { } if (batch_ == nullptr) { - batch_ = std::make_unique(pool_); + batch_ = std::make_unique(pool_, serde_); batch_->createStreamTree( std::static_pointer_cast(rows->type()), 1'000, diff --git a/velox/exec/OperatorTraceWriter.h b/velox/exec/OperatorTraceWriter.h index 9999f2c31dbd1..c0b818e93acb0 100644 --- a/velox/exec/OperatorTraceWriter.h +++ b/velox/exec/OperatorTraceWriter.h @@ -61,6 +61,7 @@ class OperatorTraceWriter { /*nullsFirst=*/true}; const std::shared_ptr fs_; memory::MemoryPool* const pool_; + VectorSerde* const serde_; const UpdateAndCheckTraceLimitCB updateAndCheckTraceLimitCB_; std::unique_ptr traceFile_; diff --git a/velox/exec/PartitionedOutput.cpp b/velox/exec/PartitionedOutput.cpp index 6b49fe22efa8e..67504891ec9e2 100644 --- a/velox/exec/PartitionedOutput.cpp +++ b/velox/exec/PartitionedOutput.cpp @@ -24,11 +24,14 @@ BlockingReason Destination::advance( uint64_t maxBytes, const std::vector& sizes, const RowVectorPtr& output, + const row::CompactRow* outputCompactRow, + const row::UnsafeRowFast* outputUnsafeRow, OutputBufferManager& bufferManager, const std::function& bufferReleaseFn, bool* atEnd, ContinueFuture* future, Scratch& scratch) { + VELOX_CHECK_LE(!!outputCompactRow + !!outputUnsafeRow, 1); if (rowIdx_ >= rows_.size()) { *atEnd = true; return BlockingReason::kNotBlocked; @@ -51,17 +54,32 @@ BlockingReason Destination::advance( } // Serialize - if (!current_) { - current_ = std::make_unique(pool_); - auto rowType = asRowType(output->type()); - serializer::presto::PrestoVectorSerde::PrestoOptions options; - options.compressionKind = - OutputBufferManager::getInstance().lock()->compressionKind(); - options.minCompressionRatio = PartitionedOutput::minCompressionRatio(); - current_->createStreamTree(rowType, rowsInCurrent_, &options); + if (current_ == nullptr) { + current_ = std::make_unique(pool_, serde_); + const auto rowType = asRowType(output->type()); + if (serde_->kind() == VectorSerde::Kind::kPresto) { + serializer::presto::PrestoVectorSerde::PrestoOptions options; + options.compressionKind = + OutputBufferManager::getInstance().lock()->compressionKind(); + options.minCompressionRatio = PartitionedOutput::minCompressionRatio(); + current_->createStreamTree(rowType, rowsInCurrent_, &options); + } else { + current_->createStreamTree(rowType, rowsInCurrent_); + } + } + + const auto rows = folly::Range(&rows_[firstRow], rowIdx_ - firstRow); + if (serde_->kind() == VectorSerde::Kind::kCompactRow) { + VELOX_CHECK_NOT_NULL(outputCompactRow); + current_->append(*outputCompactRow, rows, sizes); + } else if (serde_->kind() == VectorSerde::Kind::kUnsafeRow) { + VELOX_CHECK_NOT_NULL(outputUnsafeRow); + current_->append(*outputUnsafeRow, rows, sizes); + } else { + VELOX_CHECK_EQ(serde_->kind(), VectorSerde::Kind::kPresto); + current_->append(output, rows, scratch); } - current_->append( - output, folly::Range(&rows_[firstRow], rowIdx_ - firstRow), scratch); + // Update output state variable. if (rowIdx_ == rows_.size()) { *atEnd = true; @@ -156,7 +174,8 @@ PartitionedOutput::PartitionedOutput( maxBufferedBytes_(ctx->task->queryCtx() ->queryConfig() .maxPartitionedOutputBufferSize()), - eagerFlush_(eagerFlush) { + eagerFlush_(eagerFlush), + serde_(getNamedVectorSerde(planNode->serdeKind())) { if (!planNode->isPartitioned()) { VELOX_USER_CHECK_EQ(numDestinations_, 1); } @@ -191,6 +210,17 @@ void PartitionedOutput::initializeInput(RowVectorPtr input) { input_->size(), outputColumns); } + + // Lazy load all the input columns. + for (auto i = 0; i < output_->childrenSize(); ++i) { + output_->childAt(i)->loadedVector(); + } + + if (serde_->kind() == VectorSerde::Kind::kCompactRow) { + outputCompactRow_ = std::make_unique(output_); + } else if (serde_->kind() == VectorSerde::Kind::kUnsafeRow) { + outputUnsafeRow_ = std::make_unique(output_); + } } void PartitionedOutput::initializeDestinations() { @@ -198,7 +228,12 @@ void PartitionedOutput::initializeDestinations() { auto taskId = operatorCtx_->taskId(); for (int i = 0; i < numDestinations_; ++i) { destinations_.push_back(std::make_unique( - taskId, i, pool(), eagerFlush_, [&](uint64_t bytes, uint64_t rows) { + taskId, + i, + serde_, + pool(), + eagerFlush_, + [&](uint64_t bytes, uint64_t rows) { auto lockedStats = stats_.wlock(); lockedStats->addOutputVector(bytes, rows); })); @@ -207,7 +242,7 @@ void PartitionedOutput::initializeDestinations() { } void PartitionedOutput::initializeSizeBuffers() { - auto numInput = input_->size(); + const auto numInput = input_->size(); if (numInput > rowSize_.size()) { rowSize_.resize(numInput); sizePointers_.resize(numInput); @@ -219,16 +254,23 @@ void PartitionedOutput::initializeSizeBuffers() { } void PartitionedOutput::estimateRowSizes() { - auto numInput = input_->size(); + const auto numInput = input_->size(); std::fill(rowSize_.begin(), rowSize_.end(), 0); raw_vector storage; - auto numbers = iota(numInput, storage); - for (int i = 0; i < output_->childrenSize(); ++i) { - VectorStreamGroup::estimateSerializedSize( - output_->childAt(i).get(), - folly::Range(numbers, numInput), - sizePointers_.data(), - scratch_); + const auto numbers = iota(numInput, storage); + const auto rows = folly::Range(numbers, numInput); + if (serde_->kind() == VectorSerde::Kind::kCompactRow) { + VELOX_CHECK_NOT_NULL(outputCompactRow_); + serde_->estimateSerializedSize( + outputCompactRow_.get(), rows, sizePointers_.data()); + } else if (serde_->kind() == VectorSerde::Kind::kUnsafeRow) { + VELOX_CHECK_NOT_NULL(outputUnsafeRow_); + serde_->estimateSerializedSize( + outputUnsafeRow_.get(), rows, sizePointers_.data()); + } else { + VELOX_CHECK_EQ(serde_->kind(), VectorSerde::Kind::kPresto); + serde_->estimateSerializedSize( + output_.get(), rows, sizePointers_.data(), scratch_); } } @@ -341,6 +383,8 @@ RowVectorPtr PartitionedOutput::getOutput() { maxPageSize, rowSize_, output_, + outputCompactRow_.get(), + outputUnsafeRow_.get(), *bufferManager, bufferReleaseFn_, &atEnd, @@ -392,6 +436,8 @@ RowVectorPtr PartitionedOutput::getOutput() { // The input is fully processed, drop the reference to allow reuse. input_ = nullptr; output_ = nullptr; + outputCompactRow_.reset(); + outputUnsafeRow_.reset(); return nullptr; } @@ -399,4 +445,12 @@ bool PartitionedOutput::isFinished() { return finished_; } +void PartitionedOutput::close() { + Operator::close(); + stats_.wlock()->addRuntimeStat( + Operator::kShuffleSerdeKind, + RuntimeCounter(static_cast(serde_->kind()))); + destinations_.clear(); +} + } // namespace facebook::velox::exec diff --git a/velox/exec/PartitionedOutput.h b/velox/exec/PartitionedOutput.h index 4589646a167fc..fecdaf57c9c59 100644 --- a/velox/exec/PartitionedOutput.h +++ b/velox/exec/PartitionedOutput.h @@ -18,6 +18,8 @@ #include #include "velox/exec/Operator.h" #include "velox/exec/OutputBufferManager.h" +#include "velox/row/CompactRow.h" +#include "velox/row/UnsafeRowFast.h" #include "velox/vector/VectorStream.h" namespace facebook::velox::exec { @@ -30,11 +32,13 @@ class Destination { Destination( const std::string& taskId, int destination, + VectorSerde* serde, memory::MemoryPool* pool, bool eagerFlush, std::function recordEnqueued) : taskId_(taskId), destination_(destination), + serde_(serde), pool_(pool), eagerFlush_(eagerFlush), recordEnqueued_(std::move(recordEnqueued)) { @@ -63,6 +67,8 @@ class Destination { uint64_t maxBytes, const std::vector& sizes, const RowVectorPtr& output, + const row::CompactRow* outputCompactRow, + const row::UnsafeRowFast* outputUnsafeRow, OutputBufferManager& bufferManager, const std::function& bufferReleaseFn, bool* atEnd, @@ -105,6 +111,7 @@ class Destination { const std::string taskId_; const int destination_; + VectorSerde* const serde_; memory::MemoryPool* const pool_; const bool eagerFlush_; const std::function recordEnqueued_; @@ -182,9 +189,7 @@ class PartitionedOutput : public Operator { bool isFinished() override; - void close() override { - destinations_.clear(); - } + void close() override; static void testingSetMinCompressionRatio(float ratio) { minCompressionRatio_ = ratio; @@ -220,6 +225,7 @@ class PartitionedOutput : public Operator { const std::function bufferReleaseFn_; const int64_t maxBufferedBytes_; const bool eagerFlush_; + VectorSerde* const serde_; BlockingReason blockingReason_{BlockingReason::kNotBlocked}; ContinueFuture future_; @@ -232,6 +238,16 @@ class PartitionedOutput : public Operator { std::vector> destinations_; bool replicatedAny_{false}; RowVectorPtr output_; + // This is only set with current 'output_' in case of compact row serde + // format. It is used to accelerate serialized row size calculation and the + // actual serialization processing. + // + // NOTE: 'outputCompactRow_' construction is expensive so we cache it here to + // do it only once for an entire input processing across different + // destinations. + std::unique_ptr outputCompactRow_; + // Simialr to 'outputcompactRow_' for unsafe row serde format. + std::unique_ptr outputUnsafeRow_; // Reusable memory. SelectivityVector rows_; diff --git a/velox/exec/SpillFile.cpp b/velox/exec/SpillFile.cpp index 0d16005b20f43..6b5e4f464b925 100644 --- a/velox/exec/SpillFile.cpp +++ b/velox/exec/SpillFile.cpp @@ -17,6 +17,7 @@ #include "velox/exec/SpillFile.h" #include "velox/common/base/RuntimeMetrics.h" #include "velox/common/file/FileSystems.h" +#include "velox/vector/VectorStream.h" namespace facebook::velox::exec { namespace { @@ -93,6 +94,7 @@ SpillWriter::SpillWriter( fileCreateConfig_(fileCreateConfig), updateAndCheckSpillLimitCb_(updateAndCheckSpillLimitCb), pool_(pool), + serde_(getNamedVectorSerde(VectorSerde::Kind::kPresto)), stats_(stats) { // NOTE: if the associated spilling operator has specified the sort // comparison flags, then it must match the number of sorting keys. @@ -174,7 +176,7 @@ uint64_t SpillWriter::write( if (batch_ == nullptr) { serializer::presto::PrestoVectorSerde::PrestoOptions options = { kDefaultUseLosslessTimestamp, compressionKind_, true /*nullsFirst*/}; - batch_ = std::make_unique(pool_); + batch_ = std::make_unique(pool_, serde_); batch_->createStreamTree( std::static_pointer_cast(rows->type()), 1'000, @@ -300,6 +302,7 @@ SpillReadFile::SpillReadFile( compressionKind_, /*nullsFirst=*/true}, pool_(pool), + serde_(getNamedVectorSerde(VectorSerde::Kind::kPresto)), stats_(stats) { auto fs = filesystems::getFileSystem(path_, nullptr); auto file = fs->openFileForRead(path_); @@ -317,7 +320,7 @@ bool SpillReadFile::nextBatch(RowVectorPtr& rowVector) { { NanosecondTimer timer{&timeNs}; VectorStreamGroup::read( - input_.get(), pool_, type_, &rowVector, &readOptions_); + input_.get(), pool_, type_, serde_, &rowVector, &readOptions_); } stats_->wlock()->spillDeserializationTimeNanos += timeNs; common::updateGlobalSpillDeserializationTimeNs(timeNs); diff --git a/velox/exec/SpillFile.h b/velox/exec/SpillFile.h index 247240262e5ab..d598d2da16e87 100644 --- a/velox/exec/SpillFile.h +++ b/velox/exec/SpillFile.h @@ -189,8 +189,9 @@ class SpillWriter { // Updates the aggregated spill bytes of this query, and throws if exceeds // the max spill bytes limit. - common::UpdateAndCheckSpillLimitCB updateAndCheckSpillLimitCb_; + const common::UpdateAndCheckSpillLimitCB updateAndCheckSpillLimitCb_; memory::MemoryPool* const pool_; + VectorSerde* const serde_; folly::Synchronized* const stats_; bool finished_{false}; @@ -200,8 +201,8 @@ class SpillWriter { SpillFiles finishedFiles_; }; -/// Represents a spill file for read which turns the serialized spilled data on -/// disk back into a sequence of spilled row vectors. +/// Represents a spill file for read which turns the serialized spilled data +/// on disk back into a sequence of spilled row vectors. /// /// NOTE: The class will not delete spill file upon destruction, so the user /// needs to remove the unused spill files at some point later. For example, a @@ -269,6 +270,7 @@ class SpillReadFile { const common::CompressionKind compressionKind_; const serializer::presto::PrestoVectorSerde::PrestoOptions readOptions_; memory::MemoryPool* const pool_; + VectorSerde* const serde_; folly::Synchronized* const stats_; std::unique_ptr input_; diff --git a/velox/exec/TaskTraceReader.cpp b/velox/exec/TaskTraceReader.cpp index 6ed295190f061..8584cd226abd8 100644 --- a/velox/exec/TaskTraceReader.cpp +++ b/velox/exec/TaskTraceReader.cpp @@ -31,7 +31,10 @@ TaskTraceMetadataReader::TaskTraceMetadataReader( traceFilePath_(getTaskTraceMetaFilePath(traceDir_)), pool_(pool) { VELOX_CHECK_NOT_NULL(fs_); - VELOX_CHECK(fs_->exists(traceFilePath_)); + VELOX_CHECK( + fs_->exists(traceFilePath_), + "Task trace file not found: {}", + traceFilePath_); } void TaskTraceMetadataReader::read( diff --git a/velox/exec/benchmarks/ExchangeBenchmark.cpp b/velox/exec/benchmarks/ExchangeBenchmark.cpp index 2ac2e27501aae..74f9b9a32b0a5 100644 --- a/velox/exec/benchmarks/ExchangeBenchmark.cpp +++ b/velox/exec/benchmarks/ExchangeBenchmark.cpp @@ -132,11 +132,12 @@ class ExchangeBenchmark : public VectorTestBase { core::PlanNodePtr finalAggPlan; std::vector finalAggTaskIds; - finalAggPlan = exec::test::PlanBuilder() - .exchange(leafPlan->outputType()) - .singleAggregation({}, {"count(1)"}) - .partitionedOutput({}, 1) - .planNode(); + finalAggPlan = + exec::test::PlanBuilder() + .exchange(leafPlan->outputType(), VectorSerde::Kind::kPresto) + .singleAggregation({}, {"count(1)"}) + .partitionedOutput({}, 1) + .planNode(); std::vector finalAggSplits; for (int i = 0; i < width; i++) { @@ -149,10 +150,11 @@ class ExchangeBenchmark : public VectorTestBase { addRemoteSplits(task, leafTaskIds); } - auto plan = exec::test::PlanBuilder() - .exchange(finalAggPlan->outputType()) - .singleAggregation({}, {"sum(a0)"}) - .planNode(); + auto plan = + exec::test::PlanBuilder() + .exchange(finalAggPlan->outputType(), VectorSerde::Kind::kPresto) + .singleAggregation({}, {"sum(a0)"}) + .planNode(); auto expected = makeRowVector({makeFlatVector(1, [&](auto /*row*/) { diff --git a/velox/exec/tests/CMakeLists.txt b/velox/exec/tests/CMakeLists.txt index 9b228e302463f..a7b1a0996e036 100644 --- a/velox/exec/tests/CMakeLists.txt +++ b/velox/exec/tests/CMakeLists.txt @@ -27,6 +27,7 @@ target_link_libraries( aggregate_companion_functions_test velox_exec velox_function_registry + velox_exec_test_lib GTest::gtest GTest::gtest_main) diff --git a/velox/exec/tests/ExchangeClientTest.cpp b/velox/exec/tests/ExchangeClientTest.cpp index cf0a9eabe5571..60de09c2513f4 100644 --- a/velox/exec/tests/ExchangeClientTest.cpp +++ b/velox/exec/tests/ExchangeClientTest.cpp @@ -18,28 +18,42 @@ #include #include #include "velox/common/base/tests/GTestUtils.h" -#include "velox/exec/Exchange.h" +// #include "velox/exec/Exchange.h" #include "velox/exec/OutputBufferManager.h" #include "velox/exec/Task.h" #include "velox/exec/tests/utils/LocalExchangeSource.h" #include "velox/exec/tests/utils/PlanBuilder.h" #include "velox/exec/tests/utils/QueryAssertions.h" #include "velox/exec/tests/utils/SerializedPageUtil.h" +#include "velox/serializers/CompactRowSerializer.h" #include "velox/serializers/PrestoSerializer.h" +#include "velox/serializers/UnsafeRowSerializer.h" #include "velox/vector/tests/utils/VectorTestBase.h" namespace facebook::velox::exec { namespace { -class ExchangeClientTest : public testing::Test, - public velox::test::VectorTestBase { +class ExchangeClientTest + : public testing::Test, + public velox::test::VectorTestBase, + public testing::WithParamInterface { protected: static void SetUpTestCase() { memory::MemoryManager::testingSetInstance({}); + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kCompactRow)) { + serializer::CompactRowVectorSerde::registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kUnsafeRow)) { + serializer::spark::UnsafeRowVectorSerde::registerNamedVectorSerde(); + } } void SetUp() override { + serdeKind_ = GetParam(); test::testingStartLocalExchangeSource(); executor_ = std::make_unique(16); exec::ExchangeSource::factories().clear(); @@ -82,7 +96,8 @@ class ExchangeClientTest : public testing::Test, const std::string& taskId, int32_t destination, const RowVectorPtr& data) { - auto page = test::toSerializedPage(data, bufferManager_, pool()); + auto page = + test::toSerializedPage(data, serdeKind_, bufferManager_, pool()); const auto pageSize = page->size(); ContinueFuture unused; auto blocked = @@ -143,11 +158,12 @@ class ExchangeClientTest : public testing::Test, return executor_.get(); } + VectorSerde::Kind serdeKind_; std::unique_ptr executor_; std::shared_ptr bufferManager_; }; -TEST_F(ExchangeClientTest, nonVeloxCreateExchangeSourceException) { +TEST_P(ExchangeClientTest, nonVeloxCreateExchangeSourceException) { ExchangeSource::registerFactory( [](const auto& taskId, auto destination, auto queue, auto pool) -> std::shared_ptr { @@ -170,7 +186,7 @@ TEST_F(ExchangeClientTest, nonVeloxCreateExchangeSourceException) { client->close(); } -TEST_F(ExchangeClientTest, stats) { +TEST_P(ExchangeClientTest, stats) { auto data = { makeRowVector({makeFlatVector({1, 2, 3})}), makeRowVector({makeFlatVector({1, 2, 3, 4, 5})}), @@ -215,12 +231,12 @@ TEST_F(ExchangeClientTest, stats) { // Test scenario where fetching data from all sources at once would exceed queue // size. Verify that ExchangeClient is fetching data only from a few sources at // a time to avoid exceeding the limit. -TEST_F(ExchangeClientTest, flowControl) { +TEST_P(ExchangeClientTest, flowControl) { auto data = makeRowVector({ makeFlatVector(10'000, [](auto row) { return row; }), }); - auto page = test::toSerializedPage(data, bufferManager_, pool()); + auto page = test::toSerializedPage(data, serdeKind_, bufferManager_, pool()); // Set limit at 3.5 pages. auto client = std::make_shared( @@ -259,7 +275,7 @@ TEST_F(ExchangeClientTest, flowControl) { client->close(); } -TEST_F(ExchangeClientTest, largeSinglePage) { +TEST_P(ExchangeClientTest, largeSinglePage) { auto data = { makeRowVector({makeFlatVector(10000, folly::identity)}), makeRowVector({makeFlatVector(1, folly::identity)}), @@ -284,7 +300,7 @@ TEST_F(ExchangeClientTest, largeSinglePage) { client->close(); } -TEST_F(ExchangeClientTest, multiPageFetch) { +TEST_P(ExchangeClientTest, multiPageFetch) { auto client = std::make_shared("test", 17, 1 << 20, pool(), executor()); @@ -336,7 +352,7 @@ TEST_F(ExchangeClientTest, multiPageFetch) { client->close(); } -TEST_F(ExchangeClientTest, sourceTimeout) { +TEST_P(ExchangeClientTest, sourceTimeout) { constexpr int32_t kNumSources = 3; auto client = std::make_shared("test", 17, 1 << 20, pool(), executor()); @@ -416,7 +432,7 @@ TEST_F(ExchangeClientTest, sourceTimeout) { client->close(); } -TEST_F(ExchangeClientTest, callNextAfterClose) { +TEST_P(ExchangeClientTest, callNextAfterClose) { constexpr int32_t kNumSources = 3; common::testutil::TestValue::enable(); auto client = @@ -462,7 +478,7 @@ TEST_F(ExchangeClientTest, callNextAfterClose) { client->close(); } -TEST_F(ExchangeClientTest, acknowledge) { +TEST_P(ExchangeClientTest, acknowledge) { const int64_t pageSize = 1024; const int64_t clientBufferSize = pageSize; const int64_t serverBufferSize = 2 * pageSize; @@ -600,5 +616,13 @@ TEST_F(ExchangeClientTest, acknowledge) { ASSERT_TRUE(atEnd); } +VELOX_INSTANTIATE_TEST_SUITE_P( + ExchangeClientTest, + ExchangeClientTest, + testing::Values( + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow)); + } // namespace } // namespace facebook::velox::exec diff --git a/velox/exec/tests/ExchangeFuzzer.cpp b/velox/exec/tests/ExchangeFuzzer.cpp index 13e129b14cc63..1cfb5c6053ca0 100644 --- a/velox/exec/tests/ExchangeFuzzer.cpp +++ b/velox/exec/tests/ExchangeFuzzer.cpp @@ -141,7 +141,7 @@ class ExchangeFuzzer : public VectorTestBase { } auto partialAggPlan = exec::test::PlanBuilder() - .exchange(leafPlan->outputType()) + .exchange(leafPlan->outputType(), VectorSerde::Kind::kPresto) .partialAggregation({}, makeAggregates(rowType, 1)) .partitionedOutput({}, 1) .planNode(); @@ -156,13 +156,14 @@ class ExchangeFuzzer : public VectorTestBase { addRemoteSplits(task, leafTaskIds); } - auto plan = exec::test::PlanBuilder() - .exchange(partialAggPlan->outputType()) - .finalAggregation( - {}, - makeAggregates(*partialAggPlan->outputType(), 0), - rawInputTypes) - .planNode(); + auto plan = + exec::test::PlanBuilder() + .exchange(partialAggPlan->outputType(), VectorSerde::Kind::kPresto) + .finalAggregation( + {}, + makeAggregates(*partialAggPlan->outputType(), 0), + rawInputTypes) + .planNode(); try { // Create the Task to do the final aggregation using a TaskCursor so we diff --git a/velox/exec/tests/MultiFragmentTest.cpp b/velox/exec/tests/MultiFragmentTest.cpp index dfb4efb15eeeb..6c952b43b0dc2 100644 --- a/velox/exec/tests/MultiFragmentTest.cpp +++ b/velox/exec/tests/MultiFragmentTest.cpp @@ -15,10 +15,8 @@ */ #include "folly/experimental/EventCount.h" #include "velox/common/base/tests/GTestUtils.h" -#include "velox/common/file/FileSystems.h" #include "velox/common/testutil/TestValue.h" #include "velox/connectors/hive/HiveConnectorSplit.h" -#include "velox/dwio/common/FileSink.h" #include "velox/dwio/common/tests/utils/BatchMaker.h" #include "velox/exec/Exchange.h" #include "velox/exec/OutputBufferManager.h" @@ -39,7 +37,18 @@ using facebook::velox::test::BatchMaker; namespace facebook::velox::exec { namespace { -class MultiFragmentTest : public HiveConnectorTestBase { +class MultiFragmentTest + : public HiveConnectorTestBase, + public testing::WithParamInterface { + public: + static std::vector getTestParams() { + const std::vector kinds( + {VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}); + return kinds; + } + protected: void SetUp() override { HiveConnectorTestBase::SetUp(); @@ -159,7 +168,7 @@ class MultiFragmentTest : public HiveConnectorTestBase { const std::string& taskId, int32_t destination, const RowVectorPtr& data) { - auto page = toSerializedPage(data, bufferManager_, pool()); + auto page = toSerializedPage(data, GetParam(), bufferManager_, pool()); const auto pageSize = page->size(); ContinueFuture unused; @@ -200,20 +209,24 @@ class MultiFragmentTest : public HiveConnectorTestBase { OutputBufferManager::getInstance().lock()}; }; -TEST_F(MultiFragmentTest, aggregationSingleKey) { +TEST_P(MultiFragmentTest, aggregationSingleKey) { setupSources(10, 1000); std::vector> tasks; auto leafTaskId = makeTaskId("leaf", 0); core::PlanNodePtr partialAggPlan; + core::PlanNodeId partitionNodeId; + std::shared_ptr leafTask; { - partialAggPlan = PlanBuilder() - .tableScan(rowType_) - .project({"c0 % 10 AS c0", "c1"}) - .partialAggregation({"c0"}, {"sum(c1)"}) - .partitionedOutput({"c0"}, 3) - .planNode(); + partialAggPlan = + PlanBuilder() + .tableScan(rowType_) + .project({"c0 % 10 AS c0", "c1"}) + .partialAggregation({"c0"}, {"sum(c1)"}) + .partitionedOutput({"c0"}, 3, /*outputLayout=*/{}, GetParam()) + .capturePlanNodeId(partitionNodeId) + .planNode(); - auto leafTask = makeTask(leafTaskId, partialAggPlan, 0); + leafTask = makeTask(leafTaskId, partialAggPlan, 0); tasks.push_back(leafTask); leafTask->start(4); addHiveSplits(leafTask, filePaths_); @@ -221,21 +234,27 @@ TEST_F(MultiFragmentTest, aggregationSingleKey) { core::PlanNodePtr finalAggPlan; std::vector finalAggTaskIds; + core::PlanNodeId exchangeNodeId; + std::vector> finalTasks; for (int i = 0; i < 3; i++) { - finalAggPlan = PlanBuilder() - .exchange(partialAggPlan->outputType()) - .finalAggregation({"c0"}, {"sum(a0)"}, {{BIGINT()}}) - .partitionedOutput({}, 1) - .planNode(); + finalAggPlan = + PlanBuilder() + .exchange(partialAggPlan->outputType(), GetParam()) + .capturePlanNodeId(exchangeNodeId) + .finalAggregation({"c0"}, {"sum(a0)"}, {{BIGINT()}}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); finalAggTaskIds.push_back(makeTaskId("final-agg", i)); auto task = makeTask(finalAggTaskIds.back(), finalAggPlan, i); tasks.push_back(task); + finalTasks.push_back(task); task->start(1); addRemoteSplits(task, {leafTaskId}); } - auto op = PlanBuilder().exchange(finalAggPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(finalAggPlan->outputType(), GetParam()).planNode(); assertQuery( op, finalAggTaskIds, "SELECT c0 % 10, sum(c1) FROM tmp GROUP BY 1"); @@ -285,20 +304,38 @@ TEST_F(MultiFragmentTest, aggregationSingleKey) { ASSERT_EQ(numPools, 8); } } + auto leafPlanStats = toPlanStats(leafTask->taskStats()); + const auto serdeKindRuntimsStats = + leafPlanStats.at(partitionNodeId) + .customStats.at(Operator::kShuffleSerdeKind); + ASSERT_EQ(serdeKindRuntimsStats.count, 4); + ASSERT_EQ(serdeKindRuntimsStats.min, static_cast(GetParam())); + ASSERT_EQ(serdeKindRuntimsStats.max, static_cast(GetParam())); + + for (const auto& finalTask : finalTasks) { + auto finalPlanStats = toPlanStats(finalTask->taskStats()); + const auto serdeKindRuntimsStats = + finalPlanStats.at(exchangeNodeId) + .customStats.at(Operator::kShuffleSerdeKind); + ASSERT_EQ(serdeKindRuntimsStats.count, 1); + ASSERT_EQ(serdeKindRuntimsStats.min, static_cast(GetParam())); + ASSERT_EQ(serdeKindRuntimsStats.max, static_cast(GetParam())); + } } -TEST_F(MultiFragmentTest, aggregationMultiKey) { +TEST_P(MultiFragmentTest, aggregationMultiKey) { setupSources(10, 1'000); std::vector> tasks; auto leafTaskId = makeTaskId("leaf", 0); core::PlanNodePtr partialAggPlan; { - partialAggPlan = PlanBuilder() - .tableScan(rowType_) - .project({"c0 % 10 AS c0", "c1 % 2 AS c1", "c2"}) - .partialAggregation({"c0", "c1"}, {"sum(c2)"}) - .partitionedOutput({"c0", "c1"}, 3) - .planNode(); + partialAggPlan = + PlanBuilder() + .tableScan(rowType_) + .project({"c0 % 10 AS c0", "c1 % 2 AS c1", "c2"}) + .partialAggregation({"c0", "c1"}, {"sum(c2)"}) + .partitionedOutput({"c0", "c1"}, 3, /*outputLayout=*/{}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, partialAggPlan, 0); tasks.push_back(leafTask); @@ -311,9 +348,9 @@ TEST_F(MultiFragmentTest, aggregationMultiKey) { for (int i = 0; i < 3; i++) { finalAggPlan = PlanBuilder() - .exchange(partialAggPlan->outputType()) + .exchange(partialAggPlan->outputType(), GetParam()) .finalAggregation({"c0", "c1"}, {"sum(a0)"}, {{BIGINT()}}) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); finalAggTaskIds.push_back(makeTaskId("final-agg", i)); @@ -323,7 +360,8 @@ TEST_F(MultiFragmentTest, aggregationMultiKey) { addRemoteSplits(task, {leafTaskId}); } - auto op = PlanBuilder().exchange(finalAggPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(finalAggPlan->outputType(), GetParam()).planNode(); assertQuery( op, @@ -335,23 +373,25 @@ TEST_F(MultiFragmentTest, aggregationMultiKey) { } } -TEST_F(MultiFragmentTest, distributedTableScan) { +TEST_P(MultiFragmentTest, distributedTableScan) { setupSources(10, 1000); // Run the table scan several times to test the caching. for (int i = 0; i < 3; ++i) { auto leafTaskId = makeTaskId("leaf", 0); - auto leafPlan = PlanBuilder() - .tableScan(rowType_) - .project({"c0 % 10", "c1 % 2", "c2"}) - .partitionedOutput({}, 1, {"c2", "p1", "p0"}) - .planNode(); + auto leafPlan = + PlanBuilder() + .tableScan(rowType_) + .project({"c0 % 10", "c1 % 2", "c2"}) + .partitionedOutput({}, 1, {"c2", "p1", "p0"}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(4); addHiveSplits(leafTask, filePaths_); - auto op = PlanBuilder().exchange(leafPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(leafPlan->outputType(), GetParam()).planNode(); auto task = assertQuery(op, {leafTaskId}, "SELECT c2, c1 % 2, c0 % 10 FROM tmp"); @@ -361,7 +401,7 @@ TEST_F(MultiFragmentTest, distributedTableScan) { } } -TEST_F(MultiFragmentTest, mergeExchange) { +TEST_P(MultiFragmentTest, mergeExchange) { setupSources(20, 1000); static const core::SortOrder kAscNullsLast(true, false); @@ -378,19 +418,22 @@ TEST_F(MultiFragmentTest, mergeExchange) { std::vector partialSortTaskIds; RowTypePtr outputType; + core::PlanNodeId partitionNodeId; for (int i = 0; i < 2; ++i) { auto sortTaskId = makeTaskId("orderby", tasks.size()); partialSortTaskIds.push_back(sortTaskId); auto planNodeIdGenerator = std::make_shared(); - auto partialSortPlan = PlanBuilder(planNodeIdGenerator) - .localMerge( - {"c0"}, - {PlanBuilder(planNodeIdGenerator) - .tableScan(rowType_) - .orderBy({"c0"}, true) - .planNode()}) - .partitionedOutput({}, 1) - .planNode(); + auto partialSortPlan = + PlanBuilder(planNodeIdGenerator) + .localMerge( + {"c0"}, + {PlanBuilder(planNodeIdGenerator) + .tableScan(rowType_) + .orderBy({"c0"}, true) + .planNode()}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .capturePlanNodeId(partitionNodeId) + .planNode(); auto sortTask = makeTask(sortTaskId, partialSortPlan, tasks.size()); tasks.push_back(sortTask); @@ -401,18 +444,19 @@ TEST_F(MultiFragmentTest, mergeExchange) { auto finalSortTaskId = makeTaskId("orderby", tasks.size()); core::PlanNodeId mergeExchangeId; - auto finalSortPlan = PlanBuilder() - .mergeExchange(outputType, {"c0"}) - .capturePlanNodeId(mergeExchangeId) - .partitionedOutput({}, 1) - .planNode(); + auto finalSortPlan = + PlanBuilder() + .mergeExchange(outputType, {"c0"}, GetParam()) + .capturePlanNodeId(mergeExchangeId) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); - auto task = makeTask(finalSortTaskId, finalSortPlan, 0); - tasks.push_back(task); - task->start(1); - addRemoteSplits(task, partialSortTaskIds); + auto mergeTask = makeTask(finalSortTaskId, finalSortPlan, 0); + tasks.push_back(mergeTask); + mergeTask->start(1); + addRemoteSplits(mergeTask, partialSortTaskIds); - auto op = PlanBuilder().exchange(outputType).planNode(); + auto op = PlanBuilder().exchange(outputType, GetParam()).planNode(); assertQueryOrdered( op, {finalSortTaskId}, "SELECT * FROM tmp ORDER BY 1 NULLS LAST", {0}); @@ -420,7 +464,7 @@ TEST_F(MultiFragmentTest, mergeExchange) { ASSERT_TRUE(waitForTaskCompletion(task.get())) << task->taskId(); } - const auto finalSortStats = toPlanStats(task->taskStats()); + const auto finalSortStats = toPlanStats(mergeTask->taskStats()); const auto& mergeExchangeStats = finalSortStats.at(mergeExchangeId); EXPECT_EQ(20'000, mergeExchangeStats.inputRows); @@ -428,10 +472,16 @@ TEST_F(MultiFragmentTest, mergeExchange) { EXPECT_LT(0, mergeExchangeStats.inputBytes); EXPECT_LT(0, mergeExchangeStats.rawInputBytes); + + const auto serdeKindRuntimsStats = + mergeExchangeStats.customStats.at(Operator::kShuffleSerdeKind); + ASSERT_EQ(serdeKindRuntimsStats.count, 1); + ASSERT_EQ(serdeKindRuntimsStats.min, static_cast(GetParam())); + ASSERT_EQ(serdeKindRuntimsStats.max, static_cast(GetParam())); } // Test reordering and dropping columns in PartitionedOutput operator. -TEST_F(MultiFragmentTest, partitionedOutput) { +TEST_P(MultiFragmentTest, partitionedOutput) { setupSources(10, 1000); // Test dropping columns only @@ -439,11 +489,12 @@ TEST_F(MultiFragmentTest, partitionedOutput) { auto leafTaskId = makeTaskId("leaf", 0); auto leafPlan = PlanBuilder() .values(vectors_) - .partitionedOutput({}, 1, {"c0", "c1"}) + .partitionedOutput({}, 1, {"c0", "c1"}, GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(4); - auto op = PlanBuilder().exchange(leafPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(leafPlan->outputType(), GetParam()).planNode(); assertQuery(op, {leafTaskId}, "SELECT c0, c1 FROM tmp"); @@ -453,13 +504,15 @@ TEST_F(MultiFragmentTest, partitionedOutput) { // Test reordering and dropping at the same time { auto leafTaskId = makeTaskId("leaf", 0); - auto leafPlan = PlanBuilder() - .values(vectors_) - .partitionedOutput({}, 1, {"c3", "c0", "c2"}) - .planNode(); + auto leafPlan = + PlanBuilder() + .values(vectors_) + .partitionedOutput({}, 1, {"c3", "c0", "c2"}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(4); - auto op = PlanBuilder().exchange(leafPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(leafPlan->outputType(), GetParam()).planNode(); assertQuery(op, {leafTaskId}, "SELECT c3, c0, c2 FROM tmp"); @@ -473,11 +526,15 @@ TEST_F(MultiFragmentTest, partitionedOutput) { PlanBuilder() .values(vectors_) .partitionedOutput( - {}, 1, {"c0", "c1", "c2", "c3", "c4", "c3", "c2", "c1", "c0"}) + {}, + 1, + {"c0", "c1", "c2", "c3", "c4", "c3", "c2", "c1", "c0"}, + GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(4); - auto op = PlanBuilder().exchange(leafPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(leafPlan->outputType(), GetParam()).planNode(); assertQuery( op, {leafTaskId}, "SELECT c0, c1, c2, c3, c4, c3, c2, c1, c0 FROM tmp"); @@ -489,17 +546,19 @@ TEST_F(MultiFragmentTest, partitionedOutput) { { constexpr int32_t kFanout = 4; auto leafTaskId = makeTaskId("leaf", 0); - auto leafPlan = PlanBuilder() - .values(vectors_) - .partitionedOutput({"c5"}, kFanout, {"c2", "c0", "c3"}) - .planNode(); + auto leafPlan = + PlanBuilder() + .values(vectors_) + .partitionedOutput({"c5"}, kFanout, {"c2", "c0", "c3"}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(4); - auto intermediatePlan = PlanBuilder() - .exchange(leafPlan->outputType()) - .partitionedOutput({}, 1, {"c3", "c0", "c2"}) - .planNode(); + auto intermediatePlan = + PlanBuilder() + .exchange(leafPlan->outputType(), GetParam()) + .partitionedOutput({}, 1, {"c3", "c0", "c2"}, GetParam()) + .planNode(); std::vector intermediateTaskIds; for (auto i = 0; i < kFanout; ++i) { intermediateTaskIds.push_back(makeTaskId("intermediate", i)); @@ -509,7 +568,9 @@ TEST_F(MultiFragmentTest, partitionedOutput) { addRemoteSplits(intermediateTask, {leafTaskId}); } - auto op = PlanBuilder().exchange(intermediatePlan->outputType()).planNode(); + auto op = PlanBuilder() + .exchange(intermediatePlan->outputType(), GetParam()) + .planNode(); auto task = assertQuery(op, intermediateTaskIds, "SELECT c3, c0, c2 FROM tmp"); @@ -528,14 +589,15 @@ TEST_F(MultiFragmentTest, partitionedOutput) { [](std::string nodeId, core::PlanNodePtr source) -> core::PlanNodePtr { return core::PartitionedOutputNode::broadcast( - nodeId, 1, ROW({}), source); + nodeId, 1, ROW({}), GetParam(), source); }) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(4); leafTask->updateOutputBuffers(1, true); - auto op = PlanBuilder().exchange(leafPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(leafPlan->outputType(), GetParam()).planNode(); vector_size_t numRows = 0; for (const auto& vector : vectors_) { @@ -556,7 +618,7 @@ TEST_F(MultiFragmentTest, partitionedOutput) { auto leafTaskId = makeTaskId("leaf", 0); auto leafPlan = PlanBuilder() .values(vectors_) - .partitionedOutput({}, 1, {"c0", "c1"}) + .partitionedOutput({}, 1, {"c0", "c1"}, GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(4); @@ -567,7 +629,7 @@ TEST_F(MultiFragmentTest, partitionedOutput) { } } -TEST_F(MultiFragmentTest, noHashPartitionSkew) { +TEST_P(MultiFragmentTest, noHashPartitionSkew) { setupSources(10, 1000); // Update the key column. @@ -583,14 +645,14 @@ TEST_F(MultiFragmentTest, noHashPartitionSkew) { auto producerPlan = PlanBuilder() .values(vectors_) - .partitionedOutput({"c0"}, numPartitions, {"c0", "c1"}) + .partitionedOutput({"c0"}, numPartitions, {"c0", "c1"}, GetParam()) .planNode(); auto producerTask = makeTask(producerTaskId, producerPlan, 0); producerTask->start(1); core::PlanNodeId partialAggregationNodeId; auto consumerPlan = PlanBuilder() - .exchange(producerPlan->outputType()) + .exchange(producerPlan->outputType(), GetParam()) .localPartition({"c0"}) .partialAggregation({"c0"}, {"count(1)"}) .capturePlanNodeId(partialAggregationNodeId) @@ -641,7 +703,7 @@ TEST_F(MultiFragmentTest, noHashPartitionSkew) { } } -TEST_F(MultiFragmentTest, noHivePartitionSkew) { +TEST_P(MultiFragmentTest, noHivePartitionSkew) { setupSources(10, 1000); // Update the key column. @@ -666,14 +728,15 @@ TEST_F(MultiFragmentTest, noHivePartitionSkew) { numBuckets, std::vector{0}, std::vector{}), - {"c0", "c1"}) + {"c0", "c1"}, + GetParam()) .planNode(); auto producerTask = makeTask(producerTaskId, producerPlan, 0); producerTask->start(1); core::PlanNodeId partialAggregationNodeId; auto consumerPlan = PlanBuilder() - .exchange(producerPlan->outputType()) + .exchange(producerPlan->outputType(), GetParam()) .localPartition(numBuckets, {0}, {}) .partialAggregation({"c0"}, {"count(1)"}) .capturePlanNodeId(partialAggregationNodeId) @@ -721,7 +784,7 @@ TEST_F(MultiFragmentTest, noHivePartitionSkew) { } } -TEST_F(MultiFragmentTest, partitionedOutputWithLargeInput) { +TEST_P(MultiFragmentTest, partitionedOutputWithLargeInput) { // Verify that partitionedOutput operator is able to split a single input // vector if it hits memory or row limits. // We create a large vector that hits the row limit (70% - 120% of 10,000). @@ -731,14 +794,15 @@ TEST_F(MultiFragmentTest, partitionedOutputWithLargeInput) { // Single Partition { auto leafTaskId = makeTaskId("leaf", 0); - auto leafPlan = - PlanBuilder() - .values(vectors_) - .partitionedOutput({}, 1, {"c0", "c1", "c2", "c3", "c4"}) - .planNode(); + auto leafPlan = PlanBuilder() + .values(vectors_) + .partitionedOutput( + {}, 1, {"c0", "c1", "c2", "c3", "c4"}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0, nullptr, 4 << 20); leafTask->start(1); - auto op = PlanBuilder().exchange(leafPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(leafPlan->outputType(), GetParam()).planNode(); auto task = assertQuery(op, {leafTaskId}, "SELECT c0, c1, c2, c3, c4 FROM tmp"); @@ -760,15 +824,17 @@ TEST_F(MultiFragmentTest, partitionedOutputWithLargeInput) { kFanout, false, std::make_shared(), - {"c0", "c1", "c2", "c3", "c4"}) + {"c0", "c1", "c2", "c3", "c4"}, + GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(1); auto intermediatePlan = PlanBuilder() - .exchange(leafPlan->outputType()) - .partitionedOutput({}, 1, {"c0", "c1", "c2", "c3", "c4"}) + .exchange(leafPlan->outputType(), GetParam()) + .partitionedOutput( + {}, 1, {"c0", "c1", "c2", "c3", "c4"}, GetParam()) .planNode(); std::vector intermediateTaskIds; for (auto i = 0; i < kFanout; ++i) { @@ -779,7 +845,9 @@ TEST_F(MultiFragmentTest, partitionedOutputWithLargeInput) { addRemoteSplits(intermediateTask, {leafTaskId}); } - auto op = PlanBuilder().exchange(intermediatePlan->outputType()).planNode(); + auto op = PlanBuilder() + .exchange(intermediatePlan->outputType(), GetParam()) + .planNode(); auto task = assertQuery( op, intermediateTaskIds, "SELECT c0, c1, c2, c3, c4 FROM tmp"); @@ -790,7 +858,7 @@ TEST_F(MultiFragmentTest, partitionedOutputWithLargeInput) { } } -TEST_F(MultiFragmentTest, broadcast) { +TEST_P(MultiFragmentTest, broadcast) { auto data = makeRowVector( {makeFlatVector(1'000, [](auto row) { return row; })}); @@ -798,7 +866,10 @@ TEST_F(MultiFragmentTest, broadcast) { std::vector> tasks; auto leafTaskId = makeTaskId("leaf", 0); auto leafPlan = - PlanBuilder().values({data}).partitionedOutputBroadcast().planNode(); + PlanBuilder() + .values({data}) + .partitionedOutputBroadcast(/*outputLayout=*/{}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); tasks.emplace_back(leafTask); leafTask->start(1); @@ -807,11 +878,12 @@ TEST_F(MultiFragmentTest, broadcast) { core::PlanNodePtr finalAggPlan; std::vector finalAggTaskIds; for (int i = 0; i < 3; i++) { - finalAggPlan = PlanBuilder() - .exchange(leafPlan->outputType()) - .singleAggregation({}, {"count(1)"}) - .partitionedOutput({}, 1) - .planNode(); + finalAggPlan = + PlanBuilder() + .exchange(leafPlan->outputType(), GetParam()) + .singleAggregation({}, {"count(1)"}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); finalAggTaskIds.push_back(makeTaskId("final-agg", i)); auto task = makeTask(finalAggTaskIds.back(), finalAggPlan, i); @@ -823,7 +895,8 @@ TEST_F(MultiFragmentTest, broadcast) { leafTask->updateOutputBuffers(finalAggTaskIds.size(), true); // Collect results from multiple tasks. - auto op = PlanBuilder().exchange(finalAggPlan->outputType()).planNode(); + auto op = + PlanBuilder().exchange(finalAggPlan->outputType(), GetParam()).planNode(); assertQuery(op, finalAggTaskIds, "SELECT UNNEST(array[1000, 1000, 1000])"); @@ -836,7 +909,7 @@ TEST_F(MultiFragmentTest, broadcast) { leafTask->updateOutputBuffers(finalAggTaskIds.size(), true); } -TEST_F(MultiFragmentTest, roundRobinPartition) { +TEST_P(MultiFragmentTest, roundRobinPartition) { auto data = { makeRowVector({ makeFlatVector({1, 2, 3, 4, 5}), @@ -860,7 +933,9 @@ TEST_F(MultiFragmentTest, roundRobinPartition) { {}, 2, false, - std::make_shared()) + std::make_shared(), + /*outputLayout=*/{}, + GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); @@ -881,8 +956,8 @@ TEST_F(MultiFragmentTest, roundRobinPartition) { std::vector collectTaskIds; for (int i = 0; i < 2; i++) { collectPlan = PlanBuilder() - .exchange(leafPlan->outputType()) - .partitionedOutput({}, 1) + .exchange(leafPlan->outputType(), GetParam()) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); collectTaskIds.push_back(makeTaskId("collect", i)); @@ -891,7 +966,8 @@ TEST_F(MultiFragmentTest, roundRobinPartition) { } // Collect everything. - auto finalPlan = PlanBuilder().exchange(leafPlan->outputType()).planNode(); + auto finalPlan = + PlanBuilder().exchange(leafPlan->outputType(), GetParam()).planNode(); assertQuery(finalPlan, {collectTaskIds}, "SELECT * FROM tmp"); @@ -901,7 +977,7 @@ TEST_F(MultiFragmentTest, roundRobinPartition) { } // Test PartitionedOutput operator with constant partitioning keys. -TEST_F(MultiFragmentTest, constantKeys) { +TEST_P(MultiFragmentTest, constantKeys) { auto data = makeRowVector({ makeFlatVector( 1'000, [](auto row) { return row; }, nullEvery(7)), @@ -919,10 +995,11 @@ TEST_F(MultiFragmentTest, constantKeys) { // Make leaf task: Values -> Repartitioning (3-way) auto leafTaskId = makeTaskId("leaf", 0); - auto leafPlan = PlanBuilder() - .values({data}) - .partitionedOutput({"c0", "123"}, 3, true, {"c0"}) - .planNode(); + auto leafPlan = + PlanBuilder() + .values({data}) + .partitionedOutput({"c0", "123"}, 3, true, {"c0"}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); addTask(leafTask, {}); @@ -932,10 +1009,10 @@ TEST_F(MultiFragmentTest, constantKeys) { for (int i = 0; i < 3; i++) { finalAggPlan = PlanBuilder() - .exchange(leafPlan->outputType()) + .exchange(leafPlan->outputType(), GetParam()) .project({"c0 is null AS co_is_null"}) .partialAggregation({}, {"count_if(co_is_null)", "count(1)"}) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); finalAggTaskIds.push_back(makeTaskId("final-agg", i)); @@ -946,7 +1023,7 @@ TEST_F(MultiFragmentTest, constantKeys) { // Collect results and verify number of nulls is 3 times larger than in the // original data. auto op = PlanBuilder() - .exchange(finalAggPlan->outputType()) + .exchange(finalAggPlan->outputType(), GetParam()) .finalAggregation( {}, {"sum(a0)", "sum(a1)"}, {{BIGINT()}, {BIGINT()}}) .planNode(); @@ -961,7 +1038,7 @@ TEST_F(MultiFragmentTest, constantKeys) { } } -TEST_F(MultiFragmentTest, replicateNullsAndAny) { +TEST_P(MultiFragmentTest, replicateNullsAndAny) { auto data = makeRowVector({makeFlatVector( 1'000, [](auto row) { return row; }, nullEvery(7))}); @@ -977,10 +1054,11 @@ TEST_F(MultiFragmentTest, replicateNullsAndAny) { // Make leaf task: Values -> Repartitioning (3-way) auto leafTaskId = makeTaskId("leaf", 0); - auto leafPlan = PlanBuilder() - .values({data}) - .partitionedOutput({"c0"}, 3, true) - .planNode(); + auto leafPlan = + PlanBuilder() + .values({data}) + .partitionedOutput({"c0"}, 3, true, /*outputLayout=*/{}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); addTask(leafTask, {}); @@ -990,10 +1068,10 @@ TEST_F(MultiFragmentTest, replicateNullsAndAny) { for (int i = 0; i < 3; i++) { finalAggPlan = PlanBuilder() - .exchange(leafPlan->outputType()) + .exchange(leafPlan->outputType(), GetParam()) .project({"c0 is null AS co_is_null"}) .partialAggregation({}, {"count_if(co_is_null)", "count(1)"}) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); finalAggTaskIds.push_back(makeTaskId("final-agg", i)); @@ -1004,7 +1082,7 @@ TEST_F(MultiFragmentTest, replicateNullsAndAny) { // Collect results and verify number of nulls is 3 times larger than in the // original data. auto op = PlanBuilder() - .exchange(finalAggPlan->outputType()) + .exchange(finalAggPlan->outputType(), GetParam()) .finalAggregation( {}, {"sum(a0)", "sum(a1)"}, {{BIGINT()}, {BIGINT()}}) .planNode(); @@ -1020,7 +1098,7 @@ TEST_F(MultiFragmentTest, replicateNullsAndAny) { } // Test query finishing before all splits have been scheduled. -TEST_F(MultiFragmentTest, limit) { +TEST_P(MultiFragmentTest, limit) { auto data = makeRowVector({makeFlatVector( 1'000, [](auto row) { return row; }, nullEvery(7))}); @@ -1033,7 +1111,7 @@ TEST_F(MultiFragmentTest, limit) { PlanBuilder() .tableScan(std::dynamic_pointer_cast(data->type())) .limit(0, 10, true) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(1); @@ -1043,7 +1121,7 @@ TEST_F(MultiFragmentTest, limit) { // Make final task: Exchange -> FinalLimit(10). auto plan = PlanBuilder() - .exchange(leafPlan->outputType()) + .exchange(leafPlan->outputType(), GetParam()) .localPartition(std::vector{}) .limit(0, 10, false) .planNode(); @@ -1066,7 +1144,7 @@ TEST_F(MultiFragmentTest, limit) { ASSERT_TRUE(waitForTaskCompletion(leafTask.get())) << leafTask->taskId(); } -TEST_F(MultiFragmentTest, mergeExchangeOverEmptySources) { +TEST_P(MultiFragmentTest, mergeExchangeOverEmptySources) { std::vector> tasks; std::vector leafTaskIds; @@ -1075,8 +1153,10 @@ TEST_F(MultiFragmentTest, mergeExchangeOverEmptySources) { for (int i = 0; i < 2; ++i) { auto taskId = makeTaskId("leaf-", i); leafTaskIds.push_back(taskId); - auto plan = - PlanBuilder().values({data}).partitionedOutput({}, 1).planNode(); + auto plan = PlanBuilder() + .values({data}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); auto task = makeTask(taskId, plan, tasks.size()); tasks.push_back(task); @@ -1085,7 +1165,7 @@ TEST_F(MultiFragmentTest, mergeExchangeOverEmptySources) { auto exchangeTaskId = makeTaskId("exchange-", 0); auto plan = PlanBuilder() - .mergeExchange(rowType_, {"c0"}) + .mergeExchange(rowType_, {"c0"}, GetParam()) .singleAggregation({"c0"}, {"count(1)"}) .planNode(); @@ -1099,22 +1179,23 @@ TEST_F(MultiFragmentTest, mergeExchangeOverEmptySources) { namespace { core::PlanNodePtr makeJoinOverExchangePlan( const RowTypePtr& exchangeType, - const RowVectorPtr& buildData) { + const RowVectorPtr& buildData, + VectorSerde::Kind serdeKind) { auto planNodeIdGenerator = std::make_shared(); return PlanBuilder(planNodeIdGenerator) - .exchange(exchangeType) + .exchange(exchangeType, serdeKind) .hashJoin( {"c0"}, {"u_c0"}, PlanBuilder(planNodeIdGenerator).values({buildData}).planNode(), "", {"c0"}) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, serdeKind) .planNode(); } } // namespace -TEST_F(MultiFragmentTest, earlyCompletion) { +TEST_P(MultiFragmentTest, earlyCompletion) { // Setup a distributed query with 4 tasks: // - 1 leaf task with results partitioned 2 ways; // - 2 intermediate tasks reading from 2 partitions produced by the leaf task. @@ -1142,7 +1223,7 @@ TEST_F(MultiFragmentTest, earlyCompletion) { auto leafTaskId = makeTaskId("leaf", 0); auto plan = PlanBuilder() .values({data, data, data, data}) - .partitionedOutput({"c0"}, 2) + .partitionedOutput({"c0"}, 2, /*outputLayout=*/{}, GetParam()) .planNode(); auto task = makeTask(leafTaskId, plan, tasks.size()); @@ -1161,8 +1242,8 @@ TEST_F(MultiFragmentTest, earlyCompletion) { {"u_c0"}, {makeFlatVector({1, 2, 3, 4, 5, 6})}); } - auto joinPlan = - makeJoinOverExchangePlan(asRowType(data->type()), buildData); + auto joinPlan = makeJoinOverExchangePlan( + asRowType(data->type()), buildData, GetParam()); joinOutputType = joinPlan->outputType(); @@ -1177,7 +1258,8 @@ TEST_F(MultiFragmentTest, earlyCompletion) { } // Create output task. - auto outputPlan = PlanBuilder().exchange(joinOutputType).planNode(); + auto outputPlan = + PlanBuilder().exchange(joinOutputType, GetParam()).planNode(); assertQuery( outputPlan, joinTaskIds, "SELECT UNNEST([3, 3, 3, 3, 4, 4, 4, 4])"); @@ -1187,7 +1269,7 @@ TEST_F(MultiFragmentTest, earlyCompletion) { } } -TEST_F(MultiFragmentTest, earlyCompletionBroadcast) { +TEST_P(MultiFragmentTest, earlyCompletionBroadcast) { // Same as 'earlyCompletion' test, but broadcasts leaf task results to all // intermediate tasks. @@ -1205,7 +1287,7 @@ TEST_F(MultiFragmentTest, earlyCompletionBroadcast) { auto leafTaskId = makeTaskId("leaf", 0); auto plan = PlanBuilder() .values({data, data, data, data}) - .partitionedOutputBroadcast() + .partitionedOutputBroadcast(/*outputLayout=*/{}, GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, plan, tasks.size()); @@ -1224,8 +1306,8 @@ TEST_F(MultiFragmentTest, earlyCompletionBroadcast) { {"u_c0"}, {makeFlatVector({-7, 10, 12345678})}); } - auto joinPlan = - makeJoinOverExchangePlan(asRowType(data->type()), buildData); + auto joinPlan = makeJoinOverExchangePlan( + asRowType(data->type()), buildData, GetParam()); joinOutputType = joinPlan->outputType(); @@ -1243,7 +1325,8 @@ TEST_F(MultiFragmentTest, earlyCompletionBroadcast) { leafTask->updateOutputBuffers(joinTaskIds.size(), true); // Create output task. - auto outputPlan = PlanBuilder().exchange(joinOutputType).planNode(); + auto outputPlan = + PlanBuilder().exchange(joinOutputType, GetParam()).planNode(); assertQuery(outputPlan, joinTaskIds, "SELECT UNNEST([10, 10, 10, 10])"); @@ -1252,7 +1335,7 @@ TEST_F(MultiFragmentTest, earlyCompletionBroadcast) { } } -TEST_F(MultiFragmentTest, earlyCompletionMerge) { +TEST_P(MultiFragmentTest, earlyCompletionMerge) { // Same as 'earlyCompletion' test, but uses MergeExchange instead of Exchange. std::vector> tasks; @@ -1269,7 +1352,7 @@ TEST_F(MultiFragmentTest, earlyCompletionMerge) { auto leafTaskId = makeTaskId("leaf", 0); auto plan = PlanBuilder() .values({data, data, data, data}) - .partitionedOutput({"c0"}, 2) + .partitionedOutput({"c0"}, 2, /*outputLayout=*/{}, GetParam()) .planNode(); auto task = makeTask(leafTaskId, plan, tasks.size()); @@ -1291,14 +1374,14 @@ TEST_F(MultiFragmentTest, earlyCompletionMerge) { auto planNodeIdGenerator = std::make_shared(); auto joinPlan = PlanBuilder(planNodeIdGenerator) - .mergeExchange(asRowType(data->type()), {"c0"}) + .mergeExchange(asRowType(data->type()), {"c0"}, GetParam()) .hashJoin( {"c0"}, {"u_c0"}, PlanBuilder(planNodeIdGenerator).values({buildData}).planNode(), "", {"c0"}) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); joinOutputType = joinPlan->outputType(); @@ -1314,7 +1397,8 @@ TEST_F(MultiFragmentTest, earlyCompletionMerge) { } // Create output task. - auto outputPlan = PlanBuilder().exchange(joinOutputType).planNode(); + auto outputPlan = + PlanBuilder().exchange(joinOutputType, GetParam()).planNode(); assertQuery( outputPlan, joinTaskIds, "SELECT UNNEST([3, 3, 3, 3, 4, 4, 4, 4])"); @@ -1402,7 +1486,7 @@ class SlowOperatorTranslator : public Operator::PlanNodeTranslator { } }; -TEST_F(MultiFragmentTest, exchangeDestruction) { +TEST_P(MultiFragmentTest, exchangeDestruction) { // This unit test tests the proper destruction of ExchangeClient upon // task destruction. Operator::registerOperator(std::make_unique()); @@ -1420,7 +1504,7 @@ TEST_F(MultiFragmentTest, exchangeDestruction) { leafPlan = PlanBuilder() .tableScan(rowType_) .project({"c0 % 10 AS c0", "c1"}) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); @@ -1429,11 +1513,11 @@ TEST_F(MultiFragmentTest, exchangeDestruction) { auto rootPlan = PlanBuilder() - .exchange(leafPlan->outputType()) + .exchange(leafPlan->outputType(), GetParam()) .addNode([&leafPlan](std::string id, core::PlanNodePtr node) { return std::make_shared(id, std::move(node)); }) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); auto rootTask = makeTask("root-task", rootPlan, 0); @@ -1451,19 +1535,19 @@ TEST_F(MultiFragmentTest, exchangeDestruction) { rootTask = nullptr; } -TEST_F(MultiFragmentTest, cancelledExchange) { +TEST_P(MultiFragmentTest, cancelledExchange) { // Create a source fragment borrow the output type from it. auto planFragment = exec::test::PlanBuilder() .tableScan(rowType_) .filter("c0 % 5 = 1") - .partitionedOutput({}, 1, {"c0", "c1"}) + .partitionedOutput({}, 1, {"c0", "c1"}, GetParam()) .planFragment(); // Create task with exchange. auto planFragmentWithExchange = exec::test::PlanBuilder() - .exchange(planFragment.planNode->outputType()) - .partitionedOutput({}, 1) + .exchange(planFragment.planNode->outputType(), GetParam()) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planFragment(); auto exchangeTask = makeTask("output.0.0.1", planFragmentWithExchange.planNode, 0); @@ -1480,13 +1564,20 @@ TEST_F(MultiFragmentTest, cancelledExchange) { class TestCustomExchangeNode : public core::PlanNode { public: - TestCustomExchangeNode(const core::PlanNodeId& id, const RowTypePtr type) - : PlanNode(id), outputType_(type) {} + TestCustomExchangeNode( + const core::PlanNodeId& id, + const RowTypePtr type, + VectorSerde::Kind serdeKind) + : PlanNode(id), outputType_(type), serdeKind_(serdeKind) {} const RowTypePtr& outputType() const override { return outputType_; } + VectorSerde::Kind serdeKind() const { + return serdeKind_; + } + const std::vector& sources() const override { static std::vector kEmptySources; return kEmptySources; @@ -1509,7 +1600,8 @@ class TestCustomExchangeNode : public core::PlanNode { // Nothing to add } - RowTypePtr outputType_; + const RowTypePtr outputType_; + const VectorSerde::Kind serdeKind_; }; class TestCustomExchange : public exec::Exchange { @@ -1524,7 +1616,8 @@ class TestCustomExchange : public exec::Exchange { ctx, std::make_shared( customExchangeNode->id(), - customExchangeNode->outputType()), + customExchangeNode->outputType(), + customExchangeNode->serdeKind()), std::move(exchangeClient)) {} RowVectorPtr getOutput() override { @@ -1549,12 +1642,16 @@ class TestCustomExchangeTranslator : public exec::Operator::PlanNodeTranslator { } }; -TEST_F(MultiFragmentTest, customPlanNodeWithExchangeClient) { +TEST_P(MultiFragmentTest, customPlanNodeWithExchangeClient) { setupSources(5, 100); Operator::registerOperator(std::make_unique()); auto leafTaskId = makeTaskId("leaf", 0); - auto leafPlan = - PlanBuilder().values(vectors_).partitionedOutput({}, 1).planNode(); + core::PlanNodeId partitionNodeId; + auto leafPlan = PlanBuilder() + .values(vectors_) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .capturePlanNodeId(partitionNodeId) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(1); @@ -1565,7 +1662,7 @@ TEST_F(MultiFragmentTest, customPlanNodeWithExchangeClient) { PlanBuilder() .addNode([&leafPlan](std::string id, core::PlanNodePtr /* input */) { return std::make_shared( - id, leafPlan->outputType()); + id, leafPlan->outputType(), GetParam()); }) .capturePlanNodeId(testNodeId) .planNode(); @@ -1584,6 +1681,13 @@ TEST_F(MultiFragmentTest, customPlanNodeWithExchangeClient) { .at(testNodeId) .customStats.count("testCustomExchangeStat"), 0); + + auto planStats = toPlanStats(leafTask->taskStats()); + const auto serdeKindRuntimsStats = + planStats.at(partitionNodeId).customStats.at(Operator::kShuffleSerdeKind); + ASSERT_EQ(serdeKindRuntimsStats.count, 1); + ASSERT_EQ(serdeKindRuntimsStats.min, static_cast(GetParam())); + ASSERT_EQ(serdeKindRuntimsStats.max, static_cast(GetParam())); } // This test is to reproduce the race condition between task terminate and no @@ -1596,16 +1700,17 @@ TEST_F(MultiFragmentTest, customPlanNodeWithExchangeClient) { // task is not running. // T5: task terminate processes the pending remote splits by accessing the // associated exchange client and run into segment fault. -DEBUG_ONLY_TEST_F( +DEBUG_ONLY_TEST_P( MultiFragmentTest, raceBetweenTaskTerminateAndTaskNoMoreSplits) { setupSources(10, 1000); auto leafTaskId = makeTaskId("leaf", 0); - core::PlanNodePtr leafPlan = PlanBuilder() - .tableScan(rowType_) - .project({"c0 % 10 AS c0", "c1"}) - .partitionedOutput({}, 1) - .planNode(); + core::PlanNodePtr leafPlan = + PlanBuilder() + .tableScan(rowType_) + .project({"c0 % 10 AS c0", "c1"}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); auto leafTask = makeTask(leafTaskId, leafPlan, 0); leafTask->start(1); addHiveSplits(leafTask, filePaths_); @@ -1637,7 +1742,7 @@ DEBUG_ONLY_TEST_F( blockTerminate.await([&]() { return readyToTerminate.load(); }); }))); auto rootPlan = PlanBuilder() - .exchange(leafPlan->outputType()) + .exchange(leafPlan->outputType(), GetParam()) .finalAggregation({"c0"}, {"count(c1)"}, {{BIGINT()}}) .planNode(); @@ -1659,12 +1764,14 @@ DEBUG_ONLY_TEST_F( ASSERT_TRUE(waitForTaskFailure(rootTask.get(), 1'000'000'000)); } -TEST_F(MultiFragmentTest, taskTerminateWithPendingOutputBuffers) { +TEST_P(MultiFragmentTest, taskTerminateWithPendingOutputBuffers) { setupSources(8, 1000); auto taskId = makeTaskId("task", 0); core::PlanNodePtr leafPlan; - leafPlan = - PlanBuilder().tableScan(rowType_).partitionedOutput({}, 1).planNode(); + leafPlan = PlanBuilder() + .tableScan(rowType_) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); auto task = makeTask(taskId, leafPlan, 0); task->start(1); @@ -1725,7 +1832,7 @@ TEST_F(MultiFragmentTest, taskTerminateWithPendingOutputBuffers) { task.reset(); } -DEBUG_ONLY_TEST_F( +DEBUG_ONLY_TEST_P( MultiFragmentTest, taskTerminateWithProblematicRemainingRemoteSplits) { // Start the task with 2 drivers. @@ -1739,12 +1846,12 @@ DEBUG_ONLY_TEST_F( {"p_c0"}, {"c0"}, PlanBuilder(planNodeIdGenerator) - .exchange(rowType_) + .exchange(rowType_, GetParam()) .capturePlanNodeId(exchangeNodeId) .planNode(), "", {"c0"}) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .planNode(); auto taskId = makeTaskId("final", 0); auto task = makeTask(taskId, plan, 0); @@ -1796,7 +1903,7 @@ DEBUG_ONLY_TEST_F( failThread.join(); } -DEBUG_ONLY_TEST_F(MultiFragmentTest, mergeWithEarlyTermination) { +DEBUG_ONLY_TEST_P(MultiFragmentTest, mergeWithEarlyTermination) { setupSources(10, 1000); std::vector> filePaths( @@ -1806,15 +1913,16 @@ DEBUG_ONLY_TEST_F(MultiFragmentTest, mergeWithEarlyTermination) { auto sortTaskId = makeTaskId("orderby", 0); partialSortTaskIds.push_back(sortTaskId); auto planNodeIdGenerator = std::make_shared(); - auto partialSortPlan = PlanBuilder(planNodeIdGenerator) - .localMerge( - {"c0"}, - {PlanBuilder(planNodeIdGenerator) - .tableScan(rowType_) - .orderBy({"c0"}, true) - .planNode()}) - .partitionedOutput({}, 1) - .planNode(); + auto partialSortPlan = + PlanBuilder(planNodeIdGenerator) + .localMerge( + {"c0"}, + {PlanBuilder(planNodeIdGenerator) + .tableScan(rowType_) + .orderBy({"c0"}, true) + .planNode()}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); auto partialSortTask = makeTask(sortTaskId, partialSortPlan, 1); partialSortTask->start(1); @@ -1838,10 +1946,11 @@ DEBUG_ONLY_TEST_F(MultiFragmentTest, mergeWithEarlyTermination) { })); auto finalSortTaskId = makeTaskId("orderby", 1); - auto finalSortPlan = PlanBuilder() - .mergeExchange(partialSortPlan->outputType(), {"c0"}) - .partitionedOutput({}, 1) - .planNode(); + auto finalSortPlan = + PlanBuilder() + .mergeExchange(partialSortPlan->outputType(), {"c0"}, GetParam()) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); auto finalSortTask = makeTask(finalSortTaskId, finalSortPlan, 0); finalSortTask->start(1); addRemoteSplits(finalSortTask, partialSortTaskIds); @@ -1880,6 +1989,14 @@ class DataFetcher { int64_t averagePacketBytes() const { return numPackets > 0 ? (totalBytes / numPackets) : 0; } + + std::string toString() const { + return fmt::format( + "numPackets {} numPages {} totalBytes {}", + numPackets, + numPages, + totalBytes); + } }; Stats stats() const { @@ -1947,7 +2064,7 @@ class DataFetcher { /// granularity. It can do so only if PartitionedOutput operator limits the size /// of individual pages. PartitionedOutput operator is expected to limit page /// sizes to no more than 1MB give and take 30%. -TEST_F(MultiFragmentTest, maxBytes) { +TEST_P(MultiFragmentTest, maxBytes) { std::string s(25, 'x'); // Keep the row count under 7000 to avoid hitting the row limit in the // operator instead. @@ -1960,7 +2077,7 @@ TEST_F(MultiFragmentTest, maxBytes) { core::PlanNodeId outputNodeId; auto plan = PlanBuilder() .values({data}, false, 100) - .partitionedOutput({}, 1) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) .capturePlanNodeId(outputNodeId) .planNode(); @@ -1971,6 +2088,7 @@ TEST_F(MultiFragmentTest, maxBytes) { const auto taskId = fmt::format("test.{}", testIteration++); SCOPED_TRACE(taskId); + SCOPED_TRACE(fmt::format("maxBytes: {}", maxBytes)); auto task = makeTask(taskId, plan, 0); task->start(1); task->updateOutputBuffers(1, true); @@ -1983,11 +2101,11 @@ TEST_F(MultiFragmentTest, maxBytes) { ASSERT_TRUE(waitForTaskCompletion(task.get())); - auto stats = fetcher.stats(); + const auto stats = fetcher.stats(); if (testIteration > 1) { ASSERT_EQ(prevStats.numPages, stats.numPages); ASSERT_EQ(prevStats.totalBytes, stats.totalBytes); - ASSERT_GT(prevStats.numPackets, stats.numPackets); + ASSERT_GT(prevStats.numPackets, stats.numPackets) << stats.toString(); } ASSERT_LT(stats.averagePacketBytes(), maxBytes * 1.5); @@ -2007,11 +2125,11 @@ TEST_F(MultiFragmentTest, maxBytes) { test(5 * kMB); test(10 * kMB); test(20 * kMB); - test(32 * kMB); + test(35 * kMB); } // Verifies that ExchangeClient stats are populated even if task fails. -DEBUG_ONLY_TEST_F(MultiFragmentTest, exchangeStatsOnFailure) { +DEBUG_ONLY_TEST_P(MultiFragmentTest, exchangeStatsOnFailure) { // Triggers a failure after fetching first 10 pages. std::atomic_uint64_t expectedReceivedPages{0}; SCOPED_TESTVALUE_SET( @@ -2033,17 +2151,19 @@ DEBUG_ONLY_TEST_F(MultiFragmentTest, exchangeStatsOnFailure) { makeConstant(StringView(s), 10'000), }); - auto producerPlan = PlanBuilder() - .values({data}, false, 30) - .partitionedOutput({}, 1) - .planNode(); + auto producerPlan = + PlanBuilder() + .values({data}, false, 30) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); auto producerTaskId = makeTaskId("producer", 0); auto producerTask = makeTask(producerTaskId, producerPlan, 0); producerTask->start(1); producerTask->updateOutputBuffers(1, true); - auto plan = PlanBuilder().exchange(producerPlan->outputType()).planNode(); + auto plan = + PlanBuilder().exchange(producerPlan->outputType(), GetParam()).planNode(); auto task = makeTask("t", plan, 0, noopConsumer()); task->start(4); @@ -2064,20 +2184,21 @@ DEBUG_ONLY_TEST_F(MultiFragmentTest, exchangeStatsOnFailure) { ASSERT_TRUE(waitForTaskCompletion(producerTask.get(), 3'000'000)); } -TEST_F(MultiFragmentTest, earlyTaskFailure) { +TEST_P(MultiFragmentTest, earlyTaskFailure) { setupSources(1, 10); const auto partialSortTaskId = makeTaskId("partialSortBy", 0); auto planNodeIdGenerator = std::make_shared(); - auto partialSortPlan = PlanBuilder(planNodeIdGenerator) - .localMerge( - {"c0"}, - {PlanBuilder(planNodeIdGenerator) - .tableScan(rowType_) - .orderBy({"c0"}, true) - .planNode()}) - .partitionedOutput({}, 1) - .planNode(); + auto partialSortPlan = + PlanBuilder(planNodeIdGenerator) + .localMerge( + {"c0"}, + {PlanBuilder(planNodeIdGenerator) + .tableScan(rowType_) + .orderBy({"c0"}, true) + .planNode()}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); for (bool internalFailure : {false, true}) { SCOPED_TRACE(fmt::format("internalFailure: {}", internalFailure)); @@ -2088,7 +2209,7 @@ TEST_F(MultiFragmentTest, earlyTaskFailure) { auto finalSortTaskId = makeTaskId("finalSortBy", 0); auto finalSortPlan = PlanBuilder() - .mergeExchange(outputType, {"c0"}) + .mergeExchange(outputType, {"c0"}, GetParam()) .partitionedOutput({}, 1) .planNode(); @@ -2119,17 +2240,21 @@ TEST_F(MultiFragmentTest, earlyTaskFailure) { } } -TEST_F(MultiFragmentTest, mergeSmallBatchesInExchange) { +TEST_P(MultiFragmentTest, mergeSmallBatchesInExchange) { auto data = makeRowVector({makeFlatVector({1, 2, 3})}); const int32_t numPartitions = 100; - auto producerPlan = test::PlanBuilder() - .values({data}) - .partitionedOutput({"c0"}, numPartitions) - .planNode(); + auto producerPlan = + test::PlanBuilder() + .values({data}) + .partitionedOutput( + {"c0"}, numPartitions, /*outputLayout=*/{}, GetParam()) + .planNode(); const auto producerTaskId = "local://t1"; - auto plan = test::PlanBuilder().exchange(asRowType(data->type())).planNode(); + auto plan = test::PlanBuilder() + .exchange(asRowType(data->type()), GetParam()) + .planNode(); auto expected = makeRowVector({ makeFlatVector(3'000, [](auto row) { return 1 + row % 3; }), @@ -2172,13 +2297,29 @@ TEST_F(MultiFragmentTest, mergeSmallBatchesInExchange) { ASSERT_EQ(numPages, stats.customStats.at("numReceivedPages").sum); }; - test(1, 1'000); - test(1'000, 56); - test(10'000, 6); - test(100'000, 1); + if (GetParam() == VectorSerde::Kind::kPresto) { + test(1, 1'000); + test(1'000, 56); + test(10'000, 6); + test(100'000, 1); + } else if (GetParam() == VectorSerde::Kind::kCompactRow) { + test(1, 1'000); + test(1'000, 28); + test(10'000, 3); + test(100'000, 1); + } else { + test(1, 1'000); + test(1'000, 63); + test(10'000, 7); + test(100'000, 1); + } } -TEST_F(MultiFragmentTest, compression) { +TEST_P(MultiFragmentTest, compression) { + // NOTE: only presto format supports compression for now + if (GetParam() != VectorSerde::Kind::kPresto) { + return; + } bufferManager_->testingSetCompression( common::CompressionKind::CompressionKind_LZ4); auto guard = folly::makeGuard([&]() { @@ -2189,13 +2330,14 @@ TEST_F(MultiFragmentTest, compression) { constexpr int32_t kNumRepeats = 1'000'000; const auto data = makeRowVector({makeFlatVector({1, 2, 3})}); - const auto producerPlan = test::PlanBuilder() - .values({data}, false, kNumRepeats) - .partitionedOutput({}, 1) - .planNode(); + const auto producerPlan = + test::PlanBuilder() + .values({data}, false, kNumRepeats) + .partitionedOutput({}, 1, /*outputLayout=*/{}, GetParam()) + .planNode(); const auto plan = test::PlanBuilder() - .exchange(asRowType(data->type())) + .exchange(asRowType(data->type()), GetParam()) .singleAggregation({}, {"sum(c0)"}) .planNode(); @@ -2235,5 +2377,10 @@ TEST_F(MultiFragmentTest, compression) { test("local://t2", 0.0000001, true); } +VELOX_INSTANTIATE_TEST_SUITE_P( + MultiFragmentTest, + MultiFragmentTest, + testing::ValuesIn(MultiFragmentTest::getTestParams())); + } // namespace } // namespace facebook::velox::exec diff --git a/velox/exec/tests/OutputBufferManagerTest.cpp b/velox/exec/tests/OutputBufferManagerTest.cpp index c8414c3b2ecc5..e341ae4329fe0 100644 --- a/velox/exec/tests/OutputBufferManagerTest.cpp +++ b/velox/exec/tests/OutputBufferManagerTest.cpp @@ -21,7 +21,9 @@ #include "velox/exec/Task.h" #include "velox/exec/tests/utils/PlanBuilder.h" #include "velox/exec/tests/utils/SerializedPageUtil.h" +#include "velox/serializers/CompactRowSerializer.h" #include "velox/serializers/PrestoSerializer.h" +#include "velox/serializers/UnsafeRowSerializer.h" using namespace facebook::velox; using namespace facebook::velox::exec; @@ -29,9 +31,26 @@ using namespace facebook::velox::core; using facebook::velox::test::BatchMaker; +struct TestParam { + PartitionedOutputNode::Kind outputKind; + VectorSerde::Kind serdeKind; + + TestParam( + PartitionedOutputNode::Kind _outputKind, + VectorSerde::Kind _serdeKind) + : outputKind(_outputKind), serdeKind(_serdeKind) {} +}; + class OutputBufferManagerTest : public testing::Test { protected: - OutputBufferManagerTest() { + OutputBufferManagerTest() : serdeKind_(VectorSerde::Kind::kPresto) { + std::vector names = {"c0", "c1"}; + std::vector types = {BIGINT(), VARCHAR()}; + rowType_ = ROW(std::move(names), std::move(types)); + } + + explicit OutputBufferManagerTest(VectorSerde::Kind serdeKind) + : serdeKind_(serdeKind) { std::vector names = {"c0", "c1"}; std::vector types = {BIGINT(), VARCHAR()}; rowType_ = ROW(std::move(names), std::move(types)); @@ -52,6 +71,16 @@ class OutputBufferManagerTest : public testing::Test { serializer::presto::PrestoOutputStreamListener>(); }); } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + facebook::velox::serializer::presto::PrestoVectorSerde:: + registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kCompactRow)) { + serializer::CompactRowVectorSerde::registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kUnsafeRow)) { + serializer::spark::UnsafeRowVectorSerde::registerNamedVectorSerde(); + } } std::shared_ptr initializeTask( @@ -91,7 +120,8 @@ class OutputBufferManagerTest : public testing::Test { vector_size_t size) { auto vector = std::dynamic_pointer_cast( BatchMaker::createBatch(rowType, size, *pool_)); - return exec::test::toSerializedPage(vector, bufferManager_, pool_.get()); + return exec::test::toSerializedPage( + vector, serdeKind_, bufferManager_, pool_.get()); } void enqueue( @@ -403,6 +433,7 @@ class OutputBufferManagerTest : public testing::Test { } } + const VectorSerde::Kind serdeKind_; std::shared_ptr executor_{ std::make_shared( std::thread::hardware_concurrency())}; @@ -411,29 +442,52 @@ class OutputBufferManagerTest : public testing::Test { RowTypePtr rowType_; }; -struct TestParam { - PartitionedOutputNode::Kind kind; +class OutputBufferManagerWithDifferentSerdeKindsTest + : public OutputBufferManagerTest, + public testing::WithParamInterface { + public: + static std::vector getTestParams() { + static std::vector params = { + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}; + return params; + } }; class AllOutputBufferManagerTest : public OutputBufferManagerTest, - public testing::WithParamInterface { + public testing::WithParamInterface { public: - AllOutputBufferManagerTest() : kind_(GetParam()) {} - - static std::vector getTestParams() { - static std::vector params = { - PartitionedOutputNode::Kind::kBroadcast, - PartitionedOutputNode::Kind::kPartitioned, - PartitionedOutputNode::Kind::kArbitrary}; + static std::vector getTestParams() { + static std::vector params = { + {PartitionedOutputNode::Kind::kBroadcast, VectorSerde::Kind::kPresto}, + {PartitionedOutputNode::Kind::kBroadcast, + VectorSerde::Kind::kCompactRow}, + {PartitionedOutputNode::Kind::kBroadcast, + VectorSerde::Kind::kUnsafeRow}, + {PartitionedOutputNode::Kind::kPartitioned, VectorSerde::Kind::kPresto}, + {PartitionedOutputNode::Kind::kPartitioned, + VectorSerde::Kind::kCompactRow}, + {PartitionedOutputNode::Kind::kPartitioned, + VectorSerde::Kind::kUnsafeRow}, + {PartitionedOutputNode::Kind::kArbitrary, VectorSerde::Kind::kPresto}, + {PartitionedOutputNode::Kind::kArbitrary, + VectorSerde::Kind::kCompactRow}, + {PartitionedOutputNode::Kind::kArbitrary, + VectorSerde::Kind::kUnsafeRow}}; return params; } + AllOutputBufferManagerTest() + : OutputBufferManagerTest(GetParam().serdeKind), + outputKind_(GetParam().outputKind) {} + protected: - PartitionedOutputNode::Kind kind_; + const PartitionedOutputNode::Kind outputKind_; }; -TEST_F(OutputBufferManagerTest, arbitrayBuffer) { +TEST_P(OutputBufferManagerWithDifferentSerdeKindsTest, arbitrayBuffer) { { ArbitraryBuffer buffer; ASSERT_TRUE(buffer.empty()); @@ -509,6 +563,12 @@ TEST_F(OutputBufferManagerTest, arbitrayBuffer) { } } +VELOX_INSTANTIATE_TEST_SUITE_P( + OutputBufferManagerWithDifferentSerdeKindsTest, + OutputBufferManagerWithDifferentSerdeKindsTest, + testing::ValuesIn( + OutputBufferManagerWithDifferentSerdeKindsTest::getTestParams())); + TEST_F(OutputBufferManagerTest, outputType) { ASSERT_EQ( PartitionedOutputNode::kindString( @@ -528,7 +588,7 @@ TEST_F(OutputBufferManagerTest, outputType) { "Invalid Output Kind 100"); } -TEST_F(OutputBufferManagerTest, destinationBuffer) { +TEST_P(OutputBufferManagerWithDifferentSerdeKindsTest, destinationBuffer) { { ArbitraryBuffer buffer; DestinationBuffer destinationBuffer; @@ -717,7 +777,7 @@ TEST_F(OutputBufferManagerTest, destinationBuffer) { } } -TEST_F(OutputBufferManagerTest, basicPartitioned) { +TEST_P(OutputBufferManagerWithDifferentSerdeKindsTest, basicPartitioned) { vector_size_t size = 100; std::string taskId = "t0"; auto task = initializeTask( @@ -727,11 +787,12 @@ TEST_F(OutputBufferManagerTest, basicPartitioned) { // Duplicateb update buffers with the same settings are allowed and ignored. ASSERT_TRUE(bufferManager_->updateOutputBuffers(taskId, 5, true)); ASSERT_FALSE(bufferManager_->isFinished(taskId)); - // Partitioned output buffer doesn't allow to update with different number of - // output buffers once created. + // Partitioned output buffer doesn't allow to update with different number + // of output buffers once created. VELOX_ASSERT_THROW( bufferManager_->updateOutputBuffers(taskId, 5 + 1, true), ""); - // Partitioned output buffer doesn't expect more output buffers once created. + // Partitioned output buffer doesn't expect more output buffers once + // created. VELOX_ASSERT_THROW(bufferManager_->updateOutputBuffers(taskId, 5, false), ""); VELOX_ASSERT_THROW( bufferManager_->updateOutputBuffers(taskId, 5 - 1, true), ""); @@ -796,7 +857,7 @@ TEST_F(OutputBufferManagerTest, basicPartitioned) { EXPECT_TRUE(task->isFinished()); } -TEST_F(OutputBufferManagerTest, basicBroadcast) { +TEST_P(OutputBufferManagerWithDifferentSerdeKindsTest, basicBroadcast) { vector_size_t size = 100; std::string taskId = "t0"; @@ -867,7 +928,7 @@ TEST_F(OutputBufferManagerTest, basicBroadcast) { EXPECT_TRUE(task->isFinished()); } -TEST_F(OutputBufferManagerTest, basicArbitrary) { +TEST_P(OutputBufferManagerWithDifferentSerdeKindsTest, basicArbitrary) { const vector_size_t size = 100; int numDestinations = 5; const std::string taskId = "t0"; @@ -950,7 +1011,9 @@ TEST_F(OutputBufferManagerTest, basicArbitrary) { EXPECT_TRUE(task->isFinished()); } -TEST_F(OutputBufferManagerTest, inactiveDestinationBuffer) { +TEST_P( + OutputBufferManagerWithDifferentSerdeKindsTest, + inactiveDestinationBuffer) { const vector_size_t dataSize = 1'000; const int maxBytes = 1; int numDestinations = 2; @@ -1040,8 +1103,8 @@ TEST_F(OutputBufferManagerTest, inactiveDestinationBuffer) { ASSERT_EQ(stats.buffersStats[i].bytesBuffered, 0); } - // Set the second destination buffer active to load data with notify when data - // gets queued. + // Set the second destination buffer active to load data with notify when + // data gets queued. actives[1] = true; ASSERT_TRUE(bufferManager_->getData( taskId, @@ -1087,7 +1150,9 @@ TEST_F(OutputBufferManagerTest, inactiveDestinationBuffer) { EXPECT_TRUE(task->isFinished()); } -TEST_F(OutputBufferManagerTest, broadcastWithDynamicAddedDestination) { +TEST_P( + OutputBufferManagerWithDifferentSerdeKindsTest, + broadcastWithDynamicAddedDestination) { vector_size_t size = 100; std::string taskId = "t0"; @@ -1131,7 +1196,9 @@ TEST_F(OutputBufferManagerTest, broadcastWithDynamicAddedDestination) { EXPECT_TRUE(task->isFinished()); } -TEST_F(OutputBufferManagerTest, arbitraryWithDynamicAddedDestination) { +TEST_P( + OutputBufferManagerWithDifferentSerdeKindsTest, + arbitraryWithDynamicAddedDestination) { const vector_size_t size = 100; int numDestinations = 5; const std::string taskId = "t0"; @@ -1190,7 +1257,7 @@ TEST_F(OutputBufferManagerTest, arbitraryWithDynamicAddedDestination) { TEST_P(AllOutputBufferManagerTest, maxBytes) { const vector_size_t size = 100; const std::string taskId = "t0"; - initializeTask(taskId, rowType_, kind_, 1, 1); + initializeTask(taskId, rowType_, outputKind_, 1, 1); enqueue(taskId, 0, rowType_, size); enqueue(taskId, 0, rowType_, size); @@ -1205,7 +1272,7 @@ TEST_P(AllOutputBufferManagerTest, maxBytes) { fetchOneAndAck(taskId, 0, 1); fetchOneAndAck(taskId, 0, 2); - if (kind_ != PartitionedOutputNode::Kind::kPartitioned) { + if (outputKind_ != PartitionedOutputNode::Kind::kPartitioned) { bufferManager_->updateOutputBuffers(taskId, 0, true); } noMoreData(taskId); @@ -1216,9 +1283,10 @@ TEST_P(AllOutputBufferManagerTest, maxBytes) { TEST_P(AllOutputBufferManagerTest, outputBufferUtilization) { const std::string taskId = std::to_string(rand()); const auto destination = 0; - auto task = initializeTask(taskId, rowType_, kind_, 1, 1); + auto task = initializeTask(taskId, rowType_, outputKind_, 1, 1); verifyOutputBuffer(task, OutputBufferStatus::kInitiated); - if (kind_ == facebook::velox::core::PartitionedOutputNode::Kind::kBroadcast) { + if (outputKind_ == + facebook::velox::core::PartitionedOutputNode::Kind::kBroadcast) { bufferManager_->updateOutputBuffers(taskId, destination, true); } @@ -1259,10 +1327,10 @@ TEST_P(AllOutputBufferManagerTest, outputBufferUtilization) { TEST_P(AllOutputBufferManagerTest, outputBufferStats) { const vector_size_t vectorSize = 100; const std::string taskId = std::to_string(folly::Random::rand32()); - initializeTask(taskId, rowType_, kind_, 1, 1); + initializeTask(taskId, rowType_, outputKind_, 1, 1); { const auto stats = getStats(taskId); - ASSERT_EQ(stats.kind, kind_); + ASSERT_EQ(stats.kind, outputKind_); ASSERT_FALSE(stats.noMoreData); ASSERT_FALSE(stats.finished); ASSERT_FALSE(stats.noMoreBuffers); @@ -1283,13 +1351,13 @@ TEST_P(AllOutputBufferManagerTest, outputBufferStats) { totalNumRows += vectorSize; // Force ArbitraryBuffer to load data, otherwise the data would // not be buffered in DestinationBuffer. - if (kind_ == PartitionedOutputNode::Kind::kArbitrary) { + if (outputKind_ == PartitionedOutputNode::Kind::kArbitrary) { fetchOne(taskId, 0, pageId); } const auto statsEnqueue = getStats(taskId); ASSERT_EQ(statsEnqueue.buffersStats[0].pagesBuffered, 1); ASSERT_EQ(statsEnqueue.buffersStats[0].rowsBuffered, vectorSize); - if (kind_ == core::PartitionedOutputNode::Kind::kBroadcast) { + if (outputKind_ == core::PartitionedOutputNode::Kind::kBroadcast) { ASSERT_EQ(statsEnqueue.bufferedPages, pageId + 1); ASSERT_EQ(statsEnqueue.bufferedBytes, totalBytes); } else { @@ -1307,7 +1375,7 @@ TEST_P(AllOutputBufferManagerTest, outputBufferStats) { ASSERT_EQ(statsAck.buffersStats[0].rowsSent, totalNumRows); ASSERT_EQ(statsAck.buffersStats[0].pagesBuffered, 0); ASSERT_EQ(statsAck.buffersStats[0].rowsBuffered, 0); - if (kind_ == core::PartitionedOutputNode::Kind::kBroadcast) { + if (outputKind_ == core::PartitionedOutputNode::Kind::kBroadcast) { ASSERT_EQ(statsAck.bufferedPages, pageId + 1); ASSERT_EQ(statsAck.bufferedBytes, totalBytes); } else { @@ -1352,7 +1420,7 @@ TEST_P(AllOutputBufferManagerTest, outputBufferStats) { ASSERT_FALSE(bufferManager_->stats(taskId).has_value()); } -TEST_F(OutputBufferManagerTest, outOfOrderAcks) { +TEST_P(OutputBufferManagerWithDifferentSerdeKindsTest, outOfOrderAcks) { const vector_size_t size = 100; const std::string taskId = "t0"; auto task = initializeTask( @@ -1394,7 +1462,9 @@ TEST_F(OutputBufferManagerTest, errorInQueue) { queue->dequeueLocked(1, &atEnd, &future), "Forced failure"); } -TEST_F(OutputBufferManagerTest, setQueueErrorWithPendingPages) { +TEST_P( + OutputBufferManagerWithDifferentSerdeKindsTest, + setQueueErrorWithPendingPages) { const uint64_t kBufferSize = 128; auto iobuf = folly::IOBuf::create(kBufferSize); const std::string payload("setQueueErrorWithPendingPages"); @@ -1421,10 +1491,10 @@ TEST_F(OutputBufferManagerTest, setQueueErrorWithPendingPages) { queue->dequeueLocked(1, &atEnd, &future), "Forced failure"); } -TEST_F(OutputBufferManagerTest, getDataOnFailedTask) { +TEST_P(OutputBufferManagerWithDifferentSerdeKindsTest, getDataOnFailedTask) { // Fetching data on a task which was either never initialized in the buffer - // manager or was removed by a parallel thread must return false. The `notify` - // callback must not be registered. + // manager or was removed by a parallel thread must return false. The + // `notify` callback must not be registered. ASSERT_FALSE(bufferManager_->getData( "test.0.1", 1, @@ -1438,7 +1508,9 @@ TEST_F(OutputBufferManagerTest, getDataOnFailedTask) { ASSERT_FALSE(bufferManager_->updateNumDrivers("test.0.2", 1)); } -TEST_F(OutputBufferManagerTest, updateBrodcastBufferOnFailedTask) { +TEST_P( + OutputBufferManagerWithDifferentSerdeKindsTest, + updateBrodcastBufferOnFailedTask) { // Updating broadcast buffer count in the buffer manager for a given unknown // task must not throw exception, instead must return FALSE. ASSERT_FALSE(bufferManager_->updateOutputBuffers( @@ -1459,10 +1531,10 @@ TEST_P(AllOutputBufferManagerTest, multiFetchers) { initializeTask( taskId, rowType_, - kind_, + outputKind_, numPartitions, 1, - kind_ == PartitionedOutputNode::Kind::kBroadcast ? 256 << 20 : 0); + outputKind_ == PartitionedOutputNode::Kind::kBroadcast ? 256 << 20 : 0); std::vector threads; std::vector fetchedPages(numPartitions + extendedNumPartitions, 0); @@ -1477,7 +1549,8 @@ TEST_P(AllOutputBufferManagerTest, multiFetchers) { std::vector producedPages( numPartitions + extendedNumPartitions, 0); for (int i = 0; i < totalPages; ++i) { - const int partition = kind_ == PartitionedOutputNode::Kind::kPartitioned + const int partition = + outputKind_ == PartitionedOutputNode::Kind::kPartitioned ? folly::Random().rand32(rng) % numPartitions : 0; try { @@ -1491,7 +1564,8 @@ TEST_P(AllOutputBufferManagerTest, multiFetchers) { if (folly::Random().oneIn(4)) { std::this_thread::sleep_for(std::chrono::microseconds(5)); // NOLINT } - if (i == 1000 && (kind_ != PartitionedOutputNode::Kind::kPartitioned)) { + if (i == 1000 && + (outputKind_ != PartitionedOutputNode::Kind::kPartitioned)) { bufferManager_->updateOutputBuffers( taskId, numPartitions + extendedNumPartitions, false); for (size_t i = numPartitions; @@ -1512,11 +1586,11 @@ TEST_P(AllOutputBufferManagerTest, multiFetchers) { } if (!earlyTermination) { - if (kind_ == PartitionedOutputNode::Kind::kPartitioned) { + if (outputKind_ == PartitionedOutputNode::Kind::kPartitioned) { for (int i = 0; i < numPartitions; ++i) { ASSERT_EQ(fetchedPages[i], producedPages[i]); } - } else if (kind_ == PartitionedOutputNode::Kind::kBroadcast) { + } else if (outputKind_ == PartitionedOutputNode::Kind::kBroadcast) { int64_t totalFetchedPages{0}; for (const auto& pages : fetchedPages) { totalFetchedPages += pages; diff --git a/velox/exec/tests/PartitionedOutputTest.cpp b/velox/exec/tests/PartitionedOutputTest.cpp index 37f44de6f6f47..4ce97e38d02eb 100644 --- a/velox/exec/tests/PartitionedOutputTest.cpp +++ b/velox/exec/tests/PartitionedOutputTest.cpp @@ -15,13 +15,26 @@ */ #include "velox/exec/PartitionedOutput.h" #include +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/exec/PlanNodeStats.h" #include "velox/exec/Task.h" #include "velox/exec/tests/utils/OperatorTestBase.h" #include "velox/exec/tests/utils/PlanBuilder.h" namespace facebook::velox::exec::test { -class PartitionedOutputTest : public OperatorTestBase { +class PartitionedOutputTest + : public OperatorTestBase, + public testing::WithParamInterface { + public: + static std::vector getTestParams() { + const std::vector kinds( + {VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}); + return kinds; + } + protected: std::shared_ptr createQueryContext( std::unordered_map config) { @@ -81,13 +94,12 @@ class PartitionedOutputTest : public OperatorTestBase { OutputBufferManager::getInstance().lock()}; }; -TEST_F(PartitionedOutputTest, flush) { +TEST_P(PartitionedOutputTest, flush) { // This test verifies // - Flush thresholds are respected (flush doesn't happen neither too early // nor too late) // - Flush is done independently for each output partition (flush for one // partition doesn't trigger flush for another one) - auto input = makeRowVector( {"p1", "v1"}, {makeFlatVector({0, 1}), @@ -95,18 +107,23 @@ TEST_F(PartitionedOutputTest, flush) { // twice as large to make sure it is always flushed (even if // PartitionedOutput#setTargetSizePct rolls 120%) std::string(PartitionedOutput::kMinDestinationSize * 2, '0'), - // 10 times smaller, so the data from 13 pages is always flushed as 2 + // 10 times smaller, so the data from 13 pages is always flushed as + // 2 // pages // 130% > 120% (when PartitionedOutput#setTargetSizePct rolls 120%) - // 130% < 140% (when PartitionedOutput#setTargetSizePct rolls 70% two + // 130% < 140% (when PartitionedOutput#setTargetSizePct rolls 70% + // two // times in a row) std::string(PartitionedOutput::kMinDestinationSize / 10, '1'), })}); + core::PlanNodeId partitionNodeId; auto plan = PlanBuilder() // produce 13 pages .values({input}, false, 13) - .partitionedOutput({"p1"}, 2, std::vector{"v1"}) + .partitionedOutput( + {"p1"}, 2, std::vector{"v1"}, GetParam()) + .capturePlanNodeId(partitionNodeId) .planNode(); auto taskId = "local://test-partitioned-output-flush-0"; @@ -135,17 +152,24 @@ TEST_F(PartitionedOutputTest, flush) { // Since each row for partition 0 is over the flush threshold as // many pages as there are input pages are expected - EXPECT_EQ(partition0.size(), 13); + ASSERT_EQ(partition0.size(), 13); // Data for the second partition is much smaller and expected to be buffered // up to a defined threshold - EXPECT_EQ(partition1.size(), 2); + ASSERT_EQ(partition1.size(), 2); + + auto planStats = toPlanStats(task->taskStats()); + const auto serdeKindRuntimsStats = + planStats.at(partitionNodeId).customStats.at(Operator::kShuffleSerdeKind); + ASSERT_EQ(serdeKindRuntimsStats.count, 1); + ASSERT_EQ(serdeKindRuntimsStats.min, static_cast(GetParam())); + ASSERT_EQ(serdeKindRuntimsStats.max, static_cast(GetParam())); } -TEST_F(PartitionedOutputTest, keyChannelNotAtBeginningWithNulls) { +TEST_P(PartitionedOutputTest, keyChannelNotAtBeginningWithNulls) { // This test verifies that PartitionedOutput can handle the case where a key - // channel is not at the beginning of the input type when nulls are present in - // the key channel. This triggers collectNullRows() to run which has special - // handling logic for the key channels. + // channel is not at the beginning of the input type when nulls are present + // in the key channel. This triggers collectNullRows() to run which has + // special handling logic for the key channels. auto input = makeRowVector( // The key column p1 is the second column. @@ -159,7 +183,8 @@ TEST_F(PartitionedOutputTest, keyChannelNotAtBeginningWithNulls) { PlanBuilder() .values({input}, false, 13) // Set replicateNullsAndAny to true so we trigger the null path. - .partitionedOutput({"p1"}, 2, true, std::vector{"v1"}) + .partitionedOutput( + {"p1"}, 2, true, std::vector{"v1"}, GetParam()) .planNode(); auto taskId = "local://test-partitioned-output-0"; @@ -181,4 +206,8 @@ TEST_F(PartitionedOutputTest, keyChannelNotAtBeginningWithNulls) { .count())); } +VELOX_INSTANTIATE_TEST_SUITE_P( + PartitionedOutputTest, + PartitionedOutputTest, + testing::ValuesIn(PartitionedOutputTest::getTestParams())); } // namespace facebook::velox::exec::test diff --git a/velox/exec/tests/PlanNodeSerdeTest.cpp b/velox/exec/tests/PlanNodeSerdeTest.cpp index 8f796b963ab61..03159c2e276f5 100644 --- a/velox/exec/tests/PlanNodeSerdeTest.cpp +++ b/velox/exec/tests/PlanNodeSerdeTest.cpp @@ -173,11 +173,18 @@ TEST_F(PlanNodeSerdeTest, enforceSingleRow) { } TEST_F(PlanNodeSerdeTest, exchange) { - auto plan = - PlanBuilder() - .exchange(ROW({"a", "b", "c"}, {BIGINT(), DOUBLE(), VARCHAR()})) - .planNode(); - testSerde(plan); + for (auto serdeKind : std::vector{ + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}) { + SCOPED_TRACE(fmt::format("serdeKind: {}", serdeKind)); + auto plan = PlanBuilder() + .exchange( + ROW({"a", "b", "c"}, {BIGINT(), DOUBLE(), VARCHAR()}), + serdeKind) + .planNode(); + testSerde(plan); + } } TEST_F(PlanNodeSerdeTest, filter) { @@ -233,12 +240,18 @@ TEST_F(PlanNodeSerdeTest, limit) { } TEST_F(PlanNodeSerdeTest, mergeExchange) { - auto plan = PlanBuilder() - .mergeExchange( - ROW({"a", "b", "c"}, {BIGINT(), DOUBLE(), VARCHAR()}), - {"a DESC", "b NULLS FIRST"}) - .planNode(); - testSerde(plan); + for (auto serdeKind : std::vector{ + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}) { + auto plan = PlanBuilder() + .mergeExchange( + ROW({"a", "b", "c"}, {BIGINT(), DOUBLE(), VARCHAR()}), + {"a DESC", "b NULLS FIRST"}, + serdeKind) + .planNode(); + testSerde(plan); + } } TEST_F(PlanNodeSerdeTest, localMerge) { @@ -319,18 +332,30 @@ TEST_F(PlanNodeSerdeTest, orderBy) { } TEST_F(PlanNodeSerdeTest, partitionedOutput) { - auto plan = - PlanBuilder().values({data_}).partitionedOutputBroadcast().planNode(); - testSerde(plan); + for (auto serdeKind : std::vector{ + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}) { + SCOPED_TRACE(fmt::format("serdeKind: {}", serdeKind)); + + auto plan = PlanBuilder() + .values({data_}) + .partitionedOutputBroadcast(/*outputLayout=*/{}, serdeKind) + .planNode(); + testSerde(plan); - plan = PlanBuilder().values({data_}).partitionedOutput({"c0"}, 50).planNode(); - testSerde(plan); + plan = PlanBuilder() + .values({data_}) + .partitionedOutput({"c0"}, 50, /*outputLayout=*/{}, serdeKind) + .planNode(); + testSerde(plan); - plan = PlanBuilder() - .values({data_}) - .partitionedOutput({"c0"}, 50, {"c1", {"c2"}, "c0"}) - .planNode(); - testSerde(plan); + plan = PlanBuilder() + .values({data_}) + .partitionedOutput({"c0"}, 50, {"c1", {"c2"}, "c0"}, serdeKind) + .planNode(); + testSerde(plan); + } } TEST_F(PlanNodeSerdeTest, project) { diff --git a/velox/exec/tests/PlanNodeToStringTest.cpp b/velox/exec/tests/PlanNodeToStringTest.cpp index 59485b89478f2..280f484798b57 100644 --- a/velox/exec/tests/PlanNodeToStringTest.cpp +++ b/velox/exec/tests/PlanNodeToStringTest.cpp @@ -14,7 +14,6 @@ * limitations under the License. */ -#include "velox/connectors/hive/HiveConnectorSplit.h" #include "velox/exec/WindowFunction.h" #include "velox/exec/tests/utils/HiveConnectorTestBase.h" #include "velox/exec/tests/utils/PlanBuilder.h" @@ -589,52 +588,85 @@ TEST_F(PlanNodeToStringTest, localPartition) { } TEST_F(PlanNodeToStringTest, partitionedOutput) { - auto plan = - PlanBuilder().values({data_}).partitionedOutput({"c0"}, 4).planNode(); + for (auto serdeKind : std::vector{ + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}) { + SCOPED_TRACE(fmt::format("serdeKind: {}", serdeKind)); + auto plan = + PlanBuilder() + .values({data_}) + .partitionedOutput({"c0"}, 4, /*outputLayout=*/{}, serdeKind) + .planNode(); - ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); - ASSERT_EQ( - "-- PartitionedOutput[1][partitionFunction: HASH(c0) with 4 partitions] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", - plan->toString(true, false)); + ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); + ASSERT_EQ( + fmt::format( + "-- PartitionedOutput[1][partitionFunction: HASH(c0) with 4 partitions {}] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", + serdeKind), + plan->toString(true, false)); - plan = PlanBuilder().values({data_}).partitionedOutputBroadcast().planNode(); + plan = PlanBuilder() + .values({data_}) + .partitionedOutputBroadcast(/*outputLayout=*/{}, serdeKind) + .planNode(); - ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); - ASSERT_EQ( - "-- PartitionedOutput[1][BROADCAST] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", - plan->toString(true, false)); + ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); + ASSERT_EQ( + fmt::format( + "-- PartitionedOutput[1][BROADCAST {}] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", + serdeKind), + plan->toString(true, false)); - plan = PlanBuilder().values({data_}).partitionedOutput({}, 1).planNode(); + plan = PlanBuilder() + .values({data_}) + .partitionedOutput({}, 1, /*outputLayout=*/{}, serdeKind) + .planNode(); - ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); - ASSERT_EQ( - "-- PartitionedOutput[1][SINGLE] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", - plan->toString(true, false)); - - plan = PlanBuilder() - .values({data_}) - .partitionedOutput({"c1", "c2"}, 5, true) - .planNode(); + ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); + ASSERT_EQ( + fmt::format( + "-- PartitionedOutput[1][SINGLE {}] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", + serdeKind), + plan->toString(true, false)); - ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); - ASSERT_EQ( - "-- PartitionedOutput[1][partitionFunction: HASH(c1, c2) with 5 partitions replicate nulls and any] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", - plan->toString(true, false)); + plan = PlanBuilder() + .values({data_}) + .partitionedOutput( + {"c1", "c2"}, 5, true, /*outputLayout=*/{}, serdeKind) + .planNode(); - auto hiveSpec = std::make_shared( - 4, - std::vector{0, 1, 0, 1}, - std::vector{1, 2}, - std::vector{}); + ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); + ASSERT_EQ( + fmt::format( + "-- PartitionedOutput[1][partitionFunction: HASH(c1, c2) with 5 partitions replicate nulls and any {}] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", + serdeKind), + plan->toString(true, false)); - plan = PlanBuilder() - .values({data_}) - .partitionedOutput({"c1", "c2"}, 2, false, hiveSpec) - .planNode(); - ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); - ASSERT_EQ( - "-- PartitionedOutput[1][partitionFunction: HIVE((1, 2) buckets: 4) with 2 partitions] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", - plan->toString(true, false)); + auto hiveSpec = + std::make_shared( + 4, + std::vector{0, 1, 0, 1}, + std::vector{1, 2}, + std::vector{}); + + plan = PlanBuilder() + .values({data_}) + .partitionedOutput( + {"c1", "c2"}, + 2, + false, + hiveSpec, + /*outputLayout=*/{}, + serdeKind) + .planNode(); + ASSERT_EQ("-- PartitionedOutput[1]\n", plan->toString()); + ASSERT_EQ( + fmt::format( + "-- PartitionedOutput[1][partitionFunction: HIVE((1, 2) buckets: 4) with 2 partitions {}] -> c0:SMALLINT, c1:INTEGER, c2:BIGINT\n", + serdeKind), + plan->toString(true, false)); + } } TEST_F(PlanNodeToStringTest, localMerge) { @@ -662,12 +694,43 @@ TEST_F(PlanNodeToStringTest, localMerge) { } TEST_F(PlanNodeToStringTest, exchange) { - auto plan = - PlanBuilder().exchange(ROW({"a", "b"}, {BIGINT(), VARCHAR()})).planNode(); + for (auto serdeKind : std::vector{ + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}) { + SCOPED_TRACE(fmt::format("serdeKind: {}", serdeKind)); + + auto plan = PlanBuilder() + .exchange(ROW({"a", "b"}, {BIGINT(), VARCHAR()}), serdeKind) + .planNode(); - ASSERT_EQ("-- Exchange[0]\n", plan->toString()); - ASSERT_EQ( - "-- Exchange[0][] -> a:BIGINT, b:VARCHAR\n", plan->toString(true, false)); + ASSERT_EQ("-- Exchange[0]\n", plan->toString()); + ASSERT_EQ( + fmt::format("-- Exchange[0][{}] -> a:BIGINT, b:VARCHAR\n", serdeKind), + plan->toString(true, false)); + } +} + +TEST_F(PlanNodeToStringTest, mergeExchange) { + for (auto serdeKind : std::vector{ + VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}) { + SCOPED_TRACE(fmt::format("serdeKind: {}", serdeKind)); + + auto plan = + PlanBuilder() + .mergeExchange( + ROW({"a", "b"}, {BIGINT(), VARCHAR()}), {"a"}, serdeKind) + .planNode(); + + ASSERT_EQ("-- MergeExchange[0]\n", plan->toString()); + ASSERT_EQ( + fmt::format( + "-- MergeExchange[0][a ASC NULLS LAST, {}] -> a:BIGINT, b:VARCHAR\n", + serdeKind), + plan->toString(true, false)); + } } TEST_F(PlanNodeToStringTest, tableScan) { diff --git a/velox/exec/tests/SpillTest.cpp b/velox/exec/tests/SpillTest.cpp index 170ef2563bc5e..db5f604dde799 100644 --- a/velox/exec/tests/SpillTest.cpp +++ b/velox/exec/tests/SpillTest.cpp @@ -85,15 +85,19 @@ class SpillTest : public ::testing::TestWithParam, protected: static void SetUpTestCase() { memory::MemoryManager::testingSetInstance({}); + if (!isRegisteredVectorSerde()) { + facebook::velox::serializer::presto::PrestoVectorSerde:: + registerVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + facebook::velox::serializer::presto::PrestoVectorSerde:: + registerNamedVectorSerde(); + } } void SetUp() override { allocator_ = memory::memoryManager()->allocator(); tempDir_ = exec::test::TempDirectoryPath::create(); - if (!isRegisteredVectorSerde()) { - facebook::velox::serializer::presto::PrestoVectorSerde:: - registerVectorSerde(); - } filesystems::registerLocalFileSystem(); rng_.seed(1); compressionKind_ = GetParam().compressionKind; diff --git a/velox/exec/tests/TableScanTest.cpp b/velox/exec/tests/TableScanTest.cpp index 47cd6b626f51f..5b66d6457e000 100644 --- a/velox/exec/tests/TableScanTest.cpp +++ b/velox/exec/tests/TableScanTest.cpp @@ -1677,7 +1677,8 @@ DEBUG_ONLY_TEST_F(TableScanTest, tableScanSplitsAndWeights) { auto leafTaskId = "local://leaf-0"; auto leafPlan = PlanBuilder() .values(vectors) - .partitionedOutput({}, 1, {"c0", "c1", "c2"}) + .partitionedOutput( + {}, 1, {"c0", "c1", "c2"}, VectorSerde::Kind::kPresto) .planNode(); std::unordered_map config; auto queryCtx = core::QueryCtx::create( @@ -1696,24 +1697,23 @@ DEBUG_ONLY_TEST_F(TableScanTest, tableScanSplitsAndWeights) { // Main task plan with table scan and remote exchange. auto planNodeIdGenerator = std::make_shared(); core::PlanNodeId scanNodeId, exchangeNodeId; - auto planNode = PlanBuilder(planNodeIdGenerator, pool_.get()) - .tableScan(rowType_) - .capturePlanNodeId(scanNodeId) - .project({"c0 AS t0", "c1 AS t1", "c2 AS t2"}) - .hashJoin( - {"t0"}, - {"u0"}, - PlanBuilder(planNodeIdGenerator, pool_.get()) - .exchange(leafPlan->outputType()) - .capturePlanNodeId(exchangeNodeId) - // .values(vectors) - // .partitionedOutput({}, 1, {"c0", "c1", "c2"}) - .project({"c0 AS u0", "c1 AS u1", "c2 AS u2"}) - .planNode(), - "", - {"t1"}, - core::JoinType::kAnti) - .planNode(); + auto planNode = + PlanBuilder(planNodeIdGenerator, pool_.get()) + .tableScan(rowType_) + .capturePlanNodeId(scanNodeId) + .project({"c0 AS t0", "c1 AS t1", "c2 AS t2"}) + .hashJoin( + {"t0"}, + {"u0"}, + PlanBuilder(planNodeIdGenerator, pool_.get()) + .exchange(leafPlan->outputType(), VectorSerde::Kind::kPresto) + .capturePlanNodeId(exchangeNodeId) + .project({"c0 AS u0", "c1 AS u1", "c2 AS u2"}) + .planNode(), + "", + {"t1"}, + core::JoinType::kAnti) + .planNode(); // Create task, cursor, start the task and supply the table scan splits. const int32_t numDrivers = 6; diff --git a/velox/exec/tests/utils/OperatorTestBase.cpp b/velox/exec/tests/utils/OperatorTestBase.cpp index 0fbf9017aa638..69300dafb0b3d 100644 --- a/velox/exec/tests/utils/OperatorTestBase.cpp +++ b/velox/exec/tests/utils/OperatorTestBase.cpp @@ -21,17 +21,16 @@ #include "velox/common/memory/MallocAllocator.h" #include "velox/common/memory/SharedArbitrator.h" #include "velox/common/testutil/TestValue.h" -#include "velox/exec/OutputBufferManager.h" -#include "velox/exec/tests/utils/AssertQueryBuilder.h" #include "velox/exec/tests/utils/LocalExchangeSource.h" #include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" #include "velox/functions/prestosql/registration/RegistrationFunctions.h" #include "velox/parse/Expressions.h" #include "velox/parse/ExpressionsParser.h" #include "velox/parse/TypeResolver.h" +#include "velox/serializers/CompactRowSerializer.h" #include "velox/serializers/PrestoSerializer.h" +#include "velox/serializers/UnsafeRowSerializer.h" #include "velox/vector/tests/utils/VectorMaker.h" -#include "velox/vector/tests/utils/VectorTestBase.h" DECLARE_bool(velox_memory_leak_check_enabled); DECLARE_bool(velox_enable_memory_usage_track_in_default_memory_pool); @@ -125,6 +124,15 @@ void OperatorTestBase::SetUp() { if (!isRegisteredVectorSerde()) { this->registerVectorSerde(); } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kCompactRow)) { + serializer::CompactRowVectorSerde::registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kUnsafeRow)) { + serializer::spark::UnsafeRowVectorSerde::registerNamedVectorSerde(); + } driverExecutor_ = std::make_unique(3); ioExecutor_ = std::make_unique(3); PeriodicStatsReporter::Options options; diff --git a/velox/exec/tests/utils/PlanBuilder.cpp b/velox/exec/tests/utils/PlanBuilder.cpp index 805ad1e39fb61..e95ea01f101e1 100644 --- a/velox/exec/tests/utils/PlanBuilder.cpp +++ b/velox/exec/tests/utils/PlanBuilder.cpp @@ -246,10 +246,12 @@ PlanBuilder& PlanBuilder::traceScan( return *this; } -PlanBuilder& PlanBuilder::exchange(const RowTypePtr& outputType) { +PlanBuilder& PlanBuilder::exchange( + const RowTypePtr& outputType, + VectorSerde::Kind serdeKind) { VELOX_CHECK_NULL(planNode_, "Exchange must be the source node"); - planNode_ = - std::make_shared(nextPlanNodeId(), outputType); + planNode_ = std::make_shared( + nextPlanNodeId(), outputType, serdeKind); return *this; } @@ -284,13 +286,14 @@ parseOrderByClauses( PlanBuilder& PlanBuilder::mergeExchange( const RowTypePtr& outputType, - const std::vector& keys) { + const std::vector& keys, + VectorSerde::Kind serdeKind) { VELOX_CHECK_NULL(planNode_, "MergeExchange must be the source node"); auto [sortingKeys, sortingOrders] = parseOrderByClauses(keys, outputType, pool_); planNode_ = std::make_shared( - nextPlanNodeId(), outputType, sortingKeys, sortingOrders); + nextPlanNodeId(), outputType, sortingKeys, sortingOrders, serdeKind); return *this; } @@ -1182,15 +1185,17 @@ core::PlanNodePtr createLocalPartitionNode( PlanBuilder& PlanBuilder::partitionedOutput( const std::vector& keys, int numPartitions, - const std::vector& outputLayout) { - return partitionedOutput(keys, numPartitions, false, outputLayout); + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { + return partitionedOutput(keys, numPartitions, false, outputLayout, serdeKind); } PlanBuilder& PlanBuilder::partitionedOutput( const std::vector& keys, int numPartitions, bool replicateNullsAndAny, - const std::vector& outputLayout) { + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { VELOX_CHECK_NOT_NULL( planNode_, "PartitionedOutput cannot be the source node"); @@ -1200,7 +1205,8 @@ PlanBuilder& PlanBuilder::partitionedOutput( numPartitions, replicateNullsAndAny, createPartitionFunctionSpec(planNode_->outputType(), keyExprs, pool_), - outputLayout); + outputLayout, + serdeKind); } PlanBuilder& PlanBuilder::partitionedOutput( @@ -1208,7 +1214,8 @@ PlanBuilder& PlanBuilder::partitionedOutput( int numPartitions, bool replicateNullsAndAny, core::PartitionFunctionSpecPtr partitionFunctionSpec, - const std::vector& outputLayout) { + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { VELOX_CHECK_NOT_NULL( planNode_, "PartitionedOutput cannot be the source node"); auto outputType = outputLayout.empty() @@ -1222,31 +1229,34 @@ PlanBuilder& PlanBuilder::partitionedOutput( replicateNullsAndAny, std::move(partitionFunctionSpec), outputType, + serdeKind, planNode_); return *this; } PlanBuilder& PlanBuilder::partitionedOutputBroadcast( - const std::vector& outputLayout) { + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { VELOX_CHECK_NOT_NULL( planNode_, "PartitionedOutput cannot be the source node"); auto outputType = outputLayout.empty() ? planNode_->outputType() : extract(planNode_->outputType(), outputLayout); planNode_ = core::PartitionedOutputNode::broadcast( - nextPlanNodeId(), 1, outputType, planNode_); + nextPlanNodeId(), 1, outputType, serdeKind, planNode_); return *this; } PlanBuilder& PlanBuilder::partitionedOutputArbitrary( - const std::vector& outputLayout) { + const std::vector& outputLayout, + VectorSerde::Kind serdeKind) { VELOX_CHECK_NOT_NULL( planNode_, "PartitionedOutput cannot be the source node"); auto outputType = outputLayout.empty() ? planNode_->outputType() : extract(planNode_->outputType(), outputLayout); planNode_ = core::PartitionedOutputNode::arbitrary( - nextPlanNodeId(), outputType, planNode_); + nextPlanNodeId(), outputType, serdeKind, planNode_); return *this; } diff --git a/velox/exec/tests/utils/PlanBuilder.h b/velox/exec/tests/utils/PlanBuilder.h index 890d77e7bb381..7bb4507946091 100644 --- a/velox/exec/tests/utils/PlanBuilder.h +++ b/velox/exec/tests/utils/PlanBuilder.h @@ -335,7 +335,15 @@ class PlanBuilder { /// splits. /// /// @param outputType The type of the data coming in and out of the exchange. - PlanBuilder& exchange(const RowTypePtr& outputType); + /// @param serdekind The kind of seralized data format. +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY + PlanBuilder& exchange(const RowTypePtr& outputType) { + return exchange(outputType, VectorSerde::Kind::kPresto); + } +#endif + PlanBuilder& exchange( + const RowTypePtr& outputType, + VectorSerde::Kind serdekind); /// Add a MergeExchangeNode using specified ORDER BY clauses. /// @@ -345,9 +353,17 @@ class PlanBuilder { /// /// By default, uses ASC NULLS LAST sort order, e.g. column "a" above will use /// ASC NULLS LAST and column "b" will use DESC NULLS LAST. +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY + PlanBuilder& mergeExchange( + const RowTypePtr& outputType, + const std::vector& keys) { + return mergeExchange(outputType, keys, VectorSerde::Kind::kPresto); + } +#endif PlanBuilder& mergeExchange( const RowTypePtr& outputType, - const std::vector& keys); + const std::vector& keys, + VectorSerde::Kind serdekind); /// Add a ProjectNode using specified SQL expressions. /// @@ -787,13 +803,15 @@ class PlanBuilder { const std::vector& keys, int numPartitions, bool replicateNullsAndAny, - const std::vector& outputLayout = {}); + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); /// Same as above, but assumes 'replicateNullsAndAny' is false. PlanBuilder& partitionedOutput( const std::vector& keys, int numPartitions, - const std::vector& outputLayout = {}); + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); /// Same as above, but allows to provide custom partition function. PlanBuilder& partitionedOutput( @@ -801,7 +819,8 @@ class PlanBuilder { int numPartitions, bool replicateNullsAndAny, core::PartitionFunctionSpecPtr partitionFunctionSpec, - const std::vector& outputLayout = {}); + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); /// Adds a PartitionedOutputNode to broadcast the input data. /// @@ -810,11 +829,13 @@ class PlanBuilder { /// some input columns may be missing in the output, some columns may be /// duplicated in the output. PlanBuilder& partitionedOutputBroadcast( - const std::vector& outputLayout = {}); + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); /// Adds a PartitionedOutputNode to put data into arbitrary buffer. PlanBuilder& partitionedOutputArbitrary( - const std::vector& outputLayout = {}); + const std::vector& outputLayout = {}, + VectorSerde::Kind serdeKind = VectorSerde::Kind::kPresto); /// Adds a LocalPartitionNode to hash-partition the input on the specified /// keys using exec::HashPartitionFunction. Number of partitions is determined diff --git a/velox/exec/tests/utils/RowContainerTestBase.h b/velox/exec/tests/utils/RowContainerTestBase.h index d6c9d011b9a7d..c9d126fa869ef 100644 --- a/velox/exec/tests/utils/RowContainerTestBase.h +++ b/velox/exec/tests/utils/RowContainerTestBase.h @@ -14,15 +14,14 @@ * limitations under the License. */ #include -#include -#include -#include + #include "velox/common/file/FileSystems.h" #include "velox/dwio/common/tests/utils/BatchMaker.h" #include "velox/exec/RowContainer.h" -#include "velox/exec/VectorHasher.h" #include "velox/exec/tests/utils/TempDirectoryPath.h" +#include "velox/serializers/CompactRowSerializer.h" #include "velox/serializers/PrestoSerializer.h" +#include "velox/serializers/UnsafeRowSerializer.h" #include "velox/vector/tests/utils/VectorTestBase.h" namespace facebook::velox::exec::test { @@ -47,6 +46,18 @@ class RowContainerTestBase : public testing::Test, facebook::velox::serializer::presto::PrestoVectorSerde:: registerVectorSerde(); } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + facebook::velox::serializer::presto::PrestoVectorSerde:: + registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kCompactRow)) { + facebook::velox::serializer::CompactRowVectorSerde:: + registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kUnsafeRow)) { + facebook::velox::serializer::spark::UnsafeRowVectorSerde:: + registerNamedVectorSerde(); + } filesystems::registerLocalFileSystem(); } diff --git a/velox/exec/tests/utils/SerializedPageUtil.cpp b/velox/exec/tests/utils/SerializedPageUtil.cpp index 512c59457c794..4316f1a783fa7 100644 --- a/velox/exec/tests/utils/SerializedPageUtil.cpp +++ b/velox/exec/tests/utils/SerializedPageUtil.cpp @@ -22,9 +22,11 @@ namespace facebook::velox::exec::test { std::unique_ptr toSerializedPage( const RowVectorPtr& vector, + VectorSerde::Kind serdeKind, const std::shared_ptr& bufferManager, memory::MemoryPool* pool) { - auto data = std::make_unique(pool); + auto data = + std::make_unique(pool, getNamedVectorSerde(serdeKind)); auto size = vector->size(); auto range = IndexRange{0, size}; data->createStreamTree(asRowType(vector->type()), size); diff --git a/velox/exec/tests/utils/SerializedPageUtil.h b/velox/exec/tests/utils/SerializedPageUtil.h index 3dddd22061697..6890693d63459 100644 --- a/velox/exec/tests/utils/SerializedPageUtil.h +++ b/velox/exec/tests/utils/SerializedPageUtil.h @@ -18,13 +18,14 @@ #include "velox/exec/ExchangeQueue.h" #include "velox/exec/OutputBufferManager.h" #include "velox/vector/ComplexVector.h" -#include "velox/vector/VectorStream.h" +// #include "velox/vector/VectorStream.h" namespace facebook::velox::exec::test { /// Helper function for serializing RowVector to PrestoPage format. std::unique_ptr toSerializedPage( const RowVectorPtr& vector, + VectorSerde::Kind serdeKind, const std::shared_ptr& bufferManager, memory::MemoryPool* pool); diff --git a/velox/row/CompactRow.cpp b/velox/row/CompactRow.cpp index 8e59358db31fa..c0348951560fa 100644 --- a/velox/row/CompactRow.cpp +++ b/velox/row/CompactRow.cpp @@ -14,11 +14,14 @@ * limitations under the License. */ #include "velox/row/CompactRow.h" + +#include "velox/common/base/RawVector.h" #include "velox/vector/FlatVector.h" namespace facebook::velox::row { namespace { constexpr size_t kSizeBytes = sizeof(int32_t); +using TRowSize = uint32_t; void writeInt32(char* buffer, int32_t n) { ::memcpy(buffer, &n, kSizeBytes); @@ -209,14 +212,14 @@ void CompactRow::initialize(const TypePtr& type) { auto base = decoded_.base(); switch (typeKind_) { case TypeKind::ARRAY: { - auto arrayBase = base->as(); + auto* arrayBase = base->as(); children_.push_back(CompactRow(arrayBase->elements())); childIsFixedWidth_.push_back( arrayBase->elements()->type()->isFixedWidth()); break; } case TypeKind::MAP: { - auto mapBase = base->as(); + auto* mapBase = base->as(); children_.push_back(CompactRow(mapBase->mapKeys())); children_.push_back(CompactRow(mapBase->mapValues())); childIsFixedWidth_.push_back(mapBase->mapKeys()->type()->isFixedWidth()); @@ -225,7 +228,7 @@ void CompactRow::initialize(const TypePtr& type) { break; } case TypeKind::ROW: { - auto rowBase = base->as(); + auto* rowBase = base->as(); for (const auto& child : rowBase->children()) { children_.push_back(CompactRow(child)); childIsFixedWidth_.push_back(child->type()->isFixedWidth()); @@ -304,12 +307,12 @@ std::optional CompactRow::fixedRowSize(const RowTypePtr& rowType) { return size; } -int32_t CompactRow::rowSize(vector_size_t index) { +int32_t CompactRow::rowSize(vector_size_t index) const { return rowRowSize(index); } -int32_t CompactRow::rowRowSize(vector_size_t index) { - auto childIndex = decoded_.index(index); +int32_t CompactRow::rowRowSize(vector_size_t index) const { + const auto childIndex = decoded_.index(index); const auto numFields = children_.size(); int32_t size = rowNullBytes_; @@ -325,7 +328,23 @@ int32_t CompactRow::rowRowSize(vector_size_t index) { return size; } -int32_t CompactRow::serializeRow(vector_size_t index, char* buffer) { +void CompactRow::serializedRowSizes( + const folly::Range& rows, + vector_size_t** sizes) const { + if (const auto fixedRowSize = + row::CompactRow::fixedRowSize(asRowType(decoded_.base()->type()))) { + for (const auto row : rows) { + *sizes[row] = fixedRowSize.value() + sizeof(TRowSize); + } + return; + } + + for (const auto& row : rows) { + *sizes[row] = rowSize(row) + sizeof(TRowSize); + } +} + +int32_t CompactRow::serializeRow(vector_size_t index, char* buffer) const { auto childIndex = decoded_.index(index); int64_t valuesOffset = rowNullBytes_; @@ -365,7 +384,7 @@ void CompactRow::serializeRow( vector_size_t offset, vector_size_t size, char* buffer, - const size_t* bufferOffsets) { + const size_t* bufferOffsets) const { raw_vector rows(size); raw_vector nulls(size); if (decoded_.isIdentityMapping()) { @@ -416,16 +435,16 @@ void CompactRow::serializeRow( } } -bool CompactRow::isNullAt(vector_size_t index) { +bool CompactRow::isNullAt(vector_size_t index) const { return decoded_.isNullAt(index); } -int32_t CompactRow::variableWidthRowSize(vector_size_t index) { +int32_t CompactRow::variableWidthRowSize(vector_size_t index) const { switch (typeKind_) { case TypeKind::VARCHAR: [[fallthrough]]; case TypeKind::VARBINARY: { - auto value = decoded_.valueAt(index); + const auto value = decoded_.valueAt(index); return sizeof(int32_t) + value.size(); } case TypeKind::ARRAY: @@ -440,21 +459,21 @@ int32_t CompactRow::variableWidthRowSize(vector_size_t index) { }; } -int32_t CompactRow::arrayRowSize(vector_size_t index) { - auto baseIndex = decoded_.index(index); +int32_t CompactRow::arrayRowSize(vector_size_t index) const { + const auto baseIndex = decoded_.index(index); - auto arrayBase = decoded_.base()->asUnchecked(); - auto offset = arrayBase->offsetAt(baseIndex); - auto size = arrayBase->sizeAt(baseIndex); + auto* arrayBase = decoded_.base()->asUnchecked(); + const auto offset = arrayBase->offsetAt(baseIndex); + const auto size = arrayBase->sizeAt(baseIndex); return arrayRowSize(children_[0], offset, size, childIsFixedWidth_[0]); } int32_t CompactRow::arrayRowSize( - CompactRow& elements, + const CompactRow& elements, vector_size_t offset, vector_size_t size, - bool fixedWidth) { + bool fixedWidth) const { const int32_t nullBytes = bits::nbytes(size); // array size | null bits | elements @@ -491,7 +510,7 @@ int32_t CompactRow::arrayRowSize( return rowSize; } -int32_t CompactRow::serializeArray(vector_size_t index, char* buffer) { +int32_t CompactRow::serializeArray(vector_size_t index, char* buffer) const { auto baseIndex = decoded_.index(index); // For complex-type elements: @@ -514,11 +533,11 @@ int32_t CompactRow::serializeArray(vector_size_t index, char* buffer) { } int32_t CompactRow::serializeAsArray( - CompactRow& elements, + const CompactRow& elements, vector_size_t offset, vector_size_t size, bool fixedWidth, - char* buffer) { + char* buffer) const { // For complex-type elements: // array size | null bits | serialized size | offset e1 | offset e2 |... | e1 // | e2 |... @@ -598,20 +617,20 @@ int32_t CompactRow::serializeAsArray( return elementsOffset; } -int32_t CompactRow::mapRowSize(vector_size_t index) { +int32_t CompactRow::mapRowSize(vector_size_t index) const { auto baseIndex = decoded_.index(index); // | - auto mapBase = decoded_.base()->asUnchecked(); - auto offset = mapBase->offsetAt(baseIndex); - auto size = mapBase->sizeAt(baseIndex); + auto* mapBase = decoded_.base()->asUnchecked(); + const auto offset = mapBase->offsetAt(baseIndex); + const auto size = mapBase->sizeAt(baseIndex); return arrayRowSize(children_[0], offset, size, childIsFixedWidth_[0]) + arrayRowSize(children_[1], offset, size, childIsFixedWidth_[1]); } -int32_t CompactRow::serializeMap(vector_size_t index, char* buffer) { +int32_t CompactRow::serializeMap(vector_size_t index, char* buffer) const { auto baseIndex = decoded_.index(index); // | @@ -633,7 +652,7 @@ int32_t CompactRow::serializeMap(vector_size_t index, char* buffer) { return keysSerializedBytes + valuesSerializedBytes; } -int32_t CompactRow::serialize(vector_size_t index, char* buffer) { +int32_t CompactRow::serialize(vector_size_t index, char* buffer) const { return serializeRow(index, buffer); } @@ -641,11 +660,11 @@ void CompactRow::serialize( vector_size_t offset, vector_size_t size, const size_t* bufferOffsets, - char* buffer) { + char* buffer) const { serializeRow(offset, size, buffer, bufferOffsets); } -void CompactRow::serializeFixedWidth(vector_size_t index, char* buffer) { +void CompactRow::serializeFixedWidth(vector_size_t index, char* buffer) const { VELOX_DCHECK(fixedWidthTypeKind_); switch (typeKind_) { case TypeKind::BOOLEAN: @@ -667,7 +686,7 @@ void CompactRow::serializeFixedWidth(vector_size_t index, char* buffer) { void CompactRow::serializeFixedWidth( vector_size_t offset, vector_size_t size, - char* buffer) { + char* buffer) const { VELOX_DCHECK(supportsBulkCopy_); // decoded_.data() can be null if all values are null. if (decoded_.data()) { @@ -678,7 +697,8 @@ void CompactRow::serializeFixedWidth( } } -int32_t CompactRow::serializeVariableWidth(vector_size_t index, char* buffer) { +int32_t CompactRow::serializeVariableWidth(vector_size_t index, char* buffer) + const { switch (typeKind_) { case TypeKind::VARCHAR: [[fallthrough]]; diff --git a/velox/row/CompactRow.h b/velox/row/CompactRow.h index 323ccc66dde6b..b3e30927b88a3 100644 --- a/velox/row/CompactRow.h +++ b/velox/row/CompactRow.h @@ -15,7 +15,6 @@ */ #pragma once -#include "velox/common/base/RawVector.h" #include "velox/vector/ComplexVector.h" #include "velox/vector/DecodedVector.h" @@ -25,17 +24,24 @@ class CompactRow { public: explicit CompactRow(const RowVectorPtr& vector); + /// Returns the serialized sizes of the rows at specified indexes. + /// + /// TODO: optimizes using columnar serialization size calculation. + void serializedRowSizes( + const folly::Range& rows, + vector_size_t** sizes) const; + /// Returns row size if all fields are fixed width. Return std::nullopt if /// there are variable-width fields. static std::optional fixedRowSize(const RowTypePtr& rowType); /// Returns serialized size of the row at specified index. Use only if /// 'fixedRowSize' returned std::nullopt. - int32_t rowSize(vector_size_t index); + int32_t rowSize(vector_size_t index) const; /// Serializes row at specified index into 'buffer'. /// 'buffer' must have sufficient capacity and set to all zeros. - int32_t serialize(vector_size_t index, char* buffer); + int32_t serialize(vector_size_t index, char* buffer) const; /// Serializes rows in the range [offset, offset + size) into 'buffer' at /// given 'bufferOffsets'. 'buffer' must have sufficient capacity and set to @@ -47,7 +53,7 @@ class CompactRow { vector_size_t offset, vector_size_t size, const size_t* bufferOffsets, - char* buffer); + char* buffer) const; /// Deserializes multiple rows into a RowVector of specified type. The type /// must match the contents of the serialized rows. @@ -61,7 +67,7 @@ class CompactRow { void initialize(const TypePtr& type); - bool isNullAt(vector_size_t); + bool isNullAt(vector_size_t) const; /// Fixed-width types only. Returns number of bytes used by single value. int32_t valueBytes() const { @@ -70,56 +76,59 @@ class CompactRow { /// Writes fixed-width value at specified index into 'buffer'. Value must not /// be null. - void serializeFixedWidth(vector_size_t index, char* buffer); + void serializeFixedWidth(vector_size_t index, char* buffer) const; /// Writes range of fixed-width values between 'offset' and 'offset + size' /// into 'buffer'. Values can be null. - void - serializeFixedWidth(vector_size_t offset, vector_size_t size, char* buffer); + void serializeFixedWidth( + vector_size_t offset, + vector_size_t size, + char* buffer) const; /// Returns serialized size of variable-width row. - int32_t variableWidthRowSize(vector_size_t index); + int32_t variableWidthRowSize(vector_size_t index) const; /// Writes variable-width value at specified index into 'buffer'. Value must /// not be null. Returns number of bytes written to 'buffer'. - int32_t serializeVariableWidth(vector_size_t index, char* buffer); + int32_t serializeVariableWidth(vector_size_t index, char* buffer) const; private: /// Returns serialized size of array row. - int32_t arrayRowSize(vector_size_t index); + int32_t arrayRowSize(vector_size_t index) const; /// Serializes array value to buffer. Value must not be null. Returns number /// of bytes written to 'buffer'. - int32_t serializeArray(vector_size_t index, char* buffer); + int32_t serializeArray(vector_size_t index, char* buffer) const; /// Returns serialized size of map row. - int32_t mapRowSize(vector_size_t index); + int32_t mapRowSize(vector_size_t index) const; /// Serializes map value to buffer. Value must not be null. Returns number of /// bytes written to 'buffer'. - int32_t serializeMap(vector_size_t index, char* buffer); + int32_t serializeMap(vector_size_t index, char* buffer) const; /// Returns serialized size of a range of values. int32_t arrayRowSize( - CompactRow& elements, + const CompactRow& elements, vector_size_t offset, vector_size_t size, - bool fixedWidth); + bool fixedWidth) const; /// Serializes a range of values into buffer. Returns number of bytes written /// to 'buffer'. int32_t serializeAsArray( - CompactRow& elements, + const CompactRow& elements, vector_size_t offset, vector_size_t size, bool fixedWidth, - char* buffer); + char* buffer) const; + ; /// Returns serialized size of struct value. - int32_t rowRowSize(vector_size_t index); + int32_t rowRowSize(vector_size_t index) const; /// Serializes struct value to buffer. Value must not be null. - int32_t serializeRow(vector_size_t index, char* buffer); + int32_t serializeRow(vector_size_t index, char* buffer) const; /// Serializes struct values in range [offset, offset + size) to buffer. /// Value must not be null. @@ -127,7 +136,7 @@ class CompactRow { vector_size_t offset, vector_size_t size, char* buffer, - const size_t* bufferOffsets); + const size_t* bufferOffsets) const; const TypeKind typeKind_; DecodedVector decoded_; diff --git a/velox/row/UnsafeRowFast.cpp b/velox/row/UnsafeRowFast.cpp index 2f07c633f536b..b786f6a121165 100644 --- a/velox/row/UnsafeRowFast.cpp +++ b/velox/row/UnsafeRowFast.cpp @@ -19,6 +19,7 @@ namespace facebook::velox::row { namespace { static const int32_t kFieldWidth = 8; +using TRowSize = uint32_t; int32_t alignBits(int32_t numBits) { return bits::nwords(numBits) * 8; @@ -121,11 +122,27 @@ void UnsafeRowFast::initialize(const TypePtr& type) { } } -int32_t UnsafeRowFast::rowSize(vector_size_t index) { +void UnsafeRowFast::serializedRowSizes( + const folly::Range& rows, + vector_size_t** sizes) const { + if (const auto fixedRowSize = + UnsafeRowFast::fixedRowSize(asRowType(decoded_.base()->type()))) { + for (const auto row : rows) { + *sizes[row] = fixedRowSize.value() + sizeof(TRowSize); + } + return; + } + + for (const auto& row : rows) { + *sizes[row] = rowSize(row) + sizeof(TRowSize); + } +} + +int32_t UnsafeRowFast::rowSize(vector_size_t index) const { return rowRowSize(index); } -int32_t UnsafeRowFast::variableWidthRowSize(vector_size_t index) { +int32_t UnsafeRowFast::variableWidthRowSize(vector_size_t index) const { switch (typeKind_) { case TypeKind::VARCHAR: [[fallthrough]]; @@ -147,15 +164,16 @@ int32_t UnsafeRowFast::variableWidthRowSize(vector_size_t index) { }; } -bool UnsafeRowFast::isNullAt(vector_size_t index) { +bool UnsafeRowFast::isNullAt(vector_size_t index) const { return decoded_.isNullAt(index); } -int32_t UnsafeRowFast::serialize(vector_size_t index, char* buffer) { +int32_t UnsafeRowFast::serialize(vector_size_t index, char* buffer) const { return serializeRow(index, buffer); } -void UnsafeRowFast::serializeFixedWidth(vector_size_t index, char* buffer) { +void UnsafeRowFast::serializeFixedWidth(vector_size_t index, char* buffer) + const { VELOX_DCHECK(fixedWidthTypeKind_); switch (typeKind_) { case TypeKind::BOOLEAN: @@ -176,7 +194,7 @@ void UnsafeRowFast::serializeFixedWidth(vector_size_t index, char* buffer) { void UnsafeRowFast::serializeFixedWidth( vector_size_t offset, vector_size_t size, - char* buffer) { + char* buffer) const { VELOX_DCHECK(supportsBulkCopy_); // decoded_.data() can be null if all values are null. if (decoded_.data()) { @@ -187,9 +205,8 @@ void UnsafeRowFast::serializeFixedWidth( } } -int32_t UnsafeRowFast::serializeVariableWidth( - vector_size_t index, - char* buffer) { +int32_t UnsafeRowFast::serializeVariableWidth(vector_size_t index, char* buffer) + const { switch (typeKind_) { case TypeKind::VARCHAR: [[fallthrough]]; @@ -214,7 +231,7 @@ int32_t UnsafeRowFast::serializeVariableWidth( }; } -int32_t UnsafeRowFast::arrayRowSize(vector_size_t index) { +int32_t UnsafeRowFast::arrayRowSize(vector_size_t index) const { auto baseIndex = decoded_.index(index); // array size | null bits | fixed-width data | variable-width data @@ -225,7 +242,7 @@ int32_t UnsafeRowFast::arrayRowSize(vector_size_t index) { return arrayRowSize(children_[0], offset, size, childIsFixedWidth_[0]); } -int32_t UnsafeRowFast::serializeArray(vector_size_t index, char* buffer) { +int32_t UnsafeRowFast::serializeArray(vector_size_t index, char* buffer) const { auto baseIndex = decoded_.index(index); // array size | null bits | fixed-width data | variable-width data @@ -237,7 +254,7 @@ int32_t UnsafeRowFast::serializeArray(vector_size_t index, char* buffer) { children_[0], offset, size, childIsFixedWidth_[0], buffer); } -int32_t UnsafeRowFast::mapRowSize(vector_size_t index) { +int32_t UnsafeRowFast::mapRowSize(vector_size_t index) const { auto baseIndex = decoded_.index(index); // size of serialized keys array in bytes | | @@ -251,7 +268,7 @@ int32_t UnsafeRowFast::mapRowSize(vector_size_t index) { arrayRowSize(children_[1], offset, size, childIsFixedWidth_[1]); } -int32_t UnsafeRowFast::serializeMap(vector_size_t index, char* buffer) { +int32_t UnsafeRowFast::serializeMap(vector_size_t index, char* buffer) const { auto baseIndex = decoded_.index(index); // size of serialized keys array in bytes | | @@ -285,10 +302,10 @@ int32_t UnsafeRowFast::serializeMap(vector_size_t index, char* buffer) { } int32_t UnsafeRowFast::arrayRowSize( - UnsafeRowFast& elements, + const UnsafeRowFast& elements, vector_size_t offset, vector_size_t size, - bool fixedWidth) { + bool fixedWidth) const { int32_t nullBytes = alignBits(size); int32_t rowSize = kFieldWidth + nullBytes; @@ -308,11 +325,11 @@ int32_t UnsafeRowFast::arrayRowSize( } int32_t UnsafeRowFast::serializeAsArray( - UnsafeRowFast& elements, + const UnsafeRowFast& elements, vector_size_t offset, vector_size_t size, bool fixedWidth, - char* buffer) { + char* buffer) const { // array size | null bits | fixed-width data | variable-width data // Write array size. @@ -362,7 +379,7 @@ int32_t UnsafeRowFast::serializeAsArray( return variableWidthOffset; } -int32_t UnsafeRowFast::rowRowSize(vector_size_t index) { +int32_t UnsafeRowFast::rowRowSize(vector_size_t index) const { auto childIndex = decoded_.index(index); const auto numFields = children_.size(); @@ -377,7 +394,7 @@ int32_t UnsafeRowFast::rowRowSize(vector_size_t index) { return size; } -int32_t UnsafeRowFast::serializeRow(vector_size_t index, char* buffer) { +int32_t UnsafeRowFast::serializeRow(vector_size_t index, char* buffer) const { auto childIndex = decoded_.index(index); int64_t variableWidthOffset = rowNullBytes_ + kFieldWidth * children_.size(); diff --git a/velox/row/UnsafeRowFast.h b/velox/row/UnsafeRowFast.h index e82a83912cf48..e7a40f25a11da 100644 --- a/velox/row/UnsafeRowFast.h +++ b/velox/row/UnsafeRowFast.h @@ -24,24 +24,31 @@ class UnsafeRowFast { public: explicit UnsafeRowFast(const RowVectorPtr& vector); + /// Returns the serialized sizes of the rows at specified row indexes. + /// + /// TODO: optimizes using columnar serialization size calculation. + void serializedRowSizes( + const folly::Range& rows, + vector_size_t** sizes) const; + /// Returns row size if all fields are fixed width. Return std::nullopt if /// there are variable-width fields. static std::optional fixedRowSize(const RowTypePtr& rowType); /// Returns serialized size of the row at specified index. Use only if /// 'fixedRowSize' returned std::nullopt. - int32_t rowSize(vector_size_t index); + int32_t rowSize(vector_size_t index) const; /// Serializes row at specified index into 'buffer'. /// 'buffer' must have sufficient capacity and set to all zeros. - int32_t serialize(vector_size_t index, char* buffer); + int32_t serialize(vector_size_t index, char* buffer) const; protected: explicit UnsafeRowFast(const VectorPtr& vector); void initialize(const TypePtr& type); - bool isNullAt(vector_size_t); + bool isNullAt(vector_size_t) const; /// Fixed-width types only. Returns number of bytes used by single value. int32_t valueBytes() const { @@ -50,56 +57,58 @@ class UnsafeRowFast { /// Writes fixed-width value at specified index into 'buffer'. Value must not /// be null. - void serializeFixedWidth(vector_size_t index, char* buffer); + void serializeFixedWidth(vector_size_t index, char* buffer) const; /// Writes range of fixed-width values between 'offset' and 'offset + size' /// into 'buffer'. Values can be null. - void - serializeFixedWidth(vector_size_t offset, vector_size_t size, char* buffer); + void serializeFixedWidth( + vector_size_t offset, + vector_size_t size, + char* buffer) const; /// Returns serialized size of variable-width row. - int32_t variableWidthRowSize(vector_size_t index); + int32_t variableWidthRowSize(vector_size_t index) const; /// Writes variable-width value at specified index into 'buffer'. Value must /// not be null. Returns number of bytes written to 'buffer'. - int32_t serializeVariableWidth(vector_size_t index, char* buffer); + int32_t serializeVariableWidth(vector_size_t index, char* buffer) const; private: /// Returns serialized size of array row. - int32_t arrayRowSize(vector_size_t index); + int32_t arrayRowSize(vector_size_t index) const; /// Serializes array value to buffer. Value must not be null. Returns number /// of bytes written to 'buffer'. - int32_t serializeArray(vector_size_t index, char* buffer); + int32_t serializeArray(vector_size_t index, char* buffer) const; /// Returns serialized size of map row. - int32_t mapRowSize(vector_size_t index); + int32_t mapRowSize(vector_size_t index) const; /// Serializes map value to buffer. Value must not be null. Returns number of /// bytes written to 'buffer'. - int32_t serializeMap(vector_size_t index, char* buffer); + int32_t serializeMap(vector_size_t index, char* buffer) const; /// Returns serialized size of a range of values. int32_t arrayRowSize( - UnsafeRowFast& elements, + const UnsafeRowFast& elements, vector_size_t offset, vector_size_t size, - bool fixedWidth); + bool fixedWidth) const; /// Serializes a range of values into buffer using UnsafeRow Array /// serialization. Returns number of bytes written to 'buffer'. int32_t serializeAsArray( - UnsafeRowFast& elements, + const UnsafeRowFast& elements, vector_size_t offset, vector_size_t size, bool fixedWidth, - char* buffer); + char* buffer) const; /// Returns serialized size of struct value. - int32_t rowRowSize(vector_size_t index); + int32_t rowRowSize(vector_size_t index) const; /// Serializes struct value to buffer. Value must not be null. - int32_t serializeRow(vector_size_t index, char* buffer); + int32_t serializeRow(vector_size_t index, char* buffer) const; const TypeKind typeKind_; DecodedVector decoded_; diff --git a/velox/row/tests/CompactRowTest.cpp b/velox/row/tests/CompactRowTest.cpp index 1fc2e70c52b5b..8bfa4fc3989d7 100644 --- a/velox/row/tests/CompactRowTest.cpp +++ b/velox/row/tests/CompactRowTest.cpp @@ -69,6 +69,20 @@ class CompactRowTest : public ::testing::Test, public VectorTestBase { } } + std::vector rows(numRows); + std::iota(rows.begin(), rows.end(), 0); + std::vector serializedRowSizes(numRows); + std::vector serializedRowSizesPtr(numRows); + for (auto i = 0; i < numRows; ++i) { + serializedRowSizesPtr[i] = &serializedRowSizes[i]; + } + row.serializedRowSizes( + folly::Range(rows.data(), numRows), serializedRowSizesPtr.data()); + for (auto i = 0; i < numRows; ++i) { + // The serialized row includes the size of the row. + ASSERT_EQ(serializedRowSizes[i], row.rowSize(i) + sizeof(uint32_t)); + } + BufferPtr buffer = AlignedBuffer::allocate(totalSize, pool(), 0); auto* rawBuffer = buffer->asMutable(); { diff --git a/velox/row/tests/UnsafeRowFuzzTest.cpp b/velox/row/tests/UnsafeRowFuzzTest.cpp index 8ce4ce74eb755..9159bf73c5fd2 100644 --- a/velox/row/tests/UnsafeRowFuzzTest.cpp +++ b/velox/row/tests/UnsafeRowFuzzTest.cpp @@ -163,10 +163,26 @@ TEST_F(UnsafeRowFuzzTests, fast) { }); doTest(rowType, [&](const RowVectorPtr& data) { + const auto numRows = data->size(); std::vector> serialized; - serialized.reserve(data->size()); + serialized.reserve(numRows); UnsafeRowFast fast(data); + + std::vector rows(numRows); + std::iota(rows.begin(), rows.end(), 0); + std::vector serializedRowSizes(numRows); + std::vector serializedRowSizesPtr(numRows); + for (auto i = 0; i < numRows; ++i) { + serializedRowSizesPtr[i] = &serializedRowSizes[i]; + } + fast.serializedRowSizes( + folly::Range(rows.data(), numRows), serializedRowSizesPtr.data()); + for (auto i = 0; i < numRows; ++i) { + // The serialized row includes the size of the row. + VELOX_CHECK_EQ(serializedRowSizes[i], fast.rowSize(i) + sizeof(uint32_t)); + } + for (auto i = 0; i < data->size(); ++i) { auto rowSize = fast.serialize(i, buffers_[i]); VELOX_CHECK_LE(rowSize, kBufferSize); diff --git a/velox/serializers/CompactRowSerializer.cpp b/velox/serializers/CompactRowSerializer.cpp index 524184ca92006..6a1324dd433d9 100644 --- a/velox/serializers/CompactRowSerializer.cpp +++ b/velox/serializers/CompactRowSerializer.cpp @@ -14,24 +14,17 @@ * limitations under the License. */ #include "velox/serializers/CompactRowSerializer.h" + #include +#include "velox/common/base/Exceptions.h" #include "velox/row/CompactRow.h" namespace facebook::velox::serializer { - -void CompactRowVectorSerde::estimateSerializedSize( - const BaseVector* /* vector */, - const folly::Range& /* ranges */, - vector_size_t** /* sizes */, - Scratch& /*scratch*/) { - VELOX_UNSUPPORTED(); -} - namespace { +using TRowSize = uint32_t; + class CompactRowVectorSerializer : public IterativeVectorSerializer { public: - using TRowSize = uint32_t; - explicit CompactRowVectorSerializer(StreamArena* streamArena) : pool_{streamArena->pool()} {} @@ -98,6 +91,34 @@ class CompactRowVectorSerializer : public IterativeVectorSerializer { } } + void append( + const row::CompactRow& compactRow, + const folly::Range& rows, + const std::vector& sizes) override { + size_t totalSize = 0; + for (const auto row : rows) { + totalSize += sizes[row]; + } + if (totalSize == 0) { + return; + } + + BufferPtr buffer = AlignedBuffer::allocate(totalSize, pool_, 0); + auto* rawBuffer = buffer->asMutable(); + buffers_.push_back(std::move(buffer)); + + size_t offset = 0; + for (auto& row : rows) { + // Write row data. + const TRowSize size = + compactRow.serialize(row, rawBuffer + offset + sizeof(TRowSize)); + + // Write raw size. Needs to be in big endian order. + *(TRowSize*)(rawBuffer + offset) = folly::Endian::big(size); + offset += sizeof(TRowSize) + size; + } + } + size_t maxSerializedSize() const override { size_t totalSize = 0; for (const auto& buffer : buffers_) { @@ -113,35 +134,40 @@ class CompactRowVectorSerializer : public IterativeVectorSerializer { buffers_.clear(); } + void clear() override {} + private: memory::MemoryPool* const pool_; std::vector buffers_; }; // Read from the stream until the full row is concatenated. -std::string concatenatePartialRow( +void concatenatePartialRow( ByteInputStream* source, - std::string_view rowFragment, - CompactRowVectorSerializer::TRowSize rowSize) { - std::string rowBuffer; - rowBuffer.reserve(rowSize); - rowBuffer.append(rowFragment); - + TRowSize rowSize, + std::string& rowBuffer) { while (rowBuffer.size() < rowSize) { - rowFragment = source->nextView(rowSize - rowBuffer.size()); + const std::string_view rowFragment = + source->nextView(rowSize - rowBuffer.size()); VELOX_CHECK_GT( rowFragment.size(), 0, "Unable to read full serialized CompactRow. Needed {} but read {} bytes.", rowSize - rowBuffer.size(), rowFragment.size()); - rowBuffer += rowFragment; + rowBuffer.append(rowFragment.data(), rowFragment.size()); } - return rowBuffer; } } // namespace +void CompactRowVectorSerde::estimateSerializedSize( + const row::CompactRow* compactRow, + const folly::Range& rows, + vector_size_t** sizes) { + compactRow->serializedRowSizes(rows, sizes); +} + std::unique_ptr CompactRowVectorSerde::createIterativeSerializer( RowTypePtr /* type */, @@ -158,23 +184,25 @@ void CompactRowVectorSerde::deserialize( RowVectorPtr* result, const Options* /* options */) { std::vector serializedRows; - std::vector concatenatedRows; - + std::vector> serializedBuffers; while (!source->atEnd()) { // First read row size in big endian order. - auto rowSize = folly::Endian::big( - source->read()); - auto row = source->nextView(rowSize); + const auto rowSize = folly::Endian::big(source->read()); + auto serializedBuffer = std::make_unique(); + serializedBuffer->reserve(rowSize); + const auto row = source->nextView(rowSize); + serializedBuffer->append(row.data(), row.size()); // If we couldn't read the entire row at once, we need to concatenate it // in a different buffer. - if (row.size() < rowSize) { - concatenatedRows.push_back(concatenatePartialRow(source, row, rowSize)); - row = concatenatedRows.back(); + if (serializedBuffer->size() < rowSize) { + concatenatePartialRow(source, rowSize, *serializedBuffer); } - VELOX_CHECK_EQ(row.size(), rowSize); - serializedRows.push_back(row); + VELOX_CHECK_EQ(serializedBuffer->size(), rowSize); + serializedBuffers.emplace_back(std::move(serializedBuffer)); + serializedRows.push_back(std::string_view( + serializedBuffers.back()->data(), serializedBuffers.back()->size())); } if (serializedRows.empty()) { @@ -190,4 +218,11 @@ void CompactRowVectorSerde::registerVectorSerde() { velox::registerVectorSerde(std::make_unique()); } +// static +void CompactRowVectorSerde::registerNamedVectorSerde() { + velox::registerNamedVectorSerde( + VectorSerde::Kind::kCompactRow, + std::make_unique()); +} + } // namespace facebook::velox::serializer diff --git a/velox/serializers/CompactRowSerializer.h b/velox/serializers/CompactRowSerializer.h index 42a700282ecb1..bcf37b8469423 100644 --- a/velox/serializers/CompactRowSerializer.h +++ b/velox/serializers/CompactRowSerializer.h @@ -22,14 +22,12 @@ namespace facebook::velox::serializer { class CompactRowVectorSerde : public VectorSerde { public: - CompactRowVectorSerde() = default; + CompactRowVectorSerde() : VectorSerde(VectorSerde::Kind::kCompactRow) {} - // We do not implement this method since it is not used in production code. void estimateSerializedSize( - const BaseVector* vector, - const folly::Range& ranges, - vector_size_t** sizes, - Scratch& scratch) override; + const row::CompactRow* compactRow, + const folly::Range& rows, + vector_size_t** sizes) override; // This method is not used in production code. It is only used to // support round-trip tests for deserialization. @@ -48,6 +46,7 @@ class CompactRowVectorSerde : public VectorSerde { const Options* options) override; static void registerVectorSerde(); + static void registerNamedVectorSerde(); }; } // namespace facebook::velox::serializer diff --git a/velox/serializers/PrestoSerializer.cpp b/velox/serializers/PrestoSerializer.cpp index 952bfa2061b92..37ded5e462264 100644 --- a/velox/serializers/PrestoSerializer.cpp +++ b/velox/serializers/PrestoSerializer.cpp @@ -2948,12 +2948,12 @@ void estimateFlatSerializedSize( const BaseVector* vector, const folly::Range& ranges, vector_size_t** sizes) { - auto valueSize = vector->type()->cppSizeInBytes(); + const auto valueSize = vector->type()->cppSizeInBytes(); if (vector->mayHaveNulls()) { - auto rawNulls = vector->rawNulls(); + const auto* rawNulls = vector->rawNulls(); for (int32_t i = 0; i < ranges.size(); ++i) { - auto end = ranges[i].begin + ranges[i].size; - auto numValues = bits::countBits(rawNulls, ranges[i].begin, end); + const auto end = ranges[i].begin + ranges[i].size; + const auto numValues = bits::countBits(rawNulls, ranges[i].begin, end); // Add the size of the values. *(sizes[i]) += numValues * valueSize; // Add the size of the null bit mask if there are nulls in the range. @@ -3063,19 +3063,20 @@ void estimateFlattenedConstantSerializedSize( const folly::Range& ranges, vector_size_t** sizes, Scratch& scratch) { - VELOX_CHECK(vector->encoding() == VectorEncoding::Simple::CONSTANT); + VELOX_CHECK_EQ(vector->encoding(), VectorEncoding::Simple::CONSTANT); using T = typename KindToFlatVector::WrapperType; - auto constantVector = vector->as>(); + auto* constantVector = vector->as>(); if (constantVector->valueVector()) { estimateWrapperSerializedSize(ranges, sizes, vector, scratch); return; } + int32_t elementSize = sizeof(T); if (constantVector->isNullAt(0)) { elementSize = 1; } else if (std::is_same_v) { - auto value = constantVector->valueAt(0); - auto string = reinterpret_cast(&value); + const auto value = constantVector->valueAt(0); + const auto* string = reinterpret_cast(&value); elementSize = string->size(); } for (int32_t i = 0; i < ranges.size(); ++i) { @@ -3184,7 +3185,7 @@ void estimateSerializedSizeInt( estimateSerializedSizeInt(vector->loadedVector(), ranges, sizes, scratch); break; default: - VELOX_CHECK(false, "Unsupported vector encoding {}", vector->encoding()); + VELOX_UNSUPPORTED("Unsupported vector encoding {}", vector->encoding()); } } @@ -3280,8 +3281,8 @@ void estimateWrapperSerializedSize( ScratchPtr innerSizesHolder(scratch); const int32_t numRows = rows.size(); int32_t numInner = 0; - auto innerRows = innerRowsHolder.get(numRows); - auto innerSizes = sizes; + auto* innerRows = innerRowsHolder.get(numRows); + auto* innerSizes = sizes; const BaseVector* wrapped; if (wrapper->encoding() == VectorEncoding::Simple::DICTIONARY && !wrapper->rawNulls()) { @@ -3304,6 +3305,7 @@ void estimateWrapperSerializedSize( if (numInner == 0) { return; } + estimateSerializedSizeInt( wrapped, folly::Range(innerRows, numInner), @@ -3317,30 +3319,32 @@ void estimateFlattenedConstantSerializedSize( const folly::Range& rows, vector_size_t** sizes, Scratch& scratch) { - VELOX_CHECK(vector->encoding() == VectorEncoding::Simple::CONSTANT); + VELOX_CHECK_EQ(vector->encoding(), VectorEncoding::Simple::CONSTANT); + using T = typename KindToFlatVector::WrapperType; - auto constantVector = vector->as>(); + auto* constantVector = vector->as>(); int32_t elementSize = sizeof(T); if (constantVector->isNullAt(0)) { elementSize = 1; } else if (vector->valueVector()) { - auto values = constantVector->wrappedVector(); + const auto* values = constantVector->wrappedVector(); vector_size_t* sizePtr = &elementSize; - vector_size_t singleRow = constantVector->wrappedIndex(0); + const vector_size_t singleRow = constantVector->wrappedIndex(0); estimateSerializedSizeInt( values, folly::Range(&singleRow, 1), &sizePtr, scratch); } else if (std::is_same_v) { - auto value = constantVector->valueAt(0); - auto string = reinterpret_cast(&value); + const auto value = constantVector->valueAt(0); + const auto string = reinterpret_cast(&value); elementSize = string->size(); } for (int32_t i = 0; i < rows.size(); ++i) { *sizes[i] += elementSize; } } + void estimateSerializedSizeInt( const BaseVector* vector, const folly::Range& rows, @@ -3404,7 +3408,7 @@ void estimateSerializedSizeInt( mutableInnerRows); innerRows = mutableInnerRows; } - auto rowVector = vector->as(); + auto* rowVector = vector->as(); auto& children = rowVector->children(); for (auto& child : children) { if (child) { @@ -3418,7 +3422,7 @@ void estimateSerializedSizeInt( break; } case VectorEncoding::Simple::MAP: { - auto mapVector = vector->asUnchecked(); + auto* mapVector = vector->asUnchecked(); ScratchPtr rangeHolder(scratch); ScratchPtr sizesHolder(scratch); const auto numRanges = rowsToRanges( @@ -3447,7 +3451,7 @@ void estimateSerializedSizeInt( break; } case VectorEncoding::Simple::ARRAY: { - auto arrayVector = vector->as(); + auto* arrayVector = vector->as(); ScratchPtr rangeHolder(scratch); ScratchPtr sizesHolder(scratch); const auto numRanges = rowsToRanges( @@ -3474,7 +3478,7 @@ void estimateSerializedSizeInt( estimateSerializedSizeInt(vector->loadedVector(), rows, sizes, scratch); break; default: - VELOX_CHECK(false, "Unsupported vector encoding {}", vector->encoding()); + VELOX_UNSUPPORTED("Unsupported vector encoding {}", vector->encoding()); } } @@ -3700,7 +3704,7 @@ void estimateDictionarySerializedSize( const folly::Range& ranges, vector_size_t** sizes, Scratch& scratch) { - VELOX_CHECK(vector->encoding() == VectorEncoding::Simple::DICTIONARY); + VELOX_CHECK_EQ(vector->encoding(), VectorEncoding::Simple::DICTIONARY); using T = typename KindToFlatVector::WrapperType; auto dictionaryVector = vector->as>(); @@ -3911,8 +3915,7 @@ class PrestoBatchVectorSerializer : public BatchVectorSerializer { scratch); break; default: - VELOX_CHECK( - false, "Unsupported vector encoding {}", vector->encoding()); + VELOX_UNSUPPORTED("Unsupported vector encoding {}", vector->encoding()); } } @@ -4081,7 +4084,7 @@ void PrestoVectorSerde::estimateSerializedSize( void PrestoVectorSerde::estimateSerializedSize( const BaseVector* vector, - const folly::Range rows, + const folly::Range& rows, vector_size_t** sizes, Scratch& scratch) { estimateSerializedSizeInt(vector->loadedVector(), rows, sizes, scratch); @@ -4263,19 +4266,35 @@ void PrestoVectorSerde::deserializeSingleColumn( *result = row->childAt(0); } +namespace { +void initBitsToMapOnce() { + static folly::once_flag initOnceFlag; + folly::call_once(initOnceFlag, [&]() { + auto toByte = [](int32_t number, int32_t bit) { + return static_cast(bits::isBitSet(&number, bit)) << (bit * 8); + }; + for (auto i = 0; i < 256; ++i) { + bitsToBytesMap[i] = toByte(i, 0) | toByte(i, 1) | toByte(i, 2) | + toByte(i, 3) | toByte(i, 4) | toByte(i, 5) | toByte(i, 6) | + toByte(i, 7); + } + }); +} +} // namespace + // static void PrestoVectorSerde::registerVectorSerde() { - auto toByte = [](int32_t number, int32_t bit) { - return static_cast(bits::isBitSet(&number, bit)) << (bit * 8); - }; - for (auto i = 0; i < 256; ++i) { - bitsToBytesMap[i] = toByte(i, 0) | toByte(i, 1) | toByte(i, 2) | - toByte(i, 3) | toByte(i, 4) | toByte(i, 5) | toByte(i, 6) | - toByte(i, 7); - } + initBitsToMapOnce(); velox::registerVectorSerde(std::make_unique()); } +// static +void PrestoVectorSerde::registerNamedVectorSerde() { + initBitsToMapOnce(); + velox::registerNamedVectorSerde( + VectorSerde::Kind::kPresto, std::make_unique()); +} + namespace { class PrestoVectorLexer { public: diff --git a/velox/serializers/PrestoSerializer.h b/velox/serializers/PrestoSerializer.h index 6bd323568d116..be5ca971ffd6c 100644 --- a/velox/serializers/PrestoSerializer.h +++ b/velox/serializers/PrestoSerializer.h @@ -74,6 +74,8 @@ class PrestoVectorSerde : public VectorSerde { float minCompressionRatio{0.8}; }; + PrestoVectorSerde() : VectorSerde(Kind::kPresto) {} + /// Adds the serialized sizes of the rows of 'vector' in 'ranges[i]' to /// '*sizes[i]'. void estimateSerializedSize( @@ -82,9 +84,11 @@ class PrestoVectorSerde : public VectorSerde { vector_size_t** sizes, Scratch& scratch) override; + /// Adds the serialized sizes of the rows of 'vector' in 'rows[i]' to + /// '*sizes[i]'. void estimateSerializedSize( const BaseVector* vector, - const folly::Range rows, + const folly::Range& rows, vector_size_t** sizes, Scratch& scratch) override; @@ -188,6 +192,7 @@ class PrestoVectorSerde : public VectorSerde { const Options* options = nullptr); static void registerVectorSerde(); + static void registerNamedVectorSerde(); }; class PrestoOutputStreamListener : public OutputStreamListener { diff --git a/velox/serializers/UnsafeRowSerializer.cpp b/velox/serializers/UnsafeRowSerializer.cpp index 2e5d6e3c8b228..b675f6adce091 100644 --- a/velox/serializers/UnsafeRowSerializer.cpp +++ b/velox/serializers/UnsafeRowSerializer.cpp @@ -19,20 +19,13 @@ #include "velox/row/UnsafeRowFast.h" namespace facebook::velox::serializer::spark { - -void UnsafeRowVectorSerde::estimateSerializedSize( - const BaseVector* /* vector */, - const folly::Range& /* ranges */, - vector_size_t** /* sizes */, - Scratch& /*scratch*/) { - VELOX_UNSUPPORTED(); +namespace { +using TRowSize = uint32_t; } namespace { class UnsafeRowVectorSerializer : public IterativeVectorSerializer { public: - using TRowSize = uint32_t; - explicit UnsafeRowVectorSerializer(StreamArena* streamArena) : pool_{streamArena->pool()} {} @@ -78,6 +71,34 @@ class UnsafeRowVectorSerializer : public IterativeVectorSerializer { } } + void append( + const row::UnsafeRowFast& unsafeRow, + const folly::Range& rows, + const std::vector& sizes) override { + size_t totalSize = 0; + for (const auto row : rows) { + totalSize += sizes[row]; + } + if (totalSize == 0) { + return; + } + + BufferPtr buffer = AlignedBuffer::allocate(totalSize, pool_, 0); + auto* rawBuffer = buffer->asMutable(); + buffers_.push_back(std::move(buffer)); + + size_t offset = 0; + for (auto& row : rows) { + // Write row data. + const TRowSize size = + unsafeRow.serialize(row, rawBuffer + offset + sizeof(TRowSize)); + + // Write raw size. Needs to be in big endian order. + *(TRowSize*)(rawBuffer + offset) = folly::Endian::big(size); + offset += sizeof(TRowSize) + size; + } + } + size_t maxSerializedSize() const override { size_t totalSize = 0; for (const auto& buffer : buffers_) { @@ -93,35 +114,39 @@ class UnsafeRowVectorSerializer : public IterativeVectorSerializer { buffers_.clear(); } + void clear() override {} + private: memory::MemoryPool* const pool_; std::vector buffers_; }; // Read from the stream until the full row is concatenated. -std::string concatenatePartialRow( +void concatenatePartialRow( ByteInputStream* source, - std::string_view rowFragment, - UnsafeRowVectorSerializer::TRowSize rowSize) { - std::string rowBuffer; - rowBuffer.reserve(rowSize); - rowBuffer.append(rowFragment); - + TRowSize rowSize, + std::string& rowBuffer) { while (rowBuffer.size() < rowSize) { - rowFragment = source->nextView(rowSize - rowBuffer.size()); + const std::string_view rowFragment = + source->nextView(rowSize - rowBuffer.size()); VELOX_CHECK_GT( rowFragment.size(), 0, "Unable to read full serialized UnsafeRow. Needed {} but read {} bytes.", rowSize - rowBuffer.size(), rowFragment.size()); - rowBuffer += rowFragment; + rowBuffer.append(rowFragment.data(), rowFragment.size()); } - return rowBuffer; } - } // namespace +void UnsafeRowVectorSerde::estimateSerializedSize( + const row::UnsafeRowFast* unsafeRow, + const folly::Range& rows, + vector_size_t** sizes) { + return unsafeRow->serializedRowSizes(rows, sizes); +} + std::unique_ptr UnsafeRowVectorSerde::createIterativeSerializer( RowTypePtr /* type */, @@ -138,23 +163,26 @@ void UnsafeRowVectorSerde::deserialize( RowVectorPtr* result, const Options* /* options */) { std::vector> serializedRows; - std::vector concatenatedRows; + std::vector> serializedBuffers; while (!source->atEnd()) { // First read row size in big endian order. - auto rowSize = - folly::Endian::big(source->read()); - auto row = source->nextView(rowSize); + const auto rowSize = folly::Endian::big(source->read()); + auto serializedBuffer = std::make_unique(); + serializedBuffer->reserve(rowSize); + const auto row = source->nextView(rowSize); + serializedBuffer->append(row.data(), row.size()); // If we couldn't read the entire row at once, we need to concatenate it // in a different buffer. - if (row.size() < rowSize) { - concatenatedRows.push_back(concatenatePartialRow(source, row, rowSize)); - row = concatenatedRows.back(); + if (serializedBuffer->size() < rowSize) { + concatenatePartialRow(source, rowSize, *serializedBuffer); } - VELOX_CHECK_EQ(row.size(), rowSize); - serializedRows.push_back(row); + VELOX_CHECK_EQ(serializedBuffer->size(), rowSize); + serializedBuffers.emplace_back(std::move(serializedBuffer)); + serializedRows.push_back(std::string_view( + serializedBuffers.back()->data(), serializedBuffers.back()->size())); } if (serializedRows.empty()) { @@ -172,4 +200,10 @@ void UnsafeRowVectorSerde::registerVectorSerde() { velox::registerVectorSerde(std::make_unique()); } +// static +void UnsafeRowVectorSerde::registerNamedVectorSerde() { + velox::registerNamedVectorSerde( + VectorSerde::Kind::kUnsafeRow, std::make_unique()); +} + } // namespace facebook::velox::serializer::spark diff --git a/velox/serializers/UnsafeRowSerializer.h b/velox/serializers/UnsafeRowSerializer.h index 1940281537087..e859d8f2cc0ff 100644 --- a/velox/serializers/UnsafeRowSerializer.h +++ b/velox/serializers/UnsafeRowSerializer.h @@ -21,13 +21,12 @@ namespace facebook::velox::serializer::spark { class UnsafeRowVectorSerde : public VectorSerde { public: - UnsafeRowVectorSerde() = default; - // We do not implement this method since it is not used in production code. + UnsafeRowVectorSerde() : VectorSerde(VectorSerde::Kind::kUnsafeRow) {} + void estimateSerializedSize( - const BaseVector* vector, - const folly::Range& ranges, - vector_size_t** sizes, - Scratch& scratch) override; + const row::UnsafeRowFast* unsafeRow, + const folly::Range& rows, + vector_size_t** sizes) override; // This method is not used in production code. It is only used to // support round-trip tests for deserialization. @@ -46,5 +45,6 @@ class UnsafeRowVectorSerde : public VectorSerde { const Options* options) override; static void registerVectorSerde(); + static void registerNamedVectorSerde(); }; } // namespace facebook::velox::serializer::spark diff --git a/velox/serializers/benchmarks/RowSerializerBenchmark.cpp b/velox/serializers/benchmarks/RowSerializerBenchmark.cpp index 9443cdedbfa50..b6df7c8bd787c 100644 --- a/velox/serializers/benchmarks/RowSerializerBenchmark.cpp +++ b/velox/serializers/benchmarks/RowSerializerBenchmark.cpp @@ -43,7 +43,7 @@ class RowSerializerBenchmark { suspender.dismiss(); Scratch scratch; - auto group = std::make_unique(pool_.get()); + auto group = std::make_unique(pool_.get(), nullptr); group->createStreamTree(rowType, data->size()); group->append( data, folly::Range(indexRanges.data(), indexRanges.size()), scratch); diff --git a/velox/serializers/tests/CompactRowSerializerTest.cpp b/velox/serializers/tests/CompactRowSerializerTest.cpp index c0a091b5e0b40..277957c323bda 100644 --- a/velox/serializers/tests/CompactRowSerializerTest.cpp +++ b/velox/serializers/tests/CompactRowSerializerTest.cpp @@ -15,6 +15,8 @@ */ #include "velox/serializers/CompactRowSerializer.h" #include +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/row/CompactRow.h" #include "velox/vector/fuzzer/VectorFuzzer.h" #include "velox/vector/tests/utils/VectorTestBase.h" @@ -22,7 +24,8 @@ namespace facebook::velox::serializer { namespace { class CompactRowSerializerTest : public ::testing::Test, - public test::VectorTestBase { + public test::VectorTestBase, + public testing::WithParamInterface { protected: static void SetUpTestCase() { memory::MemoryManager::testingSetInstance({}); @@ -30,31 +33,67 @@ class CompactRowSerializerTest : public ::testing::Test, void SetUp() override { pool_ = memory::memoryManager()->addLeafPool(); - serde_ = std::make_unique(); + deregisterVectorSerde(); + deregisterNamedVectorSerde(VectorSerde::Kind::kCompactRow); + serializer::CompactRowVectorSerde::registerVectorSerde(); + serializer::CompactRowVectorSerde::registerNamedVectorSerde(); + ASSERT_EQ(getVectorSerde()->kind(), VectorSerde::Kind::kCompactRow); + ASSERT_EQ( + getNamedVectorSerde(VectorSerde::Kind::kCompactRow)->kind(), + VectorSerde::Kind::kCompactRow); + } + + void TearDown() override { + deregisterVectorSerde(); + deregisterNamedVectorSerde(VectorSerde::Kind::kCompactRow); } void serialize(RowVectorPtr rowVector, std::ostream* output) { - auto numRows = rowVector->size(); + const auto numRows = rowVector->size(); // Serialize with different range size. - std::vector rows; + std::vector ranges; vector_size_t offset = 0; vector_size_t rangeSize = 1; + std::unique_ptr compactRow; + if (GetParam()) { + compactRow = std::make_unique(rowVector); + } while (offset < numRows) { auto size = std::min(rangeSize, numRows - offset); - rows.push_back(IndexRange{offset, size}); + ranges.push_back(IndexRange{offset, size}); offset += size; rangeSize = checkedMultiply(rangeSize, 2); } auto arena = std::make_unique(pool_.get()); auto rowType = asRowType(rowVector->type()); - auto serializer = - serde_->createIterativeSerializer(rowType, numRows, arena.get()); + auto serializer = getVectorSerde()->createIterativeSerializer( + rowType, numRows, arena.get()); Scratch scratch; - serializer->append( - rowVector, folly::Range(rows.data(), rows.size()), scratch); + if (GetParam()) { + std::vector serializedRowSizes(numRows); + std::vector serializedRowSizesPtr(numRows); + for (auto i = 0; i < numRows; ++i) { + serializedRowSizesPtr[i] = &serializedRowSizes[i]; + } + for (const auto& range : ranges) { + std::vector rows(range.size); + for (auto i = 0; i < range.size; ++i) { + rows[i] = range.begin + i; + } + getVectorSerde()->estimateSerializedSize( + compactRow.get(), rows, serializedRowSizesPtr.data()); + serializer->append( + *compactRow, + folly::Range(rows.data(), rows.size()), + serializedRowSizes); + } + } else { + serializer->append( + rowVector, folly::Range(ranges.data(), ranges.size()), scratch); + } auto size = serializer->maxSerializedSize(); OStreamOutputStream out(output); serializer->flush(&out); @@ -87,7 +126,8 @@ class CompactRowSerializerTest : public ::testing::Test, auto byteStream = toByteStream(input); RowVectorPtr result; - serde_->deserialize(byteStream.get(), pool_.get(), rowType, &result); + getVectorSerde()->deserialize( + byteStream.get(), pool_.get(), rowType, &result); return result; } @@ -101,11 +141,10 @@ class CompactRowSerializerTest : public ::testing::Test, } std::shared_ptr pool_; - std::unique_ptr serde_; }; -TEST_F(CompactRowSerializerTest, fuzz) { - auto rowType = ROW({ +TEST_P(CompactRowSerializerTest, fuzz) { + const auto rowType = ROW({ BOOLEAN(), TINYINT(), SMALLINT(), @@ -135,7 +174,7 @@ TEST_F(CompactRowSerializerTest, fuzz) { VectorFuzzer::Options::TimestampPrecision::kMicroSeconds; opts.containerLength = 10; - auto seed = folly::Random::rand32(); + const auto seed = folly::Random::rand32(); LOG(ERROR) << "Seed: " << seed; SCOPED_TRACE(fmt::format("seed: {}", seed)); @@ -145,5 +184,9 @@ TEST_F(CompactRowSerializerTest, fuzz) { testRoundTrip(data); } +VELOX_INSTANTIATE_TEST_SUITE_P( + CompactRowSerializerTest, + CompactRowSerializerTest, + testing::Values(false, true)); } // namespace } // namespace facebook::velox::serializer diff --git a/velox/serializers/tests/PrestoSerializerTest.cpp b/velox/serializers/tests/PrestoSerializerTest.cpp index da03e6f39739b..933d3d2a191e8 100644 --- a/velox/serializers/tests/PrestoSerializerTest.cpp +++ b/velox/serializers/tests/PrestoSerializerTest.cpp @@ -117,10 +117,13 @@ class PrestoSerializerTest rowVector.get(), indexRanges.value(), sizes.data(), scratch); serializer->append(rowVector, indexRanges.value(), scratch); } else if (rows.has_value()) { - raw_vector sizes(rows.value().size()); - std::fill(sizes.begin(), sizes.end(), &sizeEstimate); + raw_vector sizes(rows.value().size()); + std::vector sizePointers(rows.value().size()); + for (vector_size_t i = 0; i < sizes.size(); ++i) { + sizePointers[i] = &sizes[i]; + } serde_->estimateSerializedSize( - rowVector.get(), rows.value(), sizes.data(), scratch); + rowVector.get(), rows.value(), sizePointers.data(), scratch); serializer->append(rowVector, rows.value(), scratch); } else { vector_size_t* sizes = &sizeEstimate; @@ -1593,6 +1596,13 @@ class PrestoSerializerBatchEstimateSizeTest : public testing::Test, if (!isRegisteredVectorSerde()) { serializer::presto::PrestoVectorSerde::registerVectorSerde(); } + ASSERT_EQ(getVectorSerde()->kind(), VectorSerde::Kind::kPresto); + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } + ASSERT_EQ( + getNamedVectorSerde(VectorSerde::Kind::kPresto)->kind(), + VectorSerde::Kind::kPresto); memory::MemoryManager::testingSetInstance({}); } diff --git a/velox/serializers/tests/SerializerBenchmark.cpp b/velox/serializers/tests/SerializerBenchmark.cpp index d6aeb1f3add7b..a080a65f9fedc 100644 --- a/velox/serializers/tests/SerializerBenchmark.cpp +++ b/velox/serializers/tests/SerializerBenchmark.cpp @@ -106,7 +106,7 @@ class SerializerBenchmark : public VectorTestBase { auto rowVector = vm.rowVector({vector}); { MicrosecondTimer t(&item.irTime); - auto group = std::make_unique(pool_.get()); + auto group = std::make_unique(pool_.get(), nullptr); group->createStreamTree(rowType, rowSets[selIdx].size() - kPad); for (auto repeat = 0; repeat < numRepeat; ++repeat) { group->append( @@ -119,7 +119,7 @@ class SerializerBenchmark : public VectorTestBase { { MicrosecondTimer t(&item.rrTime); - auto group = std::make_unique(pool_.get()); + auto group = std::make_unique(pool_.get(), nullptr); group->createStreamTree(rowType, rowSets[selIdx].size()); for (auto repeat = 0; repeat < numRepeat; ++repeat) { diff --git a/velox/serializers/tests/UnsafeRowSerializerTest.cpp b/velox/serializers/tests/UnsafeRowSerializerTest.cpp index cfcb2a61f8bdf..050c104e33d29 100644 --- a/velox/serializers/tests/UnsafeRowSerializerTest.cpp +++ b/velox/serializers/tests/UnsafeRowSerializerTest.cpp @@ -16,13 +16,15 @@ #include "velox/serializers/UnsafeRowSerializer.h" #include #include "velox/common/base/tests/GTestUtils.h" +#include "velox/row/UnsafeRowFast.h" #include "velox/vector/fuzzer/VectorFuzzer.h" #include "velox/vector/tests/utils/VectorTestBase.h" using namespace facebook::velox; class UnsafeRowSerializerTest : public ::testing::Test, - public test::VectorTestBase { + public test::VectorTestBase, + public testing::WithParamInterface { protected: static void SetUpTestCase() { memory::MemoryManager::testingSetInstance({}); @@ -30,24 +32,56 @@ class UnsafeRowSerializerTest : public ::testing::Test, void SetUp() override { pool_ = memory::memoryManager()->addLeafPool(); - serde_ = std::make_unique(); + deregisterVectorSerde(); + deregisterNamedVectorSerde(VectorSerde::Kind::kCompactRow); + serializer::spark::UnsafeRowVectorSerde::registerVectorSerde(); + serializer::spark::UnsafeRowVectorSerde::registerNamedVectorSerde(); + ASSERT_EQ(getVectorSerde()->kind(), VectorSerde::Kind::kUnsafeRow); + ASSERT_EQ( + getNamedVectorSerde(VectorSerde::Kind::kUnsafeRow)->kind(), + VectorSerde::Kind::kUnsafeRow); + } + + void TearDown() override { + deregisterVectorSerde(); + deregisterNamedVectorSerde(VectorSerde::Kind::kUnsafeRow); } void serialize(RowVectorPtr rowVector, std::ostream* output) { - auto numRows = rowVector->size(); + const auto numRows = rowVector->size(); - std::vector rows(numRows); + std::vector ranges(numRows); for (int i = 0; i < numRows; i++) { - rows[i] = IndexRange{i, 1}; + ranges[i] = IndexRange{i, 1}; + } + + std::unique_ptr unsafeRow; + std::vector serializedRowSizes(numRows); + std::vector serializedRowSizesPtr(numRows); + std::vector rows(numRows); + std::iota(rows.begin(), rows.end(), 0); + for (auto i = 0; i < numRows; ++i) { + serializedRowSizesPtr[i] = &serializedRowSizes[i]; + } + if (GetParam()) { + unsafeRow = std::make_unique(rowVector); + getVectorSerde()->estimateSerializedSize( + unsafeRow.get(), rows, serializedRowSizesPtr.data()); } auto arena = std::make_unique(pool_.get()); auto rowType = std::dynamic_pointer_cast(rowVector->type()); - auto serializer = - serde_->createIterativeSerializer(rowType, numRows, arena.get()); + auto serializer = getVectorSerde()->createIterativeSerializer( + rowType, numRows, arena.get()); + + if (GetParam()) { + serializer->append(*unsafeRow, rows, serializedRowSizes); + } else { + Scratch scratch; + serializer->append( + rowVector, folly::Range(ranges.data(), numRows), scratch); + } - Scratch scratch; - serializer->append(rowVector, folly::Range(rows.data(), numRows), scratch); auto size = serializer->maxSerializedSize(); OStreamOutputStream out(output); serializer->flush(&out); @@ -74,7 +108,8 @@ class UnsafeRowSerializerTest : public ::testing::Test, auto byteStream = toByteStream(input); RowVectorPtr result; - serde_->deserialize(byteStream.get(), pool_.get(), rowType, &result); + getVectorSerde()->deserialize( + byteStream.get(), pool_.get(), rowType, &result); return result; } @@ -109,11 +144,10 @@ class UnsafeRowSerializerTest : public ::testing::Test, } std::shared_ptr pool_; - std::unique_ptr serde_; }; // These expected binary buffers were samples taken using Spark's java code. -TEST_F(UnsafeRowSerializerTest, tinyint) { +TEST_P(UnsafeRowSerializerTest, tinyint) { int8_t data[20] = {0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, 0, 123, 0, 0, 0, 0, 0, 0, 0}; auto expected = makeRowVector({makeFlatVector(std::vector{123})}); @@ -122,7 +156,7 @@ TEST_F(UnsafeRowSerializerTest, tinyint) { testDeserialize(data, 20, expected); } -TEST_F(UnsafeRowSerializerTest, bigint) { +TEST_P(UnsafeRowSerializerTest, bigint) { int8_t data[20] = {0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, 0, 62, 28, -36, -33, 2, 0, 0, 0}; auto expected = @@ -132,7 +166,7 @@ TEST_F(UnsafeRowSerializerTest, bigint) { testDeserialize(data, 20, expected); } -TEST_F(UnsafeRowSerializerTest, double) { +TEST_P(UnsafeRowSerializerTest, double) { int8_t data[20] = {0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, 0, 125, 63, 53, 94, -70, 73, -109, 64}; auto expected = @@ -142,7 +176,7 @@ TEST_F(UnsafeRowSerializerTest, double) { testDeserialize(data, 20, expected); } -TEST_F(UnsafeRowSerializerTest, boolean) { +TEST_P(UnsafeRowSerializerTest, boolean) { int8_t data[20] = {0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 0, 0, 0, 0}; auto expected = makeRowVector({makeFlatVector(std::vector{true})}); @@ -151,7 +185,7 @@ TEST_F(UnsafeRowSerializerTest, boolean) { testDeserialize(data, 20, expected); } -TEST_F(UnsafeRowSerializerTest, string) { +TEST_P(UnsafeRowSerializerTest, string) { int8_t data[28] = {0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, 5, 0, 0, 0, 16, 0, 0, 0, 72, 69, 76, 76, 79, 0, 0, 0}; auto expected = @@ -161,7 +195,7 @@ TEST_F(UnsafeRowSerializerTest, string) { testDeserialize(data, 28, expected); } -TEST_F(UnsafeRowSerializerTest, null) { +TEST_P(UnsafeRowSerializerTest, null) { int8_t data[20] = {0, 0, 0, 16, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}; auto expected = makeRowVector({makeNullableFlatVector( @@ -182,7 +216,7 @@ TEST_F(UnsafeRowSerializerTest, null) { // unsafeRow.getBaseObject().asInstanceOf[Array[Byte]].foreach(b => print(b + // ", ")) print("\n") // } -TEST_F(UnsafeRowSerializerTest, decimal) { +TEST_P(UnsafeRowSerializerTest, decimal) { // short decimal int8_t data[20] = {0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, 0, 62, 28, -36, -33, 2, 0, 0, 0}; @@ -206,7 +240,7 @@ TEST_F(UnsafeRowSerializerTest, decimal) { testDeserialize(longData, 36, longExpected); } -TEST_F(UnsafeRowSerializerTest, manyRows) { +TEST_P(UnsafeRowSerializerTest, manyRows) { int8_t data[140] = {0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, 4, 0, 0, 0, 16, 0, 0, 0, 109, 97, 110, 121, 0, 0, 0, 0, 0, 0, 0, 24, 0, 0, 0, 0, 0, 0, 0, 0, 4, 0, @@ -224,7 +258,7 @@ TEST_F(UnsafeRowSerializerTest, manyRows) { testDeserialize(data, 140, expected); } -TEST_F(UnsafeRowSerializerTest, splitRow) { +TEST_P(UnsafeRowSerializerTest, splitRow) { int8_t data[20] = {0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, 0, 62, 28, -36, -33, 2, 0, 0, 0}; auto expected = @@ -254,7 +288,7 @@ TEST_F(UnsafeRowSerializerTest, splitRow) { testDeserialize(buffers, expected); } -TEST_F(UnsafeRowSerializerTest, incompleteRow) { +TEST_P(UnsafeRowSerializerTest, incompleteRow) { int8_t data[20] = {0, 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, 0, 62, 28, -36, -33, 2, 0, 0, 0}; auto expected = @@ -288,7 +322,7 @@ TEST_F(UnsafeRowSerializerTest, incompleteRow) { "(1 vs. 1) Reading past end of BufferInputStream"); } -TEST_F(UnsafeRowSerializerTest, types) { +TEST_P(UnsafeRowSerializerTest, types) { auto rowType = ROW( {BOOLEAN(), TINYINT(), @@ -330,7 +364,7 @@ TEST_F(UnsafeRowSerializerTest, types) { testRoundTrip(data); } -TEST_F(UnsafeRowSerializerTest, date) { +TEST_P(UnsafeRowSerializerTest, date) { auto rowVector = makeRowVector({ makeFlatVector({0, 1}, DATE()), }); @@ -338,7 +372,7 @@ TEST_F(UnsafeRowSerializerTest, date) { testRoundTrip(rowVector); } -TEST_F(UnsafeRowSerializerTest, unknown) { +TEST_P(UnsafeRowSerializerTest, unknown) { // UNKNOWN type. auto rowVector = makeRowVector({ BaseVector::createNullConstant(UNKNOWN(), 10, pool()), @@ -374,7 +408,7 @@ TEST_F(UnsafeRowSerializerTest, unknown) { }); } -TEST_F(UnsafeRowSerializerTest, decimalVector) { +TEST_P(UnsafeRowSerializerTest, decimalVector) { auto rowVectorDecimal = makeRowVector({makeFlatVector( { 0, @@ -397,3 +431,8 @@ TEST_F(UnsafeRowSerializerTest, decimalVector) { testRoundTrip(rowVectorArray); } + +VELOX_INSTANTIATE_TEST_SUITE_P( + UnsafeRowSerializerTest, + UnsafeRowSerializerTest, + testing::Values(false, true)); diff --git a/velox/tool/trace/PartitionedOutputReplayer.cpp b/velox/tool/trace/PartitionedOutputReplayer.cpp index 56b4c0a145eb7..8a56fed4b4a8b 100644 --- a/velox/tool/trace/PartitionedOutputReplayer.cpp +++ b/velox/tool/trace/PartitionedOutputReplayer.cpp @@ -109,7 +109,8 @@ PartitionedOutputReplayer::PartitionedOutputReplayer( const std::string& queryId, const std::string& taskId, const std::string& nodeId, - const int32_t pipelineId, + int32_t pipelineId, + VectorSerde::Kind serdeKind, const std::string& operatorType, const ConsumerCallBack& consumerCb) : OperatorReplayerBase( @@ -125,6 +126,7 @@ PartitionedOutputReplayer::PartitionedOutputReplayer( [this](const core::PlanNode* node) { return node->id() == nodeId_; }))), + serdeKind_(serdeKind), consumerCb_(consumerCb) { VELOX_CHECK_NOT_NULL(originalNode_); consumerExecutor_ = std::make_unique( @@ -165,6 +167,7 @@ core::PlanNodePtr PartitionedOutputReplayer::createPlanNode( originalNode->isReplicateNullsAndAny(), originalNode->partitionFunctionSpecPtr(), originalNode->outputType(), + serdeKind_, source); } diff --git a/velox/tool/trace/PartitionedOutputReplayer.h b/velox/tool/trace/PartitionedOutputReplayer.h index a90c222dbf657..1036aeba35354 100644 --- a/velox/tool/trace/PartitionedOutputReplayer.h +++ b/velox/tool/trace/PartitionedOutputReplayer.h @@ -45,7 +45,8 @@ class PartitionedOutputReplayer final : public OperatorReplayerBase { const std::string& queryId, const std::string& taskId, const std::string& nodeId, - const int32_t pipelineId, + int32_t pipelineId, + VectorSerde::Kind serdeKind, const std::string& operatorType, const ConsumerCallBack& consumerCb = [](auto partition, auto page) {}); @@ -58,6 +59,7 @@ class PartitionedOutputReplayer final : public OperatorReplayerBase { const core::PlanNodePtr& source) const override; const core::PartitionedOutputNode* const originalNode_; + const VectorSerde::Kind serdeKind_; const std::shared_ptr bufferManager_{ exec::OutputBufferManager::getInstance().lock()}; const std::unique_ptr executor_{ diff --git a/velox/tool/trace/TraceReplayRunner.cpp b/velox/tool/trace/TraceReplayRunner.cpp index 357ce9c0425f1..1a314f903ab15 100644 --- a/velox/tool/trace/TraceReplayRunner.cpp +++ b/velox/tool/trace/TraceReplayRunner.cpp @@ -40,6 +40,8 @@ #include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" #include "velox/functions/prestosql/registration/RegistrationFunctions.h" #include "velox/parse/TypeResolver.h" +#include "velox/serializers/CompactRowSerializer.h" +#include "velox/serializers/UnsafeRowSerializer.h" #include "velox/tool/trace/AggregationReplayer.h" #include "velox/tool/trace/FilterProjectReplayer.h" #include "velox/tool/trace/HashJoinReplayer.h" @@ -74,12 +76,30 @@ DEFINE_string( "", "Specify output directory of TableWriter."); DEFINE_double( - hiveConnectorExecutorHwMultiplier, + hive_connector_executor_hw_multiplier, 2.0, "Hardware multipler for hive connector."); +DEFINE_int32( + shuffle_serialization_format, + 0, + "Specify the shuffle serialization format, 0: presto columnar, 1: compact row, 2: spark unsafe row."); namespace facebook::velox::tool::trace { namespace { +VectorSerde::Kind getVectorSerdeKind() { + switch (FLAGS_shuffle_serialization_format) { + case 0: + return VectorSerde::Kind::kPresto; + case 1: + return VectorSerde::Kind::kCompactRow; + case 2: + return VectorSerde::Kind::kUnsafeRow; + default: + VELOX_UNSUPPORTED( + "Unsupported shuffle serialization format: {}", + static_cast(FLAGS_shuffle_serialization_format)); + } +} std::unique_ptr createReplayer() { std::unique_ptr replayer; @@ -110,6 +130,7 @@ std::unique_ptr createReplayer() { FLAGS_task_id, FLAGS_node_id, FLAGS_pipeline_id, + getVectorSerdeKind(), FLAGS_operator_type); } else if (FLAGS_operator_type == "FilterProject") { replayer = std::make_unique( @@ -236,7 +257,7 @@ void printSummary( TraceReplayRunner::TraceReplayRunner() : ioExecutor_(std::make_unique( std::thread::hardware_concurrency() * - FLAGS_hiveConnectorExecutorHwMultiplier, + FLAGS_hive_connector_executor_hw_multiplier, std::make_shared( "TraceReplayIoConnector"))) {} @@ -262,6 +283,15 @@ void TraceReplayRunner::init() { if (!isRegisteredVectorSerde()) { serializer::presto::PrestoVectorSerde::registerVectorSerde(); } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kPresto)) { + serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kCompactRow)) { + serializer::CompactRowVectorSerde::registerNamedVectorSerde(); + } + if (!isRegisteredNamedVectorSerde(VectorSerde::Kind::kUnsafeRow)) { + serializer::spark::UnsafeRowVectorSerde::registerNamedVectorSerde(); + } connector::hive::HiveTableHandle::registerSerDe(); connector::hive::LocationHandle::registerSerDe(); connector::hive::HiveColumnHandle::registerSerDe(); @@ -301,6 +331,7 @@ void TraceReplayRunner::run() { return; } + VELOX_USER_CHECK(!FLAGS_task_id.empty(), "--task_id must be provided"); VELOX_USER_CHECK( !FLAGS_operator_type.empty(), "--operator_type must be provided"); createReplayer()->run(); diff --git a/velox/tool/trace/TraceReplayRunner.h b/velox/tool/trace/TraceReplayRunner.h index f1f1d7484f621..a0bf1f15e3ae2 100644 --- a/velox/tool/trace/TraceReplayRunner.h +++ b/velox/tool/trace/TraceReplayRunner.h @@ -30,6 +30,7 @@ DECLARE_int32(pipeline_id); DECLARE_string(operator_type); DECLARE_string(table_writer_output_dir); DECLARE_double(hiveConnectorExecutorHwMultiplier); +DECLARE_int32(shuffle_serialization_format); namespace facebook::velox::tool::trace { diff --git a/velox/tool/trace/tests/PartitionedOutputReplayerTest.cpp b/velox/tool/trace/tests/PartitionedOutputReplayerTest.cpp index d2d9c2c4e49f3..79083706ca202 100644 --- a/velox/tool/trace/tests/PartitionedOutputReplayerTest.cpp +++ b/velox/tool/trace/tests/PartitionedOutputReplayerTest.cpp @@ -21,12 +21,9 @@ #include "folly/dynamic.h" #include "velox/common/base/tests/GTestUtils.h" #include "velox/common/file/FileSystems.h" -#include "velox/common/hyperloglog/SparseHll.h" +#include "velox/exec/OperatorTraceReader.h" #include "velox/exec/PartitionFunction.h" -#include "velox/exec/PartitionedOutput.h" -#include "velox/exec/TableWriter.h" #include "velox/exec/TraceUtil.h" -#include "velox/exec/tests/utils/AssertQueryBuilder.h" #include "velox/exec/tests/utils/HiveConnectorTestBase.h" #include "velox/exec/tests/utils/PlanBuilder.h" #include "velox/exec/tests/utils/TempDirectoryPath.h" @@ -40,7 +37,18 @@ using namespace facebook::velox::exec; using namespace facebook::velox::exec::test; namespace facebook::velox::tool::trace::test { -class PartitionedOutputReplayerTest : public HiveConnectorTestBase { +class PartitionedOutputReplayerTest + : public HiveConnectorTestBase, + public testing::WithParamInterface { + public: + static std::vector getTestParams() { + const std::vector kinds( + {VectorSerde::Kind::kPresto, + VectorSerde::Kind::kCompactRow, + VectorSerde::Kind::kUnsafeRow}); + return kinds; + } + protected: static void SetUpTestCase() { HiveConnectorTestBase::SetUpTestCase(); @@ -85,12 +93,13 @@ class PartitionedOutputReplayerTest : public HiveConnectorTestBase { uint32_t numPartitions, std::string& capturedPlanNodeId) { VELOX_CHECK(capturedPlanNodeId.empty()); - auto plan = PlanBuilder() - .values(inputs, false) - .partitionedOutput( - partitionKeys, numPartitions, false, outputLayout) - .capturePlanNodeId(capturedPlanNodeId) - .planNode(); + auto plan = + PlanBuilder() + .values(inputs, false) + .partitionedOutput( + partitionKeys, numPartitions, false, outputLayout, GetParam()) + .capturePlanNodeId(capturedPlanNodeId) + .planNode(); auto task = Task::create( taskId, core::PlanFragment{plan}, @@ -114,7 +123,7 @@ class PartitionedOutputReplayerTest : public HiveConnectorTestBase { exec::OutputBufferManager::getInstance().lock()}; }; -TEST_F(PartitionedOutputReplayerTest, defaultConsumer) { +TEST_P(PartitionedOutputReplayerTest, defaultConsumer) { const uint32_t numPartitions = 10; std::string planNodeId; auto input = makeRowVector( @@ -150,11 +159,12 @@ TEST_F(PartitionedOutputReplayerTest, defaultConsumer) { originalTask->taskId(), planNodeId, 0, + GetParam(), "PartitionedOutput") .run()); } -TEST_F(PartitionedOutputReplayerTest, basic) { +TEST_P(PartitionedOutputReplayerTest, basic) { struct TestParam { std::string testName; uint32_t numPartitions; @@ -222,6 +232,20 @@ TEST_F(PartitionedOutputReplayerTest, basic) { originalPartitionedResults[partition].push_back(std::move(page)); }); + // Verified that the trace summary has been written properly. + const auto taskTraceDir = + exec::trace::getTaskTraceDirectory(traceRoot, *originalTask); + const auto opTraceDir = exec::trace::getOpTraceDirectory( + taskTraceDir, + planNodeId, + /*pipelineId=*/0, + /*driverId=*/0); + const auto summary = + exec::trace::OperatorTraceSummaryReader(opTraceDir, pool()).read(); + ASSERT_GT(summary.inputRows, 0); + ASSERT_GT(summary.peakMemory, 0); + ASSERT_EQ(summary.opType, "PartitionedOutput"); + std::vector>> replayedPartitionedResults; replayedPartitionedResults.reserve(testParam.numPartitions); @@ -232,6 +256,7 @@ TEST_F(PartitionedOutputReplayerTest, basic) { originalTask->taskId(), planNodeId, 0, + GetParam(), "PartitionedOutput", [&](auto partition, auto page) { replayedPartitionedResults[partition].push_back(std::move(page)); @@ -252,4 +277,9 @@ TEST_F(PartitionedOutputReplayerTest, basic) { } } } + +VELOX_INSTANTIATE_TEST_SUITE_P( + PartitionedOutputReplayerTest, + PartitionedOutputReplayerTest, + testing::ValuesIn(PartitionedOutputReplayerTest::getTestParams())); } // namespace facebook::velox::tool::trace::test diff --git a/velox/vector/DecodedVector.cpp b/velox/vector/DecodedVector.cpp index cd740c02d7a18..fbea39d871b2d 100644 --- a/velox/vector/DecodedVector.cpp +++ b/velox/vector/DecodedVector.cpp @@ -51,14 +51,14 @@ void DecodedVector::decode( reset(end(vector.size(), rows)); partialRowsDecoded_ = rows != nullptr; loadLazy_ = loadLazy; - bool isTopLevelLazyAndLoaded = + const bool isTopLevelLazyAndLoaded = vector.isLazy() && vector.asUnchecked()->isLoaded(); if (isTopLevelLazyAndLoaded || (loadLazy_ && isLazyNotLoaded(vector))) { decode(*vector.loadedVector(), rows, loadLazy); return; } - auto encoding = vector.encoding(); + const auto encoding = vector.encoding(); switch (encoding) { case VectorEncoding::Simple::FLAT: case VectorEncoding::Simple::BIASED: diff --git a/velox/vector/VectorStream.cpp b/velox/vector/VectorStream.cpp index 18bdb103f357f..f6de9e6a27b55 100644 --- a/velox/vector/VectorStream.cpp +++ b/velox/vector/VectorStream.cpp @@ -14,8 +14,8 @@ * limitations under the License. */ #include "velox/vector/VectorStream.h" + #include -#include "velox/common/base/RawVector.h" namespace facebook::velox { namespace { @@ -66,11 +66,11 @@ std::unique_ptr& getVectorSerdeImpl() { return serde; } -std::unordered_map>& +std::unordered_map>& getNamedVectorSerdeImpl() { - static std::unordered_map> - namedSerde; - return namedSerde; + static std::unordered_map> + namedSerdes; + return namedSerdes; } } // namespace @@ -94,6 +94,35 @@ std::unique_ptr VectorSerde::createBatchSerializer( return std::make_unique(pool, this, options); } +std::string VectorSerde::kindName(Kind kind) { + switch (kind) { + case Kind::kPresto: + return "Presto"; + case Kind::kCompactRow: + return "CompactRow"; + case Kind::kUnsafeRow: + return "UnsafeRow"; + } + VELOX_UNREACHABLE( + fmt::format("Unknown vector serde kind: {}", static_cast(kind))); +} + +VectorSerde::Kind VectorSerde::kindByName(const std::string& kindName) { + static const std::unordered_map kNameToKind = { + {"Presto", Kind::kPresto}, + {"CompactRow", Kind::kCompactRow}, + {"UnsafeRow", Kind::kUnsafeRow}}; + const auto it = kNameToKind.find(kindName); + VELOX_CHECK( + it != kNameToKind.end(), "Unknown vector serde kind: {}", kindName); + return it->second; +} + +std::ostream& operator<<(std::ostream& out, VectorSerde::Kind kind) { + out << VectorSerde::kindName(kind); + return out; +} + VectorSerde* getVectorSerde() { auto serde = getVectorSerdeImpl().get(); VELOX_CHECK_NOT_NULL(serde, "Vector serde is not registered."); @@ -118,33 +147,33 @@ bool isRegisteredVectorSerde() { /// Named serde helper functions. void registerNamedVectorSerde( - std::string_view serdeName, + VectorSerde::Kind kind, std::unique_ptr serdeToRegister) { auto& namedSerdeMap = getNamedVectorSerdeImpl(); VELOX_CHECK( - namedSerdeMap.find(std::string(serdeName)) == namedSerdeMap.end(), + namedSerdeMap.find(kind) == namedSerdeMap.end(), "Vector serde '{}' is already registered.", - serdeName); - namedSerdeMap[std::string(serdeName)] = std::move(serdeToRegister); + kind); + namedSerdeMap[kind] = std::move(serdeToRegister); } -void deregisterNamedVectorSerde(std::string_view serdeName) { +void deregisterNamedVectorSerde(VectorSerde::Kind kind) { auto& namedSerdeMap = getNamedVectorSerdeImpl(); - namedSerdeMap.erase(std::string(serdeName)); + namedSerdeMap.erase(kind); } -bool isRegisteredNamedVectorSerde(std::string_view serdeName) { +bool isRegisteredNamedVectorSerde(VectorSerde::Kind kind) { auto& namedSerdeMap = getNamedVectorSerdeImpl(); - return namedSerdeMap.find(std::string(serdeName)) != namedSerdeMap.end(); + return namedSerdeMap.find(kind) != namedSerdeMap.end(); } -VectorSerde* getNamedVectorSerde(std::string_view serdeName) { +VectorSerde* getNamedVectorSerde(VectorSerde::Kind kind) { auto& namedSerdeMap = getNamedVectorSerdeImpl(); - auto it = namedSerdeMap.find(std::string(serdeName)); + auto it = namedSerdeMap.find(kind); VELOX_CHECK( it != namedSerdeMap.end(), "Named vector serde '{}' is not registered.", - serdeName); + kind); return it->second.get(); } @@ -173,6 +202,20 @@ void VectorStreamGroup::append(const RowVectorPtr& vector) { serializer_->append(vector); } +void VectorStreamGroup::append( + const row::CompactRow& compactRow, + const folly::Range& rows, + const std::vector& sizes) { + serializer_->append(compactRow, rows, sizes); +} + +void VectorStreamGroup::append( + const row::UnsafeRowFast& unsafeRow, + const folly::Range& rows, + const std::vector& sizes) { + serializer_->append(unsafeRow, rows, sizes); +} + void VectorStreamGroup::flush(OutputStream* out) { serializer_->flush(out); } @@ -181,18 +224,54 @@ void VectorStreamGroup::flush(OutputStream* out) { void VectorStreamGroup::estimateSerializedSize( const BaseVector* vector, const folly::Range& ranges, + VectorSerde* serde, vector_size_t** sizes, Scratch& scratch) { - getVectorSerde()->estimateSerializedSize(vector, ranges, sizes, scratch); + if (serde == nullptr) { + getVectorSerde()->estimateSerializedSize(vector, ranges, sizes, scratch); + } else { + serde->estimateSerializedSize(vector, ranges, sizes, scratch); + } } // static void VectorStreamGroup::estimateSerializedSize( const BaseVector* vector, - folly::Range rows, + const folly::Range& rows, + VectorSerde* serde, vector_size_t** sizes, Scratch& scratch) { - getVectorSerde()->estimateSerializedSize(vector, rows, sizes, scratch); + if (serde == nullptr) { + getVectorSerde()->estimateSerializedSize(vector, rows, sizes, scratch); + } else { + serde->estimateSerializedSize(vector, rows, sizes, scratch); + } +} + +// static +void VectorStreamGroup::estimateSerializedSize( + const row::CompactRow* compactRow, + const folly::Range& rows, + VectorSerde* serde, + vector_size_t** sizes) { + if (serde == nullptr) { + getVectorSerde()->estimateSerializedSize(compactRow, rows, sizes); + } else { + serde->estimateSerializedSize(compactRow, rows, sizes); + } +} + +// static +void VectorStreamGroup::estimateSerializedSize( + const row::UnsafeRowFast* unsafeRow, + const folly::Range& rows, + VectorSerde* serde, + vector_size_t** sizes) { + if (serde == nullptr) { + getVectorSerde()->estimateSerializedSize(unsafeRow, rows, sizes); + } else { + serde->estimateSerializedSize(unsafeRow, rows, sizes); + } } // static @@ -200,9 +279,14 @@ void VectorStreamGroup::read( ByteInputStream* source, velox::memory::MemoryPool* pool, RowTypePtr type, + VectorSerde* serde, RowVectorPtr* result, const VectorSerde::Options* options) { - getVectorSerde()->deserialize(source, pool, type, result, options); + if (serde == nullptr) { + getVectorSerde()->deserialize(source, pool, type, result, options); + } else { + serde->deserialize(source, pool, type, result, options); + } } folly::IOBuf rowVectorToIOBuf( diff --git a/velox/vector/VectorStream.h b/velox/vector/VectorStream.h index b8304bdc2e280..ca1a4ed56105a 100644 --- a/velox/vector/VectorStream.h +++ b/velox/vector/VectorStream.h @@ -16,13 +16,11 @@ #pragma once #include -#include "velox/buffer/Buffer.h" + #include "velox/common/base/RuntimeMetrics.h" #include "velox/common/base/Scratch.h" #include "velox/common/compression/Compression.h" #include "velox/common/memory/ByteStream.h" -#include "velox/common/memory/Memory.h" -#include "velox/common/memory/MemoryAllocator.h" #include "velox/common/memory/StreamArena.h" #include "velox/vector/ComplexVector.h" @@ -35,6 +33,11 @@ struct IndexRange { vector_size_t size; }; +namespace row { +class CompactRow; +class UnsafeRowFast; +}; // namespace row + /// Serializer that can iteratively build up a buffer of serialized rows from /// one or more RowVectors. /// @@ -62,12 +65,26 @@ class IterativeVectorSerializer { const RowVectorPtr& vector, const folly::Range& rows, Scratch& scratch) { - VELOX_UNSUPPORTED(); + VELOX_UNSUPPORTED("{}", __FUNCTION__); } /// Serialize all rows in a vector. void append(const RowVectorPtr& vector); + virtual void append( + const row::CompactRow& compactRow, + const folly::Range& rows, + const std::vector& sizes) { + VELOX_UNSUPPORTED("{}", __FUNCTION__); + } + + virtual void append( + const row::UnsafeRowFast& unsafeRow, + const folly::Range& rows, + const std::vector& sizes) { + VELOX_UNSUPPORTED("{}", __FUNCTION__); + } + // True if supports append with folly::Range. virtual bool supportsAppendRows() const { return false; @@ -91,7 +108,7 @@ class IterativeVectorSerializer { /// Resets 'this' to post construction state. virtual void clear() { - VELOX_UNSUPPORTED("clear"); + VELOX_UNSUPPORTED("{}", __FUNCTION__); } /// Returns serializer-dependent counters, e.g. about compression, data @@ -137,6 +154,16 @@ class BatchVectorSerializer { class VectorSerde { public: + enum class Kind { + kPresto, + kCompactRow, + kUnsafeRow, + }; + + static std::string kindName(Kind type); + + static Kind kindByName(const std::string& name); + virtual ~VectorSerde() = default; // Lets the caller pass options to the Serde. This can be extended to add @@ -153,23 +180,26 @@ class VectorSerde { common::CompressionKind::CompressionKind_NONE}; }; - /// Adds the serialized size of vector at 'rows[i]' to '*sizes[i]'. + Kind kind() const { + return kind_; + } + virtual void estimateSerializedSize( const BaseVector* /*vector*/, - folly::Range rows, - vector_size_t** sizes, - Scratch& scratch) { - VELOX_UNSUPPORTED(); + const folly::Range& /*rows*/, + vector_size_t** /*sizes*/, + Scratch& /*scratch*/) { + VELOX_UNSUPPORTED("{}", __FUNCTION__); } /// Adds the serialized sizes of the rows of 'vector' in 'ranges[i]' to /// '*sizes[i]'. virtual void estimateSerializedSize( const BaseVector* /*vector*/, - const folly::Range& ranges, - vector_size_t** sizes, - Scratch& scratch) { - VELOX_UNSUPPORTED(); + const folly::Range& /*ranges*/, + vector_size_t** /*sizes*/, + Scratch& /*scratch*/) { + VELOX_UNSUPPORTED("{}", __FUNCTION__); } virtual void estimateSerializedSize( @@ -180,6 +210,20 @@ class VectorSerde { estimateSerializedSize(vector, ranges, sizes, scratch); } + virtual void estimateSerializedSize( + const row::CompactRow* /*compactRow*/, + const folly::Range& /*rows*/, + vector_size_t** /*sizes*/) { + VELOX_UNSUPPORTED("{}", __FUNCTION__); + } + + virtual void estimateSerializedSize( + const row::UnsafeRowFast* /*unsafeRow*/, + const folly::Range& /*rows*/, + vector_size_t** /*sizes*/) { + VELOX_UNSUPPORTED("{}", __FUNCTION__); + } + /// Creates a Vector Serializer that iteratively builds up a buffer of /// serialized rows from one or more RowVectors via append, and then writes to /// an OutputSteam via flush. @@ -233,8 +277,15 @@ class VectorSerde { } VELOX_UNSUPPORTED(); } + + protected: + explicit VectorSerde(Kind kind) : kind_(kind) {} + + const Kind kind_; }; +std::ostream& operator<<(std::ostream& out, VectorSerde::Kind kind); + /// Register/deregister the "default" vector serde. void registerVectorSerde(std::unique_ptr serdeToRegister); void deregisterVectorSerde(); @@ -248,23 +299,21 @@ VectorSerde* getVectorSerde(); /// Register/deregister a named vector serde. `serdeName` is a handle that /// allows users to register multiple serde formats. void registerNamedVectorSerde( - std::string_view serdeName, + VectorSerde::Kind kind, std::unique_ptr serdeToRegister); -void deregisterNamedVectorSerde(std::string_view serdeName); +void deregisterNamedVectorSerde(VectorSerde::Kind kind); /// Check if a named vector serde has been registered with `serdeName` as a /// handle. -bool isRegisteredNamedVectorSerde(std::string_view serdeName); +bool isRegisteredNamedVectorSerde(VectorSerde::Kind kind); /// Get the vector serde identified by `serdeName`. Throws if not found. -VectorSerde* getNamedVectorSerde(std::string_view serdeName); +VectorSerde* getNamedVectorSerde(VectorSerde::Kind kind); class VectorStreamGroup : public StreamArena { public: /// If `serde` is not specified, fallback to the default registered. - explicit VectorStreamGroup( - memory::MemoryPool* pool, - VectorSerde* serde = nullptr) + VectorStreamGroup(memory::MemoryPool* pool, VectorSerde* serde) : StreamArena(pool), serde_(serde != nullptr ? serde : getVectorSerde()) {} @@ -276,24 +325,39 @@ class VectorStreamGroup : public StreamArena { /// Increments sizes[i] for each ith row in 'rows' in 'vector'. static void estimateSerializedSize( const BaseVector* vector, - folly::Range rows, + const folly::Range& rows, + VectorSerde* serde, vector_size_t** sizes, Scratch& scratch); static void estimateSerializedSize( const BaseVector* vector, const folly::Range& ranges, + VectorSerde* serde, vector_size_t** sizes, Scratch& scratch); static inline void estimateSerializedSize( const BaseVector* vector, const folly::Range& ranges, + VectorSerde* serde, vector_size_t** sizes) { Scratch scratch; - estimateSerializedSize(vector, ranges, sizes, scratch); + estimateSerializedSize(vector, ranges, serde, sizes, scratch); } + static void estimateSerializedSize( + const row::CompactRow* compactRow, + const folly::Range& rows, + VectorSerde* serde, + vector_size_t** sizes); + + static void estimateSerializedSize( + const row::UnsafeRowFast* unsafeRow, + const folly::Range& rows, + VectorSerde* serde, + vector_size_t** sizes); + void append( const RowVectorPtr& vector, const folly::Range& ranges, @@ -313,14 +377,35 @@ class VectorStreamGroup : public StreamArena { void append(const RowVectorPtr& vector); + void append( + const row::CompactRow& compactRow, + const folly::Range& rows, + const std::vector& sizes); + + void append( + const row::UnsafeRowFast& unsafeRow, + const folly::Range& rows, + const std::vector& sizes); + // Writes the contents to 'stream' in wire format. void flush(OutputStream* stream); // Reads data in wire format. Returns the RowVector in 'result'. +#ifdef VELOX_ENABLE_BACKWARD_COMPATIBILITY + static void read( + ByteInputStream* source, + velox::memory::MemoryPool* pool, + RowTypePtr type, + RowVectorPtr* result, + const VectorSerde::Options* options = nullptr) { + read(source, pool, type, nullptr, result, options); + } +#endif static void read( ByteInputStream* source, velox::memory::MemoryPool* pool, RowTypePtr type, + VectorSerde* serde, RowVectorPtr* result, const VectorSerde::Options* options = nullptr); @@ -366,3 +451,12 @@ RowVectorPtr IOBufToRowVector( VectorSerde* serde = nullptr); } // namespace facebook::velox + +template <> +struct fmt::formatter + : formatter { + auto format(facebook::velox::VectorSerde::Kind s, format_context& ctx) const { + return formatter::format( + facebook::velox::VectorSerde::kindName(s), ctx); + } +}; diff --git a/velox/vector/tests/VectorStreamTest.cpp b/velox/vector/tests/VectorStreamTest.cpp index 806bb28eda1ab..d80fcf8d1e083 100644 --- a/velox/vector/tests/VectorStreamTest.cpp +++ b/velox/vector/tests/VectorStreamTest.cpp @@ -15,11 +15,16 @@ */ #include -#include + +#include "velox/common/base/tests/GTestUtils.h" +#include "velox/vector/VectorStream.h" namespace facebook::velox::test { class MockVectorSerde : public VectorSerde { + public: + MockVectorSerde() : VectorSerde(VectorSerde::Kind::kPresto) {} + void estimateSerializedSize( const BaseVector* /*vector*/, const folly::Range& ranges, @@ -66,35 +71,40 @@ TEST(VectorStreamTest, serdeRegistration) { } TEST(VectorStreamTest, namedSerdeRegistration) { - std::string_view mySerde = "my_serde"; + const VectorSerde::Kind kind = VectorSerde::Kind::kPresto; // Nothing registered yet. - deregisterNamedVectorSerde(mySerde); - EXPECT_FALSE(isRegisteredNamedVectorSerde(mySerde)); - EXPECT_THROW(getNamedVectorSerde(mySerde), VeloxRuntimeError); + deregisterNamedVectorSerde(kind); + EXPECT_FALSE(isRegisteredNamedVectorSerde(kind)); + VELOX_ASSERT_THROW( + getNamedVectorSerde(kind), + "Named vector serde 'Presto' is not registered."); // Register a mock serde. - registerNamedVectorSerde(mySerde, std::make_unique()); + registerNamedVectorSerde(kind, std::make_unique()); - auto serde = getNamedVectorSerde(mySerde); + auto serde = getNamedVectorSerde(kind); EXPECT_NE(serde, nullptr); EXPECT_NE(dynamic_cast(serde), nullptr); + const VectorSerde::Kind otherKind = VectorSerde::Kind::kUnsafeRow; + EXPECT_FALSE(isRegisteredNamedVectorSerde(otherKind)); + VELOX_ASSERT_THROW( + getNamedVectorSerde(otherKind), + "Named vector serde 'UnsafeRow' is not registered."); + // Can't double register. - EXPECT_THROW( - registerNamedVectorSerde(mySerde, std::make_unique()), - VeloxRuntimeError); + VELOX_ASSERT_THROW( + registerNamedVectorSerde(kind, std::make_unique()), + "Vector serde 'Presto' is already registered."); // Register another one. - std::string_view myOtherSerde = "my_other_serde"; - - EXPECT_FALSE(isRegisteredNamedVectorSerde(myOtherSerde)); - EXPECT_THROW(getNamedVectorSerde(myOtherSerde), VeloxRuntimeError); - registerNamedVectorSerde(myOtherSerde, std::make_unique()); - EXPECT_TRUE(isRegisteredNamedVectorSerde(myOtherSerde)); + EXPECT_FALSE(isRegisteredNamedVectorSerde(otherKind)); + EXPECT_THROW(getNamedVectorSerde(otherKind), VeloxRuntimeError); + registerNamedVectorSerde(otherKind, std::make_unique()); + EXPECT_TRUE(isRegisteredNamedVectorSerde(otherKind)); - deregisterNamedVectorSerde(myOtherSerde); - EXPECT_FALSE(isRegisteredNamedVectorSerde(myOtherSerde)); + deregisterNamedVectorSerde(otherKind); + EXPECT_FALSE(isRegisteredNamedVectorSerde(otherKind)); } - } // namespace facebook::velox::test diff --git a/velox/vector/tests/VectorTest.cpp b/velox/vector/tests/VectorTest.cpp index b61f77f532775..44d35c1ca8b31 100644 --- a/velox/vector/tests/VectorTest.cpp +++ b/velox/vector/tests/VectorTest.cpp @@ -818,10 +818,10 @@ class VectorTest : public testing::Test, public test::VectorTestBase { auto sourceRow = makeRowVector({"c"}, {source}); auto sourceRowType = asRowType(sourceRow->type()); - VectorStreamGroup even(pool()); + VectorStreamGroup even(pool(), nullptr); even.createStreamTree(sourceRowType, source->size() / 4); - VectorStreamGroup odd(pool()); + VectorStreamGroup odd(pool(), nullptr); odd.createStreamTree(sourceRowType, source->size() / 3); std::vector evenIndices; @@ -847,9 +847,9 @@ class VectorTest : public testing::Test, public test::VectorTestBase { } VectorStreamGroup::estimateSerializedSize( - source.get(), evenIndices, evenSizePointers.data()); + source.get(), evenIndices, nullptr, evenSizePointers.data()); VectorStreamGroup::estimateSerializedSize( - source.get(), oddIndices, oddSizePointers.data()); + source.get(), oddIndices, nullptr, oddSizePointers.data()); even.append( sourceRow, folly::Range(evenIndices.data(), evenIndices.size() / 2)); even.append( @@ -878,7 +878,8 @@ class VectorTest : public testing::Test, public test::VectorTestBase { auto evenInput = prepareInput(evenString); RowVectorPtr resultRow; - VectorStreamGroup::read(evenInput.get(), pool(), sourceRowType, &resultRow); + VectorStreamGroup::read( + evenInput.get(), pool(), sourceRowType, nullptr, &resultRow); VectorPtr result = resultRow->childAt(0); switch (source->encoding()) { case VectorEncoding::Simple::FLAT: @@ -907,7 +908,8 @@ class VectorTest : public testing::Test, public test::VectorTestBase { auto oddString = oddStream.str(); auto oddInput = prepareInput(oddString); - VectorStreamGroup::read(oddInput.get(), pool(), sourceRowType, &resultRow); + VectorStreamGroup::read( + oddInput.get(), pool(), sourceRowType, nullptr, &resultRow); result = resultRow->childAt(0); for (int32_t i = 0; i < oddIndices.size(); ++i) { EXPECT_TRUE(result->equalValueAt(source.get(), i, oddIndices[i].begin))