Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/6707#discussion_r32358204
  
    --- Diff: 
streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala
 ---
    @@ -174,6 +186,120 @@ class ReceivedBlockHandlerSuite
         }
       }
     
    +  test("BlockManagerBasedBlockHandler - count messages") {
    +    // ByteBufferBlock-MEMORY_ONLY
    +    testRecordcount(true, StorageLevel.MEMORY_ONLY,
    +      ByteBufferBlock(ByteBuffer.wrap(Array.tabulate(100)(i => 
i.toByte))), blockManager, None)
    +    // ArrayBufferBlock-MEMORY_ONLY
    +    testRecordcount(true, StorageLevel.MEMORY_ONLY,
    +      ArrayBufferBlock(ArrayBuffer.fill(25)(0)), blockManager, Some(25))
    +    // ArrayBufferBlock-DISK_ONLY
    +    testRecordcount(true, StorageLevel.DISK_ONLY,
    +      ArrayBufferBlock(ArrayBuffer.fill(50)(0)), blockManager, Some(50))
    +    // ArrayBufferBlock-MEMORY_AND_DISK
    +    testRecordcount(true, StorageLevel.MEMORY_AND_DISK,
    +      ArrayBufferBlock(ArrayBuffer.fill(75)(0)), blockManager, Some(75))
    +    // IteratorBlock-MEMORY_ONLY
    +    testRecordcount(true, StorageLevel.MEMORY_ONLY,
    +      IteratorBlock((ArrayBuffer.fill(100)(0)).iterator), blockManager, 
Some(100))
    +    // IteratorBlock-DISK_ONLY
    +    testRecordcount(true, StorageLevel.DISK_ONLY,
    +      IteratorBlock((ArrayBuffer.fill(125)(0)).iterator), blockManager, 
Some(125))
    +    // IteratorBlock-MEMORY_AND_DISK
    +    testRecordcount(true, StorageLevel.MEMORY_AND_DISK,
    +      IteratorBlock((ArrayBuffer.fill(150)(0)).iterator), blockManager, 
Some(150))
    +  }
    +
    +  test("WriteAheadLogBasedBlockHandler - count messages") {
    +    // ByteBufferBlock-MEMORY_ONLY
    +    testRecordcount(false, StorageLevel.MEMORY_ONLY,
    +      ByteBufferBlock(ByteBuffer.wrap(Array.tabulate(100)(i => 
i.toByte))), blockManager, None)
    +    // ArrayBufferBlock-MEMORY_ONLY
    +    testRecordcount(false, StorageLevel.MEMORY_ONLY,
    +      ArrayBufferBlock(ArrayBuffer.fill(25)(0)), blockManager, Some(25))
    +    // ArrayBufferBlock-DISK_ONLY
    +    testRecordcount(false, StorageLevel.DISK_ONLY,
    +      ArrayBufferBlock(ArrayBuffer.fill(50)(0)), blockManager, Some(50))
    +    // ArrayBufferBlock-MEMORY_AND_DISK
    +    testRecordcount(false, StorageLevel.MEMORY_AND_DISK,
    +      ArrayBufferBlock(ArrayBuffer.fill(75)(0)), blockManager, Some(75))
    +    // IteratorBlock-MEMORY_ONLY
    +    testRecordcount(false, StorageLevel.MEMORY_ONLY,
    +      IteratorBlock((ArrayBuffer.fill(100)(0)).iterator), blockManager, 
Some(100))
    +    // IteratorBlock-DISK_ONLY
    +    testRecordcount(false, StorageLevel.DISK_ONLY,
    +      IteratorBlock((ArrayBuffer.fill(125)(0)).iterator), blockManager, 
Some(125))
    +    // IteratorBlock-MEMORY_AND_DISK
    +    testRecordcount(false, StorageLevel.MEMORY_AND_DISK,
    +      IteratorBlock((ArrayBuffer.fill(150)(0)).iterator), blockManager, 
Some(150))
    +  }
    +
    +  test("BlockManagerBasedBlockHandler-MEMORY_ONLY - isFullyConsumed") {
    +    storageLevel = StorageLevel.MEMORY_ONLY
    +    blockManager = createBlockManager(12000)
    +    val block = List.fill(70)(new Array[Byte](100))
    +    // spark.storage.unrollFraction set to 0.4 for BlockManager
    +    // With 12000 * 0.4 = 4800 bytes of free space for unroll, there is 
not enough space to store
    +    // this block With MEMORY_ONLY StorageLevel. BlockManager will not be 
able to unroll this block
    +    // and hence it will not tryToPut this block, resulting the 
SparkException
    +    withBlockManagerBasedBlockHandler { handler =>
    +      val thrown = intercept[SparkException] {
    +        val blockStoreResult = storeBlock(handler, 
IteratorBlock(block.iterator))
    +      }
    +      assert(thrown.getMessage ===
    +        "Could not store input-1-1000 to block manager with storage level 
" + storageLevel)
    +    }
    +  }
    +
    +  test("BlockManagerBasedBlockHandler-MEMORY_AND_DISK - isFullyConsumed") {
    +    blockManager = createBlockManager(12000)
    +    // spark.storage.unrollFraction set to 0.4 for BlockManager
    +    // With 12000 * 0.4 = 4800 bytes of free space for unroll, there is 
not enough space to store
    +    // this block in MEMORY, But BlockManager will be able to sereliaze 
this block to DISK
    +    // and hence count returns correct value.
    +    testRecordcount(true, StorageLevel.MEMORY_AND_DISK,
    +      IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), 
blockManager, Some(70))
    +  }
    +
    +  test("WriteAheadLogBasedBlockHandler-MEMORY_ONLY - isFullyConsumed") {
    +    blockManager = createBlockManager(12000)
    +    // spark.storage.unrollFraction set to 0.4 for BlockManager
    +    // With 12000 * 0.4 = 4800 bytes of free space for unroll, there is 
not enough space to store
    +    // this block in MEMORY, But BlockManager will be able to sereliaze 
this block to WAL
    +    // and hence count returns correct value.
    +    testRecordcount(false, StorageLevel.MEMORY_ONLY,
    +      IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), 
blockManager, Some(70))
    +  }
    +
    +  /**
    +   * Test storing of data using different types of Handler, StorageLevle 
and ReceivedBlocks
    +   * and verify the correct record count
    +   */
    +  private def testRecordcount(isBlockManagedBasedBlockHandler: Boolean,
    +      sLevel: StorageLevel,
    +      receivedBlock: ReceivedBlock,
    +      bManager: BlockManager,
    +      expectedNumRecords: Option[Long]
    +      ) {
    +   storageLevel = sLevel
    +   blockManager = bManager
    +   if (isBlockManagedBasedBlockHandler) {
    +      // test received block with BlockManager based handler
    +      withBlockManagerBasedBlockHandler { handler =>
    +        val blockStoreResult = storeBlock(handler, receivedBlock)
    +        assert(blockStoreResult.numRecords === expectedNumRecords)
    --- End diff --
    
    Can you add the assert message as well, saying "Message count not matches 
for a TypeX block being inserted in BlockHandlerTypeY with StorageLevelZ". 
Helps to debug.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to