Skip to content

Commit 3490512

Browse files
committed
Misc. cleanup
1 parent f156a8f commit 3490512

File tree

5 files changed

+34
-50
lines changed

5 files changed

+34
-50
lines changed

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

Lines changed: 9 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,21 +17,18 @@
1717

1818
package org.apache.spark.shuffle.unsafe;
1919

20-
import org.apache.spark.*;
21-
import org.apache.spark.unsafe.sort.ExternalSorterIterator;
22-
import org.apache.spark.unsafe.sort.UnsafeExternalSorter;
20+
import java.io.File;
21+
import java.io.IOException;
22+
import java.nio.ByteBuffer;
23+
2324
import scala.Option;
2425
import scala.Product2;
2526
import scala.reflect.ClassTag;
2627
import scala.reflect.ClassTag$;
2728

28-
import java.io.File;
29-
import java.io.IOException;
30-
import java.nio.ByteBuffer;
31-
import java.util.LinkedList;
32-
3329
import com.esotericsoftware.kryo.io.ByteBufferOutputStream;
3430

31+
import org.apache.spark.*;
3532
import org.apache.spark.executor.ShuffleWriteMetrics;
3633
import org.apache.spark.scheduler.MapStatus;
3734
import org.apache.spark.scheduler.MapStatus$;
@@ -44,10 +41,11 @@
4441
import org.apache.spark.storage.BlockObjectWriter;
4542
import org.apache.spark.storage.ShuffleBlockId;
4643
import org.apache.spark.unsafe.PlatformDependent;
47-
import org.apache.spark.unsafe.memory.MemoryBlock;
4844
import org.apache.spark.unsafe.memory.TaskMemoryManager;
49-
50-
import static org.apache.spark.unsafe.sort.UnsafeSorter.*;
45+
import org.apache.spark.unsafe.sort.ExternalSorterIterator;
46+
import org.apache.spark.unsafe.sort.UnsafeExternalSorter;
47+
import static org.apache.spark.unsafe.sort.UnsafeSorter.PrefixComparator;
48+
import static org.apache.spark.unsafe.sort.UnsafeSorter.RecordComparator;
5149

5250
// IntelliJ gets confused and claims that this class should be abstract, but this actually compiles
5351
public class UnsafeShuffleWriter<K, V> implements ShuffleWriter<K, V> {

core/src/main/java/org/apache/spark/unsafe/sort/UnsafeSorter.java

Lines changed: 1 addition & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -176,8 +176,7 @@ public void remove() {
176176

177177
public UnsafeSorterSpillMerger.MergeableIterator getMergeableIterator() {
178178
sorter.sort(sortBuffer, 0, sortBufferInsertPosition / 2, sortComparator);
179-
UnsafeSorterSpillMerger.MergeableIterator iter =
180-
new UnsafeSorterSpillMerger.MergeableIterator() {
179+
return new UnsafeSorterSpillMerger.MergeableIterator() {
181180

182181
private int position = 0;
183182
private Object baseObject;
@@ -213,6 +212,5 @@ public long getBaseOffset() {
213212
return baseOffset;
214213
}
215214
};
216-
return iter;
217215
}
218216
}

core/src/main/java/org/apache/spark/unsafe/sort/UnsafeSorterSpillMerger.java

Lines changed: 7 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,11 @@
1818
package org.apache.spark.unsafe.sort;
1919

2020
import java.util.Comparator;
21-
import java.util.Iterator;
2221
import java.util.PriorityQueue;
2322

2423
import static org.apache.spark.unsafe.sort.UnsafeSorter.*;
2524

26-
public final class UnsafeSorterSpillMerger {
25+
final class UnsafeSorterSpillMerger {
2726

2827
private final PriorityQueue<MergeableIterator> priorityQueue;
2928

@@ -39,13 +38,6 @@ public static abstract class MergeableIterator {
3938
public abstract long getBaseOffset();
4039
}
4140

42-
public static final class RecordAddressAndKeyPrefix {
43-
public Object baseObject;
44-
public long baseOffset;
45-
public int recordLength;
46-
public long keyPrefix;
47-
}
48-
4941
public UnsafeSorterSpillMerger(
5042
final RecordComparator recordComparator,
5143
final UnsafeSorter.PrefixComparator prefixComparator) {
@@ -74,37 +66,29 @@ public void addSpill(MergeableIterator spillReader) {
7466
priorityQueue.add(spillReader);
7567
}
7668

77-
public Iterator<RecordAddressAndKeyPrefix> getSortedIterator() {
78-
return new Iterator<RecordAddressAndKeyPrefix>() {
69+
public ExternalSorterIterator getSortedIterator() {
70+
return new ExternalSorterIterator() {
7971

8072
private MergeableIterator spillReader;
81-
private final RecordAddressAndKeyPrefix record = new RecordAddressAndKeyPrefix();
8273

8374
@Override
8475
public boolean hasNext() {
8576
return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext());
8677
}
8778

8879
@Override
89-
public RecordAddressAndKeyPrefix next() {
80+
public void loadNext() {
9081
if (spillReader != null) {
9182
if (spillReader.hasNext()) {
9283
spillReader.loadNextRecord();
9384
priorityQueue.add(spillReader);
9485
}
9586
}
9687
spillReader = priorityQueue.remove();
97-
record.baseObject = spillReader.getBaseObject();
98-
record.baseOffset = spillReader.getBaseOffset();
99-
record.keyPrefix = spillReader.getPrefix();
100-
return record;
101-
}
102-
103-
@Override
104-
public void remove() {
105-
throw new UnsupportedOperationException();
88+
baseObject = spillReader.getBaseObject();
89+
baseOffset = spillReader.getBaseOffset();
90+
keyPrefix = spillReader.getPrefix();
10691
}
10792
};
10893
}
109-
11094
}

core/src/main/java/org/apache/spark/unsafe/sort/UnsafeSorterSpillReader.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,14 @@
1717

1818
package org.apache.spark.unsafe.sort;
1919

20+
import java.io.*;
21+
2022
import com.google.common.io.ByteStreams;
23+
2124
import org.apache.spark.storage.BlockId;
2225
import org.apache.spark.storage.BlockManager;
2326
import org.apache.spark.unsafe.PlatformDependent;
2427

25-
import java.io.*;
26-
2728
final class UnsafeSorterSpillReader extends UnsafeSorterSpillMerger.MergeableIterator {
2829

2930
private final File file;

core/src/main/java/org/apache/spark/unsafe/sort/UnsafeSorterSpillWriter.java

Lines changed: 14 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,37 +17,38 @@
1717

1818
package org.apache.spark.unsafe.sort;
1919

20+
import java.io.*;
21+
import java.nio.ByteBuffer;
22+
23+
import scala.Tuple2;
24+
import scala.reflect.ClassTag;
25+
2026
import org.apache.spark.executor.ShuffleWriteMetrics;
2127
import org.apache.spark.serializer.DeserializationStream;
22-
import org.apache.spark.serializer.JavaSerializerInstance;
2328
import org.apache.spark.serializer.SerializationStream;
2429
import org.apache.spark.serializer.SerializerInstance;
2530
import org.apache.spark.storage.BlockId;
2631
import org.apache.spark.storage.BlockManager;
2732
import org.apache.spark.storage.BlockObjectWriter;
2833
import org.apache.spark.storage.TempLocalBlockId;
2934
import org.apache.spark.unsafe.PlatformDependent;
30-
import scala.Tuple2;
31-
import scala.reflect.ClassTag;
32-
33-
import java.io.*;
34-
import java.nio.ByteBuffer;
3535

3636
final class UnsafeSorterSpillWriter {
3737

3838
private static final int SER_BUFFER_SIZE = 1024 * 1024; // TODO: tune this
39-
public static final int EOF_MARKER = -1;
40-
byte[] arr = new byte[SER_BUFFER_SIZE];
39+
static final int EOF_MARKER = -1;
40+
41+
private byte[] arr = new byte[SER_BUFFER_SIZE];
4142

4243
private final File file;
4344
private final BlockId blockId;
44-
BlockObjectWriter writer;
45-
DataOutputStream dos;
45+
private BlockObjectWriter writer;
46+
private DataOutputStream dos;
4647

4748
public UnsafeSorterSpillWriter(
4849
BlockManager blockManager,
4950
int fileBufferSize,
50-
ShuffleWriteMetrics writeMetrics) throws IOException {
51+
ShuffleWriteMetrics writeMetrics) {
5152
final Tuple2<TempLocalBlockId, File> spilledFileInfo =
5253
blockManager.diskBlockManager().createTempLocalBlock();
5354
this.file = spilledFileInfo._2();
@@ -119,6 +120,8 @@ public void write(
119120
public void close() throws IOException {
120121
dos.writeInt(EOF_MARKER);
121122
writer.commitAndClose();
123+
writer = null;
124+
dos = null;
122125
arr = null;
123126
}
124127

0 commit comments

Comments
 (0)