|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package org.apache.spark.streaming.rdd |
| 18 | + |
| 19 | +import java.io.File |
| 20 | + |
| 21 | +import scala.util.Random |
| 22 | + |
| 23 | +import com.google.common.io.Files |
| 24 | +import org.apache.hadoop.conf.Configuration |
| 25 | +import org.scalatest.{BeforeAndAfterAll, FunSuite} |
| 26 | + |
| 27 | +import org.apache.spark.{SparkConf, SparkContext} |
| 28 | +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} |
| 29 | +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter} |
| 30 | + |
| 31 | +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { |
| 32 | + val conf = new SparkConf() |
| 33 | + .setMaster("local[2]") |
| 34 | + .setAppName(this.getClass.getSimpleName) |
| 35 | + val hadoopConf = new Configuration() |
| 36 | + |
| 37 | + var sparkContext: SparkContext = null |
| 38 | + var blockManager: BlockManager = null |
| 39 | + var dir: File = null |
| 40 | + |
| 41 | + override def beforeAll(): Unit = { |
| 42 | + sparkContext = new SparkContext(conf) |
| 43 | + blockManager = sparkContext.env.blockManager |
| 44 | + dir = Files.createTempDir() |
| 45 | + } |
| 46 | + |
| 47 | + override def afterAll(): Unit = { |
| 48 | + // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests. |
| 49 | + sparkContext.stop() |
| 50 | + dir.delete() |
| 51 | + System.clearProperty("spark.driver.port") |
| 52 | + } |
| 53 | + |
| 54 | + test("Read data available in block manager and write ahead log") { |
| 55 | + testRDD(5, 5) |
| 56 | + } |
| 57 | + |
| 58 | + test("Read data available only in block manager, not in write ahead log") { |
| 59 | + testRDD(5, 0) |
| 60 | + } |
| 61 | + |
| 62 | + test("Read data available only in write ahead log, not in block manager") { |
| 63 | + testRDD(0, 5) |
| 64 | + } |
| 65 | + |
| 66 | + test("Read data available only in write ahead log, and test storing in block manager") { |
| 67 | + testRDD(0, 5, testStoreInBM = true) |
| 68 | + } |
| 69 | + |
| 70 | + test("Read data with partially available in block manager, and rest in write ahead log") { |
| 71 | + testRDD(3, 2) |
| 72 | + } |
| 73 | + |
| 74 | + /** |
| 75 | + * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager |
| 76 | + * and the rest to a write ahead log, and then reading reading it all back using the RDD. |
| 77 | + * It can also test if the partitions that were read from the log were again stored in |
| 78 | + * block manager. |
| 79 | + * @param numPartitionsInBM Number of partitions to write to the Block Manager |
| 80 | + * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log |
| 81 | + * @param testStoreInBM Test whether blocks read from log are stored back into block manager |
| 82 | + */ |
| 83 | + private def testRDD(numPartitionsInBM: Int, numPartitionsInWAL: Int, testStoreInBM: Boolean = false) { |
| 84 | + val numBlocks = numPartitionsInBM + numPartitionsInWAL |
| 85 | + val data = Seq.fill(numBlocks, 10)(scala.util.Random.nextString(50)) |
| 86 | + |
| 87 | + // Put the necessary blocks in the block manager |
| 88 | + val blockIds = Array.fill(numBlocks)(StreamBlockId(Random.nextInt(), Random.nextInt())) |
| 89 | + data.zip(blockIds).take(numPartitionsInBM).foreach { case(block, blockId) => |
| 90 | + blockManager.putIterator(blockId, block.iterator, StorageLevel.MEMORY_ONLY_SER) |
| 91 | + } |
| 92 | + |
| 93 | + // Generate write ahead log segments |
| 94 | + val segments = generateFakeSegments(numPartitionsInBM) ++ |
| 95 | + writeLogSegments(data.takeRight(numPartitionsInWAL), blockIds.takeRight(numPartitionsInWAL)) |
| 96 | + |
| 97 | + // Make sure that the left `numPartitionsInBM` blocks are in block manager, and others are not |
| 98 | + require( |
| 99 | + blockIds.take(numPartitionsInBM).forall(blockManager.get(_).nonEmpty), |
| 100 | + "Expected blocks not in BlockManager" |
| 101 | + ) |
| 102 | + require( |
| 103 | + blockIds.takeRight(numPartitionsInWAL).forall(blockManager.get(_).isEmpty), |
| 104 | + "Unexpected blocks in BlockManager" |
| 105 | + ) |
| 106 | + |
| 107 | + // Make sure that the right `numPartitionsInWAL` blocks are in write ahead logs, and other are not |
| 108 | + require( |
| 109 | + segments.takeRight(numPartitionsInWAL).forall(s => |
| 110 | + new File(s.path.stripPrefix("file://")).exists()), |
| 111 | + "Expected blocks not in write ahead log" |
| 112 | + ) |
| 113 | + require( |
| 114 | + segments.take(numPartitionsInBM).forall(s => |
| 115 | + !new File(s.path.stripPrefix("file://")).exists()), |
| 116 | + "Unexpected blocks in write ahead log" |
| 117 | + ) |
| 118 | + |
| 119 | + // Create the RDD and verify whether the returned data is correct |
| 120 | + val rdd = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, |
| 121 | + segments.toArray, storeInBlockManager = false, StorageLevel.MEMORY_ONLY) |
| 122 | + assert(rdd.collect() === data.flatten) |
| 123 | + |
| 124 | + if (testStoreInBM) { |
| 125 | + val rdd2 = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, |
| 126 | + segments.toArray, storeInBlockManager = true, StorageLevel.MEMORY_ONLY) |
| 127 | + assert(rdd2.collect() === data.flatten) |
| 128 | + assert( |
| 129 | + blockIds.forall(blockManager.get(_).nonEmpty), |
| 130 | + "All blocks not found in block manager" |
| 131 | + ) |
| 132 | + } |
| 133 | + } |
| 134 | + |
| 135 | + private def writeLogSegments( |
| 136 | + blockData: Seq[Seq[String]], |
| 137 | + blockIds: Seq[BlockId] |
| 138 | + ): Seq[WriteAheadLogFileSegment] = { |
| 139 | + require(blockData.size === blockIds.size) |
| 140 | + val writer = new WriteAheadLogWriter(new File(dir, Random.nextString(10)).toString, hadoopConf) |
| 141 | + val segments = blockData.zip(blockIds).map { case (data, id) => |
| 142 | + writer.write(blockManager.dataSerialize(id, data.iterator)) |
| 143 | + } |
| 144 | + writer.close() |
| 145 | + segments |
| 146 | + } |
| 147 | + |
| 148 | + private def generateFakeSegments(count: Int): Seq[WriteAheadLogFileSegment] = { |
| 149 | + Array.fill(count)(new WriteAheadLogFileSegment("random", 0l, 0)) |
| 150 | + } |
| 151 | +} |
0 commit comments