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

[SPARK-4027][Streaming] WriteAheadLogBackedBlockRDD to read received either from BlockManager or WAL in HDFS #2931

Closed
wants to merge 13 commits into from
Closed
4 changes: 4 additions & 0 deletions core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -84,5 +84,9 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds
"Attempted to use %s after its blocks have been removed!".format(toString))
}
}

protected def getBlockIdLocations(): Map[BlockId, Seq[String]] = {
locations_
}
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/*
* 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.rdd

import scala.reflect.ClassTag

import org.apache.hadoop.conf.Configuration

import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.BlockRDD
import org.apache.spark.storage.{BlockId, StorageLevel}
import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
import org.apache.spark._
Copy link
Contributor

Choose a reason for hiding this comment

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

import out of order


private[streaming]
class HDFSBackedBlockRDDPartition(
val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
Copy link
Contributor

Choose a reason for hiding this comment

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

For classes whose constructors don't fit on a single line, I think our style is to wrap it with one field per line, indented two spaces.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

val index = idx
Copy link
Contributor

Choose a reason for hiding this comment

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

Why not just make the constructor idx a val and rename it to index?

}

private[streaming]
class HDFSBackedBlockRDD[T: ClassTag](
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't think it makes sense to tie this to WriteAheadLogFileSegment. On one hand the naming HDFSBackedBlockRDD is supposed to be general, on the other you tie it to recovery through the use of WriteAheadLogFileSegment.

Copy link
Contributor

Choose a reason for hiding this comment

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

It would be great to add javadoc explaining what this class is for. If it is used for recovery, why should we put the blocks in block manager after using them? Shouldn't recovery data be used only once during a recovery?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good point, how about renaming this class to more specific WriteAheadLogBasedBackedBlockRDD (kind-a-long).

Copy link
Contributor

Choose a reason for hiding this comment

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

I think we could still have to use the recovered data if the same RDD is used for multiple operations, correct? Maybe I am mistaken, but if I do something like

rdd1 = hdfsRdd.<blah>
and
rdd2=hdfsRdd.<blah2>

wouldn't it be better if the data recovered is now in BlockManager?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Changed to WriteAheadLogBasedBackedBlockRDD

@transient sc: SparkContext,
@transient hadoopConfiguration: Configuration,
@transient override val blockIds: Array[BlockId],
Copy link
Contributor

Choose a reason for hiding this comment

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

i don't think u need to declare this a field. it's already a field in the parent class and you can just use that?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Didnt quite get it. We do have to pass block IDs into the constructor for BlockRDD, for that block IDs need to be taken as part of the constructor of this class - either with as override blockIds or with a different name. Isnt it? Or is there a better pattern that i dont know of?

Copy link
Contributor

Choose a reason for hiding this comment

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

I think that Reynold was asking whether this needs to be declared as val here, not whether the constructor needs to accept blockIds.

Copy link
Contributor

Choose a reason for hiding this comment

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

Done

@transient val segments: Array[WriteAheadLogFileSegment],
val storeInBlockManager: Boolean,
val storageLevel: StorageLevel
) extends BlockRDD[T](sc, blockIds) {

if (blockIds.length != segments.length) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I usually like require for stuff like this.

Copy link
Contributor

Choose a reason for hiding this comment

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

By the way, I found a neat library that auto-generates really nice error messages for require statements: http://www.scalactic.org/user_guide/Requirements

throw new IllegalStateException("Number of block ids must be the same as number of segments!")
}

// Hadoop Configuration is not serializable, so broadcast it as a serializable.
val broadcastedHadoopConf = sc.broadcast(new SerializableWritable(hadoopConfiguration))
Copy link
Contributor

Choose a reason for hiding this comment

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

Over in #2935, @davies is planning to add some code to SerializableWritable to address the Hadoop Configuration constructor thread-safety issue, so you shouldn't have to do it here once we've merged that patch.

Copy link
Contributor

Choose a reason for hiding this comment

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

Does it make sense to take the SerializableWritable as the argument in the constructor (as being done in #2935) or should we just take the hadoopConf and wrap it in the SerializableWritable once that is merged? We don't want to change the interface later.

Copy link
Contributor

Choose a reason for hiding this comment

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

For now I am leaving this as is. Lets revisit this later if needed.

Copy link
Contributor

Choose a reason for hiding this comment

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

can this be private[this]?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Should definitely be private, @harishreedharan

.asInstanceOf[Broadcast[SerializableWritable[Configuration]]]
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do you need this cast?

Copy link
Contributor

Choose a reason for hiding this comment

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

We don't. I don't even remember why I added it at that time.

override def getPartitions: Array[Partition] = {
assertValid()
(0 until blockIds.size).map { i =>
Copy link
Contributor

Choose a reason for hiding this comment

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

Array.tabulate

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

new HDFSBackedBlockRDDPartition(blockIds(i), i, segments(i))
}.toArray
}

override def compute(split: Partition, context: TaskContext): Iterator[T] = {
assertValid()
val hadoopConf = broadcastedHadoopConf.value.value
val blockManager = SparkEnv.get.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) =>
block.data.asInstanceOf[Iterator[T]]
// Data not found in Block Manager, grab it from HDFS
case None =>
logInfo("Reading partition data from write ahead log " + partition.segment.path)
val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
val dataRead = reader.read(partition.segment)
reader.close()
// Currently, we support storing the data to BM only in serialized form and not in
// deserialized form
if (storeInBlockManager) {
blockManager.putBytes(blockId, dataRead, storageLevel)
}
dataRead.rewind()
blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
}
}

override def getPreferredLocations(split: Partition): Seq[String] = {
val partition = split.asInstanceOf[HDFSBackedBlockRDDPartition]
val locations = getBlockIdLocations()
locations.getOrElse(partition.blockId,
Copy link
Contributor

Choose a reason for hiding this comment

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

these few lines of getOrElse's are way too complicated to use getOrElse.

For the outer most layer, just create an if/else to make it more clear.

Copy link
Contributor

Choose a reason for hiding this comment

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

For very deeply-nested versions of this pattern, where you want to select the first non-None value among many alternatives while lazily computing them, I like doing something like

def sparkPreferredLocations = getBlockIdLocations().get(partition.blockId)
def hdfsPreferredLocations = HdfsUtils.getBlockLocations(...)
sparkPreferredLocations.orElse(hdfsPreferredLocations).getOrElse(Array[String].empty)

I kind of like this because it flattens the nested structure so the code reads as "here are the alternatives to choose among, defined in order of their precedence, and here's the line that tries them in order." I'm not sure whether this is more or less confusing than the nesting for new Scala users, though. There's probably an even nicer version of this without all of the intermediate orElse calls, but I guess that's only really necessary if you're picking among many alternatives; it might be excessively confusing here.

Copy link
Contributor

Choose a reason for hiding this comment

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

Correct, but we don't want to get the HDFS locations if we know that the sparkPreferredLocations exists, as looking up the locations from HDFS comes at a non-trivial cost of an RPC call. So I will move it to an if-else, so it is clearer.

Copy link
Contributor

Choose a reason for hiding this comment

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

we don't want to get the HDFS locations if we know that the sparkPreferredLocations exists

The snippet that I listed actually implements this behavior: if sparkPreferreredLocations is not None, then the orElse short-circuits and never evaluates hdfsPreferredLocations.

Copy link
Contributor

Choose a reason for hiding this comment

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

Ah, didn't notice the def :-)

Thanks!

Copy link
Contributor

Choose a reason for hiding this comment

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

Let's make this dead simple. There is no need to create two extra functions and call getOrElse.

Just do

locations.get(partition.blockId)) match {
  case Some(loc) =>
    loc
  case None =>
    val segment = partition.segment
    HdfsUtils.getBlockLocations(segment.path, segment.offset, segment.length, hadoopConfiguration).getOrElse(Seq.empty)
}

HdfsUtils.getBlockLocations(partition.segment.path, hadoopConfiguration)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Can you explain how this code gets the block locations of the segment of the file that the partition needs? The offsets dont seem to be passed on to the HDFSUtils.getBlockLocations

Copy link
Contributor

Choose a reason for hiding this comment

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

Fixed this one in the PR sent to your repo.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

.getOrElse(new Array[String](0)).toSeq)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,152 @@
/*
* 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.rdd

import java.io.File
Copy link
Contributor

Choose a reason for hiding this comment

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

imports out of order

import java.util.concurrent.atomic.AtomicInteger

import scala.collection.mutable.ArrayBuffer
import org.scalatest.{BeforeAndAfter, FunSuite}

import com.google.common.io.Files
import org.apache.hadoop.conf.Configuration

import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId}
import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
import org.apache.spark.{SparkConf, SparkContext}

class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter {
Copy link
Contributor

Choose a reason for hiding this comment

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

This should probably extend the SharedSparkContext test trait in Spark in order to ensure that the SparkContext is eventually cleaned up properly.

Copy link
Contributor

Choose a reason for hiding this comment

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

Getting LocalSparkContext is actually sort of painful as it is a test class, so we need to create a test-jar in mvn, which sbt does not pull in causing compilation failures even if the test-jar is added to the pom.

Copy link
Contributor

Choose a reason for hiding this comment

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

I copied the code for stopping and clearing the properties into this test though

val conf = new SparkConf()
.setMaster("local[2]")
.setAppName(this.getClass.getSimpleName)
val sparkContext = new SparkContext(conf)
val hadoopConf = new Configuration()
val blockManager = sparkContext.env.blockManager
// Since the same BM is reused in all tests, use an atomic int to generate ids
val idGenerator = new AtomicInteger(0)
var file: File = null
var dir: File = null

before {
dir = Files.createTempDir()
file = new File(dir, "BlockManagerWrite")
}

after {
file.delete()
dir.delete()
}

test("Data available in BM and HDFS") {
doTestHDFSBackedRDD(5, 5, 20, 5)
}

test("Data available in in BM but not in HDFS") {
doTestHDFSBackedRDD(5, 0, 20, 5)
}

test("Data available in in HDFS and not in BM") {
doTestHDFSBackedRDD(0, 5, 20, 5)
}

test("Data partially available in BM, and the rest in HDFS") {
doTestHDFSBackedRDD(3, 2, 20, 5)
}

/**
* Write a bunch of events into the HDFS Block RDD. Put a part of all of them to the
Copy link
Contributor

Choose a reason for hiding this comment

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

what do u mean by "a part of all of them"?

* BlockManager, so all reads need not happen from HDFS.
* @param total - Total number of Strings to write
Copy link
Contributor

Choose a reason for hiding this comment

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

You don't need a dash here, at least according to the style of the rest of the scaladocs.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed.

* @param blockCount - Number of blocks to write (therefore, total # of events per block =
* total/blockCount
*/
private def doTestHDFSBackedRDD(
writeToBMCount: Int,
writeToHDFSCount: Int,
total: Int,
Copy link
Contributor

Choose a reason for hiding this comment

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

I think numItems might be a clearer variable name

Copy link
Contributor Author

Choose a reason for hiding this comment

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

refactored.

blockCount: Int
) {
Copy link
Contributor

Choose a reason for hiding this comment

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

move this to the previous line?

val countPerBlock = total / blockCount
val blockIds = (0 until blockCount).map {
i =>
Copy link
Contributor

Choose a reason for hiding this comment

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

Style nit: place this i => on the previous line.

StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
Copy link
Contributor

Choose a reason for hiding this comment

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

indent off

}

val writtenStrings = generateData(total, countPerBlock)

if (writeToBMCount != 0) {
(0 until writeToBMCount).foreach { i =>
blockManager
.putIterator(blockIds(i), writtenStrings(i).iterator, StorageLevel.MEMORY_ONLY_SER)
}
}

val segments = new ArrayBuffer[WriteAheadLogFileSegment]
Copy link
Contributor

Choose a reason for hiding this comment

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

I'd just do segments = if (writeToHDFSCount) and return immutable segments from each branch to avoid making segments mutable.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

if (writeToHDFSCount != 0) {
// Generate some fake segments for the blocks in BM so the RDD does not complain
segments ++= generateFakeSegments(writeToBMCount)
segments ++= writeDataToHDFS(writtenStrings.slice(writeToBMCount, blockCount),
blockIds.slice(writeToBMCount, blockCount))

Copy link
Contributor

Choose a reason for hiding this comment

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

Extra blank line here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Removed

} else {
segments ++= generateFakeSegments(blockCount)
}
val rdd = new HDFSBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray,
segments.toArray, false, StorageLevel.MEMORY_ONLY)

val dataFromRDD = rdd.collect()
// verify each partition is equal to the data pulled out
assert(writtenStrings.flatten === dataFromRDD)
}

/**
* 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
* @return - Tuple of (Seq of Seqs, each of these Seqs is one block, Seq of WriteAheadLogFileSegments,
* each representing the block being written to HDFS.
*/
private def generateData(
count: Int,
countPerBlock: Int
): Seq[Seq[String]] = {
val strings = (0 until count).map { _ => scala.util.Random.nextString(50)}
Copy link
Contributor

Choose a reason for hiding this comment

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

.tabulate

Copy link
Contributor Author

Choose a reason for hiding this comment

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

refactored.

strings.grouped(countPerBlock).toSeq
}

private def writeDataToHDFS(
blockData: Seq[Seq[String]],
blockIds: Seq[BlockId]
): Seq[WriteAheadLogFileSegment] = {
assert(blockData.size === blockIds.size)
val segments = new ArrayBuffer[WriteAheadLogFileSegment]()
val writer = new WriteAheadLogWriter(file.toString, hadoopConf)
blockData.zip(blockIds).foreach {
case (data, id) =>
Copy link
Contributor

Choose a reason for hiding this comment

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

move case to previous line

segments += writer.write(blockManager.dataSerialize(id, data.iterator))
}
writer.close()
segments
}

private def generateFakeSegments(count: Int): Seq[WriteAheadLogFileSegment] = {
(0 until count).map {
_ => new WriteAheadLogFileSegment("random", 0l, 0)
}
}
}