Skip to content

Commit

Permalink
[VL] Row based sort shuffle implementation (#6475)
Browse files Browse the repository at this point in the history
  • Loading branch information
marin-ma authored Jul 24, 2024
1 parent 33d2f2d commit f75d7c1
Show file tree
Hide file tree
Showing 55 changed files with 1,343 additions and 648 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -258,27 +258,33 @@ class VeloxMetricsApi extends MetricsApi with Logging {
sparkContext: SparkContext,
isSort: Boolean): Map[String, SQLMetric] = {
val baseMetrics = Map(
"dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"),
"numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions"),
"dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"),
"bytesSpilled" -> SQLMetrics.createSizeMetric(sparkContext, "shuffle bytes spilled"),
"splitBufferSize" -> SQLMetrics.createSizeMetric(sparkContext, "split buffer size"),
"splitTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to split"),
"spillTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to spill"),
"deserializeTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to deserialize"),
"avgReadBatchNumRows" -> SQLMetrics
.createAverageMetric(sparkContext, "avg read batch num rows"),
"numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"),
"numOutputRows" -> SQLMetrics
.createMetric(sparkContext, "number of output rows"),
"inputBatches" -> SQLMetrics
.createMetric(sparkContext, "number of input batches")
.createMetric(sparkContext, "number of input batches"),
"spillTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to spill"),
"compressTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to compress"),
"decompressTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to decompress"),
"deserializeTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to deserialize"),
// For hash shuffle writer, the peak bytes represents the maximum split buffer size.
// For sort shuffle writer, the peak bytes represents the maximum
// row buffer + sort buffer size.
"peakBytes" -> SQLMetrics.createSizeMetric(sparkContext, "peak bytes allocated")
)
if (isSort) {
baseMetrics
baseMetrics ++ Map(
"sortTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to shuffle sort"),
"c2rTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to shuffle c2r")
)
} else {
baseMetrics ++ Map(
"compressTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to compress"),
"decompressTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to decompress")
"splitTime" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time to split")
)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -349,24 +349,30 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
}

def maybeAddAppendBatchesExec(plan: SparkPlan): SparkPlan = {
if (GlutenConfig.getConf.veloxCoalesceBatchesBeforeShuffle) {
VeloxAppendBatchesExec(plan, GlutenConfig.getConf.veloxMinBatchSizeForShuffle)
} else {
plan
plan match {
case shuffle: ColumnarShuffleExchangeExec
if !shuffle.useSortBasedShuffle &&
GlutenConfig.getConf.veloxCoalesceBatchesBeforeShuffle =>
val appendBatches =
VeloxAppendBatchesExec(shuffle.child, GlutenConfig.getConf.veloxMinBatchSizeForShuffle)
shuffle.withNewChildren(Seq(appendBatches))
case _ => plan
}
}

val child = shuffle.child

shuffle.outputPartitioning match {
val newShuffle = shuffle.outputPartitioning match {
case HashPartitioning(exprs, _) =>
val hashExpr = new Murmur3Hash(exprs)
val projectList = Seq(Alias(hashExpr, "hash_partition_key")()) ++ child.output
val projectTransformer = ProjectExecTransformer(projectList, child)
val validationResult = projectTransformer.doValidate()
if (validationResult.ok()) {
val newChild = maybeAddAppendBatchesExec(projectTransformer)
ColumnarShuffleExchangeExec(shuffle, newChild, newChild.output.drop(1))
ColumnarShuffleExchangeExec(
shuffle,
projectTransformer,
projectTransformer.output.drop(1))
} else {
FallbackTags.add(shuffle, validationResult)
shuffle.withNewChildren(child :: Nil)
Expand All @@ -382,8 +388,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
// null type since the value always be null.
val columnsForHash = child.output.filterNot(_.dataType == NullType)
if (columnsForHash.isEmpty) {
val newChild = maybeAddAppendBatchesExec(child)
ColumnarShuffleExchangeExec(shuffle, newChild, newChild.output)
ColumnarShuffleExchangeExec(shuffle, child, child.output)
} else {
val hashExpr = new Murmur3Hash(columnsForHash)
val projectList = Seq(Alias(hashExpr, "hash_partition_key")()) ++ child.output
Expand All @@ -404,18 +409,20 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
ProjectExecTransformer(projectList.drop(1), sortByHashCode)
val validationResult = dropSortColumnTransformer.doValidate()
if (validationResult.ok()) {
val newChild = maybeAddAppendBatchesExec(dropSortColumnTransformer)
ColumnarShuffleExchangeExec(shuffle, newChild, newChild.output)
ColumnarShuffleExchangeExec(
shuffle,
dropSortColumnTransformer,
dropSortColumnTransformer.output)
} else {
FallbackTags.add(shuffle, validationResult)
shuffle.withNewChildren(child :: Nil)
}
}
}
case _ =>
val newChild = maybeAddAppendBatchesExec(child)
ColumnarShuffleExchangeExec(shuffle, newChild, null)
ColumnarShuffleExchangeExec(shuffle, child, null)
}
maybeAddAppendBatchesExec(newShuffle)
}

/** Generate ShuffledHashJoinExecTransformer. */
Expand Down Expand Up @@ -572,11 +579,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
val numOutputRows = metrics("numOutputRows")
val deserializeTime = metrics("deserializeTime")
val readBatchNumRows = metrics("avgReadBatchNumRows")
val decompressTime: Option[SQLMetric] = if (!isSort) {
Some(metrics("decompressTime"))
} else {
None
}
val decompressTime = metrics("decompressTime")
if (GlutenConfig.getConf.isUseCelebornShuffleManager) {
val clazz = ClassUtils.getClass("org.apache.spark.shuffle.CelebornColumnarBatchSerializer")
val constructor =
Expand Down
1 change: 1 addition & 0 deletions cpp/core/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -200,6 +200,7 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS
shuffle/RoundRobinPartitioner.cc
shuffle/ShuffleMemoryPool.cc
shuffle/ShuffleReader.cc
shuffle/ShuffleWriter.cc
shuffle/SinglePartitioner.cc
shuffle/Spill.cc
shuffle/Utils.cc
Expand Down
19 changes: 6 additions & 13 deletions cpp/core/jni/JniWrapper.cc
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) {
jniByteInputStreamClose = getMethodIdOrError(env, jniByteInputStreamClass, "close", "()V");

splitResultClass = createGlobalClassReferenceOrError(env, "Lorg/apache/gluten/vectorized/GlutenSplitResult;");
splitResultConstructor = getMethodIdOrError(env, splitResultClass, "<init>", "(JJJJJJJ[J[J)V");
splitResultConstructor = getMethodIdOrError(env, splitResultClass, "<init>", "(JJJJJJJJJ[J[J)V");

columnarBatchSerializeResultClass =
createGlobalClassReferenceOrError(env, "Lorg/apache/gluten/vectorized/ColumnarBatchSerializeResult;");
Expand Down Expand Up @@ -780,14 +780,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe
.partitioning = gluten::toPartitioning(jStringToCString(env, partitioningNameJstr)),
.taskAttemptId = (int64_t)taskAttemptId,
.startPartitionId = startPartitionId,
};
auto shuffleWriterTypeC = env->GetStringUTFChars(shuffleWriterTypeJstr, JNI_FALSE);
auto shuffleWriterType = std::string(shuffleWriterTypeC);
env->ReleaseStringUTFChars(shuffleWriterTypeJstr, shuffleWriterTypeC);

if (shuffleWriterType == "sort") {
shuffleWriterOptions.shuffleWriterType = kSortShuffle;
}
.shuffleWriterType = gluten::ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterTypeJstr))};

// Build PartitionWriterOptions.
auto partitionWriterOptions = PartitionWriterOptions{
Expand Down Expand Up @@ -945,9 +938,11 @@ JNIEXPORT jobject JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrap
shuffleWriter->totalWriteTime(),
shuffleWriter->totalEvictTime(),
shuffleWriter->totalCompressTime(),
shuffleWriter->totalSortTime(),
shuffleWriter->totalC2RTime(),
shuffleWriter->totalBytesWritten(),
shuffleWriter->totalBytesEvicted(),
shuffleWriter->maxPartitionBufferSize(),
shuffleWriter->peakBytesAllocated(),
partitionLengthArr,
rawPartitionLengthArr);

Expand Down Expand Up @@ -996,9 +991,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrappe
options.batchSize = batchSize;
// TODO: Add coalesce option and maximum coalesced size.

if (jStringToCString(env, shuffleWriterType) == "sort") {
options.shuffleWriterType = kSortShuffle;
}
options.shuffleWriterType = gluten::ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterType));
std::shared_ptr<arrow::Schema> schema =
gluten::arrowGetOrThrow(arrow::ImportSchema(reinterpret_cast<struct ArrowSchema*>(cSchema)));

Expand Down
2 changes: 1 addition & 1 deletion cpp/core/operators/serializer/ColumnarBatchSerializer.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ namespace gluten {

class ColumnarBatchSerializer {
public:
ColumnarBatchSerializer(arrow::MemoryPool* arrowPool, struct ArrowSchema* cSchema) : arrowPool_(arrowPool) {}
ColumnarBatchSerializer(arrow::MemoryPool* arrowPool) : arrowPool_(arrowPool) {}

virtual ~ColumnarBatchSerializer() = default;

Expand Down
5 changes: 1 addition & 4 deletions cpp/core/shuffle/FallbackRangePartitioner.cc
Original file line number Diff line number Diff line change
Expand Up @@ -23,18 +23,15 @@ namespace gluten {
arrow::Status gluten::FallbackRangePartitioner::compute(
const int32_t* pidArr,
const int64_t numRows,
std::vector<uint32_t>& row2Partition,
std::vector<uint32_t>& partition2RowCount) {
std::vector<uint32_t>& row2Partition) {
row2Partition.resize(numRows);
std::fill(std::begin(partition2RowCount), std::end(partition2RowCount), 0);
for (auto i = 0; i < numRows; ++i) {
auto pid = pidArr[i];
if (pid >= numPartitions_) {
return arrow::Status::Invalid(
"Partition id ", std::to_string(pid), " is equal or greater than ", std::to_string(numPartitions_));
}
row2Partition[i] = pid;
partition2RowCount[pid]++;
}
return arrow::Status::OK();
}
Expand Down
6 changes: 1 addition & 5 deletions cpp/core/shuffle/FallbackRangePartitioner.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,7 @@ class FallbackRangePartitioner final : public Partitioner {
public:
FallbackRangePartitioner(int32_t numPartitions) : Partitioner(numPartitions, true) {}

arrow::Status compute(
const int32_t* pidArr,
const int64_t numRows,
std::vector<uint32_t>& row2partition,
std::vector<uint32_t>& partition2RowCount) override;
arrow::Status compute(const int32_t* pidArr, const int64_t numRows, std::vector<uint32_t>& row2partition) override;

arrow::Status compute(
const int32_t* pidArr,
Expand Down
14 changes: 2 additions & 12 deletions cpp/core/shuffle/HashPartitioner.cc
Original file line number Diff line number Diff line change
Expand Up @@ -37,23 +37,13 @@ int32_t computePid(const int32_t* pidArr, int64_t i, int32_t numPartitions) {
return pid;
}

arrow::Status gluten::HashPartitioner::compute(
const int32_t* pidArr,
const int64_t numRows,
std::vector<uint32_t>& row2partition,
std::vector<uint32_t>& partition2RowCount) {
arrow::Status
gluten::HashPartitioner::compute(const int32_t* pidArr, const int64_t numRows, std::vector<uint32_t>& row2partition) {
row2partition.resize(numRows);
std::fill(std::begin(partition2RowCount), std::end(partition2RowCount), 0);

for (auto i = 0; i < numRows; ++i) {
auto pid = computePid(pidArr, i, numPartitions_);
row2partition[i] = pid;
}

for (auto& pid : row2partition) {
partition2RowCount[pid]++;
}

return arrow::Status::OK();
}

Expand Down
6 changes: 1 addition & 5 deletions cpp/core/shuffle/HashPartitioner.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,7 @@ class HashPartitioner final : public Partitioner {
public:
HashPartitioner(int32_t numPartitions) : Partitioner(numPartitions, true) {}

arrow::Status compute(
const int32_t* pidArr,
const int64_t numRows,
std::vector<uint32_t>& row2partition,
std::vector<uint32_t>& partition2RowCount) override;
arrow::Status compute(const int32_t* pidArr, const int64_t numRows, std::vector<uint32_t>& row2partition) override;

arrow::Status compute(
const int32_t* pidArr,
Expand Down
Loading

0 comments on commit f75d7c1

Please sign in to comment.