-
Notifications
You must be signed in to change notification settings - Fork 3
Initial implementation of HDFSBackedBlockRDD. #15
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 11 commits
6f11db4
8b1b29c
bf0ac9b
6fc4cd8
7b49216
6be55a8
15d05d4
098cbd1
389acea
e70d390
cf39750
ea227f0
c4211d7
1fe3567
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,96 @@ | ||
| /* | ||
| * 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.storage | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.reflect.ClassTag | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
|
|
||
| import org.apache.spark.rdd.BlockRDD | ||
| import org.apache.spark.storage.{StorageLevel, BlockId} | ||
| import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} | ||
|
|
||
| private[spark] | ||
|
||
| class HDFSBackedBlockRDDPartition(val blockId: BlockId, idx: Int, val segment: FileSegment) | ||
| extends Partition { | ||
| val index = idx | ||
| } | ||
|
|
||
| private[spark] | ||
| class HDFSBackedBlockRDD[T: ClassTag]( | ||
| @transient sc: SparkContext, | ||
| hadoopConf: Configuration, | ||
| @transient override val blockIds: Array[BlockId], | ||
| @transient val segments: Array[FileSegment], | ||
| val storageLevel: StorageLevel | ||
| ) extends BlockRDD[T](sc, blockIds) { | ||
|
||
|
|
||
| private var isTest = false | ||
| private var bmList: ArrayBuffer[Iterable[T]] = ArrayBuffer.empty[Iterable[T]] | ||
|
||
|
|
||
| private [storage] def test() { | ||
|
||
| isTest = true | ||
| bmList = new ArrayBuffer[Iterable[T]]() | ||
| } | ||
|
|
||
| private [storage] def getBmList: ArrayBuffer[Iterable[T]] = { | ||
| bmList | ||
| } | ||
|
|
||
| override def getPartitions: Array[Partition] = { | ||
| assertValid() | ||
| (0 until blockIds.size).map { i => | ||
| new HDFSBackedBlockRDDPartition(blockIds(i), i, segments(i)) | ||
| }.toArray | ||
| } | ||
|
|
||
| override def compute(split: Partition, context: TaskContext): Iterator[T] = { | ||
|
||
| assertValid() | ||
| val blockManager = sc.env.blockManager | ||
|
||
| val partition = split.asInstanceOf[HDFSBackedBlockRDDPartition] | ||
| val blockId = partition.blockId | ||
| blockManager.get(blockId) match { | ||
| // Data is in Block Manager, grab it from there. | ||
| case Some(block) => | ||
| val data = block.data.asInstanceOf[Iterator[T]] | ||
| if (isTest) { | ||
| val dataCopies = data.duplicate | ||
| bmList += dataCopies._1.toIterable | ||
| dataCopies._2 | ||
| } else { | ||
| data | ||
| } | ||
| // Data not found in Block Manager, grab it from HDFS | ||
| case None => | ||
| // TODO: Perhaps we should cache readers at some point? | ||
| val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf) | ||
|
||
| val dataRead = reader.read(partition.segment) | ||
| reader.close() | ||
| // Should we make it configurable whether we want to insert data into BM? If we don't | ||
| // need to insert it into BM we can avoid duplicating the iterator. This is the only | ||
|
||
| // option since each of | ||
| val data = blockManager.dataDeserialize(blockId, dataRead).toIterable | ||
|
||
| blockManager.putIterator(blockId, data.iterator, storageLevel) | ||
| data.iterator.asInstanceOf[Iterator[T]] | ||
| } | ||
| } | ||
|
|
||
| override def getPreferredLocations(split: Partition): Seq[String] = { | ||
| locations_.getOrElse(split.asInstanceOf[HDFSBackedBlockRDDPartition].blockId, Seq.empty[String]) | ||
|
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,146 @@ | ||
| /* | ||
| * 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.storage | ||
|
|
||
| import java.io.File | ||
| import java.nio.ByteBuffer | ||
| import java.util.concurrent.atomic.AtomicInteger | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import com.google.common.io.Files | ||
| import org.apache.commons.lang.RandomStringUtils | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.scalatest.BeforeAndAfter | ||
|
|
||
| import org.apache.spark.{SparkEnv, TaskContext, SparkContext} | ||
| import org.apache.spark.storage.{BlockId, StreamBlockId, StorageLevel} | ||
| import org.apache.spark.streaming.TestSuiteBase | ||
|
|
||
| class HDFSBackedBlockRDDSuite extends TestSuiteBase { | ||
|
||
| val sparkContext = new SparkContext(conf) | ||
| val hadoopConf = new Configuration() | ||
| val blockIdCounter = new AtomicInteger(0) | ||
| val streamCounter = new AtomicInteger(0) | ||
| val blockManager = sparkContext.env.blockManager | ||
| var file: File = null | ||
| var dir: File = null | ||
|
|
||
| override def beforeFunction() { | ||
|
||
| super.beforeFunction() | ||
| dir = Files.createTempDir() | ||
| file = new File(dir, "BlockManagerWrite") | ||
| } | ||
|
|
||
| override def afterFunction() { | ||
| super.afterFunction() | ||
| file.delete() | ||
| dir.delete() | ||
| } | ||
|
|
||
|
||
| test("Verify all data is available when part of the data is only on HDFS") { | ||
| doTestHDFSWrites(writeAllToBM = false, 20, 5) | ||
| } | ||
|
|
||
| test("Verify all data is available when all data is in BM") { | ||
| doTestHDFSWrites(writeAllToBM = true, 20, 5) | ||
| } | ||
|
|
||
| test("Verify all data is available when part of the data is in BM with one string per block") { | ||
| doTestHDFSWrites(writeAllToBM = false, 20, 20) | ||
| } | ||
|
|
||
| test("Verify all data is available when all data is in BM with one string per block") { | ||
|
||
| doTestHDFSWrites(writeAllToBM = true, 20, 20) | ||
| } | ||
|
|
||
| /** | ||
| * Write a bunch of events into the HDFS Block RDD. Put a part of all of them to the | ||
| * BlockManager, so all reads need not happen from HDFS. | ||
| * @param writeAllToBM - If true, all data is written to BlockManager | ||
| * @param total - Total number of Strings to write | ||
| * @param blockCount - Number of blocks to write (therefore, total # of events per block = | ||
| * total/blockCount | ||
| */ | ||
| private def doTestHDFSWrites(writeAllToBM: Boolean, total: Int, blockCount: Int) { | ||
|
||
| val countPerBlock = total / blockCount | ||
| val blockIds = (0 until blockCount).map { _ => | ||
| StreamBlockId(streamCounter.incrementAndGet(), blockIdCounter.incrementAndGet()) | ||
|
||
| } | ||
|
|
||
| val (writtenStrings, segments) = writeDataToHDFS(total, countPerBlock, file, blockIds) | ||
|
|
||
| val writtenToBM = new ArrayBuffer[Iterable[String]]() | ||
| for (i <- 0 until writtenStrings.length) { | ||
| if (i % 2 == 0 || writeAllToBM) { | ||
| writtenToBM += writtenStrings(i) | ||
| blockManager.putIterator(blockIds(i), writtenStrings(i).iterator, | ||
| StorageLevel.MEMORY_ONLY) | ||
| } | ||
| } | ||
|
|
||
| val rdd = new HDFSBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, | ||
| segments.toArray, StorageLevel.MEMORY_ONLY) | ||
| rdd.test() | ||
| val partitions = rdd.getPartitions | ||
| // The task context is not used in this RDD directly, so ok to be null | ||
| val dataFromRDD = partitions.map(rdd.compute(_, null)) | ||
|
||
| val copiedData = dataFromRDD.map(_.duplicate) | ||
| // verify each partition is equal to the data pulled out | ||
| for(i <- 0 until writtenStrings.length) { | ||
| assert(writtenStrings(i) === copiedData(i)._1.toIterable) | ||
| } | ||
| assert(writtenStrings.flatten === copiedData.map(_._2.toIterable).flatten) | ||
| assert(writtenToBM === rdd.getBmList) | ||
| assert(writtenToBM.flatten === rdd.getBmList.flatten) | ||
| } | ||
|
|
||
| /** | ||
| * Write data to HDFS and get a list of Seq of Seqs in which each Seq represents the data that | ||
| * went into one block. | ||
| * @param count - Number of Strings to write | ||
| * @param countPerBlock - Number of Strings per block | ||
| * @param file - The file to write to | ||
| * @param blockIds - List of block ids to use. | ||
| * @return - Tuple of (Seq of Seqs, each of these Seqs is one block, Seq of FileSegments, | ||
| * each representing the block being written to HDFS. | ||
| */ | ||
| private def writeDataToHDFS( | ||
| count: Int, | ||
| countPerBlock: Int, | ||
| file: File, | ||
| blockIds: Seq[BlockId] | ||
| ): (Seq[Seq[String]], Seq[FileSegment]) = { | ||
|
|
||
|
||
| val strings: Seq[String] = (0 until count).map(_ => RandomStringUtils.randomAlphabetic(50)) | ||
|
||
|
|
||
| var writerOpt: Option[WriteAheadLogWriter] = None | ||
| try { | ||
|
||
| writerOpt = Some(new WriteAheadLogWriter(file.toString, hadoopConf)) | ||
| val writer = writerOpt.get | ||
| val blockData = | ||
| 0.until(count, countPerBlock).map(y => (0 until countPerBlock).map(x => strings(x + y))) | ||
|
||
| val blockIdIter = blockIds.iterator | ||
| (blockData, blockData.map { | ||
|
||
| x => | ||
| writer.write(blockManager.dataSerialize(blockIdIter.next(), x.iterator)) | ||
| }) | ||
| } finally { | ||
| writerOpt.foreach(_.close()) | ||
| } | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Cna you move this file to
streaming/rdd/and package o.a.s.streaming.rdd? Keeps things consistent with all the DStreams being defined instreaming/dstream/