Github user harishreedharan commented on a diff in the pull request:
https://github.com/apache/spark/pull/2940#discussion_r19514744
--- Diff:
streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala
---
@@ -0,0 +1,173 @@
+/*
+ * 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.receiver
+
+import java.nio.ByteBuffer
+
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.{Await, ExecutionContext, Future}
+import scala.concurrent.duration._
+import scala.language.{existentials, postfixOps}
+
+import WriteAheadLogBasedBlockHandler._
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.{Logging, SparkConf, SparkException}
+import org.apache.spark.storage._
+import org.apache.spark.streaming.util.{Clock, SystemClock,
WriteAheadLogManager}
+import org.apache.spark.util.Utils
+
+private[streaming] sealed trait ReceivedBlock
+private[streaming] case class ArrayBufferBlock(arrayBuffer:
ArrayBuffer[_]) extends ReceivedBlock
+private[streaming] case class IteratorBlock(iterator: Iterator[_]) extends
ReceivedBlock
+private[streaming] case class ByteBufferBlock(byteBuffer: ByteBuffer)
extends ReceivedBlock
+
+
+/** Trait that represents a class that handles the storage of blocks
received by receiver */
+private[streaming] trait ReceivedBlockHandler {
+
+ /** Store a received block with the given block id */
+ def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock):
Option[Any]
+
+ /** Cleanup old blocks older than the given threshold time */
+ def cleanupOldBlock(threshTime: Long)
+}
+
+/**
+ * Implementation of a
[[org.apache.spark.streaming.receiver.ReceivedBlockHandler]] which
+ * stores the received blocks into a block manager with the specified
storage level.
+ */
+private[streaming] class BlockManagerBasedBlockHandler(
+ blockManager: BlockManager, storageLevel: StorageLevel)
+ extends ReceivedBlockHandler with Logging {
+
+ def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock):
Option[Any] = {
+ val putResult: Seq[(BlockId, BlockStatus)] = receivedBlock match {
+ case ArrayBufferBlock(arrayBuffer) =>
+ blockManager.putIterator(blockId, arrayBuffer.iterator,
storageLevel, tellMaster = true)
+ case IteratorBlock(iterator) =>
+ blockManager.putIterator(blockId, iterator, storageLevel,
tellMaster = true)
+ case ByteBufferBlock(byteBuffer) =>
+ blockManager.putBytes(blockId, byteBuffer, storageLevel,
tellMaster = true)
+ case o =>
+ throw new SparkException(
+ s"Could not store $blockId to block manager, unexpected block
type ${o.getClass.getName}")
+ }
+ if (!putResult.map { _._1 }.contains(blockId)) {
+ throw new SparkException(
+ s"Could not store $blockId to block manager with storage level
$storageLevel")
+ }
+ None
+ }
+
+ def cleanupOldBlock(threshTime: Long) {
+ // this is not used as blocks inserted into the BlockManager are
cleared by DStream's clearing
+ // of BlockRDDs.
+ }
+}
+
+/**
+ * Implementation of a
[[org.apache.spark.streaming.receiver.ReceivedBlockHandler]] which
+ * stores the received blocks in both, a write ahead log and a block
manager.
+ */
+private[streaming] class WriteAheadLogBasedBlockHandler(
+ blockManager: BlockManager,
+ streamId: Int,
+ storageLevel: StorageLevel,
+ conf: SparkConf,
+ hadoopConf: Configuration,
+ checkpointDir: String,
+ clock: Clock = new SystemClock
+ ) extends ReceivedBlockHandler with Logging {
+
+ private val blockStoreTimeout = conf.getInt(
+ "spark.streaming.receiver.blockStoreTimeout", 30).seconds
+ private val rollingInterval = conf.getInt(
+ "spark.streaming.receiver.writeAheadLog.rollingInterval", 60)
+ private val maxFailures = conf.getInt(
+ "spark.streaming.receiver.writeAheadLog.maxFailures", 3)
+
+ // Manages rolling log files
+ private val logManager = new WriteAheadLogManager(
+ checkpointDirToLogDir(checkpointDir, streamId),
+ hadoopConf, rollingInterval, maxFailures,
+ callerName = this.getClass.getSimpleName,
+ clock = clock
+ )
+
+ // For processing futures used in parallel block storing into block
manager and write ahead log
+ implicit private val executionContext =
ExecutionContext.fromExecutorService(
+ Utils.newDaemonFixedThreadPool(2, this.getClass.getSimpleName))
--- End diff --
As I mentioned earlier, this might actually end up being a bottle neck.
Since you could write using multiple threads in the same receiver - we are
basically blocking more than one write from happening at any point in time.
Since the BlockManager can handle more writes in parallel, we should probably
use a much higher value than 2.
That said, the WAL Writer would still be a bottle neck - since the writes
to the WAL have to be synchronized. So I am not entirely sure if having more
than 2 threads helps a whole lot.
---
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 [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]