-
Notifications
You must be signed in to change notification settings - Fork 127
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Co-authored-by: zhangli20 <[email protected]>
- Loading branch information
Showing
7 changed files
with
301 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
169 changes: 169 additions & 0 deletions
169
.../main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeRssShuffleManagerBase.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,169 @@ | ||
/* | ||
* Copyright 2022 The Blaze Authors | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.sql.execution.blaze.shuffle | ||
|
||
import org.apache.spark.ShuffleDependency | ||
import org.apache.spark.SparkConf | ||
import org.apache.spark.TaskContext | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.shuffle._ | ||
import org.apache.spark.sql.execution.blaze.shuffle.BlazeShuffleDependency.isArrowShuffle | ||
|
||
import com.thoughtworks.enableIf | ||
|
||
abstract class BlazeRssShuffleManagerBase(conf: SparkConf) extends ShuffleManager with Logging { | ||
override def registerShuffle[K, V, C]( | ||
shuffleId: Int, | ||
dependency: ShuffleDependency[K, V, C]): ShuffleHandle | ||
|
||
override def unregisterShuffle(shuffleId: Int): Boolean | ||
|
||
def getBlazeRssShuffleReader[K, C]( | ||
handle: ShuffleHandle, | ||
startPartition: Int, | ||
endPartition: Int, | ||
context: TaskContext, | ||
metrics: ShuffleReadMetricsReporter): BlazeRssShuffleReaderBase[K, C] | ||
|
||
def getBlazeRssShuffleReader[K, C]( | ||
handle: ShuffleHandle, | ||
startMapIndex: Int, | ||
endMapIndex: Int, | ||
startPartition: Int, | ||
endPartition: Int, | ||
context: TaskContext, | ||
metrics: ShuffleReadMetricsReporter): BlazeRssShuffleReaderBase[K, C] | ||
|
||
def getRssShuffleReader[K, C]( | ||
handle: ShuffleHandle, | ||
startPartition: Int, | ||
endPartition: Int, | ||
context: TaskContext, | ||
metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] | ||
|
||
def getRssShuffleReader[K, C]( | ||
handle: ShuffleHandle, | ||
startMapIndex: Int, | ||
endMapIndex: Int, | ||
startPartition: Int, | ||
endPartition: Int, | ||
context: TaskContext, | ||
metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] | ||
|
||
def getBlazeRssShuffleWriter[K, V]( | ||
handle: ShuffleHandle, | ||
mapId: Long, | ||
context: TaskContext, | ||
metrics: ShuffleWriteMetricsReporter): BlazeRssShuffleWriterBase[K, V] | ||
|
||
def getRssShuffleWriter[K, V]( | ||
handle: ShuffleHandle, | ||
mapId: Long, | ||
context: TaskContext, | ||
metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] | ||
|
||
@enableIf( | ||
Seq("spark320", "spark324", "spark333", "spark351").contains( | ||
System.getProperty("blaze.shim"))) | ||
override def getReader[K, C]( | ||
handle: ShuffleHandle, | ||
startMapIndex: Int, | ||
endMapIndex: Int, | ||
startPartition: Int, | ||
endPartition: Int, | ||
context: TaskContext, | ||
metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { | ||
|
||
if (isArrowShuffle(handle)) { | ||
getBlazeRssShuffleReader( | ||
handle, | ||
startMapIndex, | ||
endMapIndex, | ||
startPartition, | ||
endPartition, | ||
context, | ||
metrics) | ||
} else { | ||
getRssShuffleReader( | ||
handle, | ||
startMapIndex, | ||
endMapIndex, | ||
startPartition, | ||
endPartition, | ||
context, | ||
metrics) | ||
} | ||
} | ||
|
||
@enableIf(Seq("spark303").contains(System.getProperty("blaze.shim"))) | ||
override def getReader[K, C]( | ||
handle: ShuffleHandle, | ||
startPartition: Int, | ||
endPartition: Int, | ||
context: TaskContext, | ||
metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { | ||
|
||
if (isArrowShuffle(handle)) { | ||
getBlazeRssShuffleReader(handle, startPartition, endPartition, context, metrics) | ||
} else { | ||
getRssShuffleReader(handle, startPartition, endPartition, context, metrics) | ||
} | ||
} | ||
|
||
@enableIf(Seq("spark303").contains(System.getProperty("blaze.shim"))) | ||
override def getReaderForRange[K, C]( | ||
handle: ShuffleHandle, | ||
startMapIndex: Int, | ||
endMapIndex: Int, | ||
startPartition: Int, | ||
endPartition: Int, | ||
context: TaskContext, | ||
metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { | ||
|
||
if (isArrowShuffle(handle)) { | ||
getBlazeRssShuffleReader( | ||
handle, | ||
startMapIndex, | ||
endMapIndex, | ||
startPartition, | ||
endPartition, | ||
context, | ||
metrics) | ||
} else { | ||
getRssShuffleReader( | ||
handle, | ||
startMapIndex, | ||
endMapIndex, | ||
startPartition, | ||
endPartition, | ||
context, | ||
metrics) | ||
} | ||
} | ||
|
||
override def getWriter[K, V]( | ||
handle: ShuffleHandle, | ||
mapId: Long, | ||
context: TaskContext, | ||
metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { | ||
|
||
if (isArrowShuffle(handle)) { | ||
getBlazeRssShuffleWriter(handle, mapId, context, metrics) | ||
} else { | ||
getRssShuffleWriter(handle, mapId, context, metrics) | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
24 changes: 24 additions & 0 deletions
24
...c/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeRssShuffleReaderBase.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,24 @@ | ||
/* | ||
* Copyright 2022 The Blaze Authors | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.sql.execution.blaze.shuffle | ||
|
||
import org.apache.spark.TaskContext | ||
import org.apache.spark.shuffle.BaseShuffleHandle | ||
|
||
abstract class BlazeRssShuffleReaderBase[K, C]( | ||
handle: BaseShuffleHandle[K, _, C], | ||
context: TaskContext) | ||
extends BlazeBlockStoreShuffleReaderBase[K, C](handle, context) {} |
85 changes: 85 additions & 0 deletions
85
...c/main/scala/org/apache/spark/sql/execution/blaze/shuffle/BlazeRssShuffleWriterBase.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* Copyright 2022 The Blaze Authors | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package org.apache.spark.sql.execution.blaze.shuffle | ||
|
||
import java.util.UUID | ||
|
||
import org.apache.spark.SparkEnv | ||
import org.apache.spark.shuffle.ShuffleWriteMetricsReporter | ||
import org.apache.spark.sql.blaze.JniBridge | ||
import org.apache.spark.sql.blaze.NativeHelper | ||
import org.apache.spark.sql.blaze.NativeRDD | ||
import org.apache.spark.sql.blaze.Shims | ||
import org.apache.spark.Partition | ||
import org.apache.spark.ShuffleDependency | ||
import org.apache.spark.TaskContext | ||
import org.apache.spark.scheduler.MapStatus | ||
import org.apache.spark.shuffle.ShuffleHandle | ||
import org.blaze.protobuf.PhysicalPlanNode | ||
import org.blaze.protobuf.RssShuffleWriterExecNode | ||
|
||
abstract class BlazeRssShuffleWriterBase[K, V](metrics: ShuffleWriteMetricsReporter) | ||
extends BlazeShuffleWriterBase[K, V](metrics) { | ||
|
||
def getRssPartitionWriter( | ||
handle: ShuffleHandle, | ||
mapId: Int, | ||
metrics: ShuffleWriteMetricsReporter, | ||
numPartitions: Int): RssPartitionWriterBase | ||
|
||
def nativeRssShuffleWrite( | ||
nativeShuffleRDD: NativeRDD, | ||
dep: ShuffleDependency[_, _, _], | ||
mapId: Int, | ||
context: TaskContext, | ||
partition: Partition, | ||
numPartitions: Int): MapStatus = { | ||
|
||
val rssShuffleWriterObject = | ||
getRssPartitionWriter(dep.shuffleHandle, mapId, metrics, numPartitions) | ||
if (rssShuffleWriterObject == null) { | ||
throw new RuntimeException("cannot get RssPartitionWriter") | ||
} | ||
|
||
try { | ||
val jniResourceId = s"RssPartitionWriter:${UUID.randomUUID().toString}" | ||
JniBridge.resourcesMap.put(jniResourceId, rssShuffleWriterObject) | ||
val nativeRssShuffleWriterExec = PhysicalPlanNode | ||
.newBuilder() | ||
.setRssShuffleWriter( | ||
RssShuffleWriterExecNode | ||
.newBuilder(nativeShuffleRDD.nativePlan(partition, context).getRssShuffleWriter) | ||
.setRssPartitionWriterResourceId(jniResourceId) | ||
.build()) | ||
.build() | ||
|
||
val iterator = NativeHelper.executeNativePlan( | ||
nativeRssShuffleWriterExec, | ||
nativeShuffleRDD.metrics, | ||
partition, | ||
Some(context)) | ||
assert(iterator.toArray.isEmpty) | ||
} finally { | ||
rssShuffleWriterObject.close() | ||
} | ||
|
||
val mapStatus = Shims.get.getMapStatus( | ||
SparkEnv.get.blockManager.shuffleServerId, | ||
rssShuffleWriterObject.getPartitionLengthMap, | ||
mapId) | ||
mapStatus | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters