-
Notifications
You must be signed in to change notification settings - Fork 51
[SPARK-25299] shuffle reader API #523
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 40 commits
864d1cd
c88751c
9af216f
a35b826
14c47ae
5bb4c32
584e6c8
0292fe2
71c2cc7
43c377c
9fc6a60
45172a5
4e5652b
a35d8fe
1a09ebe
c149d24
672d473
e0a3289
1e89b3f
495c7bd
88a03cb
741deed
76c0381
c7c52b0
897c0bf
34eaaf6
0548800
0637e70
f069dc1
0bba677
a82a725
ac392a1
53dd94b
4c0c791
84f7931
b59efb5
aba8a94
5ef59b6
49a1901
8c6c09c
6370b41
c442b63
2c1272a
2758a5c
bd349ca
653f67c
9f53839
94275fd
26e97c1
91db776
f0fa7b8
50c8fc3
4aa4b6e
7d23f47
363d4ab
bb7fa4c
711109b
04a135c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,64 @@ | ||
| /* | ||
| * 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.api.shuffle; | ||
|
|
||
| import java.util.Objects; | ||
|
|
||
| public class ShuffleBlockInfo { | ||
| private final int shuffleId; | ||
| private final int mapId; | ||
| private final int reduceId; | ||
| private final long length; | ||
|
|
||
| public ShuffleBlockInfo(int shuffleId, int mapId, int reduceId, long length) { | ||
| this.shuffleId = shuffleId; | ||
| this.mapId = mapId; | ||
| this.reduceId = reduceId; | ||
| this.length = length; | ||
| } | ||
|
|
||
| public int getShuffleId() { | ||
| return shuffleId; | ||
| } | ||
|
|
||
| public int getMapId() { | ||
| return mapId; | ||
| } | ||
|
|
||
| public int getReduceId() { | ||
| return reduceId; | ||
| } | ||
|
|
||
| public long getLength() { | ||
| return length; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object other) { | ||
| return other instanceof ShuffleBlockInfo | ||
| && shuffleId == ((ShuffleBlockInfo) other).shuffleId | ||
| && mapId == ((ShuffleBlockInfo) other).mapId | ||
| && reduceId == ((ShuffleBlockInfo) other).reduceId | ||
| && length == ((ShuffleBlockInfo) other).length; | ||
|
||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hash(shuffleId, mapId, reduceId, length); | ||
|
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,37 @@ | ||
| /* | ||
| * 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.api.shuffle; | ||
|
|
||
| import org.apache.spark.annotation.Experimental; | ||
|
|
||
| import java.io.IOException; | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * An interface for reading shuffle records. | ||
| * @since 3.0.0 | ||
| */ | ||
yifeih marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| @Experimental | ||
| public interface ShuffleReadSupport { | ||
| /** | ||
| * Returns an underlying {@link ShuffleReaderIterable} that will iterate through shuffle data, | ||
| * given an iterable for the shuffle blocks to fetch. | ||
| */ | ||
| ShuffleReaderIterable getPartitionReaders(Iterable<ShuffleBlockInfo> blockMetadata) | ||
| throws IOException; | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,47 @@ | ||
| /* | ||
| * 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.api.shuffle; | ||
|
|
||
| import org.apache.spark.annotation.Experimental; | ||
|
|
||
| import java.io.InputStream; | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * An interface for reading shuffle records. | ||
|
||
| * @since 3.0.0 | ||
| */ | ||
| @Experimental | ||
| public class ShuffleReaderInputStream { | ||
|
|
||
| private final ShuffleBlockInfo shuffleBlockInfo; | ||
|
||
| private final InputStream inputStream; | ||
|
|
||
| public ShuffleReaderInputStream(ShuffleBlockInfo shuffleBlockInfo, InputStream inputStream) { | ||
| this.shuffleBlockInfo = shuffleBlockInfo; | ||
| this.inputStream = inputStream; | ||
| } | ||
|
|
||
| public ShuffleBlockInfo getShuffleBlockInfo() { | ||
| return shuffleBlockInfo; | ||
| } | ||
|
|
||
| public InputStream getInputStream() { | ||
| return inputStream; | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,44 @@ | ||
| /* | ||
| * 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.api.shuffle; | ||
|
|
||
| import org.apache.spark.annotation.Experimental; | ||
|
|
||
| import java.util.Iterator; | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * An interface for iterating through shuffle blocks to read. | ||
| * @since 3.0.0 | ||
| */ | ||
| @Experimental | ||
| public interface ShuffleReaderIterable extends Iterable<ShuffleReaderInputStream> { | ||
|
|
||
| interface ShuffleReaderIterator extends Iterator<ShuffleReaderInputStream> { | ||
| /** | ||
| * Instructs the shuffle iterator to fetch the last block again. This is useful | ||
| * if the block is determined to be corrupt after decryption or decompression. | ||
| */ | ||
| default void retryLastBlock(Throwable t) { | ||
| throw new UnsupportedOperationException("Cannot retry fetching bad blocks", t); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| ShuffleReaderIterator iterator(); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,12 +17,19 @@ | |
|
|
||
| package org.apache.spark.shuffle | ||
|
|
||
| import java.io.{InputStream, IOException} | ||
| import java.nio.ByteBuffer | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.internal.{config, Logging} | ||
| import org.apache.spark.api.shuffle.{ShuffleBlockInfo, ShuffleReadSupport} | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.serializer.SerializerManager | ||
| import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} | ||
| import org.apache.spark.util.CompletionIterator | ||
| import org.apache.spark.storage.ShuffleBlockId | ||
| import org.apache.spark.util.{CompletionIterator, Utils} | ||
| import org.apache.spark.util.collection.ExternalSorter | ||
| import org.apache.spark.util.io.ChunkedByteBufferOutputStream | ||
|
|
||
| /** | ||
| * Fetches and reads the partitions in range [startPartition, endPartition) from a shuffle by | ||
|
|
@@ -34,33 +41,64 @@ private[spark] class BlockStoreShuffleReader[K, C]( | |
| endPartition: Int, | ||
| context: TaskContext, | ||
| readMetrics: ShuffleReadMetricsReporter, | ||
| serializerManager: SerializerManager = SparkEnv.get.serializerManager, | ||
| blockManager: BlockManager = SparkEnv.get.blockManager, | ||
| serializerManager: SerializerManager, | ||
| shuffleReadSupport: ShuffleReadSupport, | ||
| mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker) | ||
| extends ShuffleReader[K, C] with Logging { | ||
|
|
||
| private val dep = handle.dependency | ||
|
|
||
| /** Read the combined key-values for this reduce task */ | ||
| override def read(): Iterator[Product2[K, C]] = { | ||
| val wrappedStreams = new ShuffleBlockFetcherIterator( | ||
| context, | ||
| blockManager.shuffleClient, | ||
| blockManager, | ||
| mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), | ||
| serializerManager.wrapStream, | ||
| // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility | ||
| SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, | ||
| SparkEnv.get.conf.get(config.REDUCER_MAX_REQS_IN_FLIGHT), | ||
| SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS), | ||
| SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), | ||
| SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT), | ||
| readMetrics).toCompletionIterator | ||
| val wrappedStreams = | ||
|
||
| shuffleReadSupport.getPartitionReaders(new Iterable[ShuffleBlockInfo] { | ||
| override def iterator: Iterator[ShuffleBlockInfo] = { | ||
| /** Read the combined key-values for this reduce task */ | ||
|
||
| mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition) | ||
| .flatMap(blockManagerIdInfo => { | ||
|
||
| blockManagerIdInfo._2.map( | ||
|
||
| blockInfo => { | ||
| val block = blockInfo._1.asInstanceOf[ShuffleBlockId] | ||
| new ShuffleBlockInfo(block.shuffleId, block.mapId, block.reduceId, blockInfo._2) | ||
| } | ||
| ) | ||
| }) | ||
| } | ||
| }.asJava).iterator() | ||
|
|
||
| val serializerInstance = dep.serializer.newInstance() | ||
| val retryingWrappedStreams = new Iterator[InputStream] { | ||
| override def hasNext: Boolean = wrappedStreams.hasNext | ||
|
|
||
| // Create a key/value iterator for each stream | ||
| val recordIter = wrappedStreams.flatMap { case (blockId, wrappedStream) => | ||
| override def next(): InputStream = { | ||
| var returnStream: InputStream = null | ||
| while (wrappedStreams.hasNext && returnStream == null) { | ||
| val nextStream = wrappedStreams.next() | ||
| val blockInfo = nextStream.getShuffleBlockInfo | ||
| val blockId = ShuffleBlockId( | ||
| blockInfo.getShuffleId, | ||
| blockInfo.getMapId, | ||
| blockInfo.getReduceId) | ||
| try { | ||
| val in = serializerManager.wrapStream(blockId, nextStream.getInputStream) | ||
| val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate) | ||
| // Decompress the whole block at once to detect any corruption, which could increase | ||
| // the memory usage tne potential increase the chance of OOM. | ||
|
||
| // TODO: manage the memory used here, and spill it into disk in case of OOM. | ||
|
||
| Utils.copyStream(in, out, closeStreams = true) | ||
|
||
| returnStream = out.toChunkedByteBuffer.toInputStream(dispose = true) | ||
| } catch { | ||
| case e: IOException => | ||
|
||
| wrappedStreams.retryLastBlock(e) | ||
| } | ||
| } | ||
| if (returnStream == null) { | ||
| throw new IllegalStateException("Expected shuffle reader iterator to return a stream") | ||
| } | ||
| returnStream | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Defensive programming: if
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hmm so the retrying not working properly would result in either There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ping here? Think the last push doesn't address this. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Never mind I was looking at a stale diff |
||
| } | ||
| } | ||
|
|
||
| val serializerInstance = dep.serializer.newInstance() | ||
| val recordIter = retryingWrappedStreams.flatMap { wrappedStream => | ||
| // Note: the asKeyValueIterator below wraps a key/value iterator inside of a | ||
| // NextIterator. The NextIterator makes sure that close() is called on the | ||
| // underlying InputStream when all records have been read. | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.