Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
org.apache.spark.shuffle.VeloxCelebornColumnarBatchSerlizerFactory
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.shuffle

import org.apache.gluten.backendsapi.velox.VeloxBackend

import org.apache.spark.shuffle.gluten.celeborn.CelebornColumnarBatchSerializerFactory

class VeloxCelebornColumnarBatchSerlizerFactory extends CelebornColumnarBatchSerializerFactory {
override def backendName(): String = VeloxBackend.BACKEND_NAME

override def columnarBatchSerializerClass(): String =
"org.apache.spark.shuffle.CelebornColumnarBatchSerializer"
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,11 @@ import org.apache.gluten.execution._
import org.apache.gluten.expression._
import org.apache.gluten.expression.aggregate.{HLLAdapter, VeloxBloomFilterAggregate, VeloxCollectList, VeloxCollectSet}
import org.apache.gluten.extension.columnar.FallbackTags
import org.apache.gluten.shuffle.NeedCustomColumnarBatchSerializer
import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.gluten.vectorized.{ColumnarBatchSerializer, ColumnarBatchSerializeResult}

import org.apache.spark.{ShuffleDependency, SparkException}
import org.apache.spark.{ShuffleDependency, SparkEnv, SparkException}
import org.apache.spark.api.python.{ColumnarArrowEvalPythonExec, PullOutArrowEvalPythonPreProjectHelper}
import org.apache.spark.memory.SparkMemoryUtil
import org.apache.spark.rdd.RDD
Expand Down Expand Up @@ -553,6 +554,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
partitioning: Partitioning,
output: Seq[Attribute]): ShuffleWriterType = {
val conf = GlutenConfig.get
// todo: remove isUseCelebornShuffleManager here
if (conf.isUseCelebornShuffleManager) {
if (conf.celebornShuffleWriterType == ReservedKeys.GLUTEN_SORT_SHUFFLE_WRITER) {
if (conf.useCelebornRssSort) {
Expand Down Expand Up @@ -629,25 +631,27 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi {
val deserializeTime = metrics("deserializeTime")
val readBatchNumRows = metrics("avgReadBatchNumRows")
val decompressTime = metrics("decompressTime")
if (GlutenConfig.get.isUseCelebornShuffleManager) {
val clazz = ClassUtils.getClass("org.apache.spark.shuffle.CelebornColumnarBatchSerializer")
val constructor =
clazz.getConstructor(
classOf[StructType],
classOf[SQLMetric],
classOf[SQLMetric],
classOf[ShuffleWriterType])
constructor
.newInstance(schema, readBatchNumRows, numOutputRows, shuffleWriterType)
.asInstanceOf[Serializer]
} else {
new ColumnarBatchSerializer(
schema,
readBatchNumRows,
numOutputRows,
deserializeTime,
decompressTime,
shuffleWriterType)
SparkEnv.get.shuffleManager match {
case serializer: NeedCustomColumnarBatchSerializer =>
val className = serializer.columnarBatchSerializerClass()
val clazz = ClassUtils.getClass(className)
val constructor =
clazz.getConstructor(
classOf[StructType],
classOf[SQLMetric],
classOf[SQLMetric],
classOf[ShuffleWriterType])
constructor
.newInstance(schema, readBatchNumRows, numOutputRows, shuffleWriterType)
.asInstanceOf[Serializer]
case _ =>
new ColumnarBatchSerializer(
schema,
readBatchNumRows,
numOutputRows,
deserializeTime,
decompressTime,
shuffleWriterType)
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.shuffle.gluten.celeborn;

public interface CelebornColumnarBatchSerializerFactory {
String backendName();

String columnarBatchSerializerClass();
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.gluten.backendsapi.BackendsApiManager;
import org.apache.gluten.config.GlutenConfig;
import org.apache.gluten.exception.GlutenException;
import org.apache.gluten.shuffle.NeedCustomColumnarBatchSerializer;
import org.apache.gluten.shuffle.SupportsColumnarShuffle;

import com.google.common.base.Preconditions;
Expand All @@ -42,7 +43,8 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;

public class CelebornShuffleManager implements ShuffleManager, SupportsColumnarShuffle {
public class CelebornShuffleManager
implements ShuffleManager, SupportsColumnarShuffle, NeedCustomColumnarBatchSerializer {

private static final Logger logger = LoggerFactory.getLogger(CelebornShuffleManager.class);

Expand All @@ -63,6 +65,8 @@ public class CelebornShuffleManager implements ShuffleManager, SupportsColumnarS

private static final CelebornShuffleWriterFactory writerFactory;

private static final CelebornColumnarBatchSerializerFactory columnarBatchSerializerFactory;

static {
final ServiceLoader<CelebornShuffleWriterFactory> loader =
ServiceLoader.load(CelebornShuffleWriterFactory.class);
Expand All @@ -81,6 +85,31 @@ public class CelebornShuffleManager implements ShuffleManager, SupportsColumnarS
"No Celeborn shuffle writer factory found for backend " + backendName);
}
writerFactory = factoryMap.get(backendName);

final ServiceLoader<CelebornColumnarBatchSerializerFactory>
celebornColumnarBatchSerializerFactoriesLoader =
ServiceLoader.load(CelebornColumnarBatchSerializerFactory.class);
final List<CelebornColumnarBatchSerializerFactory> columnarBatchSerializerFactoryList =
Arrays.stream(
Iterators.toArray(
celebornColumnarBatchSerializerFactoriesLoader.iterator(),
CelebornColumnarBatchSerializerFactory.class))
.collect(Collectors.toList());
// for now, we ignore check since CH backend has not support this feature yet.
// Preconditions.checkState(
// !columnarBatchSerializerFactoryList.isEmpty(),
// "No factory found for Celeborn columnar batch serializer");
final Map<String, CelebornColumnarBatchSerializerFactory> columanrBatchSerilizerFactoryMap =
columnarBatchSerializerFactoryList.stream()
.collect(Collectors.toMap(CelebornColumnarBatchSerializerFactory::backendName, f -> f));

// for now, we ignore check since CH backend has not support this feature yet.
// if (!columanrBatchSerilizerFactoryMap.containsKey(backendName)) {
// throw new UnsupportedOperationException(
// "No Celeborn columnar batch serializer writer factory found for backend " +
// backendName);
// }
columnarBatchSerializerFactory = columanrBatchSerilizerFactoryMap.get(backendName);
}

private final SparkConf conf;
Expand Down Expand Up @@ -408,4 +437,9 @@ public <K, C> ShuffleReader<K, C> getReader(
.getReader(
handle, startMapIndex, endMapIndex, startPartition, endPartition, context, metrics);
}

@Override
public String columnarBatchSerializerClass() {
return columnarBatchSerializerFactory.columnarBatchSerializerClass();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,3 +17,7 @@
package org.apache.gluten.shuffle

trait SupportsColumnarShuffle

trait NeedCustomColumnarBatchSerializer {
def columnarBatchSerializerClass(): String
}