@@ -69,16 +69,20 @@ private[streaming] class BlockManagerBasedBlockHandler(
69
69
70
70
def storeBlock (blockId : StreamBlockId , block : ReceivedBlock ): ReceivedBlockStoreResult = {
71
71
var numRecords = None : Option [Long ]
72
- val countIterator = block match {
73
- case ArrayBufferBlock (arrayBuffer) => new CountingIterator (arrayBuffer.iterator)
74
- case IteratorBlock (iterator) => new CountingIterator (iterator)
75
- case _ => null
76
- }
72
+
77
73
val putResult : Seq [(BlockId , BlockStatus )] = block match {
78
74
case ArrayBufferBlock (arrayBuffer) =>
79
- blockManager.putIterator(blockId, countIterator, storageLevel, tellMaster = true )
75
+ val countIterator = new CountingIterator (arrayBuffer.iterator)
76
+ val putResult = blockManager.putIterator(blockId, countIterator, storageLevel,
77
+ tellMaster = true )
78
+ numRecords = Some (countIterator.count)
79
+ putResult
80
80
case IteratorBlock (iterator) =>
81
- blockManager.putIterator(blockId, countIterator, storageLevel, tellMaster = true )
81
+ val countIterator = new CountingIterator (iterator)
82
+ val putResult = blockManager.putIterator(blockId, countIterator, storageLevel,
83
+ tellMaster = true )
84
+ numRecords = Some (countIterator.count)
85
+ putResult
82
86
case ByteBufferBlock (byteBuffer) =>
83
87
blockManager.putBytes(blockId, byteBuffer, storageLevel, tellMaster = true )
84
88
case o =>
@@ -89,9 +93,6 @@ private[streaming] class BlockManagerBasedBlockHandler(
89
93
throw new SparkException (
90
94
s " Could not store $blockId to block manager with storage level $storageLevel" )
91
95
}
92
- if (countIterator != null ) {
93
- numRecords = Some (countIterator.count)
94
- }
95
96
BlockManagerBasedStoreResult (blockId, numRecords)
96
97
}
97
98
@@ -166,17 +167,18 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
166
167
def storeBlock (blockId : StreamBlockId , block : ReceivedBlock ): ReceivedBlockStoreResult = {
167
168
168
169
var numRecords = None : Option [Long ]
169
- val countIterator = block match {
170
- case ArrayBufferBlock (arrayBuffer) => new CountingIterator (arrayBuffer.iterator)
171
- case IteratorBlock (iterator) => new CountingIterator (iterator)
172
- case _ => null
173
- }
174
170
// Serialize the block so that it can be inserted into both
175
171
val serializedBlock = block match {
176
172
case ArrayBufferBlock (arrayBuffer) =>
177
- blockManager.dataSerialize(blockId, countIterator)
173
+ val countIterator = new CountingIterator (arrayBuffer.iterator)
174
+ val serializedBlock = blockManager.dataSerialize(blockId, countIterator)
175
+ numRecords = Some (countIterator.count)
176
+ serializedBlock
178
177
case IteratorBlock (iterator) =>
179
- blockManager.dataSerialize(blockId, countIterator)
178
+ val countIterator = new CountingIterator (iterator)
179
+ val serializedBlock = blockManager.dataSerialize(blockId, countIterator)
180
+ numRecords = Some (countIterator.count)
181
+ serializedBlock
180
182
case ByteBufferBlock (byteBuffer) =>
181
183
byteBuffer
182
184
case _ =>
@@ -201,9 +203,6 @@ private[streaming] class WriteAheadLogBasedBlockHandler(
201
203
// Combine the futures, wait for both to complete, and return the write ahead log record handle
202
204
val combinedFuture = storeInBlockManagerFuture.zip(storeInWriteAheadLogFuture).map(_._2)
203
205
val walRecordHandle = Await .result(combinedFuture, blockStoreTimeout)
204
- if (countIterator != null ) {
205
- numRecords = Some (countIterator.count)
206
- }
207
206
WriteAheadLogBasedStoreResult (blockId, numRecords, walRecordHandle)
208
207
}
209
208
@@ -226,7 +225,7 @@ private[streaming] object WriteAheadLogBasedBlockHandler {
226
225
/**
227
226
* A utility that will wrap the Iterator to get the count
228
227
*/
229
- private class CountingIterator [T : Manifest ](iterator : Iterator [T ]) extends Iterator [T ] {
228
+ private class CountingIterator [T ](iterator : Iterator [T ]) extends Iterator [T ] {
230
229
var count = 0
231
230
def hasNext (): Boolean = iterator.hasNext
232
231
def isFullyConsumed : Boolean = ! iterator.hasNext
0 commit comments