-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-4874] [CORE] Collect record count metrics #4067
Changes from 10 commits
1bb78b1
1aa273c
6cdb44e
57551c1
46c8186
b6f9923
17faa3a
70620a0
6f236a1
dad4d57
bd919be
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 |
---|---|---|
|
@@ -25,7 +25,7 @@ import org.apache.spark._ | |
import org.apache.spark.serializer.Serializer | ||
import org.apache.spark.shuffle.FetchFailedException | ||
import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} | ||
import org.apache.spark.util.CompletionIterator | ||
import org.apache.spark.util.{CompletionIterator} | ||
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. you don't need braces here if it is a single import. |
||
|
||
private[hash] object BlockStoreShuffleFetcher extends Logging { | ||
def fetch[T]( | ||
|
@@ -86,6 +86,12 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { | |
context.taskMetrics.updateShuffleReadMetrics() | ||
}) | ||
|
||
new InterruptibleIterator[T](context, completionIter) | ||
new InterruptibleIterator[T](context, completionIter) { | ||
val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() | ||
override def next(): T = { | ||
readMetrics.incRecordsRead(1) | ||
delegate.next() | ||
} | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -29,7 +29,8 @@ import org.apache.spark.executor.ShuffleWriteMetrics | |
* appending data to an existing block, and can guarantee atomicity in the case of faults | ||
* as it allows the caller to revert partial writes. | ||
* | ||
* This interface does not support concurrent writes. | ||
* This interface does not support concurrent writes. Also, once the writer has | ||
* been opened, it cannot be reopened again. | ||
*/ | ||
private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { | ||
|
||
|
@@ -95,6 +96,7 @@ private[spark] class DiskBlockObjectWriter( | |
private var ts: TimeTrackingOutputStream = null | ||
private var objOut: SerializationStream = null | ||
private var initialized = false | ||
private var hasBeenClosed = false | ||
|
||
/** | ||
* Cursors used to represent positions in the file. | ||
|
@@ -115,11 +117,16 @@ private[spark] class DiskBlockObjectWriter( | |
private var finalPosition: Long = -1 | ||
private var reportedPosition = initialPosition | ||
|
||
/** Calling channel.position() to update the write metrics can be a little bit expensive, so we | ||
* only call it every N writes */ | ||
private var writesSinceMetricsUpdate = 0 | ||
/** | ||
* Keep track of number of records written and also use this to periodically | ||
* output bytes written since the latter is expensive to do for each record. | ||
*/ | ||
private var numRecordsWritten = 0 | ||
|
||
override def open(): BlockObjectWriter = { | ||
if (hasBeenClosed) { | ||
throw new IllegalStateException("Writer already closed. Cannot be reopened.") | ||
} | ||
fos = new FileOutputStream(file, true) | ||
ts = new TimeTrackingOutputStream(fos) | ||
channel = fos.getChannel() | ||
|
@@ -145,6 +152,7 @@ private[spark] class DiskBlockObjectWriter( | |
ts = null | ||
objOut = null | ||
initialized = false | ||
hasBeenClosed = true | ||
} | ||
} | ||
|
||
|
@@ -168,6 +176,7 @@ private[spark] class DiskBlockObjectWriter( | |
override def revertPartialWritesAndClose() { | ||
try { | ||
writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition) | ||
writeMetrics.decShuffleRecordsWritten(numRecordsWritten) | ||
|
||
if (initialized) { | ||
objOut.flush() | ||
|
@@ -193,12 +202,11 @@ private[spark] class DiskBlockObjectWriter( | |
} | ||
|
||
objOut.writeObject(value) | ||
numRecordsWritten += 1 | ||
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. The current use of 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 the BlockObjectWriter can be reopened, wouldn't we we need to reset some other variables too? Like finalPosition, initialPosition and reportedPosition? They are set during construction? I don't see anywhere any enforcement of reopening the BlockObjectWriter. We could either reset all the variables i mentioned on open() or we can add a check that once the blockObjectWriter has been opened, it can't be reopened. I'm leaning towards the latter. what do you think? 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. What about just adding a class level comment that it can't be used after it is closed? 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. Yes, i guess that is the least we can do. Having an explicit check i think would be better. So if we are okay with it, i can add a boolean that tracks if the blockwriter has been opened before and if so, don't allow it to be reopened. Thoughts? 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. Sure, you can enforce that it is never re-opened if you want. |
||
writeMetrics.incShuffleRecordsWritten(1) | ||
|
||
if (writesSinceMetricsUpdate == 32) { | ||
writesSinceMetricsUpdate = 0 | ||
if (numRecordsWritten % 32 == 0) { | ||
updateBytesWritten() | ||
} else { | ||
writesSinceMetricsUpdate += 1 | ||
} | ||
} | ||
|
||
|
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.
to be more consistent with the presentation in the UI and the other fields here, should this be shuffleRecordsRead?
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.
@ksakellis mind fixing this one?