aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala
blob: 3c4a2716caf9012e5e96b01028164ba1f14a9e6e (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.streaming

import java.io.File
import java.nio.ByteBuffer

import scala.collection.mutable.ArrayBuffer
import scala.concurrent.duration._
import scala.language.postfixOps
import scala.reflect.ClassTag

import org.apache.hadoop.conf.Configuration
import org.scalatest.{BeforeAndAfter, Matchers}
import org.scalatest.concurrent.Eventually._

import org.apache.spark._
import org.apache.spark.broadcast.BroadcastManager
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.memory.StaticMemoryManager
import org.apache.spark.network.netty.NettyBlockTransferService
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.scheduler.LiveListenerBus
import org.apache.spark.security.CryptoStreamUtils
import org.apache.spark.serializer.{KryoSerializer, SerializerManager}
import org.apache.spark.shuffle.sort.SortShuffleManager
import org.apache.spark.storage._
import org.apache.spark.streaming.receiver._
import org.apache.spark.streaming.util._
import org.apache.spark.util.{ManualClock, Utils}
import org.apache.spark.util.io.ChunkedByteBuffer

abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean)
  extends SparkFunSuite
  with BeforeAndAfter
  with Matchers
  with LocalSparkContext
  with Logging {

  import WriteAheadLogBasedBlockHandler._
  import WriteAheadLogSuite._

  val conf = new SparkConf()
    .set("spark.streaming.receiver.writeAheadLog.rollingIntervalSecs", "1")
    .set("spark.app.id", "streaming-test")
    .set(IO_ENCRYPTION_ENABLED, enableEncryption)
  val encryptionKey =
    if (enableEncryption) {
      Some(CryptoStreamUtils.createKey(conf))
    } else {
      None
    }

  val hadoopConf = new Configuration()
  val streamId = 1
  val securityMgr = new SecurityManager(conf, encryptionKey)
  val broadcastManager = new BroadcastManager(true, conf, securityMgr)
  val mapOutputTracker = new MapOutputTrackerMaster(conf, broadcastManager, true)
  val shuffleManager = new SortShuffleManager(conf)
  val serializer = new KryoSerializer(conf)
  var serializerManager = new SerializerManager(serializer, conf, encryptionKey)
  val manualClock = new ManualClock
  val blockManagerSize = 10000000
  val blockManagerBuffer = new ArrayBuffer[BlockManager]()

  var rpcEnv: RpcEnv = null
  var blockManagerMaster: BlockManagerMaster = null
  var blockManager: BlockManager = null
  var storageLevel: StorageLevel = null
  var tempDirectory: File = null

  before {
    rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr)
    conf.set("spark.driver.port", rpcEnv.address.port.toString)

    sc = new SparkContext("local", "test", conf)
    blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager",
      new BlockManagerMasterEndpoint(rpcEnv, true, conf,
        new LiveListenerBus(sc))), conf, true)

    storageLevel = StorageLevel.MEMORY_ONLY_SER
    blockManager = createBlockManager(blockManagerSize, conf)

    tempDirectory = Utils.createTempDir()
    manualClock.setTime(0)
  }

  after {
    for ( blockManager <- blockManagerBuffer ) {
      if (blockManager != null) {
        blockManager.stop()
      }
    }
    blockManager = null
    blockManagerBuffer.clear()
    if (blockManagerMaster != null) {
      blockManagerMaster.stop()
      blockManagerMaster = null
    }
    rpcEnv.shutdown()
    rpcEnv.awaitTermination()
    rpcEnv = null

    Utils.deleteRecursively(tempDirectory)
  }

  test("BlockManagerBasedBlockHandler - store blocks") {
    withBlockManagerBasedBlockHandler { handler =>
      testBlockStoring(handler) { case (data, blockIds, storeResults) =>
        // Verify the data in block manager is correct
        val storedData = blockIds.flatMap { blockId =>
          blockManager
            .getLocalValues(blockId)
            .map(_.data.map(_.toString).toList)
            .getOrElse(List.empty)
        }.toList
        storedData shouldEqual data

        // Verify that the store results are instances of BlockManagerBasedStoreResult
        assert(
          storeResults.forall { _.isInstanceOf[BlockManagerBasedStoreResult] },
          "Unexpected store result type"
        )
      }
    }
  }

  test("BlockManagerBasedBlockHandler - handle errors in storing block") {
    withBlockManagerBasedBlockHandler { handler =>
      testErrorHandling(handler)
    }
  }

  test("WriteAheadLogBasedBlockHandler - store blocks") {
    withWriteAheadLogBasedBlockHandler { handler =>
      testBlockStoring(handler) { case (data, blockIds, storeResults) =>
        // Verify the data in block manager is correct
        val storedData = blockIds.flatMap { blockId =>
          blockManager
            .getLocalValues(blockId)
            .map(_.data.map(_.toString).toList)
            .getOrElse(List.empty)
        }.toList
        storedData shouldEqual data

        // Verify that the store results are instances of WriteAheadLogBasedStoreResult
        assert(
          storeResults.forall { _.isInstanceOf[WriteAheadLogBasedStoreResult] },
          "Unexpected store result type"
        )
        // Verify the data in write ahead log files is correct
        val walSegments = storeResults.map { result =>
          result.asInstanceOf[WriteAheadLogBasedStoreResult].walRecordHandle
        }
        val loggedData = walSegments.flatMap { walSegment =>
          val fileSegment = walSegment.asInstanceOf[FileBasedWriteAheadLogSegment]
          val reader = new FileBasedWriteAheadLogRandomReader(fileSegment.path, hadoopConf)
          val bytes = reader.read(fileSegment)
          reader.close()
          serializerManager.dataDeserializeStream(
            generateBlockId(),
            new ChunkedByteBuffer(bytes).toInputStream())(ClassTag.Any).toList
        }
        loggedData shouldEqual data
      }
    }
  }

  test("WriteAheadLogBasedBlockHandler - handle errors in storing block") {
    withWriteAheadLogBasedBlockHandler { handler =>
      testErrorHandling(handler)
    }
  }

  test("WriteAheadLogBasedBlockHandler - clean old blocks") {
    withWriteAheadLogBasedBlockHandler { handler =>
      val blocks = Seq.tabulate(10) { i => IteratorBlock(Iterator(1 to i)) }
      storeBlocks(handler, blocks)

      val preCleanupLogFiles = getWriteAheadLogFiles()
      require(preCleanupLogFiles.size > 1)

      // this depends on the number of blocks inserted using generateAndStoreData()
      manualClock.getTimeMillis() shouldEqual 5000L

      val cleanupThreshTime = 3000L
      handler.cleanupOldBlocks(cleanupThreshTime)
      eventually(timeout(10000 millis), interval(10 millis)) {
        getWriteAheadLogFiles().size should be < preCleanupLogFiles.size
      }
    }
  }

  test("Test Block - count messages") {
    // Test count with BlockManagedBasedBlockHandler
    testCountWithBlockManagerBasedBlockHandler(true)
    // Test count with WriteAheadLogBasedBlockHandler
    testCountWithBlockManagerBasedBlockHandler(false)
  }

  test("Test Block - isFullyConsumed") {
    val sparkConf = new SparkConf().set("spark.app.id", "streaming-test")
    sparkConf.set("spark.storage.unrollMemoryThreshold", "512")
    // spark.storage.unrollFraction set to 0.4 for BlockManager
    sparkConf.set("spark.storage.unrollFraction", "0.4")

    sparkConf.set(IO_ENCRYPTION_ENABLED, enableEncryption)
    // Block Manager with 12000 * 0.4 = 4800 bytes of free space for unroll
    blockManager = createBlockManager(12000, sparkConf)

    // there is not enough space to store this block in MEMORY,
    // But BlockManager will be able to serialize 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))

    // there is not enough space to store this block in MEMORY,
    // But BlockManager will be able to serialize 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))

    // 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
    storageLevel = StorageLevel.MEMORY_ONLY
    withBlockManagerBasedBlockHandler { handler =>
      val thrown = intercept[SparkException] {
        storeSingleBlock(handler, IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator))
      }
    }
  }

  private def testCountWithBlockManagerBasedBlockHandler(isBlockManagerBasedBlockHandler: Boolean) {
    // ByteBufferBlock-MEMORY_ONLY
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_ONLY,
      ByteBufferBlock(ByteBuffer.wrap(Array.tabulate(100)(i => i.toByte))), blockManager, None)
    // ByteBufferBlock-MEMORY_ONLY_SER
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_ONLY_SER,
      ByteBufferBlock(ByteBuffer.wrap(Array.tabulate(100)(i => i.toByte))), blockManager, None)
    // ArrayBufferBlock-MEMORY_ONLY
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_ONLY,
      ArrayBufferBlock(ArrayBuffer.fill(25)(0)), blockManager, Some(25))
    // ArrayBufferBlock-MEMORY_ONLY_SER
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_ONLY_SER,
      ArrayBufferBlock(ArrayBuffer.fill(25)(0)), blockManager, Some(25))
    // ArrayBufferBlock-DISK_ONLY
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.DISK_ONLY,
      ArrayBufferBlock(ArrayBuffer.fill(50)(0)), blockManager, Some(50))
    // ArrayBufferBlock-MEMORY_AND_DISK
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_AND_DISK,
      ArrayBufferBlock(ArrayBuffer.fill(75)(0)), blockManager, Some(75))
    // IteratorBlock-MEMORY_ONLY
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_ONLY,
      IteratorBlock((ArrayBuffer.fill(100)(0)).iterator), blockManager, Some(100))
    // IteratorBlock-MEMORY_ONLY_SER
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_ONLY_SER,
      IteratorBlock((ArrayBuffer.fill(100)(0)).iterator), blockManager, Some(100))
    // IteratorBlock-DISK_ONLY
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.DISK_ONLY,
      IteratorBlock((ArrayBuffer.fill(125)(0)).iterator), blockManager, Some(125))
    // IteratorBlock-MEMORY_AND_DISK
    testRecordcount(isBlockManagerBasedBlockHandler, StorageLevel.MEMORY_AND_DISK,
      IteratorBlock((ArrayBuffer.fill(150)(0)).iterator), blockManager, Some(150))
  }

  private def createBlockManager(
      maxMem: Long,
      conf: SparkConf,
      name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = {
    val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1)
    val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1)
    val blockManager = new BlockManager(name, rpcEnv, blockManagerMaster, serializerManager, conf,
      memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0)
    memManager.setMemoryStore(blockManager.memoryStore)
    blockManager.initialize("app-id")
    blockManagerBuffer += blockManager
    blockManager
  }

  /**
   * Test storing of data using different types of Handler, StorageLevel and ReceivedBlocks
   * and verify the correct record count
   */
  private def testRecordcount(isBlockManagedBasedBlockHandler: Boolean,
      sLevel: StorageLevel,
      receivedBlock: ReceivedBlock,
      bManager: BlockManager,
      expectedNumRecords: Option[Long]
      ) {
    blockManager = bManager
    storageLevel = sLevel
    var bId: StreamBlockId = null
    try {
      if (isBlockManagedBasedBlockHandler) {
        // test received block with BlockManager based handler
        withBlockManagerBasedBlockHandler { handler =>
          val (blockId, blockStoreResult) = storeSingleBlock(handler, receivedBlock)
          bId = blockId
          assert(blockStoreResult.numRecords === expectedNumRecords,
            "Message count not matches for a " +
            receivedBlock.getClass.getName +
            " being inserted using BlockManagerBasedBlockHandler with " + sLevel)
       }
      } else {
        // test received block with WAL based handler
        withWriteAheadLogBasedBlockHandler { handler =>
          val (blockId, blockStoreResult) = storeSingleBlock(handler, receivedBlock)
          bId = blockId
          assert(blockStoreResult.numRecords === expectedNumRecords,
            "Message count not matches for a " +
            receivedBlock.getClass.getName +
            " being inserted using WriteAheadLogBasedBlockHandler with " + sLevel)
        }
      }
    } finally {
     // Removing the Block Id to use same blockManager for next test
     blockManager.removeBlock(bId, true)
    }
  }

  /**
   * Test storing of data using different forms of ReceivedBlocks and verify that they succeeded
   * using the given verification function
   */
  private def testBlockStoring(receivedBlockHandler: ReceivedBlockHandler)
      (verifyFunc: (Seq[String], Seq[StreamBlockId], Seq[ReceivedBlockStoreResult]) => Unit) {
    val data = Seq.tabulate(100) { _.toString }

    def storeAndVerify(blocks: Seq[ReceivedBlock]) {
      blocks should not be empty
      val (blockIds, storeResults) = storeBlocks(receivedBlockHandler, blocks)
      withClue(s"Testing with ${blocks.head.getClass.getSimpleName}s:") {
        // Verify returns store results have correct block ids
        (storeResults.map { _.blockId }) shouldEqual blockIds

        // Call handler-specific verification function
        verifyFunc(data, blockIds, storeResults)
      }
    }

    def dataToByteBuffer(b: Seq[String]) =
      serializerManager.dataSerialize(generateBlockId, b.iterator)

    val blocks = data.grouped(10).toSeq

    storeAndVerify(blocks.map { b => IteratorBlock(b.toIterator) })
    storeAndVerify(blocks.map { b => ArrayBufferBlock(new ArrayBuffer ++= b) })
    storeAndVerify(blocks.map { b => ByteBufferBlock(dataToByteBuffer(b).toByteBuffer) })
  }

  /** Test error handling when blocks that cannot be stored */
  private def testErrorHandling(receivedBlockHandler: ReceivedBlockHandler) {
    // Handle error in iterator (e.g. divide-by-zero error)
    intercept[Exception] {
      val iterator = (10 to (-10, -1)).toIterator.map { _ / 0 }
      receivedBlockHandler.storeBlock(StreamBlockId(1, 1), IteratorBlock(iterator))
    }

    // Handler error in block manager storing (e.g. too big block)
    intercept[SparkException] {
      val byteBuffer = ByteBuffer.wrap(new Array[Byte](blockManagerSize + 1))
      receivedBlockHandler.storeBlock(StreamBlockId(1, 1), ByteBufferBlock(byteBuffer))
    }
  }

  /** Instantiate a BlockManagerBasedBlockHandler and run a code with it */
  private def withBlockManagerBasedBlockHandler(body: BlockManagerBasedBlockHandler => Unit) {
    body(new BlockManagerBasedBlockHandler(blockManager, storageLevel))
  }

  /** Instantiate a WriteAheadLogBasedBlockHandler and run a code with it */
  private def withWriteAheadLogBasedBlockHandler(body: WriteAheadLogBasedBlockHandler => Unit) {
    require(WriteAheadLogUtils.getRollingIntervalSecs(conf, isDriver = false) === 1)
    val receivedBlockHandler = new WriteAheadLogBasedBlockHandler(blockManager, serializerManager,
      1, storageLevel, conf, hadoopConf, tempDirectory.toString, manualClock)
    try {
      body(receivedBlockHandler)
    } finally {
      receivedBlockHandler.stop()
    }
  }

  /** Store blocks using a handler */
  private def storeBlocks(
      receivedBlockHandler: ReceivedBlockHandler,
      blocks: Seq[ReceivedBlock]
    ): (Seq[StreamBlockId], Seq[ReceivedBlockStoreResult]) = {
    val blockIds = Seq.fill(blocks.size)(generateBlockId())
    val storeResults = blocks.zip(blockIds).map {
      case (block, id) =>
        manualClock.advance(500) // log rolling interval set to 1000 ms through SparkConf
        logDebug("Inserting block " + id)
        receivedBlockHandler.storeBlock(id, block)
    }.toList
    logDebug("Done inserting")
    (blockIds, storeResults)
  }

  /** Store single block using a handler */
  private def storeSingleBlock(
      handler: ReceivedBlockHandler,
      block: ReceivedBlock
    ): (StreamBlockId, ReceivedBlockStoreResult) = {
    val blockId = generateBlockId
    val blockStoreResult = handler.storeBlock(blockId, block)
    logDebug("Done inserting")
    (blockId, blockStoreResult)
  }

  private def getWriteAheadLogFiles(): Seq[String] = {
    getLogFilesInDirectory(checkpointDirToLogDir(tempDirectory.toString, streamId))
  }

  private def generateBlockId(): StreamBlockId = StreamBlockId(streamId, scala.util.Random.nextLong)
}

class ReceivedBlockHandlerSuite extends BaseReceivedBlockHandlerSuite(false)

class ReceivedBlockHandlerWithEncryptionSuite extends BaseReceivedBlockHandlerSuite(true)