-
Notifications
You must be signed in to change notification settings - Fork 51
[SPARK-25299] Shuffle locations api #517
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 5 commits
86050a2
c6df7a4
b58b628
4f2f752
35c1bee
ad0b7a1
95937d1
265e9da
b2ee17c
7585916
44423b8
4fdfc11
e75ce88
e82e408
0fa63f0
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,39 @@ | ||
| /* | ||
| * 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.Serializable; | ||
|
|
||
| /** | ||
| * Represents metadata about where shuffle blocks were written in a single map task. | ||
| * <p> | ||
| * This is optionally returned by shuffle writers. The inner shuffle locations may | ||
| * be accessed by shuffle readers. Shuffle locations are only necessary when the | ||
| * location of shuffle blocks needs to be managed by the driver; shuffle plugins | ||
| * may choose to use an external database or other metadata management systems to | ||
| * track the locations of shuffle blocks instead. | ||
| */ | ||
| @Experimental | ||
| public interface MapShuffleLocations extends Serializable { | ||
|
|
||
| /** | ||
| * Get the location for a given shuffle block written by this map task. | ||
| */ | ||
| ShuffleLocation getLocationForBlock(int reduceId); | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,25 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| /** | ||
| * Marker interface representing a location of a shuffle block. Implementations of shuffle readers | ||
|
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. To be even more specific, it's not the location of a shuffle block, but a location from which to retrieve shuffle blocks. Trying to figure out how to word this so that people don't get this confused with a
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. It's not always the location of a shuffle block manager though because I can choose a server that's running just a file store for example. |
||
| * and writers are expected to cast this down to an implementation-specific representation. | ||
| */ | ||
| public interface ShuffleLocation { | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -124,7 +124,10 @@ public void write(Iterator<Product2<K, V>> records) throws IOException { | |
| if (!records.hasNext()) { | ||
| partitionLengths = new long[numPartitions]; | ||
| shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, null); | ||
| mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); | ||
| mapStatus = MapStatus$.MODULE$.apply( | ||
| blockManager.shuffleServerId(), | ||
| DefaultMapShuffleLocations.get(blockManager.shuffleServerId()), | ||
|
||
| partitionLengths); | ||
| return; | ||
| } | ||
| final SerializerInstance serInstance = serializer.newInstance(); | ||
|
|
@@ -166,7 +169,10 @@ public void write(Iterator<Product2<K, V>> records) throws IOException { | |
| logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); | ||
| } | ||
| } | ||
| mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); | ||
| mapStatus = MapStatus$.MODULE$.apply( | ||
| blockManager.shuffleServerId(), | ||
| DefaultMapShuffleLocations.get(blockManager.shuffleServerId()), | ||
| partitionLengths); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,76 @@ | ||
| /* | ||
| * 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.sort; | ||
|
|
||
| import com.google.common.cache.CacheBuilder; | ||
| import com.google.common.cache.CacheLoader; | ||
| import com.google.common.cache.LoadingCache; | ||
|
|
||
| import org.apache.spark.api.shuffle.MapShuffleLocations; | ||
| import org.apache.spark.api.shuffle.ShuffleLocation; | ||
| import org.apache.spark.storage.BlockManagerId; | ||
|
|
||
| import java.util.Objects; | ||
|
|
||
| public class DefaultMapShuffleLocations implements MapShuffleLocations, ShuffleLocation { | ||
|
|
||
| /** | ||
| * We borrow the cache size from the BlockManagerId's cache - around 1MB, which should be | ||
| * feasible. | ||
| */ | ||
| private static final LoadingCache<BlockManagerId, DefaultMapShuffleLocations> | ||
yifeih marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| DEFAULT_SHUFFLE_LOCATIONS_CACHE = | ||
| CacheBuilder.newBuilder() | ||
| .maximumSize(10000) | ||
| .build(new CacheLoader<BlockManagerId, DefaultMapShuffleLocations>() { | ||
| @Override | ||
| public DefaultMapShuffleLocations load(BlockManagerId blockManagerId) { | ||
| return new DefaultMapShuffleLocations(blockManagerId); | ||
| } | ||
| }); | ||
|
|
||
| private final BlockManagerId location; | ||
|
|
||
| public DefaultMapShuffleLocations(BlockManagerId blockManagerId) { | ||
| this.location = blockManagerId; | ||
| } | ||
|
|
||
| public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { | ||
| return DEFAULT_SHUFFLE_LOCATIONS_CACHE.getUnchecked(blockManagerId); | ||
| } | ||
|
|
||
| @Override | ||
| public ShuffleLocation getLocationForBlock(int reduceId) { | ||
| return this; | ||
| } | ||
|
|
||
| public BlockManagerId getBlockManagerId() { | ||
| return location; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean equals(Object other) { | ||
| return other instanceof DefaultMapShuffleLocations | ||
| && Objects.equals(((DefaultMapShuffleLocations) other).location, location); | ||
| } | ||
|
|
||
| @Override | ||
| public int hashCode() { | ||
| return Objects.hashCode(location); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,6 +28,7 @@ import scala.concurrent.duration.Duration | |
| import scala.reflect.ClassTag | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import org.apache.spark.api.shuffle.MapShuffleLocations | ||
| import org.apache.spark.broadcast.{Broadcast, BroadcastManager} | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.config._ | ||
|
|
@@ -282,7 +283,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging | |
|
|
||
| // For testing | ||
| def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int) | ||
| : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { | ||
| : Iterator[(MapShuffleLocations, Seq[(BlockId, Long)])] = { | ||
|
||
| getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1) | ||
| } | ||
|
|
||
|
|
@@ -296,7 +297,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging | |
| * describing the shuffle blocks that are stored at that block manager. | ||
| */ | ||
| def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) | ||
| : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] | ||
| : Iterator[(MapShuffleLocations, Seq[(BlockId, Long)])] | ||
|
|
||
| /** | ||
| * Deletes map output status information for the specified shuffle stage. | ||
|
|
@@ -646,7 +647,7 @@ private[spark] class MapOutputTrackerMaster( | |
| // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. | ||
| // This method is only called in local-mode. | ||
| def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) | ||
| : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { | ||
| : Iterator[(MapShuffleLocations, Seq[(BlockId, Long)])] = { | ||
| logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") | ||
| shuffleStatuses.get(shuffleId) match { | ||
| case Some (shuffleStatus) => | ||
|
|
@@ -683,11 +684,12 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr | |
|
|
||
| // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. | ||
| override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) | ||
| : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { | ||
| : Iterator[(MapShuffleLocations, Seq[(BlockId, Long)])] = { | ||
| logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") | ||
| val statuses = getStatuses(shuffleId) | ||
| try { | ||
| MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) | ||
| MapOutputTracker.convertMapStatuses( | ||
| shuffleId, startPartition, endPartition, statuses) | ||
| } catch { | ||
| case e: MetadataFetchFailedException => | ||
| // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: | ||
|
|
@@ -871,9 +873,9 @@ private[spark] object MapOutputTracker extends Logging { | |
| shuffleId: Int, | ||
| startPartition: Int, | ||
| endPartition: Int, | ||
| statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { | ||
| statuses: Array[MapStatus]): Iterator[(MapShuffleLocations, Seq[(BlockId, Long)])] = { | ||
|
||
| assert (statuses != null) | ||
| val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] | ||
| val splitsByAddress = new HashMap[MapShuffleLocations, ListBuffer[(BlockId, Long)]] | ||
| for ((status, mapId) <- statuses.iterator.zipWithIndex) { | ||
| if (status == null) { | ||
| val errorMessage = s"Missing an output location for shuffle $shuffleId" | ||
|
|
@@ -883,7 +885,7 @@ private[spark] object MapOutputTracker extends Logging { | |
| for (part <- startPartition until endPartition) { | ||
| val size = status.getSizeForBlock(part) | ||
| if (size != 0) { | ||
| splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += | ||
| splitsByAddress.getOrElseUpdate(status.mapShuffleLocations, ListBuffer()) += | ||
| ((ShuffleBlockId(shuffleId, mapId, part), size)) | ||
| } | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.