Skip to content
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

Initial implementation of HDFSBackedBlockRDD. #15

Merged
merged 14 commits into from
Oct 9, 2014
Merged
Show file tree
Hide file tree
Changes from 11 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,96 @@
/*
Copy link
Owner

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 in streaming/dstream/

* 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]
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please make this private[streaming]

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) {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Worth asserting that size of segments and size of block ids are equal.


private var isTest = false
private var bmList: ArrayBuffer[Iterable[T]] = ArrayBuffer.empty[Iterable[T]]
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What is the bmList supposed to store?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is basically for testing. It stores the list of blocks that we got from the BlockManager. We use it in the test to ensure that the data from BM is what we stored.


private [storage] def test() {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Its not recommended to add code like this that is test-only that messes up and confuses the true logic.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't mind removing it, but I definitely think it is worth testing where the data came from. We want to ensure that we don't go to HDFS all the time.

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The clean and reliable way to test that is by writing data to one of the places only (either BM, or HDFS, not both), and then try to read through the RDD using rdd.collect(). That eliminates the chance of wrong tests, and does not require test-only flags.

In general I am not 100% opposed to having the such test-only stuff. Ultimately, what we desire is correct AND CLEAN code, which is easy and intuitive to understand. Such test-only stuff, in most cases, makes the code look more confusing.

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] = {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You will also have to override preferred locations. If the blocks are not in the block manager (that is not preferred location from block manager), it has to fall back to HDFS locations.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like the DAGScheduler will take care of a situation where preferred locations is empty. Correct me if I wrong, but it looks like all we need to do is to return the Seq from the locations_ map if it exists, else return empty Seq.

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It will. But we are loosing opportunity for using preferred HDFS location when we can.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What do you mean by HDFS location preference? You mean short-circuit reads? I think we can investigate it, but it would be an optimization rather and is supported only in the latest versions of hadoop (like 2.4.0+).

I will check with the HDFS folks to figure out how to map DNs to the file name, but I wouldn't be too worried about it right now.

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I didnt get you. Since long before hadoopRDD gets the locations of the data blocks of a HDFS file from the name node and uses that schedule tasks corresponding to blocks. This is something MapReduce has used since long ago, the usual locality stuff.

The challenge here is to map to the desired filesegment to the block that may contain it, and then the block to node mappign is the usual (same way as HadoopRDD does it). It is fine to make this a future PR. But I want to make sure we are in the same page in understanding what this is.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, I was talking about short-circuit local reads, which gives a massive performance boost. I will add this in the next update.

assertValid()
val blockManager = sc.env.blockManager
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You cannot access sc in compute, as this runs on the workers, and sc is null there.
Use SparkEnv.get to get the environment.

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)
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I dont think there is much of a cost in creating a new Reader, so I am not worried about that.

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
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should make that an optional parameter of this RDD. The DStream may set it depending on more semantic information abotu whether we are going to ever need this data every again. If we do, then it can be put in the BlockMAnager.

// option since each of
val data = blockManager.dataDeserialize(blockId, dataRead).toIterable
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why make it iterable and then again call iterator? dataDeserialize returns an iterator.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because when we re-insert the data into the BM, the iterator is traversed. So when we return the iterator it can't be traversed again and the data will not be accessible when compute returns. We can either not insert the data into the BM or we have to do this.

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

BETTER WAY.

You have the byte buffer right?

val iterator = blockManager.dataSerialize(byteBuffer)
if (haveToPutInBlockManager)  {
     // assume that the storage level is NOT deserialized
     blockManager.putBytes(blockId, byteBuffer, storageLevel, tellMaster = true)
}
iterator

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this assumption valid? What if the storage level is deserialized?

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I am strongly thinking of making the simplifyiing assumption that this feature is not supported with storage level deserialized. Other the code get complex, here, as well as,

I think this is a fair assumption to make as of now for simplicity. Later we can add support for that. What do you think?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yep. We can document it - so we don't need to hack around it.

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])
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Instead of accessing locations_, could you make a protected method to access the locations. Call it something meaningful.

}
}
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 {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please dont use TestSuiteBase. Using that and tying into that is unnecessary. See the update WALSuite.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I used it for access to the spark conf. I am not entirely sure it is needed though

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Its not necessary. TestSutieBase's sparkconf and other methods are useful only when you are running a streamingcontext and getting results. Here just create an empty new SparkConf

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() {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use before { ... } and after { ... }

super.beforeFunction()
dir = Files.createTempDir()
file = new File(dir, "BlockManagerWrite")
}

override def afterFunction() {
super.afterFunction()
file.delete()
dir.delete()
}

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldnt there be another test which involves when all the data is in HDFS ?
Also a nit: please order the tests from simple usecase to complex - all BM, all HDFS, partial BM, ...

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") {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why the special case test of 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) {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isnt the test about reading the data through the RDD? Its not about whether HDFS writes are successful or not, as this name seems to suggest.

val countPerBlock = total / blockCount
val blockIds = (0 until blockCount).map { _ =>
StreamBlockId(streamCounter.incrementAndGet(), blockIdCounter.incrementAndGet())
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why need streamCounter?? You can use i in (0 until blockCount).map { i => StreamBlockId(i, i) }

}

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))
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldnt this be simpler? Why not just call
new HDFSBackedBlockRDD(....).collect() which should give back all the inserted records, and compare that to the intended data.

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]) = {

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

assert that count === blockIds.size, just to be safe.

val strings: Seq[String] = (0 until count).map(_ => RandomStringUtils.randomAlphabetic(50))
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

map { _ => ... }

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also dont need : Seq[String]

Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And you can use scala.util.Random.nextString(stringSize) instead of external library.


var writerOpt: Option[WriteAheadLogWriter] = None
try {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am not sure if the try...finally is really necessary. In this test code we dont expect this to ever fail. And if it every fails the test will fail and we have something bigger to worry about, not just dangling file descriptors.

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)))
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: this isnt quite exact, as if count % countPerBlock != 0, the number of elements created is > count; the last partition should have < countPerBlock elements (i.e. partial block), but this code creates full block instead of partial block.

better code

(1 to count).grouped(countPerBlock).toSeq

val blockIdIter = blockIds.iterator
(blockData, blockData.map {
Copy link
Owner

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cleaner Scala-like side-effect-free code (this code exploits the side effect of incrementing the iterator) would be.

blockData.zip(blockIds).foreach { case (data, id) => 
    writer.write(blockManager.dataSerialize(id, data.iterator))
}

x =>
writer.write(blockManager.dataSerialize(blockIdIter.next(), x.iterator))
})
} finally {
writerOpt.foreach(_.close())
}
}
}