|
| 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.storage |
| 18 | + |
| 19 | +import java.io.{RandomAccessFile, File} |
| 20 | +import java.nio.ByteBuffer |
| 21 | +import java.util.Random |
| 22 | + |
| 23 | +import scala.collection.mutable.ArrayBuffer |
| 24 | + |
| 25 | +import com.google.common.io.Files |
| 26 | +import org.apache.hadoop.conf.Configuration |
| 27 | + |
| 28 | +import org.apache.spark.streaming.TestSuiteBase |
| 29 | + |
| 30 | +class WriteAheadLogSuite extends TestSuiteBase { |
| 31 | + |
| 32 | + val hadoopConf = new Configuration() |
| 33 | + val random = new Random() |
| 34 | + |
| 35 | + test("Test successful writes") { |
| 36 | + val dir = Files.createTempDir() |
| 37 | + val file = new File(dir, "TestWriter") |
| 38 | + try { |
| 39 | + val dataToWrite = for (i <- 1 to 50) yield generateRandomData() |
| 40 | + val writer = new WriteAheadLogWriter("file:///" + file.toString, hadoopConf) |
| 41 | + val segments = dataToWrite.map(writer.write) |
| 42 | + writer.close() |
| 43 | + val writtenData = readData(segments, file) |
| 44 | + assert(writtenData.toArray === dataToWrite.toArray) |
| 45 | + } finally { |
| 46 | + file.delete() |
| 47 | + dir.delete() |
| 48 | + } |
| 49 | + } |
| 50 | + |
| 51 | + test("Test successful reads using random reader") { |
| 52 | + val file = File.createTempFile("TestRandomReads", "") |
| 53 | + file.deleteOnExit() |
| 54 | + val writtenData = writeData(50, file) |
| 55 | + val reader = new WriteAheadLogRandomReader("file:///" + file.toString, hadoopConf) |
| 56 | + writtenData.foreach { |
| 57 | + x => |
| 58 | + val length = x._1.remaining() |
| 59 | + assert(x._1 === reader.read(new FileSegment(file.toString, x._2, length))) |
| 60 | + } |
| 61 | + reader.close() |
| 62 | + } |
| 63 | + |
| 64 | + test("Test reading data using random reader written with writer") { |
| 65 | + val dir = Files.createTempDir() |
| 66 | + val file = new File(dir, "TestRandomReads") |
| 67 | + try { |
| 68 | + val dataToWrite = for (i <- 1 to 50) yield generateRandomData() |
| 69 | + val segments = writeUsingWriter(file, dataToWrite) |
| 70 | + val iter = dataToWrite.iterator |
| 71 | + val reader = new WriteAheadLogRandomReader("file:///" + file.toString, hadoopConf) |
| 72 | + val writtenData = segments.map { x => |
| 73 | + reader.read(x) |
| 74 | + } |
| 75 | + assert(dataToWrite.toArray === writtenData.toArray) |
| 76 | + } finally { |
| 77 | + file.delete() |
| 78 | + dir.delete() |
| 79 | + } |
| 80 | + } |
| 81 | + |
| 82 | + test("Test successful reads using sequential reader") { |
| 83 | + val file = File.createTempFile("TestSequentialReads", "") |
| 84 | + file.deleteOnExit() |
| 85 | + val writtenData = writeData(50, file) |
| 86 | + val reader = new WriteAheadLogReader("file:///" + file.toString, hadoopConf) |
| 87 | + val iter = writtenData.iterator |
| 88 | + iter.foreach { x => |
| 89 | + assert(reader.hasNext === true) |
| 90 | + assert(reader.next() === x._1) |
| 91 | + } |
| 92 | + reader.close() |
| 93 | + } |
| 94 | + |
| 95 | + |
| 96 | + test("Test reading data using sequential reader written with writer") { |
| 97 | + val dir = Files.createTempDir() |
| 98 | + val file = new File(dir, "TestWriter") |
| 99 | + try { |
| 100 | + val dataToWrite = for (i <- 1 to 50) yield generateRandomData() |
| 101 | + val segments = writeUsingWriter(file, dataToWrite) |
| 102 | + val iter = dataToWrite.iterator |
| 103 | + val reader = new WriteAheadLogReader("file:///" + file.toString, hadoopConf) |
| 104 | + reader.foreach { x => |
| 105 | + assert(x === iter.next()) |
| 106 | + } |
| 107 | + } finally { |
| 108 | + file.delete() |
| 109 | + dir.delete() |
| 110 | + } |
| 111 | + } |
| 112 | + |
| 113 | + /** |
| 114 | + * Writes data to the file and returns the an array of the bytes written. |
| 115 | + * @param count |
| 116 | + * @return |
| 117 | + */ |
| 118 | + // We don't want to be using the WAL writer to test the reader - it will be painful to figure |
| 119 | + // out where the bug is. Instead generate the file by hand and see if the WAL reader can |
| 120 | + // handle it. |
| 121 | + def writeData(count: Int, file: File): ArrayBuffer[(ByteBuffer, Long)] = { |
| 122 | + val writtenData = new ArrayBuffer[(ByteBuffer, Long)]() |
| 123 | + val writer = new RandomAccessFile(file, "rw") |
| 124 | + var i = 0 |
| 125 | + while (i < count) { |
| 126 | + val data = generateRandomData() |
| 127 | + writtenData += ((data, writer.getFilePointer)) |
| 128 | + data.rewind() |
| 129 | + writer.writeInt(data.remaining()) |
| 130 | + writer.write(data.array()) |
| 131 | + i += 1 |
| 132 | + } |
| 133 | + writer.close() |
| 134 | + writtenData |
| 135 | + } |
| 136 | + |
| 137 | + def readData(segments: Seq[FileSegment], file: File): Seq[ByteBuffer] = { |
| 138 | + val reader = new RandomAccessFile(file, "r") |
| 139 | + segments.map { x => |
| 140 | + reader.seek(x.offset) |
| 141 | + val data = new Array[Byte](x.length) |
| 142 | + reader.readInt() |
| 143 | + reader.readFully(data) |
| 144 | + ByteBuffer.wrap(data) |
| 145 | + } |
| 146 | + } |
| 147 | + |
| 148 | + def generateRandomData(): ByteBuffer = { |
| 149 | + val data = new Array[Byte](random.nextInt(50)) |
| 150 | + random.nextBytes(data) |
| 151 | + ByteBuffer.wrap(data) |
| 152 | + } |
| 153 | + |
| 154 | + def writeUsingWriter(file: File, input: Seq[ByteBuffer]): Seq[FileSegment] = { |
| 155 | + val writer = new WriteAheadLogWriter(file.toString, hadoopConf) |
| 156 | + val segments = input.map(writer.write) |
| 157 | + writer.close() |
| 158 | + segments |
| 159 | + } |
| 160 | +} |
0 commit comments