You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by tdas <gi...@git.apache.org> on 2014/10/24 20:58:40 UTC

[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

GitHub user tdas opened a pull request:

    https://github.com/apache/spark/pull/2931

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

    As part of the initiative of preventing data loss on streaming driver failure, this sub-task implements a BlockRDD that is backed by HDFS. This BlockRDD can either read data from the Spark's BlockManager, or read the data from file-segments in write ahead log in HDFS. 


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tdas/spark driver-ha-rdd

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/2931.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2931
    
----
commit eadde561ed26ec136273eab928b84fedc4c5fd3a
Author: Tathagata Das <ta...@gmail.com>
Date:   2014-10-23T19:54:02Z

    Transferred HDFSBackedBlockRDD for the driver-ha-working branch

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60969642
  
    This one is ready to go in?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19634884
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,125 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment)
    +  extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient blockIds: Array[BlockId],
    +    @transient segments: Array[WriteAheadLogFileSegment],
    +    storeInBlockManager: Boolean,
    +    storageLevel: StorageLevel)
    +  extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i))
    +    }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    def blockLocations = getBlockIdLocations().get(partition.blockId)
    +    def segmentLocations = HdfsUtils.getFileSegmentLocations(
    +      partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig)
    +    blockLocations.orElse(segmentLocations).getOrElse(Seq.empty)
    --- End diff --
    
    This is the final version that I am doing then. 
    ```
        val blockLocations = getBlockIdLocations().get(partition.blockId)
        def segmentLocations = HdfsUtils.getFileSegmentLocations(...)
        blockLocations.getOrElse(segmentLocations)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372365
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    --- End diff --
    
    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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19591383
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,125 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment)
    +  extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient blockIds: Array[BlockId],
    +    @transient segments: Array[WriteAheadLogFileSegment],
    +    storeInBlockManager: Boolean,
    +    storageLevel: StorageLevel)
    +  extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i))
    +    }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    def blockLocations = getBlockIdLocations().get(partition.blockId)
    +    def segmentLocations = HdfsUtils.getFileSegmentLocations(
    +      partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig)
    +    blockLocations.orElse(segmentLocations).getOrElse(Seq.empty)
    --- End diff --
    
    It's not walking over my dead body type of thing, but I think declaring two inline functions for this, coupled with orElse / getOrElse is less intuitive to most people. 
    
    Maybe others can chime in here. @shivaram @pwendell 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by shivaram <gi...@git.apache.org>.
Github user shivaram commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19612661
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,125 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment)
    +  extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient blockIds: Array[BlockId],
    +    @transient segments: Array[WriteAheadLogFileSegment],
    +    storeInBlockManager: Boolean,
    +    storageLevel: StorageLevel)
    +  extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i))
    +    }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    def blockLocations = getBlockIdLocations().get(partition.blockId)
    +    def segmentLocations = HdfsUtils.getFileSegmentLocations(
    +      partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig)
    +    blockLocations.orElse(segmentLocations).getOrElse(Seq.empty)
    --- End diff --
    
    Yeah its not very ideal as I think the most easy to understand is something like
    
    ```
    if ( ) {
      blockLocations
    } else if ( ) {
      segmentLocations
    } else {
      Seq.empty
    }
    ```
    
    but this isnt too bad if the above isn't possible


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60452179
  
    Jenkins, test this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60837536
  
    Shouldn't it be WriteAheadLogBackedRDD not WriteAheadLogBasedBackedRDD (Based and Backed together without anything in between does not make sense - no?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19428929
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    +  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
    +   * BlockManager, so all reads need not happen from HDFS.
    +   * @param total - Total number of Strings to write
    --- End diff --
    
    Removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19485339
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  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 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,
    --- End diff --
    
    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
    
    ```scala
    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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60860153
  
      [Test build #22398 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22398/consoleFull) for   PR 2931 at commit [`b0a18b1`](https://github.com/apache/spark/commit/b0a18b1562ef4821fe3aa09555f26c14540e868b).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class WriteAheadLogBackedBlockRDDPartition(`
      * `class WriteAheadLogBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19422693
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    +    .asInstanceOf[Broadcast[SerializableWritable[Configuration]]]
    +  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 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,
    +      HdfsUtils.getBlockLocations(partition.segment.path, hadoopConfiguration)
    --- End diff --
    
    Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19506143
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    (0 until blockIds.size).map { i =>
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60449113
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22152/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19573331
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    --- End diff --
    
    nitpick: the common style in spark is
    ```scala
        val storageLevel: StorageLevel)
      extends BlockRDD[T](sc, blockIds) {
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61080839
  
      [Test build #22537 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22537/consoleFull) for   PR 2931 at commit [`4a5866f`](https://github.com/apache/spark/commit/4a5866fb19fcd4fde7551294a30bd740500c834a).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class WriteAheadLogBackedBlockRDDPartition(`
      * `class WriteAheadLogBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19506112
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60653950
  
    Jenkins, test this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60459010
  
    The HdfsBackedRDDSuite is passing - not sure why there are some other failures. Maybe we are missing some cleanup?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19457227
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    --- End diff --
    
    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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60471236
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22189/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61009625
  
    @harishreedharan / @tdas I made a few more comments. Most are just nits that I've left earlier.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61011552
  
    Thanks @rxin. Updates coming soon. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454543
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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 = {
    +      if (writeToHDFSCount != 0) {
    +        // Generate some fake segments for the blocks in BM so the RDD does not complain
    +        generateFakeSegments(writeToBMCount) ++
    +          writeDataToHDFS(writtenStrings.slice(writeToBMCount, blockCount),
    +            blockIds.slice(writeToBMCount, blockCount))
    +      } else {
    +        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 Seq of Seqs, each of these Seqs is one block
    +   */
    +  private def generateData(
    +      count: Int,
    +      countPerBlock: Int
    +    ): Seq[Seq[String]] = {
    +    val strings = (0 until count).map { _ => scala.util.Random.nextString(50)}
    +    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) =>
    --- End diff --
    
    move case to previous line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61057132
  
    @tdas you also missed my two other comments


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60649869
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22300/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372560
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    +    .asInstanceOf[Broadcast[SerializableWritable[Configuration]]]
    --- End diff --
    
    Why do you need this cast?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60847372
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22387/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19373109
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    +  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
    +   * BlockManager, so all reads need not happen from HDFS.
    +   * @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 doTestHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map {
    +      i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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]
    --- End diff --
    
    I'd just do `segments = if (writeToHDFSCount)` and return immutable segments from each branch to avoid making `segments` mutable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61048431
  
      [Test build #22521 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22521/consoleFull) for   PR 2931 at commit [`ed5fbf0`](https://github.com/apache/spark/commit/ed5fbf0765136da963f6a8447f1ff69191825392).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class WriteAheadLogBackedBlockRDDPartition(`
      * `class WriteAheadLogBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19428914
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    +  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
    +   * BlockManager, so all reads need not happen from HDFS.
    +   * @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 doTestHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map {
    +      i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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]
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454599
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  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 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,
    --- End diff --
    
    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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19574168
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i)) }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockLocations = getBlockIdLocations().get(partition.blockId)
    +    lazy val segmentLocations = HdfsUtils.getFileSegmentLocations(
    --- End diff --
    
    this is a nice and creative use of lazy val, but really I think we should make it dead obvious what's happening here, rather than relying on the lazy semantics to not run getFileSegmentLocations 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61080845
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22537/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60668550
  
    jenkins, test this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61044672
  
    @rxin I updated. Only part i am not in agreement is the preferred location logic.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60678825
  
      [Test build #22306 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22306/consoleFull) for   PR 2931 at commit [`6e1bfb8`](https://github.com/apache/spark/commit/6e1bfb8dae4eb829c59dd6baf423b93bd4d2971f).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HDFSBackedBlockRDDPartition(`
      * `class HDFSBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60469691
  
    I left a pass of fairly shallow style comments; I'll loop back later to offer more substantive feedback and to actually check that I understand this logic.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19429864
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19373065
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    --- End diff --
    
    This should probably extend the `SharedSparkContext` test trait in Spark in order to ensure that the SparkContext is eventually cleaned up properly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372245
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    --- End diff --
    
    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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19483666
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    --- End diff --
    
    I think that Reynold was asking whether this needs to be declared as `val` here, not whether the constructor needs to accept blockIds.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454583
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    --- End diff --
    
    imports out of order


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19374085
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    --- End diff --
    
    For now I am leaving this as is. Lets revisit this later if needed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60853490
  
      [Test build #22398 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22398/consoleFull) for   PR 2931 at commit [`b0a18b1`](https://github.com/apache/spark/commit/b0a18b1562ef4821fe3aa09555f26c14540e868b).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372184
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    --- End diff --
    
    Why not just make the constructor `idx` a val and rename it to `index`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19506096
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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 = {
    +      if (writeToHDFSCount != 0) {
    +        // Generate some fake segments for the blocks in BM so the RDD does not complain
    +        generateFakeSegments(writeToBMCount) ++
    +          writeDataToHDFS(writtenStrings.slice(writeToBMCount, blockCount),
    +            blockIds.slice(writeToBMCount, blockCount))
    +      } else {
    +        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 Seq of Seqs, each of these Seqs is one block
    +   */
    +  private def generateData(
    +      count: Int,
    +      countPerBlock: Int
    +    ): Seq[Seq[String]] = {
    +    val strings = (0 until count).map { _ => scala.util.Random.nextString(50)}
    +    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) =>
    +        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) }
    --- End diff --
    
    Done.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19492065
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  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 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,
    --- End diff --
    
    Ah, didn't notice the def :-)
    
    Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] WriteAheadLogBackedBlo...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/2931


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454515
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * Write a bunch of events into the HDFS Block RDD. Put a part of all of them to the
    --- End diff --
    
    what do u mean by "a part of all of them"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60847367
  
      [Test build #22387 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22387/consoleFull) for   PR 2931 at commit [`29aa099`](https://github.com/apache/spark/commit/29aa0999ac72055bb51eeb6d8b51e4d21c8828ba).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class WriteAheadLogBackedBlockRDDPartition(`
      * `class WriteAheadLogBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454502
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    --- End diff --
    
    move this to the previous line?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61056802
  
    @harishreedharan I dont think so. The block location is called only once in both, and the hdfs location is called only once and only if required. I dont think there is any issue in performance. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60470185
  
      [Test build #22189 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22189/consoleFull) for   PR 2931 at commit [`c709f2f`](https://github.com/apache/spark/commit/c709f2f9e44e8b6a2f8224734389e19558e0c3d9).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19374198
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    --- End diff --
    
    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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19506198
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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 = {
    +      if (writeToHDFSCount != 0) {
    +        // Generate some fake segments for the blocks in BM so the RDD does not complain
    +        generateFakeSegments(writeToBMCount) ++
    +          writeDataToHDFS(writtenStrings.slice(writeToBMCount, blockCount),
    +            blockIds.slice(writeToBMCount, blockCount))
    +      } else {
    +        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 Seq of Seqs, each of these Seqs is one block
    +   */
    +  private def generateData(
    +      count: Int,
    +      countPerBlock: Int
    +    ): Seq[Seq[String]] = {
    +    val strings = (0 until count).map { _ => scala.util.Random.nextString(50)}
    --- End diff --
    
    refactored.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60659980
  
      [Test build #22302 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22302/consoleFull) for   PR 2931 at commit [`6e1bfb8`](https://github.com/apache/spark/commit/6e1bfb8dae4eb829c59dd6baf423b93bd4d2971f).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454664
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    (0 until blockIds.size).map { i =>
    --- End diff --
    
    Array.tabulate


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19456819
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    --- End diff --
    
    Should definitely be private, @harishreedharan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60668710
  
      [Test build #22306 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22306/consoleFull) for   PR 2931 at commit [`6e1bfb8`](https://github.com/apache/spark/commit/6e1bfb8dae4eb829c59dd6baf423b93bd4d2971f).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61071886
  
    @rxin, crap, i missed that. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19634527
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,125 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment)
    +  extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient blockIds: Array[BlockId],
    +    @transient segments: Array[WriteAheadLogFileSegment],
    +    storeInBlockManager: Boolean,
    +    storageLevel: StorageLevel)
    +  extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i))
    +    }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    def blockLocations = getBlockIdLocations().get(partition.blockId)
    +    def segmentLocations = HdfsUtils.getFileSegmentLocations(
    +      partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig)
    +    blockLocations.orElse(segmentLocations).getOrElse(Seq.empty)
    --- End diff --
    
    actually this discussion is moot because we should just let getFileSegmentLocations return Seq[String] rather than Option[Seq[String]], and then this should only consist of two branches, accomplishable with a single getOrElse.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19587321
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i)) }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockLocations = getBlockIdLocations().get(partition.blockId)
    +    lazy val segmentLocations = HdfsUtils.getFileSegmentLocations(
    --- End diff --
    
    What about using `def` instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61048438
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22521/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19490482
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  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 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,
    --- End diff --
    
    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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19495201
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    --- End diff --
    
    Done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by shaneknapp <gi...@git.apache.org>.
Github user shaneknapp commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60664941
  
    jenkins, test this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19373073
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    +  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
    +   * BlockManager, so all reads need not happen from HDFS.
    +   * @param total - Total number of Strings to write
    --- End diff --
    
    You don't need a dash here, at least according to the style of the rest of the scaladocs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61167522
  
    Alright! I think we have converged to the best solution here. I am going to wait for the tests to pass and then converge. Thanks @rxin and @JoshRosen for all the feedback!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19587457
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    --- End diff --
    
    For that matter, the `val` in the following lines were not needed either.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454823
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    --- End diff --
    
    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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61167138
  
      [Test build #22562 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22562/consoleFull) for   PR 2931 at commit [`209e49c`](https://github.com/apache/spark/commit/209e49cf7b7467f5ba10f9352745823104f6a332).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60664704
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22302/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60860158
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22398/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61073837
  
      [Test build #22537 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22537/consoleFull) for   PR 2931 at commit [`4a5866f`](https://github.com/apache/spark/commit/4a5866fb19fcd4fde7551294a30bd740500c834a).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19484412
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    --- End diff --
    
    Based on a cursory glance, it looks like this is testing that writing a HDFS-backed BlockRDD then reading it returns the same contents as the original RDD. Maybe add a comment to this effect?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60845224
  
    +1. The changes look good (apart from the style comments that Lord Jenkins made above)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19573363
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    --- End diff --
    
    and here
    ```scala
    Array.tabulate(blockIds.size) { i =>
      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i))
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] WriteAheadLogBackedBlo...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61178055
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22562/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19491638
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  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 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,
    --- End diff --
    
    > 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`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19573690
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,155 @@
    +/*
    + * 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
    +
    +import scala.util.Random
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +import org.scalatest.{BeforeAndAfterAll, FunSuite}
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var dir: File = null
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests.
    +    sparkContext.stop()
    +    dir.delete()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Read data available in block manager and write ahead log") {
    +    testRDD(5, 5)
    +  }
    +
    +  test("Read data available only in block manager, not in write ahead log") {
    +    testRDD(5, 0)
    +  }
    +
    +  test("Read data available only in write ahead log, not in block manager") {
    +    testRDD(0, 5)
    +  }
    +
    +  test("Read data available only in write ahead log, and test storing in block manager") {
    +    testRDD(0, 5, testStoreInBM = true)
    +  }
    +
    +  test("Read data with partially available in block manager, and rest in write ahead log") {
    +    testRDD(3, 2)
    +  }
    +
    +  /**
    +   * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager
    +   * and the rest to a write ahead log, and then reading reading it all back using the RDD.
    +   * It can also test if the partitions that were read from the log were again stored in
    +   * block manager.
    +   * @param numPartitionssInBM Number of partitions to write to the Block Manager
    +   * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log
    +   * @param testStoreInBM Test whether blocks read from log are stored back into block manager
    +   */
    +  private def testRDD(
    +      numPartitionssInBM: Int,
    +      numPartitionsInWAL: Int,
    +      testStoreInBM: Boolean = false
    +    ) {
    --- End diff --
    
    move this to the previous line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60983359
  
    This looks good to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60458871
  
      [Test build #420 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/420/consoleFull) for   PR 2931 at commit [`eadde56`](https://github.com/apache/spark/commit/eadde561ed26ec136273eab928b84fedc4c5fd3a).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19373061
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    --- End diff --
    
    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. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19484208
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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 = {
    +      if (writeToHDFSCount != 0) {
    +        // Generate some fake segments for the blocks in BM so the RDD does not complain
    +        generateFakeSegments(writeToBMCount) ++
    +          writeDataToHDFS(writtenStrings.slice(writeToBMCount, blockCount),
    +            blockIds.slice(writeToBMCount, blockCount))
    +      } else {
    +        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 Seq of Seqs, each of these Seqs is one block
    +   */
    +  private def generateData(
    +      count: Int,
    +      countPerBlock: Int
    +    ): Seq[Seq[String]] = {
    +    val strings = (0 until count).map { _ => scala.util.Random.nextString(50)}
    --- End diff --
    
    `.tabulate`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372156
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    +  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
    +   * BlockManager, so all reads need not happen from HDFS.
    +   * @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 doTestHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map {
    +      i =>
    --- End diff --
    
    Style nit: place this `i =>` on the previous line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372181
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    --- End diff --
    
    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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60847243
  
      [Test build #22387 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22387/consoleFull) for   PR 2931 at commit [`29aa099`](https://github.com/apache/spark/commit/29aa0999ac72055bb51eeb6d8b51e4d21c8828ba).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61044847
  
    Apart from the readability, does one have a performance benefit over the other?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19635029
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,125 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment)
    +  extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient blockIds: Array[BlockId],
    +    @transient segments: Array[WriteAheadLogFileSegment],
    +    storeInBlockManager: Boolean,
    +    storageLevel: StorageLevel)
    +  extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i))
    +    }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    def blockLocations = getBlockIdLocations().get(partition.blockId)
    +    def segmentLocations = HdfsUtils.getFileSegmentLocations(
    +      partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig)
    +    blockLocations.orElse(segmentLocations).getOrElse(Seq.empty)
    --- End diff --
    
    Correct. Once we make that change, I think both the getOrElse and the if..else solutions are equivalent - one is a scala way of doing things, and the other is the "traditional" way. The ones using def/lazy val is really a more scala way of doing it.
    
    I have no preference for any one method, but would generally consider the overhead and performance incurred by each and I am not that much of an expert in scala to know. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19374200
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    --- End diff --
    
    I copied the code for stopping and clearing the properties into this test though


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60799300
  
    @harishreedharan I am taking care of these comments to unblock other PRs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372063
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    +    .asInstanceOf[Broadcast[SerializableWritable[Configuration]]]
    +  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 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,
    +      HdfsUtils.getBlockLocations(partition.segment.path, hadoopConfiguration)
    --- End diff --
    
    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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60846631
  
    Dummy me. My IntelliJ is misconfigured to warn lines 120 chars long, not 100. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60837839
  
      [Test build #22380 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22380/consoleFull) for   PR 2931 at commit [`9e47b5b`](https://github.com/apache/spark/commit/9e47b5b74b4d568c00827e1bf2fbe15e24e2cba1).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class WriteAheadLogBackedBlockRDDPartition(`
      * `class WriteAheadLogBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19495119
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    --- End diff --
    
    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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60434330
  
      [Test build #22152 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22152/consoleFull) for   PR 2931 at commit [`eadde56`](https://github.com/apache/spark/commit/eadde561ed26ec136273eab928b84fedc4c5fd3a).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60449104
  
    **[Test build #22152 timed out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22152/consoleFull)**     for PR 2931 at commit [`eadde56`](https://github.com/apache/spark/commit/eadde561ed26ec136273eab928b84fedc4c5fd3a)     after a configured wait of `120m`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454541
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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 = {
    +      if (writeToHDFSCount != 0) {
    +        // Generate some fake segments for the blocks in BM so the RDD does not complain
    +        generateFakeSegments(writeToBMCount) ++
    +          writeDataToHDFS(writtenStrings.slice(writeToBMCount, blockCount),
    +            blockIds.slice(writeToBMCount, blockCount))
    +      } else {
    +        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 Seq of Seqs, each of these Seqs is one block
    +   */
    +  private def generateData(
    +      count: Int,
    +      countPerBlock: Int
    +    ): Seq[Seq[String]] = {
    +    val strings = (0 until count).map { _ => scala.util.Random.nextString(50)}
    +    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) =>
    +        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) }
    --- End diff --
    
    Seq.fill


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19506184
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    --- End diff --
    
    refactored.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454510
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    --- End diff --
    
    indent off


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19373105
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    +  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
    +   * BlockManager, so all reads need not happen from HDFS.
    +   * @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 doTestHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map {
    +      i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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]
    +    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))
    +
    --- End diff --
    
    Extra blank line here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] WriteAheadLogBackedBlo...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61178051
  
      [Test build #22562 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22562/consoleFull) for   PR 2931 at commit [`209e49c`](https://github.com/apache/spark/commit/209e49cf7b7467f5ba10f9352745823104f6a332).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class WriteAheadLogBackedBlockRDDPartition(`
      * `class WriteAheadLogBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60837159
  
    Updated!
    - Renamed the class, 
    - Simplified the unit test code, added one more unit test and added lots of documentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60837845
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22380/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60471234
  
      [Test build #22189 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22189/consoleFull) for   PR 2931 at commit [`c709f2f`](https://github.com/apache/spark/commit/c709f2f9e44e8b6a2f8224734389e19558e0c3d9).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HDFSBackedBlockRDDPartition(`
      * `class HDFSBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60837595
  
      [Test build #22380 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22380/consoleFull) for   PR 2931 at commit [`9e47b5b`](https://github.com/apache/spark/commit/9e47b5b74b4d568c00827e1bf2fbe15e24e2cba1).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19587299
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  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 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,
    --- End diff --
    
    Isnt it something that Josh suggested more intuitive? All the alternatives are clearly in one line. And it does not have redundant code as `case Some(loc) => loc`  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19484474
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    --- End diff --
    
    I think `numItems` might be a clearer variable name


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19587466
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,155 @@
    +/*
    + * 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
    +
    +import scala.util.Random
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +import org.scalatest.{BeforeAndAfterAll, FunSuite}
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var dir: File = null
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests.
    +    sparkContext.stop()
    +    dir.delete()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Read data available in block manager and write ahead log") {
    +    testRDD(5, 5)
    +  }
    +
    +  test("Read data available only in block manager, not in write ahead log") {
    +    testRDD(5, 0)
    +  }
    +
    +  test("Read data available only in write ahead log, not in block manager") {
    +    testRDD(0, 5)
    +  }
    +
    +  test("Read data available only in write ahead log, and test storing in block manager") {
    +    testRDD(0, 5, testStoreInBM = true)
    +  }
    +
    +  test("Read data with partially available in block manager, and rest in write ahead log") {
    +    testRDD(3, 2)
    +  }
    +
    +  /**
    +   * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager
    +   * and the rest to a write ahead log, and then reading reading it all back using the RDD.
    +   * It can also test if the partitions that were read from the log were again stored in
    +   * block manager.
    +   * @param numPartitionssInBM Number of partitions to write to the Block Manager
    +   * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log
    +   * @param testStoreInBM Test whether blocks read from log are stored back into block manager
    +   */
    +  private def testRDD(
    +      numPartitionssInBM: Int,
    +      numPartitionsInWAL: Int,
    +      testStoreInBM: Boolean = false
    +    ) {
    +    val numBlocks = numPartitionssInBM + numPartitionsInWAL
    +    val data = Seq.tabulate(numBlocks) { _ => Seq.fill(10) { scala.util.Random.nextString(50) } }
    --- End diff --
    
    Nice! Right!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454508
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map { i =>
    --- End diff --
    
    Use Array.tabulate instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60433546
  
    @JoshRosen Can you take a look?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19373068
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    +    .asInstanceOf[Broadcast[SerializableWritable[Configuration]]]
    --- End diff --
    
    We don't. I don't even remember why I added it at that time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454808
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    --- End diff --
    
    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. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19499543
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    +
    +  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 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,
    --- End diff --
    
    Let's make this dead simple. There is no need to create two extra functions and call getOrElse.
    
    Just do
    ```scala
    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)
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19428909
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -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
    +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 {
    +  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
    +   * BlockManager, so all reads need not happen from HDFS.
    +   * @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 doTestHDFSBackedRDD(
    +      writeToBMCount: Int,
    +      writeToHDFSCount: Int,
    +      total: Int,
    +      blockCount: Int
    +    ) {
    +    val countPerBlock = total / blockCount
    +    val blockIds = (0 until blockCount).map {
    +      i =>
    +        StreamBlockId(idGenerator.incrementAndGet(), idGenerator.incrementAndGet())
    +    }
    +
    +    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]
    +    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))
    +
    --- End diff --
    
    Removed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19456864
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    --- End diff --
    
    Good point, how about renaming this class to more specific `WriteAheadLogBasedBackedBlockRDD` (kind-a-long).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19506065
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    --- End diff --
    
    Added.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19587361
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(
    +    blockIds.length == segments.length,
    +    s"Number of block ids (${blockIds.length}) must be " +
    +      s"the same as number of segments (${segments.length}})!")
    +
    +  // Hadoop configuration is not serializable, so broadcast it as a serializable.
    +  private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig)
    +
    +  override def getPartitions: Array[Partition] = {
    +    assertValid()
    +    Array.tabulate(blockIds.size) { i =>
    +      new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i)) }
    +  }
    +
    +  /**
    +   * Gets the partition data by getting the corresponding block from the block manager.
    +   * If the block does not exist, then the data is read from the corresponding segment
    +   * in write ahead log files.
    +   */
    +  override def compute(split: Partition, context: TaskContext): Iterator[T] = {
    +    assertValid()
    +    val hadoopConf = broadcastedHadoopConf.value
    +    val blockManager = SparkEnv.get.blockManager
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockId = partition.blockId
    +    blockManager.get(blockId) match {
    +      case Some(block) => // Data is in Block Manager
    +        val iterator = block.data.asInstanceOf[Iterator[T]]
    +        logDebug(s"Read partition data of $this from block manager, block $blockId")
    +        iterator
    +      case None => // Data not found in Block Manager, grab it from write ahead log file
    +        val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf)
    +        val dataRead = reader.read(partition.segment)
    +        reader.close()
    +        logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}")
    +        if (storeInBlockManager) {
    +          blockManager.putBytes(blockId, dataRead, storageLevel)
    +          logDebug(s"Stored partition data of $this into block manager with level $storageLevel")
    +          dataRead.rewind()
    +        }
    +        blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]]
    +    }
    +  }
    +
    +  /**
    +   * Get the preferred location of the partition. This returns the locations of the block
    +   * if it is present in the block manager, else it returns the location of the
    +   * corresponding segment in HDFS.
    +   */
    +  override def getPreferredLocations(split: Partition): Seq[String] = {
    +    val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition]
    +    val blockLocations = getBlockIdLocations().get(partition.blockId)
    +    lazy val segmentLocations = HdfsUtils.getFileSegmentLocations(
    --- End diff --
    
    Alright, I agree about the lazy semantics. But then @JoshRosen  suggestion is more intuitive to me.  Something like 
    ```
    def blockLocations = getBlockIdLocations().get(partition.blockId)
    def segmentLocations = HdfsUtils.getBlockLocations(...)
    blockLocations.orElse(segmentLocations).getOrElse(Array.empty)
    ```
    
    All the logical alternatives are clearly in one line. And it does not have verbose, almost redundant code as `case Some(loc) => loc`  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60649863
  
      [Test build #22300 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22300/consoleFull) for   PR 2931 at commit [`9c86a61`](https://github.com/apache/spark/commit/9c86a615a97d5182103bc18695329d8de570715f).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HDFSBackedBlockRDDPartition(`
      * `class HDFSBackedBlockRDD[T: ClassTag](`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454635
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    --- End diff --
    
    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?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60454440
  
      [Test build #420 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/420/consoleFull) for   PR 2931 at commit [`eadde56`](https://github.com/apache/spark/commit/eadde561ed26ec136273eab928b84fedc4c5fd3a).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19574104
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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._
    +import org.apache.spark.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader}
    +
    +/**
    + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]].
    + * It contains information about the id of the blocks having this partition's data and
    + * the segment of the write ahead log that backs the partition.
    + * @param index index of the partition
    + * @param blockId id of the block having the partition data
    + * @param segment segment of the write ahead log having the partition data
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDDPartition(
    +    val index: Int,
    +    val blockId: BlockId,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +
    +/**
    + * This class represents a special case of the BlockRDD where the data blocks in
    + * the block manager are also backed by segments in write ahead logs. For reading
    + * the data, this RDD first looks up the blocks by their ids in the block manager.
    + * If it does not find them, it looks up the corresponding file segment.
    + *
    + * @param sc SparkContext
    + * @param hadoopConfig Hadoop configuration
    + * @param blockIds Ids of the blocks that contains this RDD's data
    + * @param segments Segments in write ahead logs that contain this RDD's data
    + * @param storeInBlockManager Whether to store in the block manager after reading from the segment
    + * @param storageLevel storage level to store when storing in block manager
    + *                     (applicable when storeInBlockManager = true)
    + */
    +private[streaming]
    +class WriteAheadLogBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfig: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    --- End diff --
    
    as i pointed out earlier, you don't need ```@transient override val" here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-61044841
  
      [Test build #22521 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22521/consoleFull) for   PR 2931 at commit [`ed5fbf0`](https://github.com/apache/spark/commit/ed5fbf0765136da963f6a8447f1ff69191825392).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by tdas <gi...@git.apache.org>.
Github user tdas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19506160
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    --- End diff --
    
    Changed to `WriteAheadLogBasedBackedBlockRDD`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454523
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    --- End diff --
    
    you should explain more clearly what this test function does.  it is long enough that it is no longer obvious, and your comment doesn't really address that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60637477
  
      [Test build #22300 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/22300/consoleFull) for   PR 2931 at commit [`9c86a61`](https://github.com/apache/spark/commit/9c86a615a97d5182103bc18695329d8de570715f).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19573756
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,155 @@
    +/*
    + * 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
    +
    +import scala.util.Random
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +import org.scalatest.{BeforeAndAfterAll, FunSuite}
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var dir: File = null
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests.
    +    sparkContext.stop()
    +    dir.delete()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Read data available in block manager and write ahead log") {
    +    testRDD(5, 5)
    +  }
    +
    +  test("Read data available only in block manager, not in write ahead log") {
    +    testRDD(5, 0)
    +  }
    +
    +  test("Read data available only in write ahead log, not in block manager") {
    +    testRDD(0, 5)
    +  }
    +
    +  test("Read data available only in write ahead log, and test storing in block manager") {
    +    testRDD(0, 5, testStoreInBM = true)
    +  }
    +
    +  test("Read data with partially available in block manager, and rest in write ahead log") {
    +    testRDD(3, 2)
    +  }
    +
    +  /**
    +   * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager
    +   * and the rest to a write ahead log, and then reading reading it all back using the RDD.
    +   * It can also test if the partitions that were read from the log were again stored in
    +   * block manager.
    +   * @param numPartitionssInBM Number of partitions to write to the Block Manager
    +   * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log
    +   * @param testStoreInBM Test whether blocks read from log are stored back into block manager
    +   */
    +  private def testRDD(
    +      numPartitionssInBM: Int,
    +      numPartitionsInWAL: Int,
    +      testStoreInBM: Boolean = false
    +    ) {
    +    val numBlocks = numPartitionssInBM + numPartitionsInWAL
    +    val data = Seq.tabulate(numBlocks) { _ => Seq.fill(10) { scala.util.Random.nextString(50) } }
    --- End diff --
    
    it seems like you just want
    ```scala
    Seq.fill(numBlocks, 10)(scala.util.Random.nextString(50))
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60678834
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22306/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19373052
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    +    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))
    +    .asInstanceOf[Broadcast[SerializableWritable[Configuration]]]
    +  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 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,
    +      HdfsUtils.getBlockLocations(partition.segment.path, hadoopConfiguration)
    --- End diff --
    
    Fixed this one in the PR sent to your repo.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454844
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    --- End diff --
    
    import out of order


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454662
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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.rdd.BlockRDD
    +import org.apache.spark.storage.{BlockId, StorageLevel}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, HdfsUtils, WriteAheadLogRandomReader}
    +import org.apache.spark._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +    val blockId: BlockId,
    +    val index: Int,
    +    val segment: WriteAheadLogFileSegment
    +  ) extends Partition
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  require(blockIds.length == segments.length,
    +    "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))
    --- End diff --
    
    can this be private[this]?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19454568
  
    --- Diff: streaming/src/test/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDDSuite.scala ---
    @@ -0,0 +1,163 @@
    +/*
    + * 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
    +import java.util.concurrent.atomic.AtomicInteger
    +
    +import org.apache.spark.{SparkConf, SparkContext}
    +
    +import scala.collection.mutable.ArrayBuffer
    +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FunSuite}
    +
    +import com.google.common.io.Files
    +import org.apache.hadoop.conf.Configuration
    +
    +import org.apache.spark.storage.{BlockManager, BlockId, StorageLevel, StreamBlockId}
    +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter}
    +
    +class HDFSBackedBlockRDDSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll {
    +  val conf = new SparkConf()
    +    .setMaster("local[2]")
    +    .setAppName(this.getClass.getSimpleName)
    +  val hadoopConf = new Configuration()
    +  // Since the same BM is reused in all tests, use an atomic int to generate ids
    +  val idGenerator = new AtomicInteger(0)
    +  
    +  var sparkContext: SparkContext = null
    +  var blockManager: BlockManager = null
    +  var file: File = null
    +  var dir: File = null
    +
    +  before {
    +    blockManager = sparkContext.env.blockManager
    +    dir = Files.createTempDir()
    +    file = new File(dir, "BlockManagerWrite")
    +  }
    +
    +  after {
    +    file.delete()
    +    dir.delete()
    +  }
    +
    +  override def beforeAll(): Unit = {
    +    sparkContext = new SparkContext(conf)
    +  }
    +
    +  override def afterAll(): Unit = {
    +    // Copied from LocalSparkContext which can't be imported since spark-core test-jar does not
    +    // get imported properly by sbt even if it is created.
    +    sparkContext.stop()
    +    System.clearProperty("spark.driver.port")
    +  }
    +
    +  test("Data available in BM and HDFS") {
    +    testHDFSBackedRDD(5, 5, 20, 5)
    +  }
    +
    +  test("Data available in in BM but not in HDFS") {
    +    testHDFSBackedRDD(5, 0, 20, 5)
    +  }
    +
    +  test("Data available in in HDFS and not in BM") {
    +    testHDFSBackedRDD(0, 5, 20, 5)
    +  }
    +
    +  test("Data partially available in BM, and the rest in HDFS") {
    +    testHDFSBackedRDD(3, 2, 20, 5)
    +  }
    +
    +  /**
    +   * 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 total Total number of Strings to write
    +   * @param blockCount Number of blocks to write (therefore, total # of events per block =
    +   *                   total/blockCount
    +   */
    +  private def testHDFSBackedRDD(
    --- End diff --
    
    basically i'm worried the test case is too complicated for others to understand without enough comment, and it will be modified incorrectly in the future.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/2931#discussion_r19372192
  
    --- Diff: streaming/src/main/scala/org/apache/spark/streaming/rdd/HDFSBackedBlockRDD.scala ---
    @@ -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._
    +
    +private[streaming]
    +class HDFSBackedBlockRDDPartition(
    +  val blockId: BlockId, idx: Int, val segment: WriteAheadLogFileSegment) extends Partition {
    +  val index = idx
    +}
    +
    +private[streaming]
    +class HDFSBackedBlockRDD[T: ClassTag](
    +    @transient sc: SparkContext,
    +    @transient hadoopConfiguration: Configuration,
    +    @transient override val blockIds: Array[BlockId],
    +    @transient val segments: Array[WriteAheadLogFileSegment],
    +    val storeInBlockManager: Boolean,
    +    val storageLevel: StorageLevel
    +  ) extends BlockRDD[T](sc, blockIds) {
    +
    +  if (blockIds.length != segments.length) {
    --- End diff --
    
    I usually like `require` for stuff like this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-4027][Streaming] HDFSBasedBlockRDD to r...

Posted by harishreedharan <gi...@git.apache.org>.
Github user harishreedharan commented on the pull request:

    https://github.com/apache/spark/pull/2931#issuecomment-60837781
  
    OK, I wasn't reading properly - still drinking coffee. Ignore my last comment please!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org