-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-11583] [Core]MapStatus Using RoaringBitmap More Properly #9661
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
75c3209
8f3eb7d
b4b7a1c
372dc4c
b9cc414
64e7295
0c4bcba
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,8 +20,8 @@ package org.apache.spark.scheduler | |
| import java.io.{Externalizable, ObjectInput, ObjectOutput} | ||
|
|
||
| import org.apache.spark.storage.BlockManagerId | ||
| import org.apache.spark.util.collection.BitSet | ||
| import org.apache.spark.util.Utils | ||
| import org.roaringbitmap.RoaringBitmap | ||
|
|
||
| /** | ||
| * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the | ||
|
|
@@ -121,8 +121,7 @@ private[spark] class CompressedMapStatus( | |
|
|
||
| /** | ||
| * A [[MapStatus]] implementation that only stores the average size of non-empty blocks, | ||
| * plus a bitmap for tracking which blocks are empty. During serialization, this bitmap | ||
| * is compressed. | ||
| * plus a bitmap for tracking which blocks are empty. | ||
| * | ||
| * @param loc location where the task is being executed | ||
| * @param numNonEmptyBlocks the number of non-empty blocks | ||
|
|
@@ -132,7 +131,7 @@ private[spark] class CompressedMapStatus( | |
| private[spark] class HighlyCompressedMapStatus private ( | ||
| private[this] var loc: BlockManagerId, | ||
| private[this] var numNonEmptyBlocks: Int, | ||
| private[this] var emptyBlocks: BitSet, | ||
| private[this] var emptyBlocks: RoaringBitmap, | ||
| private[this] var avgSize: Long) | ||
| extends MapStatus with Externalizable { | ||
|
|
||
|
|
@@ -145,7 +144,7 @@ private[spark] class HighlyCompressedMapStatus private ( | |
| override def location: BlockManagerId = loc | ||
|
|
||
| override def getSizeForBlock(reduceId: Int): Long = { | ||
| if (emptyBlocks.get(reduceId)) { | ||
| if (emptyBlocks.contains(reduceId)) { | ||
| 0 | ||
| } else { | ||
| avgSize | ||
|
|
@@ -160,7 +159,7 @@ private[spark] class HighlyCompressedMapStatus private ( | |
|
|
||
| override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { | ||
| loc = BlockManagerId(in) | ||
| emptyBlocks = new BitSet | ||
| emptyBlocks = new RoaringBitmap() | ||
| emptyBlocks.readExternal(in) | ||
| avgSize = in.readLong() | ||
| } | ||
|
|
@@ -173,18 +172,15 @@ private[spark] object HighlyCompressedMapStatus { | |
| var i = 0 | ||
| var numNonEmptyBlocks: Int = 0 | ||
| var totalSize: Long = 0 | ||
| // From a compression standpoint, it shouldn't matter whether we track empty or non-empty | ||
| // blocks. From a performance standpoint, we benefit from tracking empty blocks because | ||
| // we expect that there will be far fewer of them, so we will perform fewer bitmap insertions. | ||
| val emptyBlocks = new RoaringBitmap() | ||
| val totalNumBlocks = uncompressedSizes.length | ||
| val emptyBlocks = new BitSet(totalNumBlocks) | ||
| while (i < totalNumBlocks) { | ||
| var size = uncompressedSizes(i) | ||
| if (size > 0) { | ||
| numNonEmptyBlocks += 1 | ||
| totalSize += size | ||
| } else { | ||
| emptyBlocks.set(i) | ||
| emptyBlocks.add(i) | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If you use Though it should not help, you can also investigate whether adding
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 Does it also make sense to call
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not exactly sure why you would keep track of both the empty and non-empty blocks. Seems redundant. But maybe I misunderstand. But the comment above is probably right: you'd want to minimize the number of bitmap insertions. By design, even without So my guess is that a single call to
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let me add that even with |
||
| i += 1 | ||
| } | ||
|
|
@@ -193,6 +189,8 @@ private[spark] object HighlyCompressedMapStatus { | |
| } else { | ||
| 0 | ||
| } | ||
| emptyBlocks.runOptimize() | ||
| emptyBlocks.trim() | ||
| new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,24 +21,24 @@ import java.io.{EOFException, IOException, InputStream, OutputStream} | |
| import java.nio.ByteBuffer | ||
| import javax.annotation.Nullable | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.reflect.ClassTag | ||
|
|
||
| import com.esotericsoftware.kryo.{Kryo, KryoException} | ||
| import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} | ||
| import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} | ||
| import com.esotericsoftware.kryo.{Kryo, KryoException} | ||
| import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} | ||
| import org.apache.avro.generic.{GenericData, GenericRecord} | ||
|
|
||
| import org.apache.spark._ | ||
| import org.apache.spark.api.python.PythonBroadcast | ||
| import org.apache.spark.broadcast.HttpBroadcast | ||
| import org.apache.spark.network.util.ByteUnit | ||
| import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} | ||
| import org.apache.spark.storage._ | ||
| import org.apache.spark.util.{Utils, BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf} | ||
| import org.apache.spark.util.collection.{BitSet, CompactBuffer} | ||
| import org.apache.spark.util.{BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf, Utils} | ||
| import org.roaringbitmap._ | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable.ArrayBuffer | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you update your import rules to place scale package before third parties (as before this PR)?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. OK |
||
| import scala.reflect.ClassTag | ||
|
|
||
| /** | ||
| * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. | ||
|
|
@@ -362,6 +362,12 @@ private[serializer] object KryoSerializer { | |
| classOf[StorageLevel], | ||
| classOf[CompressedMapStatus], | ||
| classOf[HighlyCompressedMapStatus], | ||
| classOf[RoaringBitmap], | ||
| classOf[RoaringArray], | ||
| classOf[Array[Container]], | ||
| classOf[ArrayContainer], | ||
| classOf[BitmapContainer], | ||
| classOf[RunContainer], | ||
| classOf[BitSet], | ||
| classOf[CompactBuffer[_]], | ||
| classOf[BlockManagerId], | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This comment still make sense, could you keep it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK