Skip to content

Commit 1929a74

Browse files
committed
Update to reflect upstream ShuffleBlockManager -> ShuffleBlockResolver rename.
1 parent e8718dd commit 1929a74

File tree

3 files changed

+13
-13
lines changed

3 files changed

+13
-13
lines changed

core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@
4444
import org.apache.spark.serializer.SerializationStream;
4545
import org.apache.spark.serializer.Serializer;
4646
import org.apache.spark.serializer.SerializerInstance;
47-
import org.apache.spark.shuffle.IndexShuffleBlockManager;
47+
import org.apache.spark.shuffle.IndexShuffleBlockResolver;
4848
import org.apache.spark.shuffle.ShuffleMemoryManager;
4949
import org.apache.spark.shuffle.ShuffleWriter;
5050
import org.apache.spark.storage.BlockManager;
@@ -59,7 +59,7 @@ public class UnsafeShuffleWriter<K, V> extends ShuffleWriter<K, V> {
5959
private static final ClassTag<Object> OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object();
6060

6161
private final BlockManager blockManager;
62-
private final IndexShuffleBlockManager shuffleBlockManager;
62+
private final IndexShuffleBlockResolver shuffleBlockResolver;
6363
private final TaskMemoryManager memoryManager;
6464
private final ShuffleMemoryManager shuffleMemoryManager;
6565
private final SerializerInstance serializer;
@@ -87,15 +87,15 @@ public class UnsafeShuffleWriter<K, V> extends ShuffleWriter<K, V> {
8787

8888
public UnsafeShuffleWriter(
8989
BlockManager blockManager,
90-
IndexShuffleBlockManager shuffleBlockManager,
90+
IndexShuffleBlockResolver shuffleBlockResolver,
9191
TaskMemoryManager memoryManager,
9292
ShuffleMemoryManager shuffleMemoryManager,
9393
UnsafeShuffleHandle<K, V> handle,
9494
int mapId,
9595
TaskContext taskContext,
9696
SparkConf sparkConf) {
9797
this.blockManager = blockManager;
98-
this.shuffleBlockManager = shuffleBlockManager;
98+
this.shuffleBlockResolver = shuffleBlockResolver;
9999
this.memoryManager = memoryManager;
100100
this.shuffleMemoryManager = shuffleMemoryManager;
101101
this.mapId = mapId;
@@ -162,7 +162,7 @@ void closeAndWriteOutput() throws IOException {
162162
}
163163
}
164164
}
165-
shuffleBlockManager.writeIndexFile(shuffleId, mapId, partitionLengths);
165+
shuffleBlockResolver.writeIndexFile(shuffleId, mapId, partitionLengths);
166166
mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths);
167167
}
168168

@@ -192,7 +192,7 @@ void forceSorterToSpill() throws IOException {
192192
}
193193

194194
private long[] mergeSpills(SpillInfo[] spills) throws IOException {
195-
final File outputFile = shuffleBlockManager.getDataFile(shuffleId, mapId);
195+
final File outputFile = shuffleBlockResolver.getDataFile(shuffleId, mapId);
196196
try {
197197
if (spills.length == 0) {
198198
new FileOutputStream(outputFile).close(); // Create an empty file
@@ -323,7 +323,7 @@ public Option<MapStatus> stop(boolean success) {
323323
return Option.apply(mapStatus);
324324
} else {
325325
// The map task failed, so delete our output data.
326-
shuffleBlockManager.removeDataByMap(shuffleId, mapId);
326+
shuffleBlockResolver.removeDataByMap(shuffleId, mapId);
327327
return Option.apply(null);
328328
}
329329
}

core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -92,7 +92,7 @@ private[spark] class UnsafeShuffleManager(conf: SparkConf) extends ShuffleManage
9292
// TODO: do we need to do anything to register the shuffle here?
9393
new UnsafeShuffleWriter(
9494
env.blockManager,
95-
shuffleBlockResolver.asInstanceOf[IndexShuffleBlockManager],
95+
shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver],
9696
context.taskMemoryManager(),
9797
env.shuffleMemoryManager,
9898
unsafeShuffleHandle,

core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@
4141

4242
import org.apache.spark.*;
4343
import org.apache.spark.serializer.Serializer;
44-
import org.apache.spark.shuffle.IndexShuffleBlockManager;
44+
import org.apache.spark.shuffle.IndexShuffleBlockResolver;
4545
import org.apache.spark.executor.ShuffleWriteMetrics;
4646
import org.apache.spark.executor.TaskMetrics;
4747
import org.apache.spark.serializer.SerializerInstance;
@@ -67,7 +67,7 @@ public class UnsafeShuffleWriterSuite {
6767

6868
@Mock(answer = RETURNS_SMART_NULLS) ShuffleMemoryManager shuffleMemoryManager;
6969
@Mock(answer = RETURNS_SMART_NULLS) BlockManager blockManager;
70-
@Mock(answer = RETURNS_SMART_NULLS) IndexShuffleBlockManager shuffleBlockManager;
70+
@Mock(answer = RETURNS_SMART_NULLS) IndexShuffleBlockResolver shuffleBlockResolver;
7171
@Mock(answer = RETURNS_SMART_NULLS) DiskBlockManager diskBlockManager;
7272
@Mock(answer = RETURNS_SMART_NULLS) TaskContext taskContext;
7373
@Mock(answer = RETURNS_SMART_NULLS) ShuffleDependency<Object, Object, Object> shuffleDep;
@@ -124,14 +124,14 @@ public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Th
124124
when(blockManager.wrapForCompression(any(BlockId.class), any(InputStream.class)))
125125
.then(returnsSecondArg());
126126

127-
when(shuffleBlockManager.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile);
127+
when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile);
128128
doAnswer(new Answer<Void>() {
129129
@Override
130130
public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
131131
partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2];
132132
return null;
133133
}
134-
}).when(shuffleBlockManager).writeIndexFile(anyInt(), anyInt(), any(long[].class));
134+
}).when(shuffleBlockResolver).writeIndexFile(anyInt(), anyInt(), any(long[].class));
135135

136136
when(diskBlockManager.createTempShuffleBlock()).thenAnswer(
137137
new Answer<Tuple2<TempLocalBlockId, File>>() {
@@ -157,7 +157,7 @@ private UnsafeShuffleWriter<Object, Object> createWriter(boolean transferToEnabl
157157
conf.set("spark.file.transferTo", String.valueOf(transferToEnabled));
158158
return new UnsafeShuffleWriter<Object, Object>(
159159
blockManager,
160-
shuffleBlockManager,
160+
shuffleBlockResolver,
161161
taskMemoryManager,
162162
shuffleMemoryManager,
163163
new UnsafeShuffleHandle<Object, Object>(0, 1, shuffleDep),

0 commit comments

Comments
 (0)