Skip to content

Commit 3aeaff7

Browse files
committed
More refactoring and cleanup; begin cleaning iterator interfaces
1 parent 3490512 commit 3aeaff7

File tree

11 files changed

+197
-185
lines changed

11 files changed

+197
-185
lines changed

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

Lines changed: 11 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -42,10 +42,11 @@
4242
import org.apache.spark.storage.ShuffleBlockId;
4343
import org.apache.spark.unsafe.PlatformDependent;
4444
import org.apache.spark.unsafe.memory.TaskMemoryManager;
45-
import org.apache.spark.unsafe.sort.ExternalSorterIterator;
45+
import org.apache.spark.unsafe.sort.UnsafeSorterIterator;
4646
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;
47+
import org.apache.spark.unsafe.sort.PrefixComparator;
48+
49+
import org.apache.spark.unsafe.sort.RecordComparator;
4950

5051
// IntelliJ gets confused and claims that this class should be abstract, but this actually compiles
5152
public class UnsafeShuffleWriter<K, V> implements ShuffleWriter<K, V> {
@@ -104,7 +105,7 @@ private void freeMemory() {
104105
// TODO: free sorter memory
105106
}
106107

107-
private ExternalSorterIterator sortRecords(
108+
private UnsafeSorterIterator sortRecords(
108109
scala.collection.Iterator<? extends Product2<K, V>> records) throws Exception {
109110
final UnsafeExternalSorter sorter = new UnsafeExternalSorter(
110111
memoryManager,
@@ -142,7 +143,7 @@ private ExternalSorterIterator sortRecords(
142143
return sorter.getSortedIterator();
143144
}
144145

145-
private long[] writeSortedRecordsToFile(ExternalSorterIterator sortedRecords) throws IOException {
146+
private long[] writeSortedRecordsToFile(UnsafeSorterIterator sortedRecords) throws IOException {
146147
final File outputFile = shuffleBlockManager.getDataFile(shuffleId, mapId);
147148
final ShuffleBlockId blockId =
148149
new ShuffleBlockId(shuffleId, mapId, IndexShuffleBlockManager.NOOP_REDUCE_ID());
@@ -154,7 +155,7 @@ private long[] writeSortedRecordsToFile(ExternalSorterIterator sortedRecords) th
154155
final byte[] arr = new byte[SER_BUFFER_SIZE];
155156
while (sortedRecords.hasNext()) {
156157
sortedRecords.loadNext();
157-
final int partition = (int) sortedRecords.keyPrefix;
158+
final int partition = (int) sortedRecords.getKeyPrefix();
158159
assert (partition >= currentPartition);
159160
if (partition != currentPartition) {
160161
// Switch to the new partition
@@ -168,13 +169,13 @@ private long[] writeSortedRecordsToFile(ExternalSorterIterator sortedRecords) th
168169
}
169170

170171
PlatformDependent.copyMemory(
171-
sortedRecords.baseObject,
172-
sortedRecords.baseOffset + 4,
172+
sortedRecords.getBaseObject(),
173+
sortedRecords.getBaseOffset() + 4,
173174
arr,
174175
PlatformDependent.BYTE_ARRAY_OFFSET,
175-
sortedRecords.recordLength);
176+
sortedRecords.getRecordLength());
176177
assert (writer != null); // To suppress an IntelliJ warning
177-
writer.write(arr, 0, sortedRecords.recordLength);
178+
writer.write(arr, 0, sortedRecords.getRecordLength());
178179
// TODO: add a test that detects whether we leave this call out:
179180
writer.recordWritten();
180181
}

core/src/main/java/org/apache/spark/unsafe/sort/ExternalSorterIterator.java renamed to core/src/main/java/org/apache/spark/unsafe/sort/PrefixComparator.java

Lines changed: 6 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -17,15 +17,10 @@
1717

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

20-
public abstract class ExternalSorterIterator {
21-
22-
public Object baseObject;
23-
public long baseOffset;
24-
public int recordLength;
25-
public long keyPrefix;
26-
27-
public abstract boolean hasNext();
28-
29-
public abstract void loadNext();
30-
20+
/**
21+
* Compares 8-byte key prefixes in prefix sort. Subclasses may implement type-specific
22+
* comparisons, such as lexicographic comparison for strings.
23+
*/
24+
public abstract class PrefixComparator {
25+
public abstract int compare(long prefix1, long prefix2);
3126
}
Lines changed: 37 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,37 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.unsafe.sort;
19+
20+
/**
21+
* Compares records for ordering. In cases where the entire sorting key can fit in the 8-byte
22+
* prefix, this may simply return 0.
23+
*/
24+
public abstract class RecordComparator {
25+
26+
/**
27+
* Compare two records for order.
28+
*
29+
* @return a negative integer, zero, or a positive integer as the first record is less than,
30+
* equal to, or greater than the second.
31+
*/
32+
public abstract int compare(
33+
Object leftBaseObject,
34+
long leftBaseOffset,
35+
Object rightBaseObject,
36+
long rightBaseOffset);
37+
}

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

Lines changed: 10 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -33,8 +33,6 @@
3333
import java.util.Iterator;
3434
import java.util.LinkedList;
3535

36-
import static org.apache.spark.unsafe.sort.UnsafeSorter.*;
37-
3836
/**
3937
* External sorter based on {@link UnsafeSorter}.
4038
*/
@@ -111,13 +109,16 @@ public void spill() throws IOException {
111109
final UnsafeSorterSpillWriter spillWriter =
112110
new UnsafeSorterSpillWriter(blockManager, fileBufferSize, writeMetrics);
113111
spillWriters.add(spillWriter);
114-
final Iterator<RecordPointerAndKeyPrefix> sortedRecords = sorter.getSortedIterator();
112+
final UnsafeSorterIterator sortedRecords = sorter.getSortedIterator();
115113
while (sortedRecords.hasNext()) {
116-
final RecordPointerAndKeyPrefix recordPointer = sortedRecords.next();
117-
final Object baseObject = memoryManager.getPage(recordPointer.recordPointer);
118-
final long baseOffset = memoryManager.getOffsetInPage(recordPointer.recordPointer);
114+
sortedRecords.loadNext();
115+
final Object baseObject = sortedRecords.getBaseObject();
116+
final long baseOffset = sortedRecords.getBaseOffset();
117+
// TODO: this assumption that the first long holds a length is not enforced via our interfaces
118+
// We need to either always store this via the write path (e.g. not require the caller to do
119+
// it), or provide interfaces / hooks for customizing the physical storage format etc.
119120
final int recordLength = (int) PlatformDependent.UNSAFE.getLong(baseObject, baseOffset);
120-
spillWriter.write(baseObject, baseOffset, recordLength, recordPointer.keyPrefix);
121+
spillWriter.write(baseObject, baseOffset, recordLength, sortedRecords.getKeyPrefix());
121122
}
122123
spillWriter.close();
123124
final long sorterMemoryUsage = sorter.getMemoryUsage();
@@ -220,14 +221,14 @@ public void insertRecord(
220221
sorter.insertRecord(recordAddress, prefix);
221222
}
222223

223-
public ExternalSorterIterator getSortedIterator() throws IOException {
224+
public UnsafeSorterIterator getSortedIterator() throws IOException {
224225
final UnsafeSorterSpillMerger spillMerger =
225226
new UnsafeSorterSpillMerger(recordComparator, prefixComparator);
226227
for (UnsafeSorterSpillWriter spillWriter : spillWriters) {
227228
spillMerger.addSpill(spillWriter.getReader(blockManager));
228229
}
229230
spillWriters.clear();
230-
spillMerger.addSpill(sorter.getMergeableIterator());
231+
spillMerger.addSpill(sorter.getSortedIterator());
231232
return spillMerger.getSortedIterator();
232233
}
233234
}

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

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,8 @@
1717

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

20-
import static org.apache.spark.unsafe.sort.UnsafeSorter.RecordPointerAndKeyPrefix;
2120
import org.apache.spark.util.collection.SortDataFormat;
21+
import static org.apache.spark.unsafe.sort.UnsafeSortDataFormat.RecordPointerAndKeyPrefix;
2222

2323
/**
2424
* Supports sorting an array of (record pointer, key prefix) pairs. Used in {@link UnsafeSorter}.
@@ -28,6 +28,19 @@
2828
*/
2929
final class UnsafeSortDataFormat extends SortDataFormat<RecordPointerAndKeyPrefix, long[]> {
3030

31+
static final class RecordPointerAndKeyPrefix {
32+
/**
33+
* A pointer to a record; see {@link org.apache.spark.unsafe.memory.TaskMemoryManager} for a
34+
* description of how these addresses are encoded.
35+
*/
36+
public long recordPointer;
37+
38+
/**
39+
* A key prefix, for use in comparisons.
40+
*/
41+
public long keyPrefix;
42+
}
43+
3144
public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat();
3245

3346
private UnsafeSortDataFormat() { }

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

Lines changed: 13 additions & 80 deletions
Original file line numberDiff line numberDiff line change
@@ -18,10 +18,10 @@
1818
package org.apache.spark.unsafe.sort;
1919

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

2322
import org.apache.spark.util.collection.Sorter;
2423
import org.apache.spark.unsafe.memory.TaskMemoryManager;
24+
import static org.apache.spark.unsafe.sort.UnsafeSortDataFormat.RecordPointerAndKeyPrefix;
2525

2626
/**
2727
* Sorts records using an AlphaSort-style key-prefix sort. This sort stores pointers to records
@@ -32,45 +32,6 @@
3232
*/
3333
public final class UnsafeSorter {
3434

35-
public static final class RecordPointerAndKeyPrefix {
36-
/**
37-
* A pointer to a record; see {@link org.apache.spark.unsafe.memory.TaskMemoryManager} for a
38-
* description of how these addresses are encoded.
39-
*/
40-
public long recordPointer;
41-
42-
/**
43-
* A key prefix, for use in comparisons.
44-
*/
45-
public long keyPrefix;
46-
}
47-
48-
/**
49-
* Compares records for ordering. In cases where the entire sorting key can fit in the 8-byte
50-
* prefix, this may simply return 0.
51-
*/
52-
public static abstract class RecordComparator {
53-
/**
54-
* Compare two records for order.
55-
*
56-
* @return a negative integer, zero, or a positive integer as the first record is less than,
57-
* equal to, or greater than the second.
58-
*/
59-
public abstract int compare(
60-
Object leftBaseObject,
61-
long leftBaseOffset,
62-
Object rightBaseObject,
63-
long rightBaseOffset);
64-
}
65-
66-
/**
67-
* Compares 8-byte key prefixes in prefix sort. Subclasses may implement type-specific
68-
* comparisons, such as lexicographic comparison for strings.
69-
*/
70-
public static abstract class PrefixComparator {
71-
public abstract int compare(long prefix1, long prefix2);
72-
}
73-
7435
private final TaskMemoryManager memoryManager;
7536
private final Sorter<RecordPointerAndKeyPrefix, long[]> sorter;
7637
private final Comparator<RecordPointerAndKeyPrefix> sortComparator;
@@ -148,69 +109,41 @@ public void insertRecord(long objectAddress, long keyPrefix) {
148109
* Return an iterator over record pointers in sorted order. For efficiency, all calls to
149110
* {@code next()} will return the same mutable object.
150111
*/
151-
public Iterator<RecordPointerAndKeyPrefix> getSortedIterator() {
152-
sorter.sort(sortBuffer, 0, sortBufferInsertPosition / 2, sortComparator);
153-
return new Iterator<RecordPointerAndKeyPrefix>() {
154-
private int position = 0;
155-
private final RecordPointerAndKeyPrefix keyPointerAndPrefix = new RecordPointerAndKeyPrefix();
156-
157-
@Override
158-
public boolean hasNext() {
159-
return position < sortBufferInsertPosition;
160-
}
161-
162-
@Override
163-
public RecordPointerAndKeyPrefix next() {
164-
keyPointerAndPrefix.recordPointer = sortBuffer[position];
165-
keyPointerAndPrefix.keyPrefix = sortBuffer[position + 1];
166-
position += 2;
167-
return keyPointerAndPrefix;
168-
}
169-
170-
@Override
171-
public void remove() {
172-
throw new UnsupportedOperationException();
173-
}
174-
};
175-
}
176-
177-
public UnsafeSorterSpillMerger.MergeableIterator getMergeableIterator() {
112+
public UnsafeSorterIterator getSortedIterator() {
178113
sorter.sort(sortBuffer, 0, sortBufferInsertPosition / 2, sortComparator);
179-
return new UnsafeSorterSpillMerger.MergeableIterator() {
114+
return new UnsafeSorterIterator() {
180115

181116
private int position = 0;
182117
private Object baseObject;
183118
private long baseOffset;
184119
private long keyPrefix;
120+
private int recordLength;
185121

186122
@Override
187123
public boolean hasNext() {
188124
return position < sortBufferInsertPosition;
189125
}
190126

191127
@Override
192-
public void loadNextRecord() {
128+
public void loadNext() {
193129
final long recordPointer = sortBuffer[position];
194-
keyPrefix = sortBuffer[position + 1];
195-
position += 2;
196130
baseObject = memoryManager.getPage(recordPointer);
197131
baseOffset = memoryManager.getOffsetInPage(recordPointer);
132+
keyPrefix = sortBuffer[position + 1];
133+
position += 2;
198134
}
199135

200136
@Override
201-
public long getPrefix() {
202-
return keyPrefix;
203-
}
137+
public Object getBaseObject() { return baseObject; }
204138

205139
@Override
206-
public Object getBaseObject() {
207-
return baseObject;
208-
}
140+
public long getBaseOffset() { return baseOffset; }
209141

210142
@Override
211-
public long getBaseOffset() {
212-
return baseOffset;
213-
}
143+
public int getRecordLength() { return recordLength; }
144+
145+
@Override
146+
public long getKeyPrefix() { return keyPrefix; }
214147
};
215148
}
216149
}
Lines changed: 35 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,35 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.unsafe.sort;
19+
20+
import java.io.IOException;
21+
22+
public abstract class UnsafeSorterIterator {
23+
24+
public abstract boolean hasNext();
25+
26+
public abstract void loadNext() throws IOException;
27+
28+
public abstract Object getBaseObject();
29+
30+
public abstract long getBaseOffset();
31+
32+
public abstract int getRecordLength();
33+
34+
public abstract long getKeyPrefix();
35+
}

0 commit comments

Comments
 (0)