Skip to content

Commit ca65544

Browse files
author
jinxing
committed
remove spark.shuffle.accurateBlockThresholdByTimesAverage
1 parent bfea9f5 commit ca65544

File tree

4 files changed

+21
-78
lines changed

4 files changed

+21
-78
lines changed

core/src/main/scala/org/apache/spark/internal/config/package.scala

Lines changed: 2 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -282,18 +282,9 @@ package object config {
282282
private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD =
283283
ConfigBuilder("spark.shuffle.accurateBlockThreshold")
284284
.doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will " +
285-
"record the size accurately if it's above this config and " +
286-
"spark.shuffle.accurateBlockThresholdByTimesAverage * averageSize. This helps to prevent" +
287-
" OOM by avoiding underestimating shuffle block size when fetch shuffle blocks.")
285+
"record the size accurately if it's above this config. This helps to prevent OOM by " +
286+
"avoiding underestimating shuffle block size when fetch shuffle blocks.")
288287
.bytesConf(ByteUnit.BYTE)
289288
.createWithDefault(100 * 1024 * 1024)
290289

291-
private[spark] val SHUFFLE_ACCURATE_BLOCK_THRESHOLD_BY_TIMES_AVERAGE =
292-
ConfigBuilder("spark.shuffle.accurateBlockThresholdByTimesAverage")
293-
.doc("When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will " +
294-
"record the size accurately if it's above this config * averageSize and " +
295-
"spark.shuffle.accurateBlockThreshold. This helps to prevent OOM by avoiding " +
296-
"underestimating shuffle block size when fetch shuffle blocks.")
297-
.intConf
298-
.createWithDefault(2)
299290
}

core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala

Lines changed: 16 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -127,14 +127,13 @@ private[spark] class CompressedMapStatus(
127127

128128
/**
129129
* A [[MapStatus]] implementation that stores the accurate size of huge blocks, which are larger
130-
* than both spark.shuffle.accurateBlockThreshold and
131-
* spark.shuffle.accurateBlockThresholdByTimesAverage * averageSize. It stores the
132-
* average size of other non-empty blocks, plus a bitmap for tracking which blocks are empty.
130+
* than both spark.shuffle.accurateBlockThreshold. It stores the average size of other non-empty
131+
* blocks, plus a bitmap for tracking which blocks are empty.
133132
*
134133
* @param loc location where the task is being executed
135134
* @param numNonEmptyBlocks the number of non-empty blocks
136135
* @param emptyBlocks a bitmap tracking which blocks are empty
137-
* @param avgSize average size of the non-empty blocks
136+
* @param avgSize average size of the non-empty and non-huge blocks
138137
* @param hugeBlockSizes sizes of huge blocks by their reduceId.
139138
*/
140139
private[spark] class HighlyCompressedMapStatus private (
@@ -146,7 +145,7 @@ private[spark] class HighlyCompressedMapStatus private (
146145
extends MapStatus with Externalizable {
147146

148147
// loc could be null when the default constructor is called during deserialization
149-
require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0,
148+
require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0,
150149
"Average size can only be zero for map stages that produced no output")
151150

152151
protected def this() = this(null, -1, null, -1, null) // For deserialization only
@@ -204,11 +203,21 @@ private[spark] object HighlyCompressedMapStatus {
204203
// we expect that there will be far fewer of them, so we will perform fewer bitmap insertions.
205204
val emptyBlocks = new RoaringBitmap()
206205
val totalNumBlocks = uncompressedSizes.length
206+
val threshold = Option(SparkEnv.get)
207+
.map(_.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD))
208+
.getOrElse(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.defaultValue.get)
209+
val hugeBlockSizesArray = ArrayBuffer[Tuple2[Int, Byte]]()
207210
while (i < totalNumBlocks) {
208-
var size = uncompressedSizes(i)
211+
val size = uncompressedSizes(i)
209212
if (size > 0) {
210213
numNonEmptyBlocks += 1
211-
totalSize += size
214+
// Remove the huge blocks from the calculation for average size and have accurate size for
215+
// smaller blocks.
216+
if (size < threshold) {
217+
totalSize += size
218+
} else {
219+
hugeBlockSizesArray += Tuple2(i, MapStatus.compressSize(uncompressedSizes(i)))
220+
}
212221
} else {
213222
emptyBlocks.add(i)
214223
}
@@ -219,24 +228,6 @@ private[spark] object HighlyCompressedMapStatus {
219228
} else {
220229
0
221230
}
222-
val threshold1 = Option(SparkEnv.get)
223-
.map(_.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD))
224-
.getOrElse(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.defaultValue.get)
225-
val threshold2 = avgSize * Option(SparkEnv.get)
226-
.map(_.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD_BY_TIMES_AVERAGE))
227-
.getOrElse(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD_BY_TIMES_AVERAGE.defaultValue.get)
228-
val threshold = math.max(threshold1, threshold2)
229-
val hugeBlockSizesArray = ArrayBuffer[Tuple2[Int, Byte]]()
230-
if (numNonEmptyBlocks > 0) {
231-
i = 0
232-
while (i < totalNumBlocks) {
233-
if (uncompressedSizes(i) > threshold) {
234-
hugeBlockSizesArray += Tuple2(i, MapStatus.compressSize(uncompressedSizes(i)))
235-
236-
}
237-
i += 1
238-
}
239-
}
240231
emptyBlocks.trim()
241232
emptyBlocks.runOptimize()
242233
new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize,

core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala

Lines changed: 1 addition & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -133,37 +133,9 @@ class MapStatusSuite extends SparkFunSuite {
133133
assert(!success)
134134
}
135135

136-
test("When SHUFFLE_ACCURATE_BLOCK_THRESHOLD is 0, blocks which are bigger than " +
137-
"SHUFFLE_ACCURATE_BLOCK_THRESHOLD_BY_TIMES_AVERAGE * averageSize should not be " +
136+
test("Blocks which are bigger than SHUFFLE_ACCURATE_BLOCK_THRESHOLD should not be " +
138137
"underestimated.") {
139-
val conf = new SparkConf().set(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.key, "0")
140-
.set(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD_BY_TIMES_AVERAGE.key, "2")
141-
val env = mock(classOf[SparkEnv])
142-
doReturn(conf).when(env).conf
143-
SparkEnv.set(env)
144-
// Value of element in sizes is equal to the corresponding index when index >= 1000.
145-
val sizes = Array.concat(Array.fill[Long](1000)(1L), (1000L to 2000L).toArray)
146-
val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes)
147-
val arrayStream = new ByteArrayOutputStream(102400)
148-
val objectOutputStream = new ObjectOutputStream(arrayStream)
149-
assert(status1.isInstanceOf[HighlyCompressedMapStatus])
150-
objectOutputStream.writeObject(status1)
151-
objectOutputStream.flush()
152-
val array = arrayStream.toByteArray
153-
val objectInput = new ObjectInputStream(new ByteArrayInputStream(array))
154-
val status2 = objectInput.readObject().asInstanceOf[HighlyCompressedMapStatus]
155-
val avg = sizes.sum / 2001
156-
((2 * avg + 1) to 2000).foreach {
157-
case part =>
158-
assert(status2.getSizeForBlock(part.toInt) >= sizes(part.toInt))
159-
}
160-
}
161-
162-
test("When SHUFFLE_ACCURATE_BLOCK_THRESHOLD_BY_TIMES_AVERAGE is 0, blocks which are bigger than" +
163-
" SHUFFLE_ACCURATE_BLOCK_THRESHOLD should not be underestimated.")
164-
{
165138
val conf = new SparkConf().set(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.key, "1000")
166-
.set(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD_BY_TIMES_AVERAGE.key, "0")
167139
val env = mock(classOf[SparkEnv])
168140
doReturn(conf).when(env).conf
169141
SparkEnv.set(env)

docs/configuration.md

Lines changed: 2 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -617,21 +617,10 @@ Apart from these, the following properties are also available, and may be useful
617617
<td>100 * 1024 * 1024</td>
618618
<td>
619619
When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will record the
620-
size accurately if it's above this config and
621-
<code>spark.shuffle.accurateBlockThresholdByTimesAverage</code> * averageSize. This helps to
622-
prevent OOM by avoiding underestimating shuffle block size when fetch shuffle blocks.
623-
</td>
624-
</tr>
625-
<tr>
626-
<td><code>spark.shuffle.accurateBlockThresholdByTimesAverage</code></td>
627-
<td>2</td>
628-
<td>
629-
When we compress the size of shuffle blocks in HighlyCompressedMapStatus, we will record the
630-
size accurately if it's above this config * averageSize and
631-
<code>spark.shuffle.accurateBlockThreshold</code>. This helps to prevent OOM by avoiding
620+
size accurately if it's above this config. This helps to prevent OOM by avoiding
632621
underestimating shuffle block size when fetch shuffle blocks.
633622
</td>
634-
<tr>
623+
</tr>
635624
<tr>
636625
<td><code>spark.io.encryption.enabled</code></td>
637626
<td>false</td>

0 commit comments

Comments
 (0)