From 91e338b52c911366b9e6cd209c8f09ad04a2acca Mon Sep 17 00:00:00 2001 From: jinxing Date: Mon, 13 Mar 2017 21:35:51 +0800 Subject: [PATCH 01/28] Collect metrics of block sizes from executor. --- .../sort/BypassMergeSortShuffleWriter.java | 13 +++++ .../shuffle/sort/UnsafeShuffleWriter.java | 12 ++++ .../apache/spark/InternalAccumulator.scala | 3 + .../spark/executor/ShuffleWriteMetrics.scala | 55 ++++++++++++++++++- .../apache/spark/executor/TaskMetrics.scala | 11 +++- .../apache/spark/scheduler/MapStatus.scala | 2 + .../shuffle/sort/SortShuffleWriter.scala | 8 ++- .../spark/ui/jobs/JobProgressListener.scala | 14 +++++ .../spark/executor/TaskMetricsSuite.scala | 7 +++ 9 files changed, 122 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 4a15559e55cb..a6070da5d15b 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -39,6 +39,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.scheduler.HighlyCompressedMapStatus; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.Serializer; @@ -169,6 +170,18 @@ public void write(Iterator> records) throws IOException { } } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + + long maxBlockSize = 0L; + for (long partitionLength: partitionLengths) { + writeMetrics.incBlockSizeDistribution(partitionLength); + if (partitionLength > maxBlockSize) { + maxBlockSize = partitionLength; + } + } + if (mapStatus instanceof HighlyCompressedMapStatus) { + writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); + writeMetrics.setMaxBlockSize(maxBlockSize); + } } @VisibleForTesting diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 8a1771848dee..7eb600f646ee 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -44,6 +44,7 @@ import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; +import org.apache.spark.scheduler.HighlyCompressedMapStatus; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.SerializationStream; @@ -228,6 +229,17 @@ void closeAndWriteOutput() throws IOException { } } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + long maxBlockSize = 0L; + for (long partitionLength: partitionLengths) { + writeMetrics.incBlockSizeDistribution(partitionLength); + if (partitionLength > maxBlockSize) { + maxBlockSize = partitionLength; + } + } + if (mapStatus instanceof HighlyCompressedMapStatus) { + writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); + writeMetrics.setMaxBlockSize(maxBlockSize); + } } @VisibleForTesting diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index 82d3098e2e05..fb65be54774d 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -60,6 +60,9 @@ private[spark] object InternalAccumulator { val BYTES_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "bytesWritten" val RECORDS_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "recordsWritten" val WRITE_TIME = SHUFFLE_WRITE_METRICS_PREFIX + "writeTime" + val BLOCK_SIZE_DISTRIBUTION_PREFIX = SHUFFLE_WRITE_METRICS_PREFIX + "blockSizeDistribution." + val AVERAGE_BLOCK_SIZE = SHUFFLE_WRITE_METRICS_PREFIX + "averageBlockSize" + val MAX_BLOCK_SIZE = SHUFFLE_WRITE_METRICS_PREFIX + "maxBlockSize" } // Names of output metrics diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index ada2e1bc0859..689e57540946 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,8 +17,12 @@ package org.apache.spark.executor +import java.{lang => jl} + +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.LongAccumulator +import org.apache.spark.util.{CollectionAccumulator, LongAccumulator} /** @@ -31,6 +35,12 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { private[executor] val _bytesWritten = new LongAccumulator private[executor] val _recordsWritten = new LongAccumulator private[executor] val _writeTime = new LongAccumulator + private[executor] val _blockSizeDistribution = new Array[LongAccumulator](9) + (0 until 9).foreach { + case i => _blockSizeDistribution(i) = new LongAccumulator + } + private[executor] val _averageBlockSize = new LongAccumulator + private[executor] val _maxBlockSize = new LongAccumulator /** * Number of bytes written for the shuffle by this task. @@ -47,6 +57,26 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { */ def writeTime: Long = _writeTime.sum + /** + * Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), + * [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). + */ + def blockSizeDistribution: Seq[jl.Long] = { + _blockSizeDistribution.map(_.value).toSeq + } + + /** + * The average size of blocks in HighlyCompressedMapStatus. + * This is not set if CompressedMapStatus is returned. + */ + def averageBlockSize: Long = _averageBlockSize.value + + /** + * The max size of blocks in HighlyCompressedMapStatus. + * This is not set if CompressedMapStatus is returned. + */ + def maxBlockSize: Long = _maxBlockSize.value + private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) private[spark] def incWriteTime(v: Long): Unit = _writeTime.add(v) @@ -57,6 +87,29 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { _recordsWritten.setValue(recordsWritten - v) } + private[spark] def incBlockSizeDistribution(len: Long): Unit = { + len match { + case len: Long if len >= 0L && len < 1024L => _blockSizeDistribution(0).add(1) + case len: Long if len >= 1024L && len < 10240L => _blockSizeDistribution(1).add(1) + case len: Long if len >= 10240L && len < 102400L => _blockSizeDistribution(2).add(1) + case len: Long if len >= 102400L && len < 1048576L => _blockSizeDistribution(3).add(1) + case len: Long if len >= 1048576L && len < 10485760L => _blockSizeDistribution(4).add(1) + case len: Long if len >= 10485760L && len < 104857600L => _blockSizeDistribution(5).add(1) + case len: Long if len >= 104857600L && len < 1073741824L => _blockSizeDistribution(6).add(1) + case len: Long if len >= 1073741824L && len < 10737418240L => _blockSizeDistribution(7).add(1) + case len: Long if len >= 10737418240L => _blockSizeDistribution(8).add(1) + + } + } + + private[spark] def setAverageBlockSize(avg: Long): Unit = { + _averageBlockSize.setValue(avg) + } + + private[spark] def setMaxBlockSize(avg: Long): Unit = { + _maxBlockSize.setValue(avg) + } + // Legacy methods for backward compatibility. // TODO: remove these once we make this class private. @deprecated("use bytesWritten instead", "2.0.0") diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index dfd2f818acda..f3b46ebab734 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -221,11 +221,20 @@ class TaskMetrics private[spark] () extends Serializable { shuffleWrite.BYTES_WRITTEN -> shuffleWriteMetrics._bytesWritten, shuffleWrite.RECORDS_WRITTEN -> shuffleWriteMetrics._recordsWritten, shuffleWrite.WRITE_TIME -> shuffleWriteMetrics._writeTime, + shuffleWrite.AVERAGE_BLOCK_SIZE -> shuffleWriteMetrics._averageBlockSize, + shuffleWrite.MAX_BLOCK_SIZE -> shuffleWriteMetrics._maxBlockSize, input.BYTES_READ -> inputMetrics._bytesRead, input.RECORDS_READ -> inputMetrics._recordsRead, output.BYTES_WRITTEN -> outputMetrics._bytesWritten, output.RECORDS_WRITTEN -> outputMetrics._recordsWritten - ) ++ testAccum.map(TEST_ACCUM -> _) + ) ++ testAccum.map(TEST_ACCUM -> _) ++ blockSizeDistributionAccums + + def blockSizeDistributionAccums(): Map[String, AccumulatorV2[_, _]] = { + shuffleWriteMetrics._blockSizeDistribution.zipWithIndex.map { + case (accum, index) => + (shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX + index, accum) + }.toMap + } @transient private[spark] lazy val internalAccums: Seq[AccumulatorV2[_, _]] = nameToAccums.values.toIndexedSeq diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index b2e9a97129f0..dbc0ca66dd8a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -164,6 +164,8 @@ private[spark] class HighlyCompressedMapStatus private ( emptyBlocks.readExternal(in) avgSize = in.readLong() } + + def getAvgSize(): Long = avgSize } private[spark] object HighlyCompressedMapStatus { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 636b88e792bf..1b28ee022c68 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -19,7 +19,7 @@ package org.apache.spark.shuffle.sort import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.scheduler.{HighlyCompressedMapStatus, MapStatus} import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.Utils @@ -72,6 +72,12 @@ private[spark] class SortShuffleWriter[K, V, C]( val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + partitionLengths.foreach(writeMetrics.incBlockSizeDistribution(_)) + if (mapStatus.isInstanceOf[HighlyCompressedMapStatus]) { + writeMetrics.setAverageBlockSize( + mapStatus.asInstanceOf[HighlyCompressedMapStatus].getAvgSize()); + writeMetrics.setMaxBlockSize(partitionLengths.max) + } } finally { if (tmp.exists() && !tmp.delete()) { logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index e87caff42643..155d22e5cca4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -400,6 +400,20 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskMetrics.foreach { m => val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) + + if (m.shuffleWriteMetrics.averageBlockSize == 0) { + logInfo(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + + s"the block sizes in MapStatus are accurate, distribution is:") + } else { + logInfo(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + + s"the block sizes in MapStatus are inaccurate(average=" + + s"${m.shuffleWriteMetrics.averageBlockSize}, " + + s"max=${m.shuffleWriteMetrics.maxBlockSize}), distribution is:") + } + val ranges = List[String]("[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", + "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g") + logDebug(ranges.map("%11s".format(_)).mkString(" ")) + logDebug(m.shuffleWriteMetrics.blockSizeDistribution.map("%11d".format(_)).mkString(" ")) } val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index eae26fa742a2..6dfe7039f423 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -133,10 +133,17 @@ class TaskMetricsSuite extends SparkFunSuite { sw.decRecordsWritten(2L) sw.incWriteTime(300L) sw.incWriteTime(30L) + sw.incBlockSizeDistribution(10) + sw.incBlockSizeDistribution(2048) + sw.setAverageBlockSize(1024) + sw.setMaxBlockSize(10240) // assert new values exist assert(sw.bytesWritten == 108L) assert(sw.recordsWritten == 216L) assert(sw.writeTime == 330L) + assert(sw.blockSizeDistribution(0) === 1 && sw.blockSizeDistribution(1) ===1) + assert(sw.averageBlockSize === 1024) + assert(sw.maxBlockSize === 10240) } test("mutating input metrics values") { From a95e75c31dd94960064244b4fcd79a7ca7cafb63 Mon Sep 17 00:00:00 2001 From: jinxing Date: Thu, 16 Mar 2017 08:48:38 +0800 Subject: [PATCH 02/28] change in JsonProtocolSuite --- .../apache/spark/util/JsonProtocolSuite.scala | 88 ++++++++++++++++++- 1 file changed, 84 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 9f76c74bce89..75ce7e32ded2 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -714,6 +714,9 @@ private[spark] object JsonProtocolSuite extends Assertions { val expectedJson = pretty(parse(expected)) val actualJson = pretty(parse(actual)) if (expectedJson != actualJson) { + println(expected) + println("=================================================") + println(actualJson) // scalastyle:off // This prints something useful if the JSON strings don't match println("=== EXPECTED ===\n" + expectedJson + "\n") @@ -1956,38 +1959,115 @@ private[spark] object JsonProtocolSuite extends Assertions { | }, | { | "ID": 20, + | "Name": "${shuffleWrite.AVERAGE_BLOCK_SIZE}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 21, + | "Name": "${shuffleWrite.MAX_BLOCK_SIZE}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 22, | "Name": "${input.BYTES_READ}", | "Update": 2100, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 21, + | "ID": 23, | "Name": "${input.RECORDS_READ}", | "Update": 21, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 22, + | "ID": 24, | "Name": "${output.BYTES_WRITTEN}", | "Update": 1200, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 23, + | "ID": 25, | "Name": "${output.RECORDS_WRITTEN}", | "Update": 12, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 24, + | "ID": 26, | "Name": "$TEST_ACCUM", | "Update": 0, | "Internal": true, | "Count Failed Values": true + | }, + | { + | "ID": 27, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.0", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 28, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.1", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 29, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.2", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 30, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.3", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 31, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.4", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 32, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.5", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 33, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.6", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 34, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.7", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 35, + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.8", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true | } | ] | } From 7cd290da23ce85440fb9bdef00ed87c317295a74 Mon Sep 17 00:00:00 2001 From: jinxing Date: Thu, 16 Mar 2017 14:48:55 +0800 Subject: [PATCH 03/28] fix JsonProtocolSuite. --- .../apache/spark/executor/TaskMetrics.scala | 10 +++++---- .../apache/spark/util/JsonProtocolSuite.scala | 21 ++++++++----------- 2 files changed, 15 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index f3b46ebab734..e8ea3bd8689a 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -229,11 +229,13 @@ class TaskMetrics private[spark] () extends Serializable { output.RECORDS_WRITTEN -> outputMetrics._recordsWritten ) ++ testAccum.map(TEST_ACCUM -> _) ++ blockSizeDistributionAccums - def blockSizeDistributionAccums(): Map[String, AccumulatorV2[_, _]] = { - shuffleWriteMetrics._blockSizeDistribution.zipWithIndex.map { + def blockSizeDistributionAccums(): LinkedHashMap[String, AccumulatorV2[_, _]] = { + val linkedHashMap = LinkedHashMap[String, AccumulatorV2[_, _]]() + shuffleWriteMetrics._blockSizeDistribution.zipWithIndex.foreach { case (accum, index) => - (shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX + index, accum) - }.toMap + linkedHashMap.put(shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX + index, accum) + } + linkedHashMap } @transient private[spark] lazy val internalAccums: Seq[AccumulatorV2[_, _]] = diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 75ce7e32ded2..9c4a67b633eb 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -714,9 +714,6 @@ private[spark] object JsonProtocolSuite extends Assertions { val expectedJson = pretty(parse(expected)) val actualJson = pretty(parse(actual)) if (expectedJson != actualJson) { - println(expected) - println("=================================================") - println(actualJson) // scalastyle:off // This prints something useful if the JSON strings don't match println("=== EXPECTED ===\n" + expectedJson + "\n") @@ -2008,63 +2005,63 @@ private[spark] object JsonProtocolSuite extends Assertions { | }, | { | "ID": 27, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.0", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}0", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 28, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.1", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}1", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 29, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.2", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}2", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 30, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.3", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}3", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 31, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.4", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}4", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 32, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.5", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}5", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 33, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.6", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}6", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 34, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.7", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}7", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 35, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}.8", + | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}8", | "Update": 0, | "Internal": true, | "Count Failed Values": true From b91ac1384775f8dbf8b2c3025e18b088a4bf1015 Mon Sep 17 00:00:00 2001 From: jinxing Date: Mon, 20 Mar 2017 14:39:37 +0800 Subject: [PATCH 04/28] Remove MAX_BLOCK_SIZE, add UNDERESTIMATED_BLOCKS_NUM and UNDERESTIMATED_BLOCKS_SIZE. --- .../sort/BypassMergeSortShuffleWriter.java | 7 +++- .../shuffle/sort/UnsafeShuffleWriter.java | 18 +++++++--- .../apache/spark/InternalAccumulator.scala | 3 +- .../spark/executor/ShuffleWriteMetrics.scala | 22 ++++++++---- .../apache/spark/executor/TaskMetrics.scala | 3 +- .../shuffle/sort/SortShuffleWriter.scala | 8 ++++- .../spark/ui/jobs/JobProgressListener.scala | 12 ++++--- .../spark/executor/TaskMetricsSuite.scala | 6 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 35 +++++++++++-------- 9 files changed, 77 insertions(+), 37 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index a6070da5d15b..ef21f2bb5f4e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -180,7 +180,12 @@ public void write(Iterator> records) throws IOException { } if (mapStatus instanceof HighlyCompressedMapStatus) { writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); - writeMetrics.setMaxBlockSize(maxBlockSize); + for (int i = 0; i < partitionLengths.length; i++) { + if (partitionLengths[i] < mapStatus.getSizeForBlock(i)) { + writeMetrics.incUnderestimatedBlocksNum(); + writeMetrics.incUnderestimatedBlocksSize(partitionLengths[i]); + } + } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 7eb600f646ee..58b0a85d82c4 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -17,10 +17,10 @@ package org.apache.spark.shuffle.sort; -import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; import java.util.Iterator; +import javax.annotation.Nullable; import scala.Option; import scala.Product2; @@ -35,13 +35,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.*; +import org.apache.commons.io.output.CloseShieldOutputStream; +import org.apache.commons.io.output.CountingOutputStream; +import org.apache.spark.Partitioner; +import org.apache.spark.ShuffleDependency; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; import org.apache.spark.annotation.Private; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; -import org.apache.commons.io.output.CloseShieldOutputStream; -import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.HighlyCompressedMapStatus; @@ -238,7 +241,12 @@ void closeAndWriteOutput() throws IOException { } if (mapStatus instanceof HighlyCompressedMapStatus) { writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); - writeMetrics.setMaxBlockSize(maxBlockSize); + for (int i = 0; i < partitionLengths.length; i++) { + if (partitionLengths[i] < mapStatus.getSizeForBlock(i)) { + writeMetrics.incUnderestimatedBlocksNum(); + writeMetrics.incUnderestimatedBlocksSize(partitionLengths[i]); + } + } } } diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index fb65be54774d..50c78614d6f5 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -62,7 +62,8 @@ private[spark] object InternalAccumulator { val WRITE_TIME = SHUFFLE_WRITE_METRICS_PREFIX + "writeTime" val BLOCK_SIZE_DISTRIBUTION_PREFIX = SHUFFLE_WRITE_METRICS_PREFIX + "blockSizeDistribution." val AVERAGE_BLOCK_SIZE = SHUFFLE_WRITE_METRICS_PREFIX + "averageBlockSize" - val MAX_BLOCK_SIZE = SHUFFLE_WRITE_METRICS_PREFIX + "maxBlockSize" + val UNDERESTIMATED_BLOCKS_NUM = SHUFFLE_WRITE_METRICS_PREFIX + "underestimatedBlocksNum" + val UNDERESTIMATED_BLOCKS_SIZE = SHUFFLE_WRITE_METRICS_PREFIX + "underestimatedBlocksSize" } // Names of output metrics diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 689e57540946..7e81caa1a7b5 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -40,7 +40,8 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { case i => _blockSizeDistribution(i) = new LongAccumulator } private[executor] val _averageBlockSize = new LongAccumulator - private[executor] val _maxBlockSize = new LongAccumulator + private[executor] val _underestimatedBlocksNum = new LongAccumulator + private[executor] val _underestimatedBlocksSize = new LongAccumulator /** * Number of bytes written for the shuffle by this task. @@ -72,10 +73,14 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { def averageBlockSize: Long = _averageBlockSize.value /** - * The max size of blocks in HighlyCompressedMapStatus. - * This is not set if CompressedMapStatus is returned. + * The num of blocks whose sizes are underestimated in MapStatus. + */ + def underestimatedBlocksNum: Long = _underestimatedBlocksNum.value + + /** + * The total amount of blocks whose sizes are underestimated in MapStatus. */ - def maxBlockSize: Long = _maxBlockSize.value + def underestimatedBlocksSize: Long = _underestimatedBlocksSize.value private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) @@ -98,7 +103,6 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { case len: Long if len >= 104857600L && len < 1073741824L => _blockSizeDistribution(6).add(1) case len: Long if len >= 1073741824L && len < 10737418240L => _blockSizeDistribution(7).add(1) case len: Long if len >= 10737418240L => _blockSizeDistribution(8).add(1) - } } @@ -106,8 +110,12 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { _averageBlockSize.setValue(avg) } - private[spark] def setMaxBlockSize(avg: Long): Unit = { - _maxBlockSize.setValue(avg) + private[spark] def incUnderestimatedBlocksNum() = { + _underestimatedBlocksNum.add(1) + } + + private[spark] def incUnderestimatedBlocksSize(v: Long) = { + _underestimatedBlocksSize.add(v) } // Legacy methods for backward compatibility. diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e8ea3bd8689a..36ac292938e5 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -222,7 +222,8 @@ class TaskMetrics private[spark] () extends Serializable { shuffleWrite.RECORDS_WRITTEN -> shuffleWriteMetrics._recordsWritten, shuffleWrite.WRITE_TIME -> shuffleWriteMetrics._writeTime, shuffleWrite.AVERAGE_BLOCK_SIZE -> shuffleWriteMetrics._averageBlockSize, - shuffleWrite.MAX_BLOCK_SIZE -> shuffleWriteMetrics._maxBlockSize, + shuffleWrite.UNDERESTIMATED_BLOCKS_NUM -> shuffleWriteMetrics._underestimatedBlocksNum, + shuffleWrite.UNDERESTIMATED_BLOCKS_SIZE -> shuffleWriteMetrics._underestimatedBlocksSize, input.BYTES_READ -> inputMetrics._bytesRead, input.RECORDS_READ -> inputMetrics._recordsRead, output.BYTES_WRITTEN -> outputMetrics._bytesWritten, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 1b28ee022c68..31f6e1f58c8f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -76,7 +76,13 @@ private[spark] class SortShuffleWriter[K, V, C]( if (mapStatus.isInstanceOf[HighlyCompressedMapStatus]) { writeMetrics.setAverageBlockSize( mapStatus.asInstanceOf[HighlyCompressedMapStatus].getAvgSize()); - writeMetrics.setMaxBlockSize(partitionLengths.max) + (0 until partitionLengths.length).foreach { + case i => + if (partitionLengths(i) < mapStatus.getSizeForBlock(i)) { + writeMetrics.incUnderestimatedBlocksNum() + writeMetrics.incUnderestimatedBlocksSize(partitionLengths(i)) + } + } } } finally { if (tmp.exists() && !tmp.delete()) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 155d22e5cca4..1e2940eff1c7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -402,13 +402,15 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) if (m.shuffleWriteMetrics.averageBlockSize == 0) { - logInfo(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + + logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + s"the block sizes in MapStatus are accurate, distribution is:") } else { - logInfo(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + - s"the block sizes in MapStatus are inaccurate(average=" + - s"${m.shuffleWriteMetrics.averageBlockSize}, " + - s"max=${m.shuffleWriteMetrics.maxBlockSize}), distribution is:") + logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + + s"the block sizes in MapStatus are inaccurate" + + s"(average=${m.shuffleWriteMetrics.averageBlockSize}, " + + s"${m.shuffleWriteMetrics.underestimatedBlocksNum} blocks underestimated, " + + s"sum of sizes is ${m.shuffleWriteMetrics.underestimatedBlocksSize}), " + + s"distribution is:") } val ranges = List[String]("[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g") diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index 6dfe7039f423..f2fb9d1af696 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -136,14 +136,16 @@ class TaskMetricsSuite extends SparkFunSuite { sw.incBlockSizeDistribution(10) sw.incBlockSizeDistribution(2048) sw.setAverageBlockSize(1024) - sw.setMaxBlockSize(10240) + sw.incUnderestimatedBlocksNum() + sw.incUnderestimatedBlocksSize(1024) // assert new values exist assert(sw.bytesWritten == 108L) assert(sw.recordsWritten == 216L) assert(sw.writeTime == 330L) assert(sw.blockSizeDistribution(0) === 1 && sw.blockSizeDistribution(1) ===1) assert(sw.averageBlockSize === 1024) - assert(sw.maxBlockSize === 10240) + assert(sw.underestimatedBlocksNum === 1) + assert(sw.underestimatedBlocksSize === 1024) } test("mutating input metrics values") { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 9c4a67b633eb..d1002378507d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1963,104 +1963,111 @@ private[spark] object JsonProtocolSuite extends Assertions { | }, | { | "ID": 21, - | "Name": "${shuffleWrite.MAX_BLOCK_SIZE}", + | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_NUM}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 22, + | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_SIZE}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 23, | "Name": "${input.BYTES_READ}", | "Update": 2100, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 23, + | "ID": 24, | "Name": "${input.RECORDS_READ}", | "Update": 21, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 24, + | "ID": 25, | "Name": "${output.BYTES_WRITTEN}", | "Update": 1200, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 25, + | "ID": 26, | "Name": "${output.RECORDS_WRITTEN}", | "Update": 12, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 26, + | "ID": 27, | "Name": "$TEST_ACCUM", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 27, + | "ID": 28, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}0", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 28, + | "ID": 29, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}1", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 29, + | "ID": 30, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}2", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 30, + | "ID": 31, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}3", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 31, + | "ID": 32, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}4", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 32, + | "ID": 33, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}5", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 33, + | "ID": 34, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}6", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 34, + | "ID": 35, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}7", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 35, + | "ID": 36, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}8", | "Update": 0, | "Internal": true, From b1cd87946cdc5f1b4d0cf4eebbaf9976d68f0528 Mon Sep 17 00:00:00 2001 From: jinxing Date: Mon, 20 Mar 2017 15:18:33 +0800 Subject: [PATCH 05/28] wip --- .../apache/spark/InternalAccumulator.scala | 2 ++ .../spark/executor/ShuffleReadMetrics.scala | 26 +++++++++++++++++++ .../apache/spark/executor/TaskMetrics.scala | 2 ++ .../spark/executor/TaskMetricsSuite.scala | 6 +++++ 4 files changed, 36 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index 50c78614d6f5..fdf07feb3d6c 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -50,6 +50,8 @@ private[spark] object InternalAccumulator { val REMOTE_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "remoteBlocksFetched" val LOCAL_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "localBlocksFetched" val REMOTE_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesRead" + val REMOTE_BYTES_READ_TO_MEM = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesReadToMem" + val REMOTE_BYTES_READ_TO_DISK = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesReadToDisk" val LOCAL_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "localBytesRead" val FETCH_WAIT_TIME = SHUFFLE_READ_METRICS_PREFIX + "fetchWaitTime" val RECORDS_READ = SHUFFLE_READ_METRICS_PREFIX + "recordsRead" diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 8dd1a1ea059b..17fd5acc58ca 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -31,6 +31,8 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[executor] val _remoteBlocksFetched = new LongAccumulator private[executor] val _localBlocksFetched = new LongAccumulator private[executor] val _remoteBytesRead = new LongAccumulator + private[executor] val _remoteBytesReadToMem = new LongAccumulator + private[executor] val _remoteBytesReadToDisk = new LongAccumulator private[executor] val _localBytesRead = new LongAccumulator private[executor] val _fetchWaitTime = new LongAccumulator private[executor] val _recordsRead = new LongAccumulator @@ -50,6 +52,16 @@ class ShuffleReadMetrics private[spark] () extends Serializable { */ def remoteBytesRead: Long = _remoteBytesRead.sum + /** + * Total number of remotes bytes read to memory from the shuffle by this task. + */ + def remoteBytesReadToMem: Long = _remoteBytesReadToMem.sum + + /** + * Total number of remotes bytes read to disk from the shuffle by this task. + */ + def remoteBytesReadToDisk: Long = _remoteBytesReadToDisk.sum + /** * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ @@ -80,6 +92,8 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[spark] def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched.add(v) private[spark] def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) + private[spark] def incRemoteBytesReadToMem(v: Long): Unit = _remoteBytesReadToMem.add(v) + private[spark] def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.add(v) private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) @@ -87,6 +101,8 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[spark] def setRemoteBlocksFetched(v: Int): Unit = _remoteBlocksFetched.setValue(v) private[spark] def setLocalBlocksFetched(v: Int): Unit = _localBlocksFetched.setValue(v) private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) + private[spark] def setRemoteBytesReadToMem(v: Long): Unit = _remoteBytesReadToMem.setValue(v) + private[spark] def setRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.setValue(v) private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) @@ -99,6 +115,8 @@ class ShuffleReadMetrics private[spark] () extends Serializable { _remoteBlocksFetched.setValue(0) _localBlocksFetched.setValue(0) _remoteBytesRead.setValue(0) + _remoteBytesReadToMem.setValue(0) + _remoteBytesReadToDisk.setValue(0) _localBytesRead.setValue(0) _fetchWaitTime.setValue(0) _recordsRead.setValue(0) @@ -106,6 +124,8 @@ class ShuffleReadMetrics private[spark] () extends Serializable { _remoteBlocksFetched.add(metric.remoteBlocksFetched) _localBlocksFetched.add(metric.localBlocksFetched) _remoteBytesRead.add(metric.remoteBytesRead) + _remoteBytesReadToMem.add(metric.remoteBytesReadToMem) + _remoteBytesReadToDisk.add(metric.remoteBytesReadToDisk) _localBytesRead.add(metric.localBytesRead) _fetchWaitTime.add(metric.fetchWaitTime) _recordsRead.add(metric.recordsRead) @@ -122,6 +142,8 @@ private[spark] class TempShuffleReadMetrics { private[this] var _remoteBlocksFetched = 0L private[this] var _localBlocksFetched = 0L private[this] var _remoteBytesRead = 0L + private[this] var _remoteBytesReadToMem = 0L + private[this] var _remoteBytesReadToDisk = 0L private[this] var _localBytesRead = 0L private[this] var _fetchWaitTime = 0L private[this] var _recordsRead = 0L @@ -129,6 +151,8 @@ private[spark] class TempShuffleReadMetrics { def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched += v def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched += v def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead += v + def incRemoteBytesReadToMem(v: Long): Unit = _remoteBytesReadToMem += v + def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk += v def incLocalBytesRead(v: Long): Unit = _localBytesRead += v def incFetchWaitTime(v: Long): Unit = _fetchWaitTime += v def incRecordsRead(v: Long): Unit = _recordsRead += v @@ -136,6 +160,8 @@ private[spark] class TempShuffleReadMetrics { def remoteBlocksFetched: Long = _remoteBlocksFetched def localBlocksFetched: Long = _localBlocksFetched def remoteBytesRead: Long = _remoteBytesRead + def remoteBytesReadToMem: Long = _remoteBytesReadToMem + def remoteBytesReadToDisk: Long = _remoteBytesReadToDisk def localBytesRead: Long = _localBytesRead def fetchWaitTime: Long = _fetchWaitTime def recordsRead: Long = _recordsRead diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 36ac292938e5..d22f48301695 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -215,6 +215,8 @@ class TaskMetrics private[spark] () extends Serializable { shuffleRead.REMOTE_BLOCKS_FETCHED -> shuffleReadMetrics._remoteBlocksFetched, shuffleRead.LOCAL_BLOCKS_FETCHED -> shuffleReadMetrics._localBlocksFetched, shuffleRead.REMOTE_BYTES_READ -> shuffleReadMetrics._remoteBytesRead, + shuffleRead.REMOTE_BYTES_READ_TO_MEM -> shuffleReadMetrics._remoteBytesReadToMem, + shuffleRead.REMOTE_BYTES_READ_TO_DISK -> shuffleReadMetrics._remoteBytesReadToDisk, shuffleRead.LOCAL_BYTES_READ -> shuffleReadMetrics._localBytesRead, shuffleRead.FETCH_WAIT_TIME -> shuffleReadMetrics._fetchWaitTime, shuffleRead.RECORDS_READ -> shuffleReadMetrics._recordsRead, diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index f2fb9d1af696..c63b2e8ec54d 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -94,6 +94,10 @@ class TaskMetricsSuite extends SparkFunSuite { sr.setRemoteBytesRead(30L) sr.incRemoteBytesRead(3L) sr.incRemoteBytesRead(3L) + sr.setRemoteBytesReadToMem(10L) + sr.incRemoteBytesReadToMem(8L) + sr.setRemoteBytesReadToDisk(10L) + sr.incRemoteBytesReadToDisk(8L) sr.setLocalBytesRead(400L) sr.setLocalBytesRead(40L) sr.incLocalBytesRead(4L) @@ -110,6 +114,8 @@ class TaskMetricsSuite extends SparkFunSuite { assert(sr.remoteBlocksFetched == 12) assert(sr.localBlocksFetched == 24) assert(sr.remoteBytesRead == 36L) + assert(sr.remoteBytesReadToMem === 18L) + assert(sr.remoteBytesReadToDisk === 18L) assert(sr.localBytesRead == 48L) assert(sr.fetchWaitTime == 60L) assert(sr.recordsRead == 72L) From f21f1688f0c9c7dfa85090dbc08219fe0502c7c6 Mon Sep 17 00:00:00 2001 From: jinxing Date: Mon, 20 Mar 2017 16:20:26 +0800 Subject: [PATCH 06/28] fix taskMetricsFromJson and taskMetricsToJson --- .../spark/executor/ShuffleWriteMetrics.scala | 4 ++ .../org/apache/spark/util/JsonProtocol.scala | 27 ++++++++- .../apache/spark/util/JsonProtocolSuite.scala | 58 ++++++++++++------- 3 files changed, 66 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 7e81caa1a7b5..3bd9b7473f44 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -114,6 +114,10 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { _underestimatedBlocksNum.add(1) } + private[spark] def setUnderestimatedBlocksNum(value: Long) = { + _underestimatedBlocksNum.setValue(value) + } + private[spark] def incUnderestimatedBlocksSize(v: Long) = { _underestimatedBlocksSize.add(v) } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 4b4d2d10cbf8..b17d207088c5 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,6 +21,7 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map +import scala.collection.mutable.LinkedHashMap import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule @@ -330,12 +331,25 @@ private[spark] object JsonProtocol { ("Local Blocks Fetched" -> taskMetrics.shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> taskMetrics.shuffleReadMetrics.fetchWaitTime) ~ ("Remote Bytes Read" -> taskMetrics.shuffleReadMetrics.remoteBytesRead) ~ + ("Remote Bytes Read To Mem" -> taskMetrics.shuffleReadMetrics.remoteBytesReadToMem) ~ + ("Remote Bytes Read To Disk" -> taskMetrics.shuffleReadMetrics.remoteBytesReadToDisk) ~ ("Local Bytes Read" -> taskMetrics.shuffleReadMetrics.localBytesRead) ~ ("Total Records Read" -> taskMetrics.shuffleReadMetrics.recordsRead) + val linkedHashMap = LinkedHashMap[String, Long]() + taskMetrics.shuffleWriteMetrics.blockSizeDistribution.zipWithIndex.foreach { + case (size, index) => + linkedHashMap.put(s"Shuffle Write Block Size Distribution $index", size) + } val shuffleWriteMetrics: JValue = ("Shuffle Bytes Written" -> taskMetrics.shuffleWriteMetrics.bytesWritten) ~ ("Shuffle Write Time" -> taskMetrics.shuffleWriteMetrics.writeTime) ~ - ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) + ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) ~ + ("Shuffle Write Average Block Size" -> taskMetrics.shuffleWriteMetrics.averageBlockSize) ~ + ("Shuffle Write Underestimated Blocks Num" -> + taskMetrics.shuffleWriteMetrics.underestimatedBlocksNum) ~ + ("Shuffle Write Underestimated Blocks Size" -> + taskMetrics.shuffleWriteMetrics.underestimatedBlocksSize) ++ linkedHashMap + val inputMetrics: JValue = ("Bytes Read" -> taskMetrics.inputMetrics.bytesRead) ~ ("Records Read" -> taskMetrics.inputMetrics.recordsRead) @@ -807,6 +821,17 @@ private[spark] object JsonProtocol { writeMetrics.incRecordsWritten( Utils.jsonOption(writeJson \ "Shuffle Records Written").map(_.extract[Long]).getOrElse(0L)) writeMetrics.incWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) + (0 until 9).foreach { + case i => + writeMetrics.incBlockSizeDistribution((writeJson \ + s"Shuffle Write Block Size Distribution $i").extract[Long]) + } + writeMetrics.setAverageBlockSize( + (writeJson \ "Shuffle Write Average Block Size").extract[Long]) + writeMetrics.setUnderestimatedBlocksNum( + (writeJson \ "Shuffle Write Underestimated Blocks Num").extract[Long]) + writeMetrics.incUnderestimatedBlocksSize( + (writeJson \ "Shuffle Write Underestimated Blocks Size").extract[Long]) } // Output metrics diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index d1002378507d..82a4b8aff4d7 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1914,160 +1914,174 @@ private[spark] object JsonProtocolSuite extends Assertions { | }, | { | "ID": 14, - | "Name": "${shuffleRead.LOCAL_BYTES_READ}", + | "Name": "${shuffleRead.REMOTE_BYTES_READ_TO_MEM}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 15, - | "Name": "${shuffleRead.FETCH_WAIT_TIME}", + | "Name": "${shuffleRead.REMOTE_BYTES_READ_TO_DISK}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 16, - | "Name": "${shuffleRead.RECORDS_READ}", + | "Name": "${shuffleRead.LOCAL_BYTES_READ}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 17, - | "Name": "${shuffleWrite.BYTES_WRITTEN}", + | "Name": "${shuffleRead.FETCH_WAIT_TIME}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 18, - | "Name": "${shuffleWrite.RECORDS_WRITTEN}", + | "Name": "${shuffleRead.RECORDS_READ}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 19, - | "Name": "${shuffleWrite.WRITE_TIME}", + | "Name": "${shuffleWrite.BYTES_WRITTEN}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 20, - | "Name": "${shuffleWrite.AVERAGE_BLOCK_SIZE}", + | "Name": "${shuffleWrite.RECORDS_WRITTEN}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 21, - | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_NUM}", + | "Name": "${shuffleWrite.WRITE_TIME}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 22, - | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_SIZE}", + | "Name": "${shuffleWrite.AVERAGE_BLOCK_SIZE}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { | "ID": 23, + | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_NUM}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 24, + | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_SIZE}", + | "Update": 0, + | "Internal": true, + | "Count Failed Values": true + | }, + | { + | "ID": 25, | "Name": "${input.BYTES_READ}", | "Update": 2100, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 24, + | "ID": 26, | "Name": "${input.RECORDS_READ}", | "Update": 21, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 25, + | "ID": 27, | "Name": "${output.BYTES_WRITTEN}", | "Update": 1200, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 26, + | "ID": 28, | "Name": "${output.RECORDS_WRITTEN}", | "Update": 12, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 27, + | "ID": 29, | "Name": "$TEST_ACCUM", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 28, + | "ID": 30, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}0", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 29, + | "ID": 31, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}1", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 30, + | "ID": 32, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}2", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 31, + | "ID": 33, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}3", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 32, + | "ID": 34, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}4", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 33, + | "ID": 35, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}5", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 34, + | "ID": 36, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}6", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 35, + | "ID": 37, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}7", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 36, + | "ID": 38, | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}8", | "Update": 0, | "Internal": true, From 0e6d7833321e50d3781e773e39f6ddcdc0cc7132 Mon Sep 17 00:00:00 2001 From: jinxing Date: Mon, 20 Mar 2017 16:23:19 +0800 Subject: [PATCH 07/28] fix --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b17d207088c5..393ea6e4ea6d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -805,6 +805,8 @@ private[spark] object JsonProtocol { readMetrics.incRemoteBlocksFetched((readJson \ "Remote Blocks Fetched").extract[Int]) readMetrics.incLocalBlocksFetched((readJson \ "Local Blocks Fetched").extract[Int]) readMetrics.incRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) + readMetrics.incRemoteBytesReadToMem((readJson \ "Remote Bytes Read To Mem").extract[Long]) + readMetrics.incRemoteBytesReadToDisk((readJson \ "Remote Bytes Read To Disk").extract[Long]) readMetrics.incLocalBytesRead( Utils.jsonOption(readJson \ "Local Bytes Read").map(_.extract[Long]).getOrElse(0L)) readMetrics.incFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) From 0e853322a903018bd971f1983865fc2ab09246d7 Mon Sep 17 00:00:00 2001 From: jinxing Date: Mon, 20 Mar 2017 22:39:55 +0800 Subject: [PATCH 08/28] Fix json. --- .../spark/executor/ShuffleWriteMetrics.scala | 4 ++ .../org/apache/spark/util/JsonProtocol.scala | 17 ++--- .../apache/spark/util/JsonProtocolSuite.scala | 62 +++++++++++++++++-- 3 files changed, 69 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 3bd9b7473f44..00da56025e0a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -106,6 +106,10 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { } } + private[spark] def setBlockSizeDistribution(index: Int, value: Long): Unit = { + _blockSizeDistribution(index).setValue(value) + } + private[spark] def setAverageBlockSize(avg: Long): Unit = { _averageBlockSize.setValue(avg) } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 393ea6e4ea6d..83960ce0f6f2 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -335,12 +335,9 @@ private[spark] object JsonProtocol { ("Remote Bytes Read To Disk" -> taskMetrics.shuffleReadMetrics.remoteBytesReadToDisk) ~ ("Local Bytes Read" -> taskMetrics.shuffleReadMetrics.localBytesRead) ~ ("Total Records Read" -> taskMetrics.shuffleReadMetrics.recordsRead) - val linkedHashMap = LinkedHashMap[String, Long]() - taskMetrics.shuffleWriteMetrics.blockSizeDistribution.zipWithIndex.foreach { - case (size, index) => - linkedHashMap.put(s"Shuffle Write Block Size Distribution $index", size) - } - val shuffleWriteMetrics: JValue = + + + var shuffleWriteMetrics: JValue = ("Shuffle Bytes Written" -> taskMetrics.shuffleWriteMetrics.bytesWritten) ~ ("Shuffle Write Time" -> taskMetrics.shuffleWriteMetrics.writeTime) ~ ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) ~ @@ -348,7 +345,11 @@ private[spark] object JsonProtocol { ("Shuffle Write Underestimated Blocks Num" -> taskMetrics.shuffleWriteMetrics.underestimatedBlocksNum) ~ ("Shuffle Write Underestimated Blocks Size" -> - taskMetrics.shuffleWriteMetrics.underestimatedBlocksSize) ++ linkedHashMap + taskMetrics.shuffleWriteMetrics.underestimatedBlocksSize) merge + taskMetrics.shuffleWriteMetrics.blockSizeDistribution.zipWithIndex.map { + case (size, index) => + render(s"Shuffle Write Block Size Distribution $index" -> size.asInstanceOf[Long]) + }.reduceLeft(_ merge _) val inputMetrics: JValue = ("Bytes Read" -> taskMetrics.inputMetrics.bytesRead) ~ @@ -825,7 +826,7 @@ private[spark] object JsonProtocol { writeMetrics.incWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) (0 until 9).foreach { case i => - writeMetrics.incBlockSizeDistribution((writeJson \ + writeMetrics.setBlockSizeDistribution(i, (writeJson \ s"Shuffle Write Block Size Distribution $i").extract[Long]) } writeMetrics.setAverageBlockSize( diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 82a4b8aff4d7..8a27e25331f6 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -847,6 +847,8 @@ private[spark] object JsonProtocolSuite extends Assertions { } else { val sr = t.createTempShuffleReadMetrics() sr.incRemoteBytesRead(b + d) + sr.incRemoteBytesReadToMem(b) + sr.incRemoteBytesReadToMem(d) sr.incLocalBlocksFetched(e) sr.incFetchWaitTime(a + d) sr.incRemoteBlocksFetched(f) @@ -861,6 +863,12 @@ private[spark] object JsonProtocolSuite extends Assertions { val sw = t.shuffleWriteMetrics sw.incBytesWritten(a + b + c) sw.incWriteTime(b + c + d) + sw.setBlockSizeDistribution(0, a) + sw.setBlockSizeDistribution(1, b) + sw.setBlockSizeDistribution(2, c) + sw.setAverageBlockSize(b) + sw.setUnderestimatedBlocksNum(c) + sw.incUnderestimatedBlocksSize(a + b + c) sw.incRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) } // Make at most 6 blocks @@ -1127,13 +1135,27 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, | "Remote Bytes Read": 1000, + | "Remote Bytes Read To Mem": 1000, + | "Remote Bytes Read To Disk": 0, | "Local Bytes Read": 1100, | "Total Records Read": 10 | }, - | "Shuffle Write Metrics": { - | "Shuffle Bytes Written": 1200, - | "Shuffle Write Time": 1500, - | "Shuffle Records Written": 12 + | "Shuffle Write Metrics" : { + | "Shuffle Bytes Written" : 1200, + | "Shuffle Write Time" : 1500, + | "Shuffle Records Written" : 12, + | "Shuffle Write Average Block Size" : 400, + | "Shuffle Write Underestimated Blocks Num" : 500, + | "Shuffle Write Underestimated Blocks Size" : 1200, + | "Shuffle Write Block Size Distribution 0" : 300, + | "Shuffle Write Block Size Distribution 1" : 400, + | "Shuffle Write Block Size Distribution 2" : 500, + | "Shuffle Write Block Size Distribution 3" : 0, + | "Shuffle Write Block Size Distribution 4" : 0, + | "Shuffle Write Block Size Distribution 5" : 0, + | "Shuffle Write Block Size Distribution 6" : 0, + | "Shuffle Write Block Size Distribution 7" : 0, + | "Shuffle Write Block Size Distribution 8" : 0 | }, | "Input Metrics" : { | "Bytes Read" : 0, @@ -1227,13 +1249,27 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched" : 0, | "Fetch Wait Time" : 0, | "Remote Bytes Read" : 0, + | "Remote Bytes Read To Mem" : 0, + | "Remote Bytes Read To Disk" : 0, | "Local Bytes Read" : 0, | "Total Records Read" : 0 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, | "Shuffle Write Time": 1500, - | "Shuffle Records Written": 12 + | "Shuffle Records Written": 12, + | "Shuffle Write Average Block Size" : 400, + | "Shuffle Write Underestimated Blocks Num" : 500, + | "Shuffle Write Underestimated Blocks Size" : 1200, + | "Shuffle Write Block Size Distribution 0" : 300, + | "Shuffle Write Block Size Distribution 1" : 400, + | "Shuffle Write Block Size Distribution 2" : 500, + | "Shuffle Write Block Size Distribution 3" : 0, + | "Shuffle Write Block Size Distribution 4" : 0, + | "Shuffle Write Block Size Distribution 5" : 0, + | "Shuffle Write Block Size Distribution 6" : 0, + | "Shuffle Write Block Size Distribution 7" : 0, + | "Shuffle Write Block Size Distribution 8" : 0 | }, | "Input Metrics": { | "Bytes Read": 2100, @@ -1327,13 +1363,27 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched" : 0, | "Fetch Wait Time" : 0, | "Remote Bytes Read" : 0, + | "Remote Bytes Read To Mem" : 0, + | "Remote Bytes Read To Disk" : 0, | "Local Bytes Read" : 0, | "Total Records Read" : 0 | }, | "Shuffle Write Metrics" : { | "Shuffle Bytes Written" : 0, | "Shuffle Write Time" : 0, - | "Shuffle Records Written" : 0 + | "Shuffle Records Written" : 0, + | "Shuffle Write Average Block Size" : 0, + | "Shuffle Write Underestimated Blocks Num" : 0, + | "Shuffle Write Underestimated Blocks Size" : 0, + | "Shuffle Write Block Size Distribution 0" : 0, + | "Shuffle Write Block Size Distribution 1" : 0, + | "Shuffle Write Block Size Distribution 2" : 0, + | "Shuffle Write Block Size Distribution 3" : 0, + | "Shuffle Write Block Size Distribution 4" : 0, + | "Shuffle Write Block Size Distribution 5" : 0, + | "Shuffle Write Block Size Distribution 6" : 0, + | "Shuffle Write Block Size Distribution 7" : 0, + | "Shuffle Write Block Size Distribution 8" : 0 | }, | "Input Metrics": { | "Bytes Read": 2100, From 7ac639f9239ef943e55a1cc76dc7346bbe617bfd Mon Sep 17 00:00:00 2001 From: jinxing Date: Tue, 21 Mar 2017 11:14:43 +0800 Subject: [PATCH 09/28] remove maxBlockSize and fix remoteBytesReadToMem. --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 4 ---- .../org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 4 ---- .../apache/spark/storage/ShuffleBlockFetcherIterator.scala | 2 ++ 3 files changed, 2 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index ef21f2bb5f4e..4994cfff435c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -171,12 +171,8 @@ public void write(Iterator> records) throws IOException { } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - long maxBlockSize = 0L; for (long partitionLength: partitionLengths) { writeMetrics.incBlockSizeDistribution(partitionLength); - if (partitionLength > maxBlockSize) { - maxBlockSize = partitionLength; - } } if (mapStatus instanceof HighlyCompressedMapStatus) { writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 58b0a85d82c4..5063a8643a60 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -232,12 +232,8 @@ void closeAndWriteOutput() throws IOException { } } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - long maxBlockSize = 0L; for (long partitionLength: partitionLengths) { writeMetrics.incBlockSizeDistribution(partitionLength); - if (partitionLength > maxBlockSize) { - maxBlockSize = partitionLength; - } } if (mapStatus instanceof HighlyCompressedMapStatus) { writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index f8906117638b..737f25a885f1 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -157,6 +157,7 @@ final class ShuffleBlockFetcherIterator( case SuccessFetchResult(_, address, _, buf, _) => if (address != blockManager.blockManagerId) { shuffleMetrics.incRemoteBytesRead(buf.size) + shuffleMetrics.incRemoteBytesReadToMem(buf.size) shuffleMetrics.incRemoteBlocksFetched(1) } buf.release() @@ -337,6 +338,7 @@ final class ShuffleBlockFetcherIterator( case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) => if (address != blockManager.blockManagerId) { shuffleMetrics.incRemoteBytesRead(buf.size) + shuffleMetrics.incRemoteBytesReadToMem(buf.size) shuffleMetrics.incRemoteBlocksFetched(1) } bytesInFlight -= size From fea66c557aafca2925c2831c60caef46b3b7c436 Mon Sep 17 00:00:00 2001 From: jinxing Date: Tue, 21 Mar 2017 12:09:29 +0800 Subject: [PATCH 10/28] small fix --- .../test/scala/org/apache/spark/util/JsonProtocolSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 8a27e25331f6..798d4251e038 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1368,7 +1368,7 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Bytes Read" : 0, | "Total Records Read" : 0 | }, - | "Shuffle Write Metrics" : { + | "Shuffle Write Metrics": { | "Shuffle Bytes Written" : 0, | "Shuffle Write Time" : 0, | "Shuffle Records Written" : 0, From e6091b69271ade48a4eaaf3c770203f35ab06001 Mon Sep 17 00:00:00 2001 From: jinxing Date: Tue, 21 Mar 2017 12:18:35 +0800 Subject: [PATCH 11/28] Remove the new metrics from event log --- .../org/apache/spark/util/JsonProtocol.scala | 28 +--------- .../apache/spark/util/JsonProtocolSuite.scala | 56 +------------------ 2 files changed, 4 insertions(+), 80 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 83960ce0f6f2..a3b6dae70775 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -331,26 +331,13 @@ private[spark] object JsonProtocol { ("Local Blocks Fetched" -> taskMetrics.shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> taskMetrics.shuffleReadMetrics.fetchWaitTime) ~ ("Remote Bytes Read" -> taskMetrics.shuffleReadMetrics.remoteBytesRead) ~ - ("Remote Bytes Read To Mem" -> taskMetrics.shuffleReadMetrics.remoteBytesReadToMem) ~ - ("Remote Bytes Read To Disk" -> taskMetrics.shuffleReadMetrics.remoteBytesReadToDisk) ~ ("Local Bytes Read" -> taskMetrics.shuffleReadMetrics.localBytesRead) ~ ("Total Records Read" -> taskMetrics.shuffleReadMetrics.recordsRead) - var shuffleWriteMetrics: JValue = ("Shuffle Bytes Written" -> taskMetrics.shuffleWriteMetrics.bytesWritten) ~ ("Shuffle Write Time" -> taskMetrics.shuffleWriteMetrics.writeTime) ~ - ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) ~ - ("Shuffle Write Average Block Size" -> taskMetrics.shuffleWriteMetrics.averageBlockSize) ~ - ("Shuffle Write Underestimated Blocks Num" -> - taskMetrics.shuffleWriteMetrics.underestimatedBlocksNum) ~ - ("Shuffle Write Underestimated Blocks Size" -> - taskMetrics.shuffleWriteMetrics.underestimatedBlocksSize) merge - taskMetrics.shuffleWriteMetrics.blockSizeDistribution.zipWithIndex.map { - case (size, index) => - render(s"Shuffle Write Block Size Distribution $index" -> size.asInstanceOf[Long]) - }.reduceLeft(_ merge _) - + ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) val inputMetrics: JValue = ("Bytes Read" -> taskMetrics.inputMetrics.bytesRead) ~ ("Records Read" -> taskMetrics.inputMetrics.recordsRead) @@ -806,8 +793,6 @@ private[spark] object JsonProtocol { readMetrics.incRemoteBlocksFetched((readJson \ "Remote Blocks Fetched").extract[Int]) readMetrics.incLocalBlocksFetched((readJson \ "Local Blocks Fetched").extract[Int]) readMetrics.incRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) - readMetrics.incRemoteBytesReadToMem((readJson \ "Remote Bytes Read To Mem").extract[Long]) - readMetrics.incRemoteBytesReadToDisk((readJson \ "Remote Bytes Read To Disk").extract[Long]) readMetrics.incLocalBytesRead( Utils.jsonOption(readJson \ "Local Bytes Read").map(_.extract[Long]).getOrElse(0L)) readMetrics.incFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) @@ -824,17 +809,6 @@ private[spark] object JsonProtocol { writeMetrics.incRecordsWritten( Utils.jsonOption(writeJson \ "Shuffle Records Written").map(_.extract[Long]).getOrElse(0L)) writeMetrics.incWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) - (0 until 9).foreach { - case i => - writeMetrics.setBlockSizeDistribution(i, (writeJson \ - s"Shuffle Write Block Size Distribution $i").extract[Long]) - } - writeMetrics.setAverageBlockSize( - (writeJson \ "Shuffle Write Average Block Size").extract[Long]) - writeMetrics.setUnderestimatedBlocksNum( - (writeJson \ "Shuffle Write Underestimated Blocks Num").extract[Long]) - writeMetrics.incUnderestimatedBlocksSize( - (writeJson \ "Shuffle Write Underestimated Blocks Size").extract[Long]) } // Output metrics diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 798d4251e038..e511dbfec5bb 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -847,8 +847,6 @@ private[spark] object JsonProtocolSuite extends Assertions { } else { val sr = t.createTempShuffleReadMetrics() sr.incRemoteBytesRead(b + d) - sr.incRemoteBytesReadToMem(b) - sr.incRemoteBytesReadToMem(d) sr.incLocalBlocksFetched(e) sr.incFetchWaitTime(a + d) sr.incRemoteBlocksFetched(f) @@ -863,12 +861,6 @@ private[spark] object JsonProtocolSuite extends Assertions { val sw = t.shuffleWriteMetrics sw.incBytesWritten(a + b + c) sw.incWriteTime(b + c + d) - sw.setBlockSizeDistribution(0, a) - sw.setBlockSizeDistribution(1, b) - sw.setBlockSizeDistribution(2, c) - sw.setAverageBlockSize(b) - sw.setUnderestimatedBlocksNum(c) - sw.incUnderestimatedBlocksSize(a + b + c) sw.incRecordsWritten(if (hasRecords) (a + b + c) / 100 else -1) } // Make at most 6 blocks @@ -1135,27 +1127,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, | "Remote Bytes Read": 1000, - | "Remote Bytes Read To Mem": 1000, - | "Remote Bytes Read To Disk": 0, | "Local Bytes Read": 1100, | "Total Records Read": 10 | }, | "Shuffle Write Metrics" : { | "Shuffle Bytes Written" : 1200, | "Shuffle Write Time" : 1500, - | "Shuffle Records Written" : 12, - | "Shuffle Write Average Block Size" : 400, - | "Shuffle Write Underestimated Blocks Num" : 500, - | "Shuffle Write Underestimated Blocks Size" : 1200, - | "Shuffle Write Block Size Distribution 0" : 300, - | "Shuffle Write Block Size Distribution 1" : 400, - | "Shuffle Write Block Size Distribution 2" : 500, - | "Shuffle Write Block Size Distribution 3" : 0, - | "Shuffle Write Block Size Distribution 4" : 0, - | "Shuffle Write Block Size Distribution 5" : 0, - | "Shuffle Write Block Size Distribution 6" : 0, - | "Shuffle Write Block Size Distribution 7" : 0, - | "Shuffle Write Block Size Distribution 8" : 0 + | "Shuffle Records Written" : 12 | }, | "Input Metrics" : { | "Bytes Read" : 0, @@ -1249,27 +1227,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched" : 0, | "Fetch Wait Time" : 0, | "Remote Bytes Read" : 0, - | "Remote Bytes Read To Mem" : 0, - | "Remote Bytes Read To Disk" : 0, | "Local Bytes Read" : 0, | "Total Records Read" : 0 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, | "Shuffle Write Time": 1500, - | "Shuffle Records Written": 12, - | "Shuffle Write Average Block Size" : 400, - | "Shuffle Write Underestimated Blocks Num" : 500, - | "Shuffle Write Underestimated Blocks Size" : 1200, - | "Shuffle Write Block Size Distribution 0" : 300, - | "Shuffle Write Block Size Distribution 1" : 400, - | "Shuffle Write Block Size Distribution 2" : 500, - | "Shuffle Write Block Size Distribution 3" : 0, - | "Shuffle Write Block Size Distribution 4" : 0, - | "Shuffle Write Block Size Distribution 5" : 0, - | "Shuffle Write Block Size Distribution 6" : 0, - | "Shuffle Write Block Size Distribution 7" : 0, - | "Shuffle Write Block Size Distribution 8" : 0 + | "Shuffle Records Written": 12 | }, | "Input Metrics": { | "Bytes Read": 2100, @@ -1363,27 +1327,13 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched" : 0, | "Fetch Wait Time" : 0, | "Remote Bytes Read" : 0, - | "Remote Bytes Read To Mem" : 0, - | "Remote Bytes Read To Disk" : 0, | "Local Bytes Read" : 0, | "Total Records Read" : 0 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written" : 0, | "Shuffle Write Time" : 0, - | "Shuffle Records Written" : 0, - | "Shuffle Write Average Block Size" : 0, - | "Shuffle Write Underestimated Blocks Num" : 0, - | "Shuffle Write Underestimated Blocks Size" : 0, - | "Shuffle Write Block Size Distribution 0" : 0, - | "Shuffle Write Block Size Distribution 1" : 0, - | "Shuffle Write Block Size Distribution 2" : 0, - | "Shuffle Write Block Size Distribution 3" : 0, - | "Shuffle Write Block Size Distribution 4" : 0, - | "Shuffle Write Block Size Distribution 5" : 0, - | "Shuffle Write Block Size Distribution 6" : 0, - | "Shuffle Write Block Size Distribution 7" : 0, - | "Shuffle Write Block Size Distribution 8" : 0 + | "Shuffle Records Written" : 0 | }, | "Input Metrics": { | "Bytes Read": 2100, From f7ff868ccba47e3ef15051eb6da2dad43ec16c44 Mon Sep 17 00:00:00 2001 From: jinxing Date: Tue, 21 Mar 2017 12:23:36 +0800 Subject: [PATCH 12/28] small fix --- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index e511dbfec5bb..46eaf908c52f 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1130,10 +1130,10 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Bytes Read": 1100, | "Total Records Read": 10 | }, - | "Shuffle Write Metrics" : { - | "Shuffle Bytes Written" : 1200, - | "Shuffle Write Time" : 1500, - | "Shuffle Records Written" : 12 + | "Shuffle Write Metrics": { + | "Shuffle Bytes Written": 1200, + | "Shuffle Write Time": 1500, + | "Shuffle Records Written": 12 | }, | "Input Metrics" : { | "Bytes Read" : 0, From d86f985fb63204715189c1e1a964f127c396df59 Mon Sep 17 00:00:00 2001 From: jinxing Date: Tue, 21 Mar 2017 21:29:10 +0800 Subject: [PATCH 13/28] add log for shuffle read. --- .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 1e2940eff1c7..e476660c30e4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -401,6 +401,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) + logDebug(s"${m.shuffleReadMetrics.remoteBytesReadToMem} bytes shuffled to memory.") + logDebug(s"${m.shuffleReadMetrics.remoteBytesReadToDisk} bytes shuffled to disk.") + if (m.shuffleWriteMetrics.averageBlockSize == 0) { logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + s"the block sizes in MapStatus are accurate, distribution is:") From dc7ae19e4ea5c1b6992b6fdd1f809e1e66842be9 Mon Sep 17 00:00:00 2001 From: jinxing Date: Tue, 21 Mar 2017 23:25:57 +0800 Subject: [PATCH 14/28] fix code style. --- .../sort/BypassMergeSortShuffleWriter.java | 2 +- .../shuffle/sort/UnsafeShuffleWriter.java | 2 +- .../spark/executor/ShuffleWriteMetrics.scala | 4 ++-- .../apache/spark/scheduler/MapStatus.scala | 2 +- .../shuffle/sort/SortShuffleWriter.scala | 21 ++++++++++--------- 5 files changed, 16 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 4994cfff435c..39b7df82c568 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -177,7 +177,7 @@ public void write(Iterator> records) throws IOException { if (mapStatus instanceof HighlyCompressedMapStatus) { writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); for (int i = 0; i < partitionLengths.length; i++) { - if (partitionLengths[i] < mapStatus.getSizeForBlock(i)) { + if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { writeMetrics.incUnderestimatedBlocksNum(); writeMetrics.incUnderestimatedBlocksSize(partitionLengths[i]); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 5063a8643a60..229f633a4bc7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -238,7 +238,7 @@ void closeAndWriteOutput() throws IOException { if (mapStatus instanceof HighlyCompressedMapStatus) { writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); for (int i = 0; i < partitionLengths.length; i++) { - if (partitionLengths[i] < mapStatus.getSizeForBlock(i)) { + if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { writeMetrics.incUnderestimatedBlocksNum(); writeMetrics.incUnderestimatedBlocksSize(partitionLengths[i]); } diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index 00da56025e0a..ead991e4b962 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import java.{lang => jl} +import java.lang.{Long => JLong} import scala.collection.JavaConverters._ @@ -62,7 +62,7 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { * Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), * [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). */ - def blockSizeDistribution: Seq[jl.Long] = { + def blockSizeDistribution: Seq[JLong] = { _blockSizeDistribution.map(_.value).toSeq } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index dbc0ca66dd8a..246ae8c55f42 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -165,7 +165,7 @@ private[spark] class HighlyCompressedMapStatus private ( avgSize = in.readLong() } - def getAvgSize(): Long = avgSize + def getAvgSize: Long = avgSize } private[spark] object HighlyCompressedMapStatus { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 31f6e1f58c8f..5955baa59523 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -73,16 +73,17 @@ private[spark] class SortShuffleWriter[K, V, C]( shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) partitionLengths.foreach(writeMetrics.incBlockSizeDistribution(_)) - if (mapStatus.isInstanceOf[HighlyCompressedMapStatus]) { - writeMetrics.setAverageBlockSize( - mapStatus.asInstanceOf[HighlyCompressedMapStatus].getAvgSize()); - (0 until partitionLengths.length).foreach { - case i => - if (partitionLengths(i) < mapStatus.getSizeForBlock(i)) { - writeMetrics.incUnderestimatedBlocksNum() - writeMetrics.incUnderestimatedBlocksSize(partitionLengths(i)) - } - } + mapStatus match { + case hc: HighlyCompressedMapStatus => + writeMetrics.setAverageBlockSize(hc.getAvgSize()) + (0 until partitionLengths.length).foreach { + case i => + if (partitionLengths(i) > mapStatus.getSizeForBlock(i)) { + writeMetrics.incUnderestimatedBlocksNum() + writeMetrics.incUnderestimatedBlocksSize(partitionLengths(i)) + } + } + case _ => // no-op } } finally { if (tmp.exists() && !tmp.delete()) { From c7aae80b972d0e24d5dc5413209be053eae4c015 Mon Sep 17 00:00:00 2001 From: jinxing Date: Wed, 22 Mar 2017 13:49:34 +0800 Subject: [PATCH 15/28] Remove some metrics. --- .../sort/BypassMergeSortShuffleWriter.java | 56 +++++++-- .../shuffle/sort/UnsafeShuffleWriter.java | 53 ++++++++- .../apache/spark/InternalAccumulator.scala | 4 - .../spark/executor/ShuffleReadMetrics.scala | 13 --- .../spark/executor/ShuffleWriteMetrics.scala | 61 +--------- .../apache/spark/executor/TaskMetrics.scala | 14 +-- .../shuffle/sort/SortShuffleWriter.scala | 36 ++++-- .../spark/ui/jobs/JobProgressListener.scala | 21 +--- .../spark/executor/TaskMetricsSuite.scala | 10 -- .../apache/spark/util/JsonProtocolSuite.scala | 106 ++---------------- 10 files changed, 140 insertions(+), 234 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 39b7df82c568..6952c8b09594 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -76,6 +76,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private static final Logger logger = LoggerFactory.getLogger(BypassMergeSortShuffleWriter.class); + private final TaskContext taskContext; private final int fileBufferSize; private final boolean transferToEnabled; private final int numPartitions; @@ -116,6 +117,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); + this.taskContext = taskContext; this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; @@ -170,16 +172,56 @@ public void write(Iterator> records) throws IOException { } } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - - for (long partitionLength: partitionLengths) { - writeMetrics.incBlockSizeDistribution(partitionLength); - } if (mapStatus instanceof HighlyCompressedMapStatus) { - writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); + HighlyCompressedMapStatus hc = (HighlyCompressedMapStatus) mapStatus; + long underestimatedBlocksSize = 0L; for (int i = 0; i < partitionLengths.length; i++) { if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { - writeMetrics.incUnderestimatedBlocksNum(); - writeMetrics.incUnderestimatedBlocksSize(partitionLengths[i]); + underestimatedBlocksSize += partitionLengths[i]; + } + } + writeMetrics.incUnderestimatedBlocksSize(underestimatedBlocksSize); + if (logger.isDebugEnabled()) { + int underestimatedBlocksNum = 0; + // Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), + // [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). + int[] lenDistribution = {0, 0, 0, 0, 0, 0, 0, 0, 0}; + for (int i = 0; i < partitionLengths.length; i++) { + long len = partitionLengths[i]; + if (len > mapStatus.getSizeForBlock(i)) { + underestimatedBlocksNum++; + if (len >= 0L && len < 1024L) { + lenDistribution[0]++; + } else if (len >= 1024L && len < 10240L) { + lenDistribution[1]++; + } else if (len >= 10240L && len < 102400L) { + lenDistribution[2]++; + } else if (len >= 102400L && len < 1048576L ) { + lenDistribution[3]++; + } else if (len >= 1048576L && len < 10485760L) { + lenDistribution[4]++; + } else if (len >= 10485760L && len < 104857600L) { + lenDistribution[5]++; + } else if (len >= 104857600L && len < 1073741824L) { + lenDistribution[6]++; + } else if (len >= 1073741824L && len < 10737418240L) { + lenDistribution[7]++; + } else { + lenDistribution[8]++; + } + } + String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", + "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; + String[] rangesAndDistribute = new String[9]; + for (int j = 0; j < 9; j++) { + rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; + } + logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + + "inaccurate (average is {}, {} blocks underestimated, sum of sizes is {})," + + " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), + taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), + underestimatedBlocksNum, underestimatedBlocksSize, + String.join(", ", rangesAndDistribute)); } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 229f633a4bc7..5b9eb6312996 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -232,15 +232,56 @@ void closeAndWriteOutput() throws IOException { } } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - for (long partitionLength: partitionLengths) { - writeMetrics.incBlockSizeDistribution(partitionLength); - } if (mapStatus instanceof HighlyCompressedMapStatus) { - writeMetrics.setAverageBlockSize(((HighlyCompressedMapStatus) mapStatus).getAvgSize()); + HighlyCompressedMapStatus hc = (HighlyCompressedMapStatus) mapStatus; + long underestimatedBlocksSize = 0L; for (int i = 0; i < partitionLengths.length; i++) { if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { - writeMetrics.incUnderestimatedBlocksNum(); - writeMetrics.incUnderestimatedBlocksSize(partitionLengths[i]); + underestimatedBlocksSize += partitionLengths[i]; + } + } + writeMetrics.incUnderestimatedBlocksSize(underestimatedBlocksSize); + if (logger.isDebugEnabled()) { + int underestimatedBlocksNum = 0; + // Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), + // [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). + int[] lenDistribution = {0, 0, 0, 0, 0, 0, 0, 0, 0}; + for (int i = 0; i < partitionLengths.length; i++) { + long len = partitionLengths[i]; + if (len > mapStatus.getSizeForBlock(i)) { + underestimatedBlocksNum++; + if (len >= 0L && len < 1024L) { + lenDistribution[0]++; + } else if (len >= 1024L && len < 10240L) { + lenDistribution[1]++; + } else if (len >= 10240L && len < 102400L) { + lenDistribution[2]++; + } else if (len >= 102400L && len < 1048576L ) { + lenDistribution[3]++; + } else if (len >= 1048576L && len < 10485760L) { + lenDistribution[4]++; + } else if (len >= 10485760L && len < 104857600L) { + lenDistribution[5]++; + } else if (len >= 104857600L && len < 1073741824L) { + lenDistribution[6]++; + } else if (len >= 1073741824L && len < 10737418240L) { + lenDistribution[7]++; + } else { + lenDistribution[8]++; + } + } + String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", + "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; + String[] rangesAndDistribute = new String[9]; + for (int j = 0; j < 9; j++) { + rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; + } + logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + + "inaccurate (average is {}, {} blocks underestimated, sum of sizes is {})," + + " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), + taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), + underestimatedBlocksNum, underestimatedBlocksSize, + String.join(", ", rangesAndDistribute)); } } } diff --git a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala index fdf07feb3d6c..9144957f24cc 100644 --- a/core/src/main/scala/org/apache/spark/InternalAccumulator.scala +++ b/core/src/main/scala/org/apache/spark/InternalAccumulator.scala @@ -51,7 +51,6 @@ private[spark] object InternalAccumulator { val LOCAL_BLOCKS_FETCHED = SHUFFLE_READ_METRICS_PREFIX + "localBlocksFetched" val REMOTE_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesRead" val REMOTE_BYTES_READ_TO_MEM = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesReadToMem" - val REMOTE_BYTES_READ_TO_DISK = SHUFFLE_READ_METRICS_PREFIX + "remoteBytesReadToDisk" val LOCAL_BYTES_READ = SHUFFLE_READ_METRICS_PREFIX + "localBytesRead" val FETCH_WAIT_TIME = SHUFFLE_READ_METRICS_PREFIX + "fetchWaitTime" val RECORDS_READ = SHUFFLE_READ_METRICS_PREFIX + "recordsRead" @@ -62,9 +61,6 @@ private[spark] object InternalAccumulator { val BYTES_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "bytesWritten" val RECORDS_WRITTEN = SHUFFLE_WRITE_METRICS_PREFIX + "recordsWritten" val WRITE_TIME = SHUFFLE_WRITE_METRICS_PREFIX + "writeTime" - val BLOCK_SIZE_DISTRIBUTION_PREFIX = SHUFFLE_WRITE_METRICS_PREFIX + "blockSizeDistribution." - val AVERAGE_BLOCK_SIZE = SHUFFLE_WRITE_METRICS_PREFIX + "averageBlockSize" - val UNDERESTIMATED_BLOCKS_NUM = SHUFFLE_WRITE_METRICS_PREFIX + "underestimatedBlocksNum" val UNDERESTIMATED_BLOCKS_SIZE = SHUFFLE_WRITE_METRICS_PREFIX + "underestimatedBlocksSize" } diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 17fd5acc58ca..8e5a8a446ad4 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -32,7 +32,6 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[executor] val _localBlocksFetched = new LongAccumulator private[executor] val _remoteBytesRead = new LongAccumulator private[executor] val _remoteBytesReadToMem = new LongAccumulator - private[executor] val _remoteBytesReadToDisk = new LongAccumulator private[executor] val _localBytesRead = new LongAccumulator private[executor] val _fetchWaitTime = new LongAccumulator private[executor] val _recordsRead = new LongAccumulator @@ -57,11 +56,6 @@ class ShuffleReadMetrics private[spark] () extends Serializable { */ def remoteBytesReadToMem: Long = _remoteBytesReadToMem.sum - /** - * Total number of remotes bytes read to disk from the shuffle by this task. - */ - def remoteBytesReadToDisk: Long = _remoteBytesReadToDisk.sum - /** * Shuffle data that was read from the local disk (as opposed to from a remote executor). */ @@ -93,7 +87,6 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[spark] def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched.add(v) private[spark] def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead.add(v) private[spark] def incRemoteBytesReadToMem(v: Long): Unit = _remoteBytesReadToMem.add(v) - private[spark] def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.add(v) private[spark] def incLocalBytesRead(v: Long): Unit = _localBytesRead.add(v) private[spark] def incFetchWaitTime(v: Long): Unit = _fetchWaitTime.add(v) private[spark] def incRecordsRead(v: Long): Unit = _recordsRead.add(v) @@ -102,7 +95,6 @@ class ShuffleReadMetrics private[spark] () extends Serializable { private[spark] def setLocalBlocksFetched(v: Int): Unit = _localBlocksFetched.setValue(v) private[spark] def setRemoteBytesRead(v: Long): Unit = _remoteBytesRead.setValue(v) private[spark] def setRemoteBytesReadToMem(v: Long): Unit = _remoteBytesReadToMem.setValue(v) - private[spark] def setRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk.setValue(v) private[spark] def setLocalBytesRead(v: Long): Unit = _localBytesRead.setValue(v) private[spark] def setFetchWaitTime(v: Long): Unit = _fetchWaitTime.setValue(v) private[spark] def setRecordsRead(v: Long): Unit = _recordsRead.setValue(v) @@ -116,7 +108,6 @@ class ShuffleReadMetrics private[spark] () extends Serializable { _localBlocksFetched.setValue(0) _remoteBytesRead.setValue(0) _remoteBytesReadToMem.setValue(0) - _remoteBytesReadToDisk.setValue(0) _localBytesRead.setValue(0) _fetchWaitTime.setValue(0) _recordsRead.setValue(0) @@ -125,7 +116,6 @@ class ShuffleReadMetrics private[spark] () extends Serializable { _localBlocksFetched.add(metric.localBlocksFetched) _remoteBytesRead.add(metric.remoteBytesRead) _remoteBytesReadToMem.add(metric.remoteBytesReadToMem) - _remoteBytesReadToDisk.add(metric.remoteBytesReadToDisk) _localBytesRead.add(metric.localBytesRead) _fetchWaitTime.add(metric.fetchWaitTime) _recordsRead.add(metric.recordsRead) @@ -143,7 +133,6 @@ private[spark] class TempShuffleReadMetrics { private[this] var _localBlocksFetched = 0L private[this] var _remoteBytesRead = 0L private[this] var _remoteBytesReadToMem = 0L - private[this] var _remoteBytesReadToDisk = 0L private[this] var _localBytesRead = 0L private[this] var _fetchWaitTime = 0L private[this] var _recordsRead = 0L @@ -152,7 +141,6 @@ private[spark] class TempShuffleReadMetrics { def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched += v def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead += v def incRemoteBytesReadToMem(v: Long): Unit = _remoteBytesReadToMem += v - def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk += v def incLocalBytesRead(v: Long): Unit = _localBytesRead += v def incFetchWaitTime(v: Long): Unit = _fetchWaitTime += v def incRecordsRead(v: Long): Unit = _recordsRead += v @@ -161,7 +149,6 @@ private[spark] class TempShuffleReadMetrics { def localBlocksFetched: Long = _localBlocksFetched def remoteBytesRead: Long = _remoteBytesRead def remoteBytesReadToMem: Long = _remoteBytesReadToMem - def remoteBytesReadToDisk: Long = _remoteBytesReadToDisk def localBytesRead: Long = _localBytesRead def fetchWaitTime: Long = _fetchWaitTime def recordsRead: Long = _recordsRead diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index ead991e4b962..ba0382d07b28 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -17,12 +17,8 @@ package org.apache.spark.executor -import java.lang.{Long => JLong} - -import scala.collection.JavaConverters._ - import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.{CollectionAccumulator, LongAccumulator} +import org.apache.spark.util.LongAccumulator /** @@ -35,12 +31,6 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { private[executor] val _bytesWritten = new LongAccumulator private[executor] val _recordsWritten = new LongAccumulator private[executor] val _writeTime = new LongAccumulator - private[executor] val _blockSizeDistribution = new Array[LongAccumulator](9) - (0 until 9).foreach { - case i => _blockSizeDistribution(i) = new LongAccumulator - } - private[executor] val _averageBlockSize = new LongAccumulator - private[executor] val _underestimatedBlocksNum = new LongAccumulator private[executor] val _underestimatedBlocksSize = new LongAccumulator /** @@ -58,25 +48,6 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { */ def writeTime: Long = _writeTime.sum - /** - * Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), - * [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). - */ - def blockSizeDistribution: Seq[JLong] = { - _blockSizeDistribution.map(_.value).toSeq - } - - /** - * The average size of blocks in HighlyCompressedMapStatus. - * This is not set if CompressedMapStatus is returned. - */ - def averageBlockSize: Long = _averageBlockSize.value - - /** - * The num of blocks whose sizes are underestimated in MapStatus. - */ - def underestimatedBlocksNum: Long = _underestimatedBlocksNum.value - /** * The total amount of blocks whose sizes are underestimated in MapStatus. */ @@ -92,36 +63,6 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { _recordsWritten.setValue(recordsWritten - v) } - private[spark] def incBlockSizeDistribution(len: Long): Unit = { - len match { - case len: Long if len >= 0L && len < 1024L => _blockSizeDistribution(0).add(1) - case len: Long if len >= 1024L && len < 10240L => _blockSizeDistribution(1).add(1) - case len: Long if len >= 10240L && len < 102400L => _blockSizeDistribution(2).add(1) - case len: Long if len >= 102400L && len < 1048576L => _blockSizeDistribution(3).add(1) - case len: Long if len >= 1048576L && len < 10485760L => _blockSizeDistribution(4).add(1) - case len: Long if len >= 10485760L && len < 104857600L => _blockSizeDistribution(5).add(1) - case len: Long if len >= 104857600L && len < 1073741824L => _blockSizeDistribution(6).add(1) - case len: Long if len >= 1073741824L && len < 10737418240L => _blockSizeDistribution(7).add(1) - case len: Long if len >= 10737418240L => _blockSizeDistribution(8).add(1) - } - } - - private[spark] def setBlockSizeDistribution(index: Int, value: Long): Unit = { - _blockSizeDistribution(index).setValue(value) - } - - private[spark] def setAverageBlockSize(avg: Long): Unit = { - _averageBlockSize.setValue(avg) - } - - private[spark] def incUnderestimatedBlocksNum() = { - _underestimatedBlocksNum.add(1) - } - - private[spark] def setUnderestimatedBlocksNum(value: Long) = { - _underestimatedBlocksNum.setValue(value) - } - private[spark] def incUnderestimatedBlocksSize(v: Long) = { _underestimatedBlocksSize.add(v) } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index d22f48301695..7b00235f5362 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -216,30 +216,18 @@ class TaskMetrics private[spark] () extends Serializable { shuffleRead.LOCAL_BLOCKS_FETCHED -> shuffleReadMetrics._localBlocksFetched, shuffleRead.REMOTE_BYTES_READ -> shuffleReadMetrics._remoteBytesRead, shuffleRead.REMOTE_BYTES_READ_TO_MEM -> shuffleReadMetrics._remoteBytesReadToMem, - shuffleRead.REMOTE_BYTES_READ_TO_DISK -> shuffleReadMetrics._remoteBytesReadToDisk, shuffleRead.LOCAL_BYTES_READ -> shuffleReadMetrics._localBytesRead, shuffleRead.FETCH_WAIT_TIME -> shuffleReadMetrics._fetchWaitTime, shuffleRead.RECORDS_READ -> shuffleReadMetrics._recordsRead, shuffleWrite.BYTES_WRITTEN -> shuffleWriteMetrics._bytesWritten, shuffleWrite.RECORDS_WRITTEN -> shuffleWriteMetrics._recordsWritten, shuffleWrite.WRITE_TIME -> shuffleWriteMetrics._writeTime, - shuffleWrite.AVERAGE_BLOCK_SIZE -> shuffleWriteMetrics._averageBlockSize, - shuffleWrite.UNDERESTIMATED_BLOCKS_NUM -> shuffleWriteMetrics._underestimatedBlocksNum, shuffleWrite.UNDERESTIMATED_BLOCKS_SIZE -> shuffleWriteMetrics._underestimatedBlocksSize, input.BYTES_READ -> inputMetrics._bytesRead, input.RECORDS_READ -> inputMetrics._recordsRead, output.BYTES_WRITTEN -> outputMetrics._bytesWritten, output.RECORDS_WRITTEN -> outputMetrics._recordsWritten - ) ++ testAccum.map(TEST_ACCUM -> _) ++ blockSizeDistributionAccums - - def blockSizeDistributionAccums(): LinkedHashMap[String, AccumulatorV2[_, _]] = { - val linkedHashMap = LinkedHashMap[String, AccumulatorV2[_, _]]() - shuffleWriteMetrics._blockSizeDistribution.zipWithIndex.foreach { - case (accum, index) => - linkedHashMap.put(shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX + index, accum) - } - linkedHashMap - } + ) ++ testAccum.map(TEST_ACCUM -> _) @transient private[spark] lazy val internalAccums: Seq[AccumulatorV2[_, _]] = nameToAccums.values.toIndexedSeq diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 5955baa59523..05b5ca236f4d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -72,16 +72,36 @@ private[spark] class SortShuffleWriter[K, V, C]( val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) - partitionLengths.foreach(writeMetrics.incBlockSizeDistribution(_)) + mapStatus match { case hc: HighlyCompressedMapStatus => - writeMetrics.setAverageBlockSize(hc.getAvgSize()) - (0 until partitionLengths.length).foreach { - case i => - if (partitionLengths(i) > mapStatus.getSizeForBlock(i)) { - writeMetrics.incUnderestimatedBlocksNum() - writeMetrics.incUnderestimatedBlocksSize(partitionLengths(i)) - } + val underestimatedLengths = partitionLengths.filter(_ > hc.getAvgSize) + writeMetrics.incUnderestimatedBlocksSize(underestimatedLengths.sum) + if (log.isDebugEnabled()) { + // Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), + // [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). + val lenDistribution = Array[Int](0, 0, 0, 0, 0, 0, 0, 0, 0) + partitionLengths.foreach { + case len: Long if len >= 0L && len < 1024L => lenDistribution(0) += 1 + case len: Long if len >= 1024L && len < 10240L => lenDistribution(1) += 1 + case len: Long if len >= 10240L && len < 102400L => lenDistribution(2) += 1 + case len: Long if len >= 102400L && len < 1048576L => lenDistribution(3) += 1 + case len: Long if len >= 1048576L && len < 10485760L => lenDistribution(4) += 1 + case len: Long if len >= 10485760L && len < 104857600L => lenDistribution(5) += 1 + case len: Long if len >= 104857600L && len < 1073741824L => lenDistribution(6) += 1 + case len: Long if len >= 1073741824L && len < 10737418240L => lenDistribution(7) += 1 + case len => lenDistribution(8) += 1 + } + val ranges = List[String]("[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", + "[1m, 10m)", "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g") + val distributeStr = ranges.zip(lenDistribution).map { + case (range, num) => s"$range:$num" + }.mkString(", ") + logDebug(s"For task ${context.partitionId()}.${context.attemptNumber()} in stage " + + s"${context.stageId()} (TID ${context.taskAttemptId()}), " + + s"the block sizes in MapStatus are inaccurate (average is ${hc.getAvgSize}, " + + s"${underestimatedLengths.length} blocks underestimated, " + + s"sum of sizes is ${underestimatedLengths.sum}), distribution is $distributeStr.") } case _ => // no-op } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index e476660c30e4..3f6261dcbff3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -400,25 +400,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskMetrics.foreach { m => val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) - logDebug(s"${m.shuffleReadMetrics.remoteBytesReadToMem} bytes shuffled to memory.") - logDebug(s"${m.shuffleReadMetrics.remoteBytesReadToDisk} bytes shuffled to disk.") - - if (m.shuffleWriteMetrics.averageBlockSize == 0) { - logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + - s"the block sizes in MapStatus are accurate, distribution is:") - } else { - logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + - s"the block sizes in MapStatus are inaccurate" + - s"(average=${m.shuffleWriteMetrics.averageBlockSize}, " + - s"${m.shuffleWriteMetrics.underestimatedBlocksNum} blocks underestimated, " + - s"sum of sizes is ${m.shuffleWriteMetrics.underestimatedBlocksSize}), " + - s"distribution is:") - } - val ranges = List[String]("[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", - "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g") - logDebug(ranges.map("%11s".format(_)).mkString(" ")) - logDebug(m.shuffleWriteMetrics.blockSizeDistribution.map("%11d".format(_)).mkString(" ")) + logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + + s"the block sizes in MapStatus are inaccurate, sum of sizes of underestimated" + + s" blocks is ${m.shuffleWriteMetrics.underestimatedBlocksSize}") } val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) diff --git a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala index c63b2e8ec54d..b546bcfb4acd 100644 --- a/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/TaskMetricsSuite.scala @@ -96,8 +96,6 @@ class TaskMetricsSuite extends SparkFunSuite { sr.incRemoteBytesRead(3L) sr.setRemoteBytesReadToMem(10L) sr.incRemoteBytesReadToMem(8L) - sr.setRemoteBytesReadToDisk(10L) - sr.incRemoteBytesReadToDisk(8L) sr.setLocalBytesRead(400L) sr.setLocalBytesRead(40L) sr.incLocalBytesRead(4L) @@ -115,7 +113,6 @@ class TaskMetricsSuite extends SparkFunSuite { assert(sr.localBlocksFetched == 24) assert(sr.remoteBytesRead == 36L) assert(sr.remoteBytesReadToMem === 18L) - assert(sr.remoteBytesReadToDisk === 18L) assert(sr.localBytesRead == 48L) assert(sr.fetchWaitTime == 60L) assert(sr.recordsRead == 72L) @@ -139,18 +136,11 @@ class TaskMetricsSuite extends SparkFunSuite { sw.decRecordsWritten(2L) sw.incWriteTime(300L) sw.incWriteTime(30L) - sw.incBlockSizeDistribution(10) - sw.incBlockSizeDistribution(2048) - sw.setAverageBlockSize(1024) - sw.incUnderestimatedBlocksNum() sw.incUnderestimatedBlocksSize(1024) // assert new values exist assert(sw.bytesWritten == 108L) assert(sw.recordsWritten == 216L) assert(sw.writeTime == 330L) - assert(sw.blockSizeDistribution(0) === 1 && sw.blockSizeDistribution(1) ===1) - assert(sw.averageBlockSize === 1024) - assert(sw.underestimatedBlocksNum === 1) assert(sw.underestimatedBlocksSize === 1024) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 46eaf908c52f..af36707262cf 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1921,171 +1921,87 @@ private[spark] object JsonProtocolSuite extends Assertions { | }, | { | "ID": 15, - | "Name": "${shuffleRead.REMOTE_BYTES_READ_TO_DISK}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 16, | "Name": "${shuffleRead.LOCAL_BYTES_READ}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 17, + | "ID": 16, | "Name": "${shuffleRead.FETCH_WAIT_TIME}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 18, + | "ID": 17, | "Name": "${shuffleRead.RECORDS_READ}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 19, + | "ID": 18, | "Name": "${shuffleWrite.BYTES_WRITTEN}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 20, + | "ID": 19, | "Name": "${shuffleWrite.RECORDS_WRITTEN}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 21, + | "ID": 20, | "Name": "${shuffleWrite.WRITE_TIME}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 22, - | "Name": "${shuffleWrite.AVERAGE_BLOCK_SIZE}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 23, - | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_NUM}", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 24, + | "ID": 21, | "Name": "${shuffleWrite.UNDERESTIMATED_BLOCKS_SIZE}", | "Update": 0, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 25, + | "ID": 22, | "Name": "${input.BYTES_READ}", | "Update": 2100, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 26, + | "ID": 23, | "Name": "${input.RECORDS_READ}", | "Update": 21, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 27, + | "ID": 24, | "Name": "${output.BYTES_WRITTEN}", | "Update": 1200, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 28, + | "ID": 25, | "Name": "${output.RECORDS_WRITTEN}", | "Update": 12, | "Internal": true, | "Count Failed Values": true | }, | { - | "ID": 29, + | "ID": 26, | "Name": "$TEST_ACCUM", | "Update": 0, | "Internal": true, | "Count Failed Values": true - | }, - | { - | "ID": 30, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}0", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 31, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}1", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 32, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}2", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 33, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}3", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 34, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}4", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 35, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}5", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 36, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}6", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 37, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}7", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true - | }, - | { - | "ID": 38, - | "Name": "${shuffleWrite.BLOCK_SIZE_DISTRIBUTION_PREFIX}8", - | "Update": 0, - | "Internal": true, - | "Count Failed Values": true | } | ] | } From b2ba1923f59f31117597e60be6c02c6477aca5de Mon Sep 17 00:00:00 2001 From: jinxing Date: Wed, 22 Mar 2017 14:48:14 +0800 Subject: [PATCH 16/28] change log --- .../sort/BypassMergeSortShuffleWriter.java | 42 +++++++++---------- .../shuffle/sort/UnsafeShuffleWriter.java | 40 +++++++++--------- .../spark/ui/jobs/JobProgressListener.scala | 3 +- 3 files changed, 43 insertions(+), 42 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 6952c8b09594..291b0c537204 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -190,25 +190,25 @@ public void write(Iterator> records) throws IOException { long len = partitionLengths[i]; if (len > mapStatus.getSizeForBlock(i)) { underestimatedBlocksNum++; - if (len >= 0L && len < 1024L) { - lenDistribution[0]++; - } else if (len >= 1024L && len < 10240L) { - lenDistribution[1]++; - } else if (len >= 10240L && len < 102400L) { - lenDistribution[2]++; - } else if (len >= 102400L && len < 1048576L ) { - lenDistribution[3]++; - } else if (len >= 1048576L && len < 10485760L) { - lenDistribution[4]++; - } else if (len >= 10485760L && len < 104857600L) { - lenDistribution[5]++; - } else if (len >= 104857600L && len < 1073741824L) { - lenDistribution[6]++; - } else if (len >= 1073741824L && len < 10737418240L) { - lenDistribution[7]++; - } else { - lenDistribution[8]++; - } + } + if (len >= 0L && len < 1024L) { + lenDistribution[0]++; + } else if (len >= 1024L && len < 10240L) { + lenDistribution[1]++; + } else if (len >= 10240L && len < 102400L) { + lenDistribution[2]++; + } else if (len >= 102400L && len < 1048576L ) { + lenDistribution[3]++; + } else if (len >= 1048576L && len < 10485760L) { + lenDistribution[4]++; + } else if (len >= 10485760L && len < 104857600L) { + lenDistribution[5]++; + } else if (len >= 104857600L && len < 1073741824L) { + lenDistribution[6]++; + } else if (len >= 1073741824L && len < 10737418240L) { + lenDistribution[7]++; + } else { + lenDistribution[8]++; } String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; @@ -217,8 +217,8 @@ public void write(Iterator> records) throws IOException { rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; } logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + - "inaccurate (average is {}, {} blocks underestimated, sum of sizes is {})," + - " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), + "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + + " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), underestimatedBlocksNum, underestimatedBlocksSize, String.join(", ", rangesAndDistribute)); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 5b9eb6312996..8492de7d6b79 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -250,25 +250,25 @@ void closeAndWriteOutput() throws IOException { long len = partitionLengths[i]; if (len > mapStatus.getSizeForBlock(i)) { underestimatedBlocksNum++; - if (len >= 0L && len < 1024L) { - lenDistribution[0]++; - } else if (len >= 1024L && len < 10240L) { - lenDistribution[1]++; - } else if (len >= 10240L && len < 102400L) { - lenDistribution[2]++; - } else if (len >= 102400L && len < 1048576L ) { - lenDistribution[3]++; - } else if (len >= 1048576L && len < 10485760L) { - lenDistribution[4]++; - } else if (len >= 10485760L && len < 104857600L) { - lenDistribution[5]++; - } else if (len >= 104857600L && len < 1073741824L) { - lenDistribution[6]++; - } else if (len >= 1073741824L && len < 10737418240L) { - lenDistribution[7]++; - } else { - lenDistribution[8]++; - } + } + if (len >= 0L && len < 1024L) { + lenDistribution[0]++; + } else if (len >= 1024L && len < 10240L) { + lenDistribution[1]++; + } else if (len >= 10240L && len < 102400L) { + lenDistribution[2]++; + } else if (len >= 102400L && len < 1048576L ) { + lenDistribution[3]++; + } else if (len >= 1048576L && len < 10485760L) { + lenDistribution[4]++; + } else if (len >= 10485760L && len < 104857600L) { + lenDistribution[5]++; + } else if (len >= 104857600L && len < 1073741824L) { + lenDistribution[6]++; + } else if (len >= 1073741824L && len < 10737418240L) { + lenDistribution[7]++; + } else { + lenDistribution[8]++; } String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; @@ -277,7 +277,7 @@ void closeAndWriteOutput() throws IOException { rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; } logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + - "inaccurate (average is {}, {} blocks underestimated, sum of sizes is {})," + + "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), underestimatedBlocksNum, underestimatedBlocksSize, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 3f6261dcbff3..76f1e8b9df53 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -400,7 +400,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskMetrics.foreach { m => val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) - logDebug(s"${m.shuffleReadMetrics.remoteBytesReadToMem} bytes shuffled to memory.") + logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + + s"${m.shuffleReadMetrics.remoteBytesReadToMem} bytes shuffled to memory.") logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + s"the block sizes in MapStatus are inaccurate, sum of sizes of underestimated" + s" blocks is ${m.shuffleWriteMetrics.underestimatedBlocksSize}") From e86ec285bfa87d3801352df79470b7f2893f8b76 Mon Sep 17 00:00:00 2001 From: jinxing Date: Wed, 22 Mar 2017 14:56:57 +0800 Subject: [PATCH 17/28] small fix --- .../apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 9 +++------ .../main/scala/org/apache/spark/util/JsonProtocol.scala | 1 - 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 8492de7d6b79..64a59e96ef85 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -35,16 +35,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.io.output.CloseShieldOutputStream; -import org.apache.commons.io.output.CountingOutputStream; -import org.apache.spark.Partitioner; -import org.apache.spark.ShuffleDependency; -import org.apache.spark.SparkConf; -import org.apache.spark.TaskContext; +import org.apache.spark.*; import org.apache.spark.annotation.Private; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; +import org.apache.commons.io.output.CloseShieldOutputStream; +import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.HighlyCompressedMapStatus; diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index a3b6dae70775..557a689c8e8c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -21,7 +21,6 @@ import java.util.{Properties, UUID} import scala.collection.JavaConverters._ import scala.collection.Map -import scala.collection.mutable.LinkedHashMap import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule From 1720b5e8e1d1c142cc414ae39cb620727bb92d5f Mon Sep 17 00:00:00 2001 From: jinxing Date: Wed, 22 Mar 2017 14:59:57 +0800 Subject: [PATCH 18/28] small fix --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 557a689c8e8c..4b4d2d10cbf8 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -332,8 +332,7 @@ private[spark] object JsonProtocol { ("Remote Bytes Read" -> taskMetrics.shuffleReadMetrics.remoteBytesRead) ~ ("Local Bytes Read" -> taskMetrics.shuffleReadMetrics.localBytesRead) ~ ("Total Records Read" -> taskMetrics.shuffleReadMetrics.recordsRead) - - var shuffleWriteMetrics: JValue = + val shuffleWriteMetrics: JValue = ("Shuffle Bytes Written" -> taskMetrics.shuffleWriteMetrics.bytesWritten) ~ ("Shuffle Write Time" -> taskMetrics.shuffleWriteMetrics.writeTime) ~ ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) From c9276c2051436a1b27c844c4aae87110bc39c002 Mon Sep 17 00:00:00 2001 From: jinxing Date: Wed, 22 Mar 2017 16:27:03 +0800 Subject: [PATCH 19/28] changed log. --- .../org/apache/spark/ui/jobs/JobProgressListener.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 76f1e8b9df53..ca3cc8d9c97b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -401,10 +401,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + - s"${m.shuffleReadMetrics.remoteBytesReadToMem} bytes shuffled to memory.") - logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + - s"the block sizes in MapStatus are inaccurate, sum of sizes of underestimated" + - s" blocks is ${m.shuffleWriteMetrics.underestimatedBlocksSize}") + s"${m.shuffleReadMetrics.remoteBytesReadToMem} bytes shuffle-read to memory.") + logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), total " + + s"size of underestimated blocks is ${m.shuffleWriteMetrics.underestimatedBlocksSize}") } val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) From 2b891666c5daf014b464ff1ca623a9a7b6b17879 Mon Sep 17 00:00:00 2001 From: jinxing Date: Thu, 23 Mar 2017 22:40:42 +0800 Subject: [PATCH 20/28] add remoteBytesReadToMem and underestimatedBlocksSize to api --- .../spark/status/api/v1/AllStagesResource.scala | 8 ++++++-- .../scala/org/apache/spark/status/api/v1/api.scala | 8 ++++++-- .../apache/spark/ui/jobs/JobProgressListener.scala | 4 ---- .../main/scala/org/apache/spark/ui/jobs/UIData.scala | 12 ++++++++---- .../scala/org/apache/spark/util/JsonProtocol.scala | 9 ++++++++- 5 files changed, 28 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala index 1818935392eb..6037c62d3957 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala @@ -200,6 +200,7 @@ private[v1] object AllStagesResource { readBytes = submetricQuantiles(_.totalBytesRead), readRecords = submetricQuantiles(_.recordsRead), remoteBytesRead = submetricQuantiles(_.remoteBytesRead), + remoteBytesReadToMem = submetricQuantiles(_.remoteBytesReadToMem), remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched), localBlocksFetched = submetricQuantiles(_.localBlocksFetched), totalBlocksFetched = submetricQuantiles(_.totalBlocksFetched), @@ -216,7 +217,8 @@ private[v1] object AllStagesResource { def build: ShuffleWriteMetricDistributions = new ShuffleWriteMetricDistributions( writeBytes = submetricQuantiles(_.bytesWritten), writeRecords = submetricQuantiles(_.recordsWritten), - writeTime = submetricQuantiles(_.writeTime) + writeTime = submetricQuantiles(_.writeTime), + underestimatedBlocksSize = submetricQuantiles(_.underestimatedBlocksSize) ) }.build @@ -281,6 +283,7 @@ private[v1] object AllStagesResource { localBlocksFetched = internal.localBlocksFetched, fetchWaitTime = internal.fetchWaitTime, remoteBytesRead = internal.remoteBytesRead, + remoteBytesReadToMem = internal.remoteBytesReadToMem, localBytesRead = internal.localBytesRead, recordsRead = internal.recordsRead ) @@ -290,7 +293,8 @@ private[v1] object AllStagesResource { new ShuffleWriteMetrics( bytesWritten = internal.bytesWritten, writeTime = internal.writeTime, - recordsWritten = internal.recordsWritten + recordsWritten = internal.recordsWritten, + underestimatedBlocksSize = internal.underestimatedBlocksSize ) } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 5b9227350eda..ebf62bf0d66c 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -196,13 +196,15 @@ class ShuffleReadMetrics private[spark]( val localBlocksFetched: Long, val fetchWaitTime: Long, val remoteBytesRead: Long, + val remoteBytesReadToMem: Long, val localBytesRead: Long, val recordsRead: Long) class ShuffleWriteMetrics private[spark]( val bytesWritten: Long, val writeTime: Long, - val recordsWritten: Long) + val recordsWritten: Long, + val underestimatedBlocksSize: Long) class TaskMetricDistributions private[spark]( val quantiles: IndexedSeq[Double], @@ -237,12 +239,14 @@ class ShuffleReadMetricDistributions private[spark]( val localBlocksFetched: IndexedSeq[Double], val fetchWaitTime: IndexedSeq[Double], val remoteBytesRead: IndexedSeq[Double], + val remoteBytesReadToMem: IndexedSeq[Double], val totalBlocksFetched: IndexedSeq[Double]) class ShuffleWriteMetricDistributions private[spark]( val writeBytes: IndexedSeq[Double], val writeRecords: IndexedSeq[Double], - val writeTime: IndexedSeq[Double]) + val writeTime: IndexedSeq[Double], + val underestimatedBlocksSize: IndexedSeq[Double]) class AccumulableInfo private[spark]( val id: Long, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ca3cc8d9c97b..e87caff42643 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -400,10 +400,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskMetrics.foreach { m => val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.metrics) updateAggregateMetrics(stageData, info.executorId, m, oldMetrics) - logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), " + - s"${m.shuffleReadMetrics.remoteBytesReadToMem} bytes shuffle-read to memory.") - logDebug(s"For task ${info.id} in stage ${taskEnd.stageId} (TID ${info.taskId}), total " + - s"size of underestimated blocks is ${m.shuffleWriteMetrics.underestimatedBlocksSize}") } val taskData = stageData.taskData.getOrElseUpdate(info.taskId, TaskUIData(info, None)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 073f7edfc2fe..25c2ab293503 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -238,6 +238,7 @@ private[spark] object UIData { remoteBlocksFetched: Long, localBlocksFetched: Long, remoteBytesRead: Long, + remoteBytesReadToMem: Long, localBytesRead: Long, fetchWaitTime: Long, recordsRead: Long, @@ -261,6 +262,7 @@ private[spark] object UIData { remoteBlocksFetched = metrics.remoteBlocksFetched, localBlocksFetched = metrics.localBlocksFetched, remoteBytesRead = metrics.remoteBytesRead, + remoteBytesReadToMem = metrics.remoteBytesReadToMem, localBytesRead = metrics.localBytesRead, fetchWaitTime = metrics.fetchWaitTime, recordsRead = metrics.recordsRead, @@ -269,13 +271,14 @@ private[spark] object UIData { ) } } - private val EMPTY = ShuffleReadMetricsUIData(0, 0, 0, 0, 0, 0, 0, 0) + private val EMPTY = ShuffleReadMetricsUIData(0, 0, 0, 0, 0, 0, 0, 0, 0) } case class ShuffleWriteMetricsUIData( bytesWritten: Long, recordsWritten: Long, - writeTime: Long) + writeTime: Long, + underestimatedBlocksSize: Long) object ShuffleWriteMetricsUIData { def apply(metrics: ShuffleWriteMetrics): ShuffleWriteMetricsUIData = { @@ -285,11 +288,12 @@ private[spark] object UIData { new ShuffleWriteMetricsUIData( bytesWritten = metrics.bytesWritten, recordsWritten = metrics.recordsWritten, - writeTime = metrics.writeTime + writeTime = metrics.writeTime, + underestimatedBlocksSize = metrics.underestimatedBlocksSize ) } } - private val EMPTY = ShuffleWriteMetricsUIData(0, 0, 0) + private val EMPTY = ShuffleWriteMetricsUIData(0, 0, 0, 0) } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 4b4d2d10cbf8..92470cca908c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -330,12 +330,15 @@ private[spark] object JsonProtocol { ("Local Blocks Fetched" -> taskMetrics.shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> taskMetrics.shuffleReadMetrics.fetchWaitTime) ~ ("Remote Bytes Read" -> taskMetrics.shuffleReadMetrics.remoteBytesRead) ~ + ("Remote Bytes Read To Mem" -> taskMetrics.shuffleReadMetrics.remoteBytesReadToMem) ~ ("Local Bytes Read" -> taskMetrics.shuffleReadMetrics.localBytesRead) ~ ("Total Records Read" -> taskMetrics.shuffleReadMetrics.recordsRead) val shuffleWriteMetrics: JValue = ("Shuffle Bytes Written" -> taskMetrics.shuffleWriteMetrics.bytesWritten) ~ ("Shuffle Write Time" -> taskMetrics.shuffleWriteMetrics.writeTime) ~ - ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) + ("Shuffle Records Written" -> taskMetrics.shuffleWriteMetrics.recordsWritten) ~ + ("Underestimated Blocks Size Writen" -> + taskMetrics.shuffleWriteMetrics.underestimatedBlocksSize) val inputMetrics: JValue = ("Bytes Read" -> taskMetrics.inputMetrics.bytesRead) ~ ("Records Read" -> taskMetrics.inputMetrics.recordsRead) @@ -791,6 +794,8 @@ private[spark] object JsonProtocol { readMetrics.incRemoteBlocksFetched((readJson \ "Remote Blocks Fetched").extract[Int]) readMetrics.incLocalBlocksFetched((readJson \ "Local Blocks Fetched").extract[Int]) readMetrics.incRemoteBytesRead((readJson \ "Remote Bytes Read").extract[Long]) + Utils.jsonOption(readJson \ "Remote Bytes Read To Mem") + .foreach { v => readMetrics.incRemoteBytesReadToMem(v.extract[Long])} readMetrics.incLocalBytesRead( Utils.jsonOption(readJson \ "Local Bytes Read").map(_.extract[Long]).getOrElse(0L)) readMetrics.incFetchWaitTime((readJson \ "Fetch Wait Time").extract[Long]) @@ -807,6 +812,8 @@ private[spark] object JsonProtocol { writeMetrics.incRecordsWritten( Utils.jsonOption(writeJson \ "Shuffle Records Written").map(_.extract[Long]).getOrElse(0L)) writeMetrics.incWriteTime((writeJson \ "Shuffle Write Time").extract[Long]) + Utils.jsonOption(writeJson \ "Underestimated Blocks Size Writen") + .foreach { v => writeMetrics.incUnderestimatedBlocksSize(v.extract[Long])} } // Output metrics From 6a96c3b2858ee53bad073040217182c19d5e7db0 Mon Sep 17 00:00:00 2001 From: jinxing Date: Thu, 23 Mar 2017 23:03:02 +0800 Subject: [PATCH 21/28] fix log --- .../sort/BypassMergeSortShuffleWriter.java | 24 +++++++++---------- .../shuffle/sort/UnsafeShuffleWriter.java | 24 +++++++++---------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 291b0c537204..886b77f02ad2 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -210,19 +210,19 @@ public void write(Iterator> records) throws IOException { } else { lenDistribution[8]++; } - String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", - "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; - String[] rangesAndDistribute = new String[9]; - for (int j = 0; j < 9; j++) { - rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; - } - logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + - "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + - " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), - taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), - underestimatedBlocksNum, underestimatedBlocksSize, - String.join(", ", rangesAndDistribute)); } + String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", + "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; + String[] rangesAndDistribute = new String[9]; + for (int j = 0; j < 9; j++) { + rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; + } + logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + + "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + + " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), + taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), + underestimatedBlocksNum, underestimatedBlocksSize, + String.join(", ", rangesAndDistribute)); } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 64a59e96ef85..6ffb91123718 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -267,19 +267,19 @@ void closeAndWriteOutput() throws IOException { } else { lenDistribution[8]++; } - String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", - "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; - String[] rangesAndDistribute = new String[9]; - for (int j = 0; j < 9; j++) { - rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; - } - logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + - "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + - " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), - taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), - underestimatedBlocksNum, underestimatedBlocksSize, - String.join(", ", rangesAndDistribute)); } + String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", + "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; + String[] rangesAndDistribute = new String[9]; + for (int j = 0; j < 9; j++) { + rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; + } + logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + + "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + + " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), + taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), + underestimatedBlocksNum, underestimatedBlocksSize, + String.join(", ", rangesAndDistribute)); } } } From 4f992fcd363bc7632da71093dfc8f61477d0cd1e Mon Sep 17 00:00:00 2001 From: jinxing Date: Fri, 24 Mar 2017 00:08:07 +0800 Subject: [PATCH 22/28] show distribution with probabilities. --- .../sort/BypassMergeSortShuffleWriter.java | 49 +++++-------------- .../shuffle/sort/UnsafeShuffleWriter.java | 49 +++++-------------- .../shuffle/sort/SortShuffleWriter.scala | 36 +++++--------- .../org/apache/spark/util/Distribution.scala | 2 +- 4 files changed, 37 insertions(+), 99 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 886b77f02ad2..fd6c08607636 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -47,6 +47,7 @@ import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.*; +import org.apache.spark.util.Distribution; import org.apache.spark.util.Utils; /** @@ -181,48 +182,22 @@ public void write(Iterator> records) throws IOException { } } writeMetrics.incUnderestimatedBlocksSize(underestimatedBlocksSize); - if (logger.isDebugEnabled()) { + if (logger.isDebugEnabled() && partitionLengths.length > 0) { int underestimatedBlocksNum = 0; - // Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), - // [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). - int[] lenDistribution = {0, 0, 0, 0, 0, 0, 0, 0, 0}; + // Distribution of sizes in MapStatus. + double[] cp = new double[partitionLengths.length]; for (int i = 0; i < partitionLengths.length; i++) { - long len = partitionLengths[i]; - if (len > mapStatus.getSizeForBlock(i)) { - underestimatedBlocksNum++; - } - if (len >= 0L && len < 1024L) { - lenDistribution[0]++; - } else if (len >= 1024L && len < 10240L) { - lenDistribution[1]++; - } else if (len >= 10240L && len < 102400L) { - lenDistribution[2]++; - } else if (len >= 102400L && len < 1048576L ) { - lenDistribution[3]++; - } else if (len >= 1048576L && len < 10485760L) { - lenDistribution[4]++; - } else if (len >= 10485760L && len < 104857600L) { - lenDistribution[5]++; - } else if (len >= 104857600L && len < 1073741824L) { - lenDistribution[6]++; - } else if (len >= 1073741824L && len < 10737418240L) { - lenDistribution[7]++; - } else { - lenDistribution[8]++; - } - } - String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", - "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; - String[] rangesAndDistribute = new String[9]; - for (int j = 0; j < 9; j++) { - rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; + cp[i] = partitionLengths[i]; } + Distribution distribution = new Distribution(cp, 0, cp.length); + double[] probabilities = {0.0, 0.25, 0.5, 0.75, 1.0}; + String distributionStr = distribution.getQuantiles(probabilities).mkString(", "); logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + - " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), - taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), - underestimatedBlocksNum, underestimatedBlocksSize, - String.join(", ", rangesAndDistribute)); + " distribution at the given probabilities(0, 0.25, 0.5, 0.75, 1.0) is {}.", + taskContext.partitionId(), taskContext.attemptNumber(), taskContext.stageId(), + taskContext.taskAttemptId(), hc.getAvgSize(), + underestimatedBlocksNum, underestimatedBlocksSize, distributionStr); } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 6ffb91123718..2eb7873207b9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -54,6 +54,7 @@ import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; +import org.apache.spark.util.Distribution; import org.apache.spark.util.Utils; @Private @@ -238,48 +239,22 @@ void closeAndWriteOutput() throws IOException { } } writeMetrics.incUnderestimatedBlocksSize(underestimatedBlocksSize); - if (logger.isDebugEnabled()) { + if (logger.isDebugEnabled() && partitionLengths.length > 0) { int underestimatedBlocksNum = 0; - // Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), - // [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). - int[] lenDistribution = {0, 0, 0, 0, 0, 0, 0, 0, 0}; + // Distribution of sizes in MapStatus. + double[] cp = new double[partitionLengths.length]; for (int i = 0; i < partitionLengths.length; i++) { - long len = partitionLengths[i]; - if (len > mapStatus.getSizeForBlock(i)) { - underestimatedBlocksNum++; - } - if (len >= 0L && len < 1024L) { - lenDistribution[0]++; - } else if (len >= 1024L && len < 10240L) { - lenDistribution[1]++; - } else if (len >= 10240L && len < 102400L) { - lenDistribution[2]++; - } else if (len >= 102400L && len < 1048576L ) { - lenDistribution[3]++; - } else if (len >= 1048576L && len < 10485760L) { - lenDistribution[4]++; - } else if (len >= 10485760L && len < 104857600L) { - lenDistribution[5]++; - } else if (len >= 104857600L && len < 1073741824L) { - lenDistribution[6]++; - } else if (len >= 1073741824L && len < 10737418240L) { - lenDistribution[7]++; - } else { - lenDistribution[8]++; - } - } - String[] ranges = {"[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", "[1m, 10m)", - "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g"}; - String[] rangesAndDistribute = new String[9]; - for (int j = 0; j < 9; j++) { - rangesAndDistribute[j] = ranges[j] + ":" + lenDistribution[j]; + cp[i] = partitionLengths[i]; } + Distribution distribution = new Distribution(cp, 0, cp.length); + double[] probabilities = {0.0, 0.25, 0.5, 0.75, 1.0}; + String distributionStr = distribution.getQuantiles(probabilities).mkString(", "); logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + - " distribution is {}.", taskContext.partitionId(), taskContext.attemptNumber(), - taskContext.stageId(), taskContext.taskAttemptId(), hc.getAvgSize(), - underestimatedBlocksNum, underestimatedBlocksSize, - String.join(", ", rangesAndDistribute)); + " distribution at the given probabilities(0, 0.25, 0.5, 0.75, 1.0) is {}.", + taskContext.partitionId(), taskContext.attemptNumber(), taskContext.stageId(), + taskContext.taskAttemptId(), hc.getAvgSize(), + underestimatedBlocksNum, underestimatedBlocksSize, distributionStr); } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 05b5ca236f4d..c7bea2472b67 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -22,7 +22,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{HighlyCompressedMapStatus, MapStatus} import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} import org.apache.spark.storage.ShuffleBlockId -import org.apache.spark.util.Utils +import org.apache.spark.util.{Distribution, Utils} import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( @@ -78,30 +78,18 @@ private[spark] class SortShuffleWriter[K, V, C]( val underestimatedLengths = partitionLengths.filter(_ > hc.getAvgSize) writeMetrics.incUnderestimatedBlocksSize(underestimatedLengths.sum) if (log.isDebugEnabled()) { - // Distribution of sizes in MapStatus. The ranges are: [0, 1k), [1k, 10k), [10k, 100k), - // [100k, 1m), [1m, 10m), [10m, 100m), [100m, 1g), [1g, 10g), [10g, Long.MaxValue). - val lenDistribution = Array[Int](0, 0, 0, 0, 0, 0, 0, 0, 0) - partitionLengths.foreach { - case len: Long if len >= 0L && len < 1024L => lenDistribution(0) += 1 - case len: Long if len >= 1024L && len < 10240L => lenDistribution(1) += 1 - case len: Long if len >= 10240L && len < 102400L => lenDistribution(2) += 1 - case len: Long if len >= 102400L && len < 1048576L => lenDistribution(3) += 1 - case len: Long if len >= 1048576L && len < 10485760L => lenDistribution(4) += 1 - case len: Long if len >= 10485760L && len < 104857600L => lenDistribution(5) += 1 - case len: Long if len >= 104857600L && len < 1073741824L => lenDistribution(6) += 1 - case len: Long if len >= 1073741824L && len < 10737418240L => lenDistribution(7) += 1 - case len => lenDistribution(8) += 1 + // Distribution of sizes in MapStatus. + Distribution(partitionLengths.map(_.toDouble)) match { + case Some(distribution) => + val distributionStr = distribution.getQuantiles().mkString(", ") + logDebug(s"For task ${context.partitionId()}.${context.attemptNumber()} in stage" + + s" ${context.stageId()} (TID ${context.taskAttemptId()}), the block sizes in" + + s" MapStatus are inaccurate (average is ${hc.getAvgSize}," + + s" ${underestimatedLengths.length} blocks underestimated, sum of sizes is" + + s" ${underestimatedLengths.sum}), distribution at the given probabilities" + + s" (0, 0.25, 0.5, 0.75, 1.0) is $distributionStr.") + case None => // no-op } - val ranges = List[String]("[0, 1k)", "[1k, 10k)", "[10k, 100k)", "[100k, 1m)", - "[1m, 10m)", "[10m, 100m)", "[100m, 1g)", "[1g, 10g)", ">10g") - val distributeStr = ranges.zip(lenDistribution).map { - case (range, num) => s"$range:$num" - }.mkString(", ") - logDebug(s"For task ${context.partitionId()}.${context.attemptNumber()} in stage " + - s"${context.stageId()} (TID ${context.taskAttemptId()}), " + - s"the block sizes in MapStatus are inaccurate (average is ${hc.getAvgSize}, " + - s"${underestimatedLengths.length} blocks underestimated, " + - s"sum of sizes is ${underestimatedLengths.sum}), distribution is $distributeStr.") } case _ => // no-op } diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 950b69f7db64..b06ce9416450 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -42,7 +42,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va * given from 0 to 1 * @param probabilities */ - def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) + def getQuantiles(probabilities: Array[Double] = defaultProbabilities) : IndexedSeq[Double] = { probabilities.toIndexedSeq.map { p: Double => data(closestIndex(p)) } } From c58cb7eefe8915c10090da0a43c74c2582e7d773 Mon Sep 17 00:00:00 2001 From: jinxing Date: Fri, 24 Mar 2017 12:51:07 +0800 Subject: [PATCH 23/28] fix --- .../java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 2eb7873207b9..b87f1ec6fd58 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -17,10 +17,10 @@ package org.apache.spark.shuffle.sort; +import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; import java.util.Iterator; -import javax.annotation.Nullable; import scala.Option; import scala.Product2; From 0efa348d1beccf76da702d06ee77c8aac2aebb12 Mon Sep 17 00:00:00 2001 From: jinxing Date: Fri, 24 Mar 2017 22:40:52 +0800 Subject: [PATCH 24/28] change MimaExcludes. --- project/MimaExcludes.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bd4528bd2126..56b44af61b94 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -955,6 +955,12 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.classification.RandomForestClassificationModel.setFeatureSubsetStrategy"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.numTrees"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.regression.RandomForestRegressionModel.setFeatureSubsetStrategy") + ) ++ Seq( + // [SPARK-19937] Add remote bytes read to memory and underestimated blocks size to metrics. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetrics.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleReadMetricDistributions.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleWriteMetrics.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.status.api.v1.ShuffleWriteMetricDistributions.this") ) } From c26ea561579a76f3370fc2ebc07314e788dc32a1 Mon Sep 17 00:00:00 2001 From: jinxing Date: Sat, 25 Mar 2017 23:00:00 +0800 Subject: [PATCH 25/28] fix json expectations. --- .../one_stage_attempt_json_expectation.json | 32 ++- .../one_stage_json_expectation.json | 32 ++- .../stage_task_list_expectation.json | 80 +++++-- ...multi_attempt_app_json_1__expectation.json | 32 ++- ...multi_attempt_app_json_2__expectation.json | 32 ++- ...k_list_w__offset___length_expectation.json | 200 +++++++++++++----- ...stage_task_list_w__sortBy_expectation.json | 80 +++++-- ...tBy_short_names___runtime_expectation.json | 80 +++++-- ...rtBy_short_names__runtime_expectation.json | 80 +++++-- ...mmary_w__custom_quantiles_expectation.json | 4 +- ...sk_summary_w_shuffle_read_expectation.json | 4 +- ...k_summary_w_shuffle_write_expectation.json | 4 +- ...age_with_accumulable_json_expectation.json | 32 ++- .../apache/spark/util/JsonProtocolSuite.scala | 12 +- 14 files changed, 528 insertions(+), 176 deletions(-) diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index c2f450ba87c6..7c7b4cb31079 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -60,13 +60,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 94000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -105,13 +107,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 98000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -150,13 +154,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 76000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -195,13 +201,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1647, "writeTime" : 83000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -240,13 +248,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1645, "writeTime" : 101000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -285,13 +295,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 73000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -330,13 +342,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 88000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -375,13 +389,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 79000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } } diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 506859ae545b..ba70852743bc 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -60,13 +60,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 94000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -105,13 +107,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 98000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -150,13 +154,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 76000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -195,13 +201,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1647, "writeTime" : 83000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -240,13 +248,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1645, "writeTime" : 101000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -285,13 +295,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 73000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -330,13 +342,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 88000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -375,13 +389,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1648, "writeTime" : 79000, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json index f4cec68fbfdf..10bf32aeba55 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -33,13 +33,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3842811, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -77,13 +79,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3934399, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -121,13 +125,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89885, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -165,13 +171,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 1311694, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -209,13 +217,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 83022, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -253,13 +263,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3675510, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -297,13 +309,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 4016617, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -341,13 +355,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 2579051, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -385,13 +401,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 121551, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -429,13 +447,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 101664, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -473,13 +493,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94709, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -517,13 +539,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94507, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -561,13 +585,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -605,13 +631,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -649,13 +677,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95646, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -693,13 +723,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 602780, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -737,13 +769,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108320, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -781,13 +815,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -825,13 +861,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -869,13 +907,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95788, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json index 496a21c328da..4c3af6064fbb 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -38,13 +38,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -87,13 +89,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -136,13 +140,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -185,13 +191,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -234,13 +242,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -283,13 +293,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -332,13 +344,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -381,13 +395,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json index 4328dc753c5d..8d01b10a30f4 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -38,13 +38,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -87,13 +89,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -136,13 +140,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -185,13 +191,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -234,13 +242,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -283,13 +293,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -332,13 +344,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, { @@ -381,13 +395,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json index 8c571430f3a1..6b5424b86773 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -33,13 +33,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94709, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -77,13 +79,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94507, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -121,13 +125,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -165,13 +171,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -209,13 +217,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95646, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -253,13 +263,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 602780, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -297,13 +309,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108320, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -341,13 +355,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -385,13 +401,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -429,13 +447,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95788, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -473,13 +493,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97716, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -517,13 +539,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100270, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -561,13 +585,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 143427, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -605,13 +631,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 91844, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -649,13 +677,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 157194, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -693,13 +723,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94134, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -737,13 +769,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108213, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -781,13 +815,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102019, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -825,13 +861,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 104299, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -869,13 +907,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 114938, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -913,13 +953,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 119770, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -957,13 +999,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 92619, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1001,13 +1045,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89603, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1045,13 +1091,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 118329, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1089,13 +1137,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 127746, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1133,13 +1183,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 160963, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1177,13 +1229,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 123855, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1221,13 +1275,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 111869, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1265,13 +1321,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 131158, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1309,13 +1367,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 98748, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1353,13 +1413,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94792, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1397,13 +1459,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90765, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1441,13 +1505,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 103713, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1485,13 +1551,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 171516, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1529,13 +1597,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 98293, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1573,13 +1643,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 92985, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1617,13 +1689,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 113322, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1661,13 +1735,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 103015, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1705,13 +1781,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 139844, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1749,13 +1827,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94984, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1793,13 +1873,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90836, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1837,13 +1919,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 96013, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1881,13 +1965,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89664, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1925,13 +2011,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 92835, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -1969,13 +2057,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90506, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -2013,13 +2103,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108309, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -2057,13 +2149,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90329, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -2101,13 +2195,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 96849, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -2145,13 +2241,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97521, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -2189,13 +2287,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100753, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index 0bd614bdc756..334eb6dd61dc 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -33,13 +33,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 4016617, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -77,13 +79,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3934399, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -121,13 +125,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3675510, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -165,13 +171,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3842811, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -209,13 +217,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 1311694, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -253,13 +263,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 83022, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -297,13 +309,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 2579051, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -341,13 +355,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89885, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -385,13 +401,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 143427, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -429,13 +447,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -473,13 +493,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -517,13 +539,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100270, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -561,13 +585,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 101664, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -605,13 +631,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108320, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -649,13 +677,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95788, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -693,13 +723,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95646, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -737,13 +769,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97716, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -781,13 +815,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 121551, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -825,13 +861,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -869,13 +907,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index 0bd614bdc756..334eb6dd61dc 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -33,13 +33,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 4016617, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -77,13 +79,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3934399, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -121,13 +125,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3675510, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -165,13 +171,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 3842811, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -209,13 +217,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 1311694, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -253,13 +263,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 83022, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -297,13 +309,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 2579051, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -341,13 +355,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89885, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -385,13 +401,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 143427, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -429,13 +447,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100836, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -473,13 +493,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 99944, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -517,13 +539,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100270, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -561,13 +585,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 101664, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -605,13 +631,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 108320, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -649,13 +677,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95788, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -693,13 +723,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95646, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -737,13 +769,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97716, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -781,13 +815,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 121551, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -825,13 +861,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102476, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -869,13 +907,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95004, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index b58f1a51ba48..e2c3ffbf6a1e 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -33,13 +33,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 94792, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -77,13 +79,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90765, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -121,13 +125,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 171516, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -165,13 +171,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 96849, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -209,13 +217,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 97521, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -253,13 +263,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 101750, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -297,13 +309,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 95848, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -341,13 +355,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89603, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -385,13 +401,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 98748, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -429,13 +447,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 103713, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -473,13 +493,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 96013, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -517,13 +539,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 100753, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -561,13 +585,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102779, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -605,13 +631,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 102159, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -649,13 +677,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 98472, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -693,13 +723,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 133964, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -737,13 +769,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 98293, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -781,13 +815,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 103015, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -825,13 +861,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 90836, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } }, { @@ -869,13 +907,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 1710, "writeTime" : 89664, - "recordsWritten" : 10 + "recordsWritten" : 10, + "underestimatedBlocksSize": 0 } } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json index 0ed609d5b7f9..a6032afa56cb 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w__custom_quantiles_expectation.json @@ -24,11 +24,13 @@ "localBlocksFetched" : [ 0.0, 0.0, 0.0 ], "fetchWaitTime" : [ 0.0, 0.0, 0.0 ], "remoteBytesRead" : [ 0.0, 0.0, 0.0 ], + "remoteBytesReadToMem" : [ 0.0, 0.0, 0.0 ], "totalBlocksFetched" : [ 0.0, 0.0, 0.0 ] }, "shuffleWriteMetrics" : { "writeBytes" : [ 1710.0, 1710.0, 1710.0 ], "writeRecords" : [ 10.0, 10.0, 10.0 ], - "writeTime" : [ 89437.0, 102159.0, 4016617.0 ] + "writeTime" : [ 89437.0, 102159.0, 4016617.0 ], + "underestimatedBlocksSize": [0.0, 0.0, 0.0] } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json index 6d230ac65377..2700d6de3c1b 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_read_expectation.json @@ -24,11 +24,13 @@ "localBlocksFetched" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ], "fetchWaitTime" : [ 0.0, 0.0, 0.0, 1.0, 1.0 ], "remoteBytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBytesReadToMem" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "totalBlocksFetched" : [ 100.0, 100.0, 100.0, 100.0, 100.0 ] }, "shuffleWriteMetrics" : { "writeBytes" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "writeRecords" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], - "writeTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] + "writeTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "underestimatedBlocksSize": [ 0.0, 0.0, 0.0, 0.0, 0.0 ] } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json index aea0f5413d8b..9c46ac302fd1 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_summary_w_shuffle_write_expectation.json @@ -24,11 +24,13 @@ "localBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "fetchWaitTime" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "remoteBytesRead" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], + "remoteBytesReadToMem" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ], "totalBlocksFetched" : [ 0.0, 0.0, 0.0, 0.0, 0.0 ] }, "shuffleWriteMetrics" : { "writeBytes" : [ 1710.0, 1710.0, 1710.0, 1710.0, 1710.0 ], "writeRecords" : [ 10.0, 10.0, 10.0, 10.0, 10.0 ], - "writeTime" : [ 90329.0, 95848.0, 102159.0, 121551.0, 2579051.0 ] + "writeTime" : [ 90329.0, 95848.0, 102159.0, 121551.0, 2579051.0 ], + "underestimatedBlocksSize": [ 0.0, 0.0, 0.0, 0.0, 0.0 ] } } diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index a449926ee7dc..bf89496fd0a7 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -69,13 +69,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -119,13 +121,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -169,13 +173,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -219,13 +225,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -269,13 +277,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -319,13 +329,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -369,13 +381,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } }, @@ -419,13 +433,15 @@ "localBlocksFetched" : 0, "fetchWaitTime" : 0, "remoteBytesRead" : 0, + "remoteBytesReadToMem" : 0, "localBytesRead" : 0, "recordsRead" : 0 }, "shuffleWriteMetrics" : { "bytesWritten" : 0, "writeTime" : 0, - "recordsWritten" : 0 + "recordsWritten" : 0, + "underestimatedBlocksSize": 0 } } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index af36707262cf..af0bb6f275ee 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1127,13 +1127,15 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, | "Remote Bytes Read": 1000, + | "Remote Bytes Read To Mem": 0, | "Local Bytes Read": 1100, | "Total Records Read": 10 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, | "Shuffle Write Time": 1500, - | "Shuffle Records Written": 12 + | "Shuffle Records Written": 12, + | "Underestimated Blocks Size Writen" : 0 | }, | "Input Metrics" : { | "Bytes Read" : 0, @@ -1227,13 +1229,15 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched" : 0, | "Fetch Wait Time" : 0, | "Remote Bytes Read" : 0, + | "Remote Bytes Read To Mem" : 0, | "Local Bytes Read" : 0, | "Total Records Read" : 0 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written": 1200, | "Shuffle Write Time": 1500, - | "Shuffle Records Written": 12 + | "Shuffle Records Written": 12, + | "Underestimated Blocks Size Writen" : 0 | }, | "Input Metrics": { | "Bytes Read": 2100, @@ -1327,13 +1331,15 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Local Blocks Fetched" : 0, | "Fetch Wait Time" : 0, | "Remote Bytes Read" : 0, + | "Remote Bytes Read To Mem" : 0, | "Local Bytes Read" : 0, | "Total Records Read" : 0 | }, | "Shuffle Write Metrics": { | "Shuffle Bytes Written" : 0, | "Shuffle Write Time" : 0, - | "Shuffle Records Written" : 0 + | "Shuffle Records Written" : 0, + | "Underestimated Blocks Size Writen" : 0 | }, | "Input Metrics": { | "Bytes Read": 2100, From 8801fc6454fc09d417dea563cd07cc255ae009b9 Mon Sep 17 00:00:00 2001 From: jinxing Date: Sun, 26 Mar 2017 11:02:04 +0800 Subject: [PATCH 26/28] small fix --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 3 +++ .../org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java | 3 +++ .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 2 +- 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index fd6c08607636..f0a627a12ed3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -188,6 +188,9 @@ public void write(Iterator> records) throws IOException { double[] cp = new double[partitionLengths.length]; for (int i = 0; i < partitionLengths.length; i++) { cp[i] = partitionLengths[i]; + if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { + underestimatedBlocksNum++; + } } Distribution distribution = new Distribution(cp, 0, cp.length); double[] probabilities = {0.0, 0.25, 0.5, 0.75, 1.0}; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index b87f1ec6fd58..446ce60cb50c 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -245,6 +245,9 @@ void closeAndWriteOutput() throws IOException { double[] cp = new double[partitionLengths.length]; for (int i = 0; i < partitionLengths.length; i++) { cp[i] = partitionLengths[i]; + if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { + underestimatedBlocksNum++; + } } Distribution distribution = new Distribution(cp, 0, cp.length); double[] probabilities = {0.0, 0.25, 0.5, 0.75, 1.0}; diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index c7bea2472b67..f3db7d04fd7a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -77,7 +77,7 @@ private[spark] class SortShuffleWriter[K, V, C]( case hc: HighlyCompressedMapStatus => val underestimatedLengths = partitionLengths.filter(_ > hc.getAvgSize) writeMetrics.incUnderestimatedBlocksSize(underestimatedLengths.sum) - if (log.isDebugEnabled()) { + if (log.isDebugEnabled() && partitionLengths.length > 0) { // Distribution of sizes in MapStatus. Distribution(partitionLengths.map(_.toDouble)) match { case Some(distribution) => From cf5de4a19c28d50c828ac8648c250c8eaf717949 Mon Sep 17 00:00:00 2001 From: jinxing Date: Sun, 26 Mar 2017 21:28:27 +0800 Subject: [PATCH 27/28] change size of underestimated blocks and remove the duplicate code. --- .../sort/BypassMergeSortShuffleWriter.java | 30 ++---------- .../shuffle/sort/UnsafeShuffleWriter.java | 47 +++++-------------- .../spark/executor/ShuffleWriteMetrics.scala | 2 +- .../shuffle/sort/SortShuffleWriter.scala | 33 +++++++------ 4 files changed, 36 insertions(+), 76 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index f0a627a12ed3..fcf136f41c40 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -174,33 +174,13 @@ public void write(Iterator> records) throws IOException { } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); if (mapStatus instanceof HighlyCompressedMapStatus) { - HighlyCompressedMapStatus hc = (HighlyCompressedMapStatus) mapStatus; - long underestimatedBlocksSize = 0L; - for (int i = 0; i < partitionLengths.length; i++) { - if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { - underestimatedBlocksSize += partitionLengths[i]; - } - } - writeMetrics.incUnderestimatedBlocksSize(underestimatedBlocksSize); if (logger.isDebugEnabled() && partitionLengths.length > 0) { - int underestimatedBlocksNum = 0; - // Distribution of sizes in MapStatus. - double[] cp = new double[partitionLengths.length]; - for (int i = 0; i < partitionLengths.length; i++) { - cp[i] = partitionLengths[i]; - if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { - underestimatedBlocksNum++; - } + Tuple2 tuple = SortShuffleWriter$.MODULE$.genBlocksDistributionStr( + partitionLengths, (HighlyCompressedMapStatus) mapStatus, taskContext); + if (!tuple._1.isEmpty()) { + logger.debug(tuple._1); } - Distribution distribution = new Distribution(cp, 0, cp.length); - double[] probabilities = {0.0, 0.25, 0.5, 0.75, 1.0}; - String distributionStr = distribution.getQuantiles(probabilities).mkString(", "); - logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + - "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + - " distribution at the given probabilities(0, 0.25, 0.5, 0.75, 1.0) is {}.", - taskContext.partitionId(), taskContext.attemptNumber(), taskContext.stageId(), - taskContext.taskAttemptId(), hc.getAvgSize(), - underestimatedBlocksNum, underestimatedBlocksSize, distributionStr); + writeMetrics.incUnderestimatedBlocksSize((Long)(tuple._2)); } } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 446ce60cb50c..ef355de5bbb7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -19,34 +19,30 @@ import javax.annotation.Nullable; import java.io.*; +import java.lang.Long; import java.nio.channels.FileChannel; import java.util.Iterator; -import scala.Option; -import scala.Product2; +import scala.*; import scala.collection.JavaConverters; import scala.reflect.ClassTag; import scala.reflect.ClassTag$; import com.google.common.annotations.VisibleForTesting; -import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; -import com.google.common.io.Files; +import com.google.common.io.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.commons.io.output.CloseShieldOutputStream; +import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.*; import org.apache.spark.annotation.Private; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; -import org.apache.commons.io.output.CloseShieldOutputStream; -import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; -import org.apache.spark.scheduler.HighlyCompressedMapStatus; -import org.apache.spark.scheduler.MapStatus; -import org.apache.spark.scheduler.MapStatus$; +import org.apache.spark.scheduler.*; import org.apache.spark.serializer.SerializationStream; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; @@ -54,7 +50,6 @@ import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; -import org.apache.spark.util.Distribution; import org.apache.spark.util.Utils; @Private @@ -231,33 +226,13 @@ void closeAndWriteOutput() throws IOException { } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); if (mapStatus instanceof HighlyCompressedMapStatus) { - HighlyCompressedMapStatus hc = (HighlyCompressedMapStatus) mapStatus; - long underestimatedBlocksSize = 0L; - for (int i = 0; i < partitionLengths.length; i++) { - if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { - underestimatedBlocksSize += partitionLengths[i]; - } - } - writeMetrics.incUnderestimatedBlocksSize(underestimatedBlocksSize); if (logger.isDebugEnabled() && partitionLengths.length > 0) { - int underestimatedBlocksNum = 0; - // Distribution of sizes in MapStatus. - double[] cp = new double[partitionLengths.length]; - for (int i = 0; i < partitionLengths.length; i++) { - cp[i] = partitionLengths[i]; - if (partitionLengths[i] > mapStatus.getSizeForBlock(i)) { - underestimatedBlocksNum++; - } + Tuple2 tuple = SortShuffleWriter$.MODULE$.genBlocksDistributionStr( + partitionLengths, (HighlyCompressedMapStatus) mapStatus, taskContext); + if (!tuple._1.isEmpty()) { + logger.debug(tuple._1); } - Distribution distribution = new Distribution(cp, 0, cp.length); - double[] probabilities = {0.0, 0.25, 0.5, 0.75, 1.0}; - String distributionStr = distribution.getQuantiles(probabilities).mkString(", "); - logger.debug("For task {}.{} in stage {} (TID {}), the block sizes in MapStatus are " + - "inaccurate (average is {}, {} blocks underestimated, size of underestimated is {})," + - " distribution at the given probabilities(0, 0.25, 0.5, 0.75, 1.0) is {}.", - taskContext.partitionId(), taskContext.attemptNumber(), taskContext.stageId(), - taskContext.taskAttemptId(), hc.getAvgSize(), - underestimatedBlocksNum, underestimatedBlocksSize, distributionStr); + writeMetrics.incUnderestimatedBlocksSize((Long)(tuple._2)); } } } diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index ba0382d07b28..a24d3e45ee60 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -49,7 +49,7 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { def writeTime: Long = _writeTime.sum /** - * The total amount of blocks whose sizes are underestimated in MapStatus. + * The sum of underestimated sizes of blocks in MapStatus. */ def underestimatedBlocksSize: Long = _underestimatedBlocksSize.value diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index f3db7d04fd7a..6ad30b46504f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -72,23 +72,13 @@ private[spark] class SortShuffleWriter[K, V, C]( val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) - mapStatus match { case hc: HighlyCompressedMapStatus => - val underestimatedLengths = partitionLengths.filter(_ > hc.getAvgSize) - writeMetrics.incUnderestimatedBlocksSize(underestimatedLengths.sum) if (log.isDebugEnabled() && partitionLengths.length > 0) { - // Distribution of sizes in MapStatus. - Distribution(partitionLengths.map(_.toDouble)) match { - case Some(distribution) => - val distributionStr = distribution.getQuantiles().mkString(", ") - logDebug(s"For task ${context.partitionId()}.${context.attemptNumber()} in stage" + - s" ${context.stageId()} (TID ${context.taskAttemptId()}), the block sizes in" + - s" MapStatus are inaccurate (average is ${hc.getAvgSize}," + - s" ${underestimatedLengths.length} blocks underestimated, sum of sizes is" + - s" ${underestimatedLengths.sum}), distribution at the given probabilities" + - s" (0, 0.25, 0.5, 0.75, 1.0) is $distributionStr.") - case None => // no-op + SortShuffleWriter.genBlocksDistributionStr(partitionLengths, hc, context) match { + case (logStr, underestimatedSize) if logStr.nonEmpty => + logDebug(logStr) + writeMetrics.incUnderestimatedBlocksSize(underestimatedSize) } } case _ => // no-op @@ -135,4 +125,19 @@ private[spark] object SortShuffleWriter { dep.partitioner.numPartitions <= bypassMergeThreshold } } + def genBlocksDistributionStr(lens: Array[Long], hc: HighlyCompressedMapStatus, + ctx: TaskContext): (String, Long) = { + // Distribution of sizes in MapStatus. + Distribution(lens.map(_.toDouble)) match { + case Some(distribution) => + val underestimatedLengths = lens.filter(_ > hc.getAvgSize).map(_ - hc.getAvgSize) + val distributionStr = distribution.getQuantiles().mkString(", ") + (s"For task ${ctx.partitionId()}.${ctx.attemptNumber()} in stage ${ctx.stageId()} " + + s"(TID ${ctx.taskAttemptId()}), the block sizes in MapStatus are highly compressed" + + s" (average is ${hc.getAvgSize}, ${underestimatedLengths.length} blocks underestimated," + + s" the size of underestimated is ${underestimatedLengths.sum}), distribution at " + + s"probabilities(0, 0.25, 0.5, 0.75, 1.0) is $distributionStr.", underestimatedLengths.sum) + case None => ("", 0L) + } + } } From 873129f783d154c96803e13b94f8f16c2922cb68 Mon Sep 17 00:00:00 2001 From: jinxing Date: Sun, 26 Mar 2017 22:07:44 +0800 Subject: [PATCH 28/28] remote unused import --- .../apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 1 - core/src/main/scala/org/apache/spark/util/Distribution.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index fcf136f41c40..389028dabd9f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -47,7 +47,6 @@ import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.*; -import org.apache.spark.util.Distribution; import org.apache.spark.util.Utils; /** diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index b06ce9416450..950b69f7db64 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -42,7 +42,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va * given from 0 to 1 * @param probabilities */ - def getQuantiles(probabilities: Array[Double] = defaultProbabilities) + def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) : IndexedSeq[Double] = { probabilities.toIndexedSeq.map { p: Double => data(closestIndex(p)) } }