From 6d6776a45f30e3594a15bda2582f99819c28a583 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 8 May 2014 23:16:56 -0700 Subject: [PATCH 01/32] SPARK-1729. Make Flume pull data from source, rather than the current push model Currently Spark uses Flume's internal Avro Protocol to ingest data from Flume. If the executor running the receiver fails, it currently has to be restarted on the same node to be able to receive data. This commit adds a new Sink which can be deployed to a Flume agent. This sink can be polled by a new DStream that is also included in this commit. This model ensures that data can be pulled into Spark from Flume even if the receiver is restarted on a new node. This also allows the receiver to receive data on multiple threads for better performance. --- external/flume-sink/pom.xml | 82 ++++ .../flume-sink/src/main/avro/sparkflume.avdl | 40 ++ .../apache/spark/flume/sink/SparkSink.scala | 365 ++++++++++++++++++ external/flume/pom.xml | 4 + .../streaming/flume/EventTransformer.scala | 70 ++++ .../streaming/flume/FlumeInputDStream.scala | 38 +- .../flume/FlumePollingInputDStream.scala | 140 +++++++ .../spark/streaming/flume/FlumeUtils.scala | 52 ++- .../flume/FlumePollingReceiverSuite.scala | 104 +++++ pom.xml | 1 + project/SparkBuild.scala | 26 +- project/plugins.sbt | 4 + 12 files changed, 885 insertions(+), 41 deletions(-) create mode 100644 external/flume-sink/pom.xml create mode 100644 external/flume-sink/src/main/avro/sparkflume.avdl create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala create mode 100644 external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml new file mode 100644 index 0000000000000..40d7e09edaf16 --- /dev/null +++ b/external/flume-sink/pom.xml @@ -0,0 +1,82 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-flume-sink_2.10 + jar + Spark Project External Flume Sink + http://spark.apache.org/ + + + org.apache.flume + flume-ng-sdk + 1.4.0 + + + org.jboss.netty + netty + + + org.apache.thrift + libthrift + + + + + org.apache.flume + flume-ng-core + 1.4.0 + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + org.apache.avro + avro-maven-plugin + 1.7.3 + + String + + ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro + + + + generate-sources + + idl-protocol + + + + + + + diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl new file mode 100644 index 0000000000000..9dcc709de079a --- /dev/null +++ b/external/flume-sink/src/main/avro/sparkflume.avdl @@ -0,0 +1,40 @@ +/** + * 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. + */ + +@namespace("org.apache.spark.flume") + +protocol SparkFlumeProtocol { + + record SparkSinkEvent { + map headers; + bytes body; + } + + record EventBatch { + string sequenceNumber; + array eventBatch; + } + + EventBatch getEventBatch (int n); + + void ack (string sequenceNumber); + + void nack (string sequenceNumber); + +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala new file mode 100644 index 0000000000000..6243463a475b6 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -0,0 +1,365 @@ +/* + * 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.flume.sink + +import org.apache.flume.sink.AbstractSink +import java.util.concurrent.locks.ReentrantLock +import org.apache.flume.Sink.Status +import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} +import scala.util.control.Breaks +import java.nio.ByteBuffer +import org.apache.flume.{FlumeException, Context} +import org.slf4j.LoggerFactory +import java.util.concurrent.atomic.AtomicLong +import org.apache.commons.lang.RandomStringUtils +import java.util.concurrent._ +import java.util +import org.apache.flume.conf.{ConfigurationException, Configurable} +import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.avro.ipc.{NettyTransceiver, NettyServer} +import org.apache.avro.ipc.specific.SpecificResponder +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory +import java.net.InetSocketAddress + +class SparkSink() extends AbstractSink with Configurable { + private val LOG = LoggerFactory.getLogger(this.getClass) + private val lock = new ReentrantLock() + private val blockingCondition = lock.newCondition() + + // This sink will not persist sequence numbers and reuses them if it gets restarted. + // So it is possible to commit a transaction which may have been meant for the sink before the + // restart. + // Since the new txn may not have the same sequence number we must guard against accidentally + // committing + // a new transaction. To reduce the probability of that happening a random string is prepended + // to the sequence number. + // Does not change for life of sink + private val seqBase = RandomStringUtils.randomAlphanumeric(8) + // Incremented for each transaction + private val seqNum = new AtomicLong(0) + + private var transactionExecutorOpt: Option[ExecutorService] = None + + private var numProcessors: Integer = SparkSinkConfig.DEFAULT_PROCESSOR_COUNT + private var transactionTimeout = SparkSinkConfig.DEFAULT_TRANSACTION_TIMEOUT + + private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() + + private var processorFactory: Option[SparkHandlerFactory] = None + private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME + private var port: Int = 0 + private var maxThreads: Int = SparkSinkConfig.DEFAULT_MAX_THREADS + private var serverOpt: Option[NettyServer] = None + private var running = false + + override def start() { + transactionExecutorOpt = Option(Executors.newFixedThreadPool(numProcessors, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("Spark Sink, " + getName + " Processor Thread - %d").build())) + + processorFactory = Option(new SparkHandlerFactory(numProcessors)) + + val responder = new SpecificResponder(classOf[SparkFlumeProtocol], new AvroCallbackHandler()) + + serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port), + new NioServerSocketChannelFactory( + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat( + "Spark Sink " + classOf[NettyTransceiver].getSimpleName + " Boss-%d").build), + Executors.newFixedThreadPool(maxThreads, new ThreadFactoryBuilder().setNameFormat( + "Spark Sink " + classOf[NettyTransceiver].getSimpleName + " I/O Worker-%d").build)))) + + serverOpt.map(server => server.start()) + lock.lock() + try { + running = true + } finally { + lock.unlock() + } + super.start() + } + + override def stop() { + lock.lock() + try { + running = false + transactionExecutorOpt.map(executor => executor.shutdownNow()) + blockingCondition.signalAll() + } finally { + lock.unlock() + } + } + + override def configure(ctx: Context) { + import SparkSinkConfig._ + hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) + val portOpt = Option(ctx.getInteger(CONF_PORT)) + if(portOpt.isDefined) { + port = portOpt.get + } else { + throw new ConfigurationException("The Port to bind must be specified") + } + numProcessors = ctx.getInteger(PROCESSOR_COUNT, DEFAULT_PROCESSOR_COUNT) + transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) + maxThreads = ctx.getInteger(CONF_MAX_THREADS, DEFAULT_MAX_THREADS) + } + + override def process(): Status = { + // This method is called in a loop by the Flume framework - block it until the sink is + // stopped to save CPU resources + lock.lock() + try { + while(running) { + blockingCondition.await() + } + } finally { + lock.unlock() + } + Status.BACKOFF + } + + private class AvroCallbackHandler() extends SparkFlumeProtocol { + + override def getEventBatch(n: Int): EventBatch = { + val processor = processorFactory.get.checkOut(n) + transactionExecutorOpt.map(executor => executor.submit(processor)) + // Wait until a batch is available - can be null if some error was thrown + val eventBatch = Option(processor.eventQueue.take()) + if (eventBatch.isDefined) { + val eventsToBeSent = eventBatch.get + processorMap.put(eventsToBeSent.getSequenceNumber, processor) + if (LOG.isDebugEnabled) { + LOG.debug("Sent " + eventsToBeSent.getEventBatch.size() + + " events with sequence number: " + eventsToBeSent.getSequenceNumber) + } + eventsToBeSent + } else { + throw new FlumeException("Error while trying to retrieve events from the channel.") + } + } + + override def ack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = true) + null + } + + override def nack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = false) + LOG.info("Spark failed to commit transaction. Will reattempt events.") + null + } + + def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { + val processorOpt = Option(processorMap.remove(sequenceNumber)) + if (processorOpt.isDefined) { + val processor = processorOpt.get + processor.resultQueueUpdateLock.lock() + try { + // Is the sequence number the same as the one the processor is processing? If not, + // don't update { + if (processor.eventBatch.getSequenceNumber.equals(sequenceNumber)) { + processor.resultQueue.put(success) + } + } finally { + processor.resultQueueUpdateLock.unlock() + } + } + } + } + + // Flume forces transactions to be thread-local (horrible, I know!) + // So the sink basically spawns a new thread to pull the events out within a transaction. + // The thread fills in the event batch object that is set before the thread is scheduled. + // After filling it in, the thread waits on a condition - which is released only + // when the success message comes back for the specific sequence number for that event batch. + /** + * This class represents a transaction on the Flume channel. This class runs a separate thread + * which owns the transaction. It is blocked until the success call for that transaction comes + * back. + * @param maxBatchSize + */ + private class TransactionProcessor(var maxBatchSize: Int) extends Callable[Void] { + // Must be set to a new event batch before scheduling this!! + val eventBatch = new EventBatch("", new util.LinkedList[SparkSinkEvent]) + val eventQueue = new SynchronousQueue[EventBatch]() + val resultQueue = new SynchronousQueue[Boolean]() + val resultQueueUpdateLock = new ReentrantLock() + + object Zero { + val zero = "0" // Oh, I miss static finals + } + + + override def call(): Void = { + val tx = getChannel.getTransaction + tx.begin() + try { + eventBatch.setSequenceNumber(seqBase + seqNum.incrementAndGet()) + val events = eventBatch.getEventBatch + events.clear() + val loop = new Breaks + loop.breakable { + for (i <- 0 until maxBatchSize) { + val eventOpt = Option(getChannel.take()) + + eventOpt.map(event => { + events.add(new SparkSinkEvent(toCharSequenceMap(event + .getHeaders), + ByteBuffer.wrap(event.getBody))) + }) + if (eventOpt.isEmpty) { + loop.break() + } + } + } + // Make the data available to the sender thread + eventQueue.put(eventBatch) + + // Wait till timeout for the ack/nack + val maybeResult = Option(resultQueue.poll(transactionTimeout, TimeUnit.SECONDS)) + // There is a race condition here. + // 1. This times out. + // 2. The result is empty, so timeout exception is thrown. + // 3. The ack comes in before the finally block is entered + // 4. The thread with the ack has a handle to this processor, + // and another thread has the same processor checked out + // (since the finally block was executed and the processor checked back in) + // 5. The thread with the ack now updates the result queue, + // so the processor thinks it is the ack for the current batch. + // To avoid this - update the sequence number to "0" (with or without a result - does not + // matter). + // In the ack method, check if the seq number is the same as the processor's - + // if they are then update the result queue. Now if the + // processor updates the seq number first - the ack/nack never updates the result. If the + // ack/nack updates the + // result after the timeout but before the seq number is updated to "0" it does not + // matter - the processor would + // still timeout and the result is cleared before reusing the processor. + // Unfortunately, this needs to be done from within a lock + // to make sure that the new sequence number is actually visible to the ack thread + // (happens-before) + resultQueueUpdateLock.lock() + try { + eventBatch.setSequenceNumber(Zero.zero) + } finally { + resultQueueUpdateLock.unlock() + } + eventBatch.getEventBatch.clear() + // If the batch failed on spark side, throw a FlumeException + maybeResult.map(success => + if (!success) { + throw new + FlumeException("Spark could not accept events. The transaction will be retried.") + } + ) + // If the operation timed out, throw a TimeoutException + if (maybeResult.isEmpty) { + throw new TimeoutException("Spark did not respond within the timeout period of " + + transactionTimeout + "seconds. Transaction will be retried") + } + null + } catch { + case e: Throwable => + try { + LOG.warn("Error while attempting to remove events from the channel.", e) + tx.rollback() + } catch { + case e1: Throwable => LOG.error( + "Rollback failed while attempting to rollback due to commit failure.", e1) + } + null // No point rethrowing the exception + } finally { + // Must *always* release the caller thread + eventQueue.put(null) + // In the case of success coming after the timeout, but before resetting the seq number + // remove the event from the map and then clear the value + resultQueue.clear() + processorMap.remove(eventBatch.getSequenceNumber) + processorFactory.get.checkIn(this) + tx.close() + } + } + + def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, + CharSequence] = { + val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) + charSeqMap.putAll(inMap) + charSeqMap + } + } + + private class SparkHandlerFactory(val maxInstances: Int) { + val queue = new scala.collection.mutable.Queue[TransactionProcessor] + val queueModificationLock = new ReentrantLock() + var currentSize = 0 + val waitForCheckIn = queueModificationLock.newCondition() + + def checkOut(n: Int): TransactionProcessor = { + def getProcessor = { + val processor = queue.dequeue() + processor.maxBatchSize = n + processor + } + queueModificationLock.lock() + try { + if (queue.size > 0) { + getProcessor + } + else { + if (currentSize < maxInstances) { + currentSize += 1 + new TransactionProcessor(n) + } else { + // No events in queue and cannot initialize more! + // Since currentSize never reduces, queue size increasing is the only hope + while (queue.size == 0 && currentSize >= maxInstances) { + waitForCheckIn.await() + } + getProcessor + } + } + } finally { + queueModificationLock.unlock() + } + } + + def checkIn(processor: TransactionProcessor) { + queueModificationLock.lock() + try { + queue.enqueue(processor) + waitForCheckIn.signal() + } finally { + queueModificationLock.unlock() + } + } + } +} + +object SparkSinkConfig { + val PROCESSOR_COUNT = "processorCount" + val DEFAULT_PROCESSOR_COUNT = 10 + + val CONF_TRANSACTION_TIMEOUT = "timeout" + val DEFAULT_TRANSACTION_TIMEOUT = 60 + + val CONF_HOSTNAME = "hostname" + val DEFAULT_HOSTNAME = "0.0.0.0" + + val CONF_PORT = "port" + + val CONF_MAX_THREADS = "maxThreads" + val DEFAULT_MAX_THREADS = 5 +} diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 6aec215687fe0..93d8ec02ac69e 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -74,6 +74,10 @@ junit-interface test + + org.apache.spark + spark-streaming-flume-sink_2.10 + target/scala-${scala.binary.version}/classes diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala new file mode 100644 index 0000000000000..91f6171d57368 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -0,0 +1,70 @@ +/* + * 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.flume + +import java.io.{ObjectOutput, ObjectInput} +import org.apache.spark.util.Utils +import scala.collection.JavaConversions._ +import org.apache.spark.Logging + +/** + * A simple object that provides the implementation of readExternal and writeExternal for both + * the wrapper classes for Flume-style Events. + */ +object EventTransformer extends Logging { + def readExternal(in: ObjectInput): (java.util.HashMap[CharSequence, CharSequence], + Array[Byte]) = { + val bodyLength = in.readInt() + val bodyBuff = new Array[Byte](bodyLength) + in.read(bodyBuff) + + val numHeaders = in.readInt() + val headers = new java.util.HashMap[CharSequence, CharSequence] + + for (i <- 0 until numHeaders) { + val keyLength = in.readInt() + val keyBuff = new Array[Byte](keyLength) + in.read(keyBuff) + val key: String = Utils.deserialize(keyBuff) + + val valLength = in.readInt() + val valBuff = new Array[Byte](valLength) + in.read(valBuff) + val value: String = Utils.deserialize(valBuff) + + headers.put(key, value) + } + (headers, bodyBuff) + } + + def writeExternal(out: ObjectOutput, headers: java.util.Map[CharSequence, CharSequence], + body: Array[Byte]) { + out.writeInt(body.length) + out.write(body) + val numHeaders = headers.size() + out.writeInt(numHeaders) + for ((k,v) <- headers) { + val keyBuff = Utils.serialize(k.toString) + out.writeInt(keyBuff.length) + out.write(keyBuff) + val valBuff = Utils.serialize(v.toString) + out.writeInt(valBuff.length) + out.write(valBuff) + } + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index df7605fe579f8..78715226ab402 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -30,7 +30,6 @@ import org.apache.flume.source.avro.Status import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer -import org.apache.spark.util.Utils import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ @@ -61,47 +60,14 @@ class SparkFlumeEvent() extends Externalizable { /* De-serialize from bytes. */ def readExternal(in: ObjectInput) { - val bodyLength = in.readInt() - val bodyBuff = new Array[Byte](bodyLength) - in.read(bodyBuff) - - val numHeaders = in.readInt() - val headers = new java.util.HashMap[CharSequence, CharSequence] - - for (i <- 0 until numHeaders) { - val keyLength = in.readInt() - val keyBuff = new Array[Byte](keyLength) - in.read(keyBuff) - val key : String = Utils.deserialize(keyBuff) - - val valLength = in.readInt() - val valBuff = new Array[Byte](valLength) - in.read(valBuff) - val value : String = Utils.deserialize(valBuff) - - headers.put(key, value) - } - + val (headers, bodyBuff) = EventTransformer.readExternal(in) event.setBody(ByteBuffer.wrap(bodyBuff)) event.setHeaders(headers) } /* Serialize to bytes. */ def writeExternal(out: ObjectOutput) { - val body = event.getBody.array() - out.writeInt(body.length) - out.write(body) - - val numHeaders = event.getHeaders.size() - out.writeInt(numHeaders) - for ((k, v) <- event.getHeaders) { - val keyBuff = Utils.serialize(k.toString) - out.writeInt(keyBuff.length) - out.write(keyBuff) - val valBuff = Utils.serialize(v.toString) - out.writeInt(valBuff.length) - out.write(valBuff) - } + EventTransformer.writeExternal(out, event.getHeaders, event.getBody.array()) } } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala new file mode 100644 index 0000000000000..71b0f72f85f53 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -0,0 +1,140 @@ +/* + * 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.flume + +import scala.reflect.ClassTag +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.Logging +import java.net.InetSocketAddress +import java.util.concurrent.{TimeUnit, Executors} +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.spark.flume.{SparkSinkEvent, SparkFlumeProtocol} +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import com.google.common.util.concurrent.ThreadFactoryBuilder +import java.io.{ObjectOutput, ObjectInput, Externalizable} +import java.nio.ByteBuffer +import scala.collection.JavaConversions._ + +class FlumePollingInputDStream[T: ClassTag]( + @transient ssc_ : StreamingContext, + val host: String, + val port: Int, + val maxBatchSize: Int, + val parallelism: Int, + storageLevel: StorageLevel +) extends ReceiverInputDStream[SparkPollingEvent](ssc_) { + /** + * Gets the receiver object that will be sent to the worker nodes + * to receive data. This method needs to defined by any specific implementation + * of a NetworkInputDStream. + */ + override def getReceiver(): Receiver[SparkPollingEvent] = { + new FlumePollingReceiver(host, port, maxBatchSize, parallelism, storageLevel) + } +} + +private[streaming] class FlumePollingReceiver( + host: String, + port: Int, + maxBatchSize: Int, + parallelism: Int, + storageLevel: StorageLevel +) extends Receiver[SparkPollingEvent](storageLevel) with Logging { + + lazy val channelFactory = + new NioClientSocketChannelFactory(Executors.newSingleThreadExecutor(), + Executors.newSingleThreadExecutor()) + lazy val transceiver = new NettyTransceiver(new InetSocketAddress(host, port), channelFactory) + lazy val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) + + override def onStart(): Unit = { + val dataReceiver = new Runnable { + override def run(): Unit = { + while (true) { + val batch = client.getEventBatch(maxBatchSize) + val seq = batch.getSequenceNumber + val events: java.util.List[SparkSinkEvent] = batch.getEventBatch + logDebug("Received batch of " + events.size() + " events with sequence number: " + seq) + try { + events.foreach(event => store(SparkPollingEvent.fromSparkSinkEvent(event))) + client.ack(seq) + } catch { + case e: Throwable => + client.nack(seq) + TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. + logWarning("Error while attempting to store events", e) + } + } + } + } + for (i <- 0 until parallelism) { + logInfo("Starting Flume Polling Receiver worker threads starting..") + receiverExecutor.submit(dataReceiver) + } + } + + override def store(dataItem: SparkPollingEvent) { + // Not entirely sure store is thread-safe for all storage levels - so wrap it in synchronized + // This takes a performance hit, since the parallelism is useful only for pulling data now. + this.synchronized { + super.store(dataItem) + } + } + + override def onStop(): Unit = { + logInfo("Shutting down Flume Polling Receiver") + receiverExecutor.shutdownNow() + transceiver.close() + channelFactory.releaseExternalResources() + } +} + +private[streaming] object SparkPollingEvent { + def fromSparkSinkEvent(in: SparkSinkEvent): SparkPollingEvent = { + val event = new SparkPollingEvent() + event.event = in + event + } +} +/* + * Unfortunately Avro does not allow including pre-compiled classes - so even though + * SparkSinkEvent is identical to AvroFlumeEvent, we need to create a new class and a wrapper + * around that to make it externalizable. + */ +class SparkPollingEvent() extends Externalizable with Logging { + var event : SparkSinkEvent = new SparkSinkEvent() + + /* De-serialize from bytes. */ + def readExternal(in: ObjectInput) { + val (headers, bodyBuff) = EventTransformer.readExternal(in) + event.setBody(ByteBuffer.wrap(bodyBuff)) + event.setHeaders(headers) + } + + /* Serialize to bytes. */ + def writeExternal(out: ObjectOutput) { + EventTransformer.writeExternal(out, event.getHeaders, event.getBody.array()) + } +} + + diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 499f3560ef768..f7d9bd3c6e2ab 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.flume import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaInputDStream, JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.ReceiverInputDStream object FlumeUtils { /** @@ -68,4 +68,52 @@ object FlumeUtils { ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port, storageLevel) } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param host The host on which the Flume agent is running + * @param port The port the Spark Sink is accepting connections on + * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream ( + ssc: StreamingContext, + host: String, + port: Int, + maxBatchSize: Int = 100, + parallelism: Int = 5, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[SparkPollingEvent] = { + new FlumePollingInputDStream[SparkPollingEvent](ssc, host, port, maxBatchSize, + parallelism, storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param host The host on which the Flume agent is running + * @param port The port the Spark Sink is accepting connections on + * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + def createJavaPollingStream ( + ssc: StreamingContext, + host: String, + port: Int, + maxBatchSize: Int = 100, + parallelism: Int = 5, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): JavaReceiverInputDStream[SparkPollingEvent] = { + new FlumePollingInputDStream[SparkPollingEvent](ssc, host, port, maxBatchSize, + parallelism, storageLevel) + } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala new file mode 100644 index 0000000000000..579f0b1091df3 --- /dev/null +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -0,0 +1,104 @@ +/* + * 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.flume + +import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} +import org.apache.spark.storage.StorageLevel +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import org.apache.spark.streaming.util.ManualClock +import java.nio.charset.Charset +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.Context +import org.apache.flume.conf.Configurables +import org.apache.spark.flume.sink.{SparkSinkConfig, SparkSink} +import scala.collection.JavaConversions._ +import org.apache.flume.event.EventBuilder +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +class FlumePollingReceiverSuite extends TestSuiteBase { + + val testPort = 9999 + + test("flume polling test") { + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val flumeStream: ReceiverInputDStream[SparkPollingEvent] = + FlumeUtils.createPollingStream(ssc, "localhost", testPort, 100, 5, + StorageLevel.MEMORY_AND_DISK) + val outputBuffer = new ArrayBuffer[Seq[SparkPollingEvent]] + with SynchronizedBuffer[Seq[SparkPollingEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + + // Start the channel and sink. + val context = new Context() + context.put("capacity", "5000") + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + ssc.start() + + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = Seq(1, 2, 3, 4, 5) + for (i <- 0 until 5) { + val tx = channel.getTransaction + tx.begin() + for (j <- 0 until input.size) { + channel.put(EventBuilder.withBody( + (String.valueOf(i) + input(j)).getBytes("utf-8"), + Map[String, String]("test-" + input(j).toString -> "header"))) + } + tx.commit() + tx.close() + Thread.sleep(500) // Allow some time for the events to reach + clock.addToTime(batchDuration.milliseconds) + } + val startTime = System.currentTimeMillis() + while (outputBuffer.size < 5 && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + logInfo("output.size = " + outputBuffer.size) + Thread.sleep(100) + } + val timeTaken = System.currentTimeMillis() - startTime + assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") + logInfo("Stopping context") + ssc.stop() + + val decoder = Charset.forName("UTF-8").newDecoder() + + assert(outputBuffer.size === 5) + var counter = 0 + for (i <- 0 until outputBuffer.size; + j <- 0 until outputBuffer(i).size) { + counter += 1 + val eventToVerify = outputBuffer(i)(j).event + val str = decoder.decode(eventToVerify.getBody) + assert(str.toString === (String.valueOf(i) + input(j))) + assert(eventToVerify.getHeaders.get("test-" + input(j).toString) === "header") + } + } + +} diff --git a/pom.xml b/pom.xml index 86264d1132ec4..cf7bf3d2ee1ec 100644 --- a/pom.xml +++ b/pom.xml @@ -100,6 +100,7 @@ external/twitter external/kafka external/flume + external/flume-sink external/zeromq external/mqtt examples diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 29dcd8678b476..df21813ff983a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -16,10 +16,15 @@ */ import sbt._ +import sbt.ClasspathDependency import sbt.Classpaths.publishTask +import sbt.ExclusionRule import sbt.Keys._ +import sbt.Task import sbtassembly.Plugin._ import AssemblyKeys._ +import sbtavro.SbtAvro._ +import scala.Some import scala.util.Properties import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact @@ -140,8 +145,11 @@ object SparkBuild extends Build { lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings) .dependsOn(streaming % "compile->compile;test->test") + lazy val externalFlumeSink = Project("external-flume-sink", file("external/flume-sink"), settings = flumeSinkSettings) + lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings) - .dependsOn(streaming % "compile->compile;test->test") + .dependsOn(streaming % "compile->compile;test->test").dependsOn(externalFlumeSink) + lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings) .dependsOn(streaming % "compile->compile;test->test") @@ -149,8 +157,8 @@ object SparkBuild extends Build { lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings) .dependsOn(streaming % "compile->compile;test->test") - lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) - lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) + lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalFlumeSink, externalZeromq, externalMqtt) + lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalFlumeSink, externalZeromq, externalMqtt) lazy val examples = Project("examples", file("examples"), settings = examplesSettings) .dependsOn(core, mllib, graphx, bagel, streaming, hive) dependsOn(allExternal: _*) @@ -622,6 +630,18 @@ object SparkBuild extends Build { ) ) + def flumeSinkSettings() = { + sharedSettings ++ Seq( + name := "spark-streaming-flume-sink", + previousArtifact := sparkPreviousArtifact("spark-streaming-flume-sink"), + libraryDependencies ++= Seq( + "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" + excludeAll(excludeJBossNetty, excludeThrift), + "org.apache.flume" % "flume-ng-core" % "1.4.0" % "compile" + excludeAll(excludeJBossNetty, excludeThrift) + ) + ) ++ sbtavro.SbtAvro.avroSettings + } def zeromqSettings() = sharedSettings ++ Seq( name := "spark-streaming-zeromq", previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), diff --git a/project/plugins.sbt b/project/plugins.sbt index 0cd16fd5bedd4..eadf71707ba19 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -4,6 +4,8 @@ resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline. resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" +resolvers += "sbt-plugins" at "http://repo.scala-sbt.org/scalasbt/sbt-plugin-releases" + addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.10.2") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") @@ -24,3 +26,5 @@ addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") + +addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") From d24d9d47795fe0a81fa2d70a4f81c24d2efd8914 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 18 May 2014 00:58:45 -0700 Subject: [PATCH 02/32] SPARK-1729. Make Flume pull data from source, rather than the current push model Update to the previous patch fixing some error cases and also excluding Netty dependencies. Also updated the unit tests. --- .../apache/spark/flume/sink/SparkSink.scala | 73 +++++++++++++------ .../flume/FlumePollingReceiverSuite.scala | 44 +++++++---- project/SparkBuild.scala | 1 - 3 files changed, 78 insertions(+), 40 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index 6243463a475b6..9e70be74d7e6d 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -30,9 +30,8 @@ import java.util.concurrent._ import java.util import org.apache.flume.conf.{ConfigurationException, Configurable} import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.avro.ipc.{NettyTransceiver, NettyServer} +import org.apache.avro.ipc.NettyServer import org.apache.avro.ipc.specific.SpecificResponder -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import java.net.InetSocketAddress class SparkSink() extends AbstractSink with Configurable { @@ -75,12 +74,10 @@ class SparkSink() extends AbstractSink with Configurable { val responder = new SpecificResponder(classOf[SparkFlumeProtocol], new AvroCallbackHandler()) - serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port), - new NioServerSocketChannelFactory( - Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat( - "Spark Sink " + classOf[NettyTransceiver].getSimpleName + " Boss-%d").build), - Executors.newFixedThreadPool(maxThreads, new ThreadFactoryBuilder().setNameFormat( - "Spark Sink " + classOf[NettyTransceiver].getSimpleName + " I/O Worker-%d").build)))) + // Using the constructor that takes specific thread-pools requires bringing in netty + // dependencies which are being excluded in the build. In practice, + // Netty dependencies are already available on the JVM as Flume would have pulled them in. + serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) serverOpt.map(server => server.start()) lock.lock() @@ -93,10 +90,14 @@ class SparkSink() extends AbstractSink with Configurable { } override def stop() { + transactionExecutorOpt.map(executor => executor.shutdownNow()) + serverOpt.map(server => { + server.close() + server.join() + }) lock.lock() try { running = false - transactionExecutorOpt.map(executor => executor.shutdownNow()) blockingCondition.signalAll() } finally { lock.unlock() @@ -131,23 +132,28 @@ class SparkSink() extends AbstractSink with Configurable { Status.BACKOFF } + + // Object representing an empty batch returned by the txn processor due to some error. + case object ErrorEventBatch extends EventBatch + private class AvroCallbackHandler() extends SparkFlumeProtocol { override def getEventBatch(n: Int): EventBatch = { val processor = processorFactory.get.checkOut(n) transactionExecutorOpt.map(executor => executor.submit(processor)) // Wait until a batch is available - can be null if some error was thrown - val eventBatch = Option(processor.eventQueue.take()) - if (eventBatch.isDefined) { - val eventsToBeSent = eventBatch.get - processorMap.put(eventsToBeSent.getSequenceNumber, processor) - if (LOG.isDebugEnabled) { - LOG.debug("Sent " + eventsToBeSent.getEventBatch.size() + - " events with sequence number: " + eventsToBeSent.getSequenceNumber) + val eventBatch = processor.eventQueue.take() + eventBatch match { + case ErrorEventBatch => throw new FlumeException("Something went wrong. No events" + + " retrieved from channel.") + case events => { + processorMap.put(events.getSequenceNumber, processor) + if (LOG.isDebugEnabled) { + LOG.debug("Sent " + events.getEventBatch.size() + + " events with sequence number: " + events.getSequenceNumber) + } + events } - eventsToBeSent - } else { - throw new FlumeException("Error while trying to retrieve events from the channel.") } } @@ -211,17 +217,38 @@ class SparkSink() extends AbstractSink with Configurable { val events = eventBatch.getEventBatch events.clear() val loop = new Breaks + var gotEventsInThisTxn = false loop.breakable { - for (i <- 0 until maxBatchSize) { + var i = 0 + // Using for here causes the maxBatchSize change to be ineffective as the Range gets + // pregenerated + while (i < maxBatchSize) { + i += 1 val eventOpt = Option(getChannel.take()) - eventOpt.map(event => { events.add(new SparkSinkEvent(toCharSequenceMap(event .getHeaders), ByteBuffer.wrap(event.getBody))) + gotEventsInThisTxn = true }) if (eventOpt.isEmpty) { - loop.break() + if (!gotEventsInThisTxn) { + // To avoid sending empty batches, we wait till events are available backing off + // between attempts to get events. Each attempt to get an event though causes one + // iteration to be lost. To ensure that we still send back maxBatchSize number of + // events, we cheat and increase the maxBatchSize by 1 to account for the lost + // iteration. Even throwing an exception is expensive as Avro will serialize it + // and send it over the wire, which is useless. Before incrementing though, + // ensure that we are not anywhere near INT_MAX. + if (maxBatchSize >= Int.MaxValue / 2) { + // Random sanity check + throw new RuntimeException("Safety exception - polled too many times, no events!") + } + maxBatchSize += 1 + Thread.sleep(500) + } else { + loop.break() + } } } } @@ -283,7 +310,7 @@ class SparkSink() extends AbstractSink with Configurable { null // No point rethrowing the exception } finally { // Must *always* release the caller thread - eventQueue.put(null) + eventQueue.put(ErrorEventBatch) // In the case of success coming after the timeout, but before resetting the seq number // remove the event from the map and then clear the value resultQueue.clear() diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala index 579f0b1091df3..aa5db4d94ff17 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -22,7 +22,6 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon import org.apache.spark.storage.StorageLevel import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import org.apache.spark.streaming.util.ManualClock -import java.nio.charset.Charset import org.apache.flume.channel.MemoryChannel import org.apache.flume.Context import org.apache.flume.conf.Configurables @@ -39,7 +38,7 @@ class FlumePollingReceiverSuite extends TestSuiteBase { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkPollingEvent] = - FlumeUtils.createPollingStream(ssc, "localhost", testPort, 100, 5, + FlumeUtils.createPollingStream(ssc, "localhost", testPort, 100, 1, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkPollingEvent]] with SynchronizedBuffer[Seq[SparkPollingEvent]] @@ -63,15 +62,17 @@ class FlumePollingReceiverSuite extends TestSuiteBase { ssc.start() val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val input = Seq(1, 2, 3, 4, 5) + var t = 0 for (i <- 0 until 5) { val tx = channel.getTransaction tx.begin() - for (j <- 0 until input.size) { + for (j <- 0 until 5) { channel.put(EventBuilder.withBody( - (String.valueOf(i) + input(j)).getBytes("utf-8"), - Map[String, String]("test-" + input(j).toString -> "header"))) + String.valueOf(t).getBytes("utf-8"), + Map[String, String]("test-" + t.toString -> "header"))) + t += 1 } + tx.commit() tx.close() Thread.sleep(500) // Allow some time for the events to reach @@ -86,19 +87,30 @@ class FlumePollingReceiverSuite extends TestSuiteBase { assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") logInfo("Stopping context") ssc.stop() + sink.stop() + channel.stop() - val decoder = Charset.forName("UTF-8").newDecoder() - - assert(outputBuffer.size === 5) + val flattenedBuffer = outputBuffer.flatten + assert(flattenedBuffer.size === 25) var counter = 0 - for (i <- 0 until outputBuffer.size; - j <- 0 until outputBuffer(i).size) { - counter += 1 - val eventToVerify = outputBuffer(i)(j).event - val str = decoder.decode(eventToVerify.getBody) - assert(str.toString === (String.valueOf(i) + input(j))) - assert(eventToVerify.getHeaders.get("test-" + input(j).toString) === "header") + for (i <- 0 until 25) { + val eventToVerify = EventBuilder.withBody( + String.valueOf(i).getBytes("utf-8"), + Map[String, String]("test-" + i.toString -> "header")) + var found = false + var j = 0 + while (j < flattenedBuffer.size && !found) { + val strToCompare = new String(flattenedBuffer(j).event.getBody.array(), "utf-8") + if (new String(eventToVerify.getBody, "utf-8") == strToCompare && + eventToVerify.getHeaders.get("test-" + i.toString) + .equals(flattenedBuffer(j).event.getHeaders.get("test-" + i.toString))) { + found = true + counter += 1 + } + j += 1 + } } + assert (counter === 25) } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index df21813ff983a..223c37d729fa6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -23,7 +23,6 @@ import sbt.Keys._ import sbt.Task import sbtassembly.Plugin._ import AssemblyKeys._ -import sbtavro.SbtAvro._ import scala.Some import scala.util.Properties import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} From 08176adc2a1a4f17562f486e0f897abfb7eba84d Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 18 May 2014 01:06:22 -0700 Subject: [PATCH 03/32] SPARK-1729. Make Flume pull data from source, rather than the current push model Exclude IO Netty in the Flume sink. --- project/SparkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 223c37d729fa6..dc9797694f527 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -635,9 +635,9 @@ object SparkBuild extends Build { previousArtifact := sparkPreviousArtifact("spark-streaming-flume-sink"), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" - excludeAll(excludeJBossNetty, excludeThrift), + excludeAll(excludeIONetty, excludeThrift), "org.apache.flume" % "flume-ng-core" % "1.4.0" % "compile" - excludeAll(excludeJBossNetty, excludeThrift) + excludeAll(excludeIONetty, excludeThrift) ) ) ++ sbtavro.SbtAvro.avroSettings } From 03d6c1c45bb5e1e00ba0a3618b920481ec3ec51a Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 19 May 2014 09:24:55 -0700 Subject: [PATCH 04/32] SPARK-1729. Make Flume pull data from source, rather than the current push model Removing previousArtifact from build spec, so that the build runs fine. --- project/SparkBuild.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index dc9797694f527..1c2aba66c9212 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -632,7 +632,6 @@ object SparkBuild extends Build { def flumeSinkSettings() = { sharedSettings ++ Seq( name := "spark-streaming-flume-sink", - previousArtifact := sparkPreviousArtifact("spark-streaming-flume-sink"), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeIONetty, excludeThrift), From 8df37e4911f74253a901502c9232c3db26dc8856 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 19 May 2014 23:09:02 -0700 Subject: [PATCH 05/32] SPARK-1729. Make Flume pull data from source, rather than the current push model Updated Maven build to be equivalent of the sbt build. --- external/flume-sink/pom.xml | 13 +++++++++++-- project/plugins.sbt | 2 -- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 40d7e09edaf16..4937e1ca42357 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -36,7 +36,7 @@ 1.4.0 - org.jboss.netty + io.netty netty @@ -49,6 +49,16 @@ org.apache.flume flume-ng-core 1.4.0 + + + io.netty + netty + + + org.apache.thrift + libthrift + + @@ -64,7 +74,6 @@ avro-maven-plugin 1.7.3 - String ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro diff --git a/project/plugins.sbt b/project/plugins.sbt index eadf71707ba19..0165382a57541 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -4,8 +4,6 @@ resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline. resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" -resolvers += "sbt-plugins" at "http://repo.scala-sbt.org/scalasbt/sbt-plugin-releases" - addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.10.2") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") From 87775aa52e21804680ed43dc4f789adf718ddb6c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 20 May 2014 17:42:40 -0700 Subject: [PATCH 06/32] SPARK-1729. Make Flume pull data from source, rather than the current push model Fix build with maven. --- external/flume/pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 93d8ec02ac69e..f90f88ae6e076 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -77,6 +77,7 @@ org.apache.spark spark-streaming-flume-sink_2.10 + ${project.version} From 0f10788487f10234aa39277d4c20556f7c846796 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sat, 24 May 2014 01:32:32 -0700 Subject: [PATCH 07/32] SPARK-1729. Make Flume pull data from source, rather than the current push model Added support for polling several Flume agents from a single receiver. --- .../flume/FlumePollingInputDStream.scala | 39 +++++-- .../spark/streaming/flume/FlumeUtils.scala | 17 ++- .../flume/FlumePollingReceiverSuite.scala | 108 ++++++++++++++---- 3 files changed, 120 insertions(+), 44 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 71b0f72f85f53..2571f2d36f3f1 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -32,11 +32,11 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder import java.io.{ObjectOutput, ObjectInput, Externalizable} import java.nio.ByteBuffer import scala.collection.JavaConversions._ +import scala.collection.mutable class FlumePollingInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, - val host: String, - val port: Int, + val addresses: Seq[InetSocketAddress], val maxBatchSize: Int, val parallelism: Int, storageLevel: StorageLevel @@ -47,30 +47,44 @@ class FlumePollingInputDStream[T: ClassTag]( * of a NetworkInputDStream. */ override def getReceiver(): Receiver[SparkPollingEvent] = { - new FlumePollingReceiver(host, port, maxBatchSize, parallelism, storageLevel) + new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) } } private[streaming] class FlumePollingReceiver( - host: String, - port: Int, + addresses: Seq[InetSocketAddress], maxBatchSize: Int, parallelism: Int, storageLevel: StorageLevel ) extends Receiver[SparkPollingEvent](storageLevel) with Logging { + lazy val channelFactoryExecutor = + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). + setNameFormat("Flume Receiver Channel Thread - %d").build()) + lazy val channelFactory = - new NioClientSocketChannelFactory(Executors.newSingleThreadExecutor(), - Executors.newSingleThreadExecutor()) - lazy val transceiver = new NettyTransceiver(new InetSocketAddress(host, port), channelFactory) - lazy val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) + private var connections = Array.empty[FlumeConnection] // temporarily empty, filled in later + override def onStart(): Unit = { + val connectionBuilder = new mutable.ArrayBuilder.ofRef[FlumeConnection]() + addresses.map(host => { + val transceiver = new NettyTransceiver(host, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + connectionBuilder += new FlumeConnection(transceiver, client) + }) + connections = connectionBuilder.result() val dataReceiver = new Runnable { override def run(): Unit = { + var counter = 0 while (true) { + counter = counter % connections.size + val client = connections(counter).client + counter += 1 val batch = client.getEventBatch(maxBatchSize) val seq = batch.getSequenceNumber val events: java.util.List[SparkSinkEvent] = batch.getEventBatch @@ -104,11 +118,16 @@ private[streaming] class FlumePollingReceiver( override def onStop(): Unit = { logInfo("Shutting down Flume Polling Receiver") receiverExecutor.shutdownNow() - transceiver.close() + connections.map(connection => { + connection.tranceiver.close() + }) channelFactory.releaseExternalResources() } } +private class FlumeConnection(val tranceiver: NettyTransceiver, + val client: SparkFlumeProtocol.Callback) + private[streaming] object SparkPollingEvent { def fromSparkSinkEvent(in: SparkSinkEvent): SparkPollingEvent = { val event = new SparkPollingEvent() diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index f7d9bd3c6e2ab..ca0059ff04dab 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -21,6 +21,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.ReceiverInputDStream +import java.net.InetSocketAddress object FlumeUtils { /** @@ -72,8 +73,7 @@ object FlumeUtils { /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. - * @param host The host on which the Flume agent is running - * @param port The port the Spark Sink is accepting connections on + * @param addresses List of InetSocketAddresses representing the hosts to connect to. * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a * single RPC call * @param parallelism Number of concurrent requests this stream should send to the sink. Note @@ -83,21 +83,19 @@ object FlumeUtils { */ def createPollingStream ( ssc: StreamingContext, - host: String, - port: Int, + addresses: Seq[InetSocketAddress], maxBatchSize: Int = 100, parallelism: Int = 5, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): ReceiverInputDStream[SparkPollingEvent] = { - new FlumePollingInputDStream[SparkPollingEvent](ssc, host, port, maxBatchSize, + new FlumePollingInputDStream[SparkPollingEvent](ssc, addresses, maxBatchSize, parallelism, storageLevel) } /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. - * @param host The host on which the Flume agent is running - * @param port The port the Spark Sink is accepting connections on + * @param addresses List of InetSocketAddresses representing the hosts to connect to. * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a * single RPC call * @param parallelism Number of concurrent requests this stream should send to the sink. Note @@ -107,13 +105,12 @@ object FlumeUtils { */ def createJavaPollingStream ( ssc: StreamingContext, - host: String, - port: Int, + addresses: Seq[InetSocketAddress], maxBatchSize: Int = 100, parallelism: Int = 5, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): JavaReceiverInputDStream[SparkPollingEvent] = { - new FlumePollingInputDStream[SparkPollingEvent](ssc, host, port, maxBatchSize, + new FlumePollingInputDStream[SparkPollingEvent](ssc, addresses, maxBatchSize, parallelism, storageLevel) } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala index aa5db4d94ff17..404759f291f39 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -29,6 +29,8 @@ import org.apache.spark.flume.sink.{SparkSinkConfig, SparkSink} import scala.collection.JavaConversions._ import org.apache.flume.event.EventBuilder import org.apache.spark.streaming.dstream.ReceiverInputDStream +import java.net.InetSocketAddress +import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} class FlumePollingReceiverSuite extends TestSuiteBase { @@ -38,7 +40,7 @@ class FlumePollingReceiverSuite extends TestSuiteBase { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkPollingEvent] = - FlumeUtils.createPollingStream(ssc, "localhost", testPort, 100, 1, + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), 100, 5, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkPollingEvent]] with SynchronizedBuffer[Seq[SparkPollingEvent]] @@ -60,26 +62,67 @@ class FlumePollingReceiverSuite extends TestSuiteBase { sink.setChannel(channel) sink.start() ssc.start() + writeAndVerify(Seq(channel), ssc, outputBuffer) + sink.stop() + channel.stop() + } - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - var t = 0 - for (i <- 0 until 5) { - val tx = channel.getTransaction - tx.begin() - for (j <- 0 until 5) { - channel.put(EventBuilder.withBody( - String.valueOf(t).getBytes("utf-8"), - Map[String, String]("test-" + t.toString -> "header"))) - t += 1 - } + test("flume polling test multiple hosts") { + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val flumeStream: ReceiverInputDStream[SparkPollingEvent] = + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort), + new InetSocketAddress("localhost", testPort + 1)), 100, 5, + StorageLevel.MEMORY_AND_DISK) + val outputBuffer = new ArrayBuffer[Seq[SparkPollingEvent]] + with SynchronizedBuffer[Seq[SparkPollingEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + + // Start the channel and sink. + val context = new Context() + context.put("capacity", "5000") + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val channel2 = new MemoryChannel() + Configurables.configure(channel2, context) - tx.commit() - tx.close() - Thread.sleep(500) // Allow some time for the events to reach - clock.addToTime(batchDuration.milliseconds) + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + + val sink2 = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort + 1)) + Configurables.configure(sink2, context) + sink2.setChannel(channel2) + sink2.start() + ssc.start() + writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) + sink.stop() + channel.stop() + + } + + def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, + outputBuffer: ArrayBuffer[Seq[SparkPollingEvent]]) { + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val executor = Executors.newCachedThreadPool() + val executorCompletion = new ExecutorCompletionService[Void](executor) + channels.map(channel => { + executorCompletion.submit(new TxnSubmitter(channel, clock)) + }) + for(i <- 0 until channels.size) { + executorCompletion.take() } val startTime = System.currentTimeMillis() - while (outputBuffer.size < 5 && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + while (outputBuffer.size < 5 * channels.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { logInfo("output.size = " + outputBuffer.size) Thread.sleep(100) } @@ -87,15 +130,13 @@ class FlumePollingReceiverSuite extends TestSuiteBase { assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") logInfo("Stopping context") ssc.stop() - sink.stop() - channel.stop() val flattenedBuffer = outputBuffer.flatten - assert(flattenedBuffer.size === 25) + assert(flattenedBuffer.size === 25 * channels.size) var counter = 0 - for (i <- 0 until 25) { - val eventToVerify = EventBuilder.withBody( - String.valueOf(i).getBytes("utf-8"), + for (k <- 0 until channels.size; i <- 0 until 25) { + val eventToVerify = EventBuilder.withBody((channels(k).getName + " - " + + String.valueOf(i)).getBytes("utf-8"), Map[String, String]("test-" + i.toString -> "header")) var found = false var j = 0 @@ -110,7 +151,26 @@ class FlumePollingReceiverSuite extends TestSuiteBase { j += 1 } } - assert (counter === 25) + assert(counter === 25 * channels.size) } + private class TxnSubmitter(channel: MemoryChannel, clock: ManualClock) extends Callable[Void] { + override def call(): Void = { + var t = 0 + for (i <- 0 until 5) { + val tx = channel.getTransaction + tx.begin() + for (j <- 0 until 5) { + channel.put(EventBuilder.withBody((channel.getName + " - " + String.valueOf(t)).getBytes("utf-8"), + Map[String, String]("test-" + t.toString -> "header"))) + t += 1 + } + tx.commit() + tx.close() + Thread.sleep(500) // Allow some time for the events to reach + clock.addToTime(batchDuration.milliseconds) + } + null + } + } } From c604a3c0fee085679967460f50b563a8d58aedf1 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 5 Jun 2014 09:17:05 -0700 Subject: [PATCH 08/32] SPARK-1729. Optimize imports. --- .../streaming/flume/EventTransformer.scala | 4 ++- .../flume/FlumePollingInputDStream.scala | 30 +++++++++++-------- .../spark/streaming/flume/FlumeUtils.scala | 4 ++- .../flume/FlumePollingReceiverSuite.scala | 19 +++++++----- 4 files changed, 35 insertions(+), 22 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala index 91f6171d57368..75b224afca39b 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -18,8 +18,10 @@ package org.apache.spark.streaming.flume import java.io.{ObjectOutput, ObjectInput} -import org.apache.spark.util.Utils + import scala.collection.JavaConversions._ + +import org.apache.spark.util.Utils import org.apache.spark.Logging /** diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 2571f2d36f3f1..495f569f6359d 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -16,23 +16,29 @@ */ package org.apache.spark.streaming.flume -import scala.reflect.ClassTag -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.Logging + +import java.io.{ObjectOutput, ObjectInput, Externalizable} import java.net.InetSocketAddress +import java.nio.ByteBuffer import java.util.concurrent.{TimeUnit, Executors} + +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.reflect.ClassTag + +import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor -import org.apache.spark.flume.{SparkSinkEvent, SparkFlumeProtocol} import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import com.google.common.util.concurrent.ThreadFactoryBuilder -import java.io.{ObjectOutput, ObjectInput, Externalizable} -import java.nio.ByteBuffer -import scala.collection.JavaConversions._ -import scala.collection.mutable + +import org.apache.spark.Logging +import org.apache.spark.flume.{SparkSinkEvent, SparkFlumeProtocol} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.receiver.Receiver + + class FlumePollingInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index ca0059ff04dab..2ff8ce2a77d2d 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -17,11 +17,13 @@ package org.apache.spark.streaming.flume +import java.net.InetSocketAddress + import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} import org.apache.spark.streaming.dstream.ReceiverInputDStream -import java.net.InetSocketAddress + object FlumeUtils { /** diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala index 404759f291f39..dc64405872a2d 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -18,19 +18,22 @@ */ package org.apache.spark.streaming.flume -import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} -import org.apache.spark.storage.StorageLevel +import java.net.InetSocketAddress +import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} + +import scala.collection.JavaConversions._ import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} -import org.apache.spark.streaming.util.ManualClock -import org.apache.flume.channel.MemoryChannel + import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel import org.apache.flume.conf.Configurables -import org.apache.spark.flume.sink.{SparkSinkConfig, SparkSink} -import scala.collection.JavaConversions._ import org.apache.flume.event.EventBuilder + +import org.apache.spark.flume.sink.{SparkSinkConfig, SparkSink} +import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream -import java.net.InetSocketAddress -import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} class FlumePollingReceiverSuite extends TestSuiteBase { From 9741683173c5dad3148c77d1a0f47b92387b8bdc Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 5 Jun 2014 23:38:12 -0700 Subject: [PATCH 09/32] SPARK-1729. Fixes based on review. --- .../flume-sink/src/main/avro/sparkflume.avdl | 2 +- .../apache/spark/flume/sink/SparkSink.scala | 112 ++++++------------ .../flume/FlumePollingInputDStream.scala | 18 +-- 3 files changed, 47 insertions(+), 85 deletions(-) diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl index 9dcc709de079a..fa00b2310a17b 100644 --- a/external/flume-sink/src/main/avro/sparkflume.avdl +++ b/external/flume-sink/src/main/avro/sparkflume.avdl @@ -28,7 +28,7 @@ protocol SparkFlumeProtocol { record EventBatch { string sequenceNumber; - array eventBatch; + array events; } EventBatch getEventBatch (int n); diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index 9e70be74d7e6d..521800be64f2c 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -34,10 +34,8 @@ import org.apache.avro.ipc.NettyServer import org.apache.avro.ipc.specific.SpecificResponder import java.net.InetSocketAddress -class SparkSink() extends AbstractSink with Configurable { +class SparkSink extends AbstractSink with Configurable { private val LOG = LoggerFactory.getLogger(this.getClass) - private val lock = new ReentrantLock() - private val blockingCondition = lock.newCondition() // This sink will not persist sequence numbers and reuses them if it gets restarted. // So it is possible to commit a transaction which may have been meant for the sink before the @@ -58,19 +56,20 @@ class SparkSink() extends AbstractSink with Configurable { private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() - private var processorFactory: Option[SparkHandlerFactory] = None + private var processorManager: Option[TransactionProcessorManager] = None private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME private var port: Int = 0 private var maxThreads: Int = SparkSinkConfig.DEFAULT_MAX_THREADS private var serverOpt: Option[NettyServer] = None - private var running = false + + private val blockingLatch = new CountDownLatch(1) override def start() { transactionExecutorOpt = Option(Executors.newFixedThreadPool(numProcessors, new ThreadFactoryBuilder().setDaemon(true) .setNameFormat("Spark Sink, " + getName + " Processor Thread - %d").build())) - processorFactory = Option(new SparkHandlerFactory(numProcessors)) + processorManager = Option(new TransactionProcessorManager(numProcessors)) val responder = new SpecificResponder(classOf[SparkFlumeProtocol], new AvroCallbackHandler()) @@ -80,12 +79,6 @@ class SparkSink() extends AbstractSink with Configurable { serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) serverOpt.map(server => server.start()) - lock.lock() - try { - running = true - } finally { - lock.unlock() - } super.start() } @@ -95,24 +88,15 @@ class SparkSink() extends AbstractSink with Configurable { server.close() server.join() }) - lock.lock() - try { - running = false - blockingCondition.signalAll() - } finally { - lock.unlock() - } + blockingLatch.countDown() + super.stop() } override def configure(ctx: Context) { import SparkSinkConfig._ hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) - val portOpt = Option(ctx.getInteger(CONF_PORT)) - if(portOpt.isDefined) { - port = portOpt.get - } else { - throw new ConfigurationException("The Port to bind must be specified") - } + port = Option(ctx.getInteger(CONF_PORT)). + getOrElse(throw new ConfigurationException("The port to bind to must be specified")) numProcessors = ctx.getInteger(PROCESSOR_COUNT, DEFAULT_PROCESSOR_COUNT) transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) maxThreads = ctx.getInteger(CONF_MAX_THREADS, DEFAULT_MAX_THREADS) @@ -120,15 +104,9 @@ class SparkSink() extends AbstractSink with Configurable { override def process(): Status = { // This method is called in a loop by the Flume framework - block it until the sink is - // stopped to save CPU resources - lock.lock() - try { - while(running) { - blockingCondition.await() - } - } finally { - lock.unlock() - } + // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is + // being shut down. + blockingLatch.await() Status.BACKOFF } @@ -136,24 +114,22 @@ class SparkSink() extends AbstractSink with Configurable { // Object representing an empty batch returned by the txn processor due to some error. case object ErrorEventBatch extends EventBatch - private class AvroCallbackHandler() extends SparkFlumeProtocol { + private class AvroCallbackHandler extends SparkFlumeProtocol { override def getEventBatch(n: Int): EventBatch = { - val processor = processorFactory.get.checkOut(n) + val processor = processorManager.get.checkOut(n) transactionExecutorOpt.map(executor => executor.submit(processor)) // Wait until a batch is available - can be null if some error was thrown - val eventBatch = processor.eventQueue.take() - eventBatch match { + processor.eventQueue.take() match { case ErrorEventBatch => throw new FlumeException("Something went wrong. No events" + " retrieved from channel.") - case events => { - processorMap.put(events.getSequenceNumber, processor) + case eventBatch: EventBatch => + processorMap.put(eventBatch.getSequenceNumber, processor) if (LOG.isDebugEnabled) { - LOG.debug("Sent " + events.getEventBatch.size() + - " events with sequence number: " + events.getSequenceNumber) + LOG.debug("Sent " + eventBatch.getEvents.size() + + " events with sequence number: " + eventBatch.getSequenceNumber) } - events - } + eventBatch } } @@ -214,41 +190,23 @@ class SparkSink() extends AbstractSink with Configurable { tx.begin() try { eventBatch.setSequenceNumber(seqBase + seqNum.incrementAndGet()) - val events = eventBatch.getEventBatch + val events = eventBatch.getEvents events.clear() val loop = new Breaks var gotEventsInThisTxn = false loop.breakable { - var i = 0 - // Using for here causes the maxBatchSize change to be ineffective as the Range gets - // pregenerated - while (i < maxBatchSize) { - i += 1 - val eventOpt = Option(getChannel.take()) - eventOpt.map(event => { - events.add(new SparkSinkEvent(toCharSequenceMap(event - .getHeaders), - ByteBuffer.wrap(event.getBody))) - gotEventsInThisTxn = true - }) - if (eventOpt.isEmpty) { - if (!gotEventsInThisTxn) { - // To avoid sending empty batches, we wait till events are available backing off - // between attempts to get events. Each attempt to get an event though causes one - // iteration to be lost. To ensure that we still send back maxBatchSize number of - // events, we cheat and increase the maxBatchSize by 1 to account for the lost - // iteration. Even throwing an exception is expensive as Avro will serialize it - // and send it over the wire, which is useless. Before incrementing though, - // ensure that we are not anywhere near INT_MAX. - if (maxBatchSize >= Int.MaxValue / 2) { - // Random sanity check - throw new RuntimeException("Safety exception - polled too many times, no events!") + while (events.size() < maxBatchSize) { + Option(getChannel.take()) match { + case Some(event) => + events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), + ByteBuffer.wrap(event.getBody))) + gotEventsInThisTxn = true + case None => + if (!gotEventsInThisTxn) { + Thread.sleep(500) + } else { + loop.break() } - maxBatchSize += 1 - Thread.sleep(500) - } else { - loop.break() - } } } } @@ -284,7 +242,7 @@ class SparkSink() extends AbstractSink with Configurable { } finally { resultQueueUpdateLock.unlock() } - eventBatch.getEventBatch.clear() + eventBatch.getEvents.clear() // If the batch failed on spark side, throw a FlumeException maybeResult.map(success => if (!success) { @@ -315,7 +273,7 @@ class SparkSink() extends AbstractSink with Configurable { // remove the event from the map and then clear the value resultQueue.clear() processorMap.remove(eventBatch.getSequenceNumber) - processorFactory.get.checkIn(this) + processorManager.get.checkIn(this) tx.close() } } @@ -328,7 +286,7 @@ class SparkSink() extends AbstractSink with Configurable { } } - private class SparkHandlerFactory(val maxInstances: Int) { + private class TransactionProcessorManager(val maxInstances: Int) { val queue = new scala.collection.mutable.Queue[TransactionProcessor] val queueModificationLock = new ReentrantLock() var currentSize = 0 diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 495f569f6359d..ee337b5f5507f 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -77,13 +77,13 @@ private[streaming] class FlumePollingReceiver( private var connections = Array.empty[FlumeConnection] // temporarily empty, filled in later override def onStart(): Unit = { - val connectionBuilder = new mutable.ArrayBuilder.ofRef[FlumeConnection]() - addresses.map(host => { + // Create the connections to each Flume agent. + connections = addresses.map(host => { val transceiver = new NettyTransceiver(host, channelFactory) val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) - connectionBuilder += new FlumeConnection(transceiver, client) - }) - connections = connectionBuilder.result() + new FlumeConnection(transceiver, client) + }).toArray + val dataReceiver = new Runnable { override def run(): Unit = { var counter = 0 @@ -93,14 +93,18 @@ private[streaming] class FlumePollingReceiver( counter += 1 val batch = client.getEventBatch(maxBatchSize) val seq = batch.getSequenceNumber - val events: java.util.List[SparkSinkEvent] = batch.getEventBatch + val events: java.util.List[SparkSinkEvent] = batch.getEvents logDebug("Received batch of " + events.size() + " events with sequence number: " + seq) try { events.foreach(event => store(SparkPollingEvent.fromSparkSinkEvent(event))) client.ack(seq) } catch { case e: Throwable => - client.nack(seq) + try { + client.nack(seq) // If the agent is down, even this could fail and throw + } catch { + case e: Throwable => logError("Sending Nack also failed. A Flume agent is down.") + } TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. logWarning("Error while attempting to store events", e) } From e7da5128be13130538e41fb5e976089e93f1e149 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 5 Jun 2014 23:43:13 -0700 Subject: [PATCH 10/32] SPARK-1729. Fixing import order --- .../apache/spark/flume/sink/SparkSink.scala | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index 521800be64f2c..030396f12844a 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -16,23 +16,29 @@ */ package org.apache.spark.flume.sink -import org.apache.flume.sink.AbstractSink -import java.util.concurrent.locks.ReentrantLock -import org.apache.flume.Sink.Status -import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} -import scala.util.control.Breaks + +import java.net.InetSocketAddress import java.nio.ByteBuffer -import org.apache.flume.{FlumeException, Context} -import org.slf4j.LoggerFactory -import java.util.concurrent.atomic.AtomicLong -import org.apache.commons.lang.RandomStringUtils -import java.util.concurrent._ import java.util -import org.apache.flume.conf.{ConfigurationException, Configurable} +import java.util.concurrent._ +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.locks.ReentrantLock + +import scala.util.control.Breaks + import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.avro.ipc.NettyServer import org.apache.avro.ipc.specific.SpecificResponder -import java.net.InetSocketAddress +import org.apache.commons.lang.RandomStringUtils +import org.apache.flume.Sink.Status +import org.apache.flume.conf.{ConfigurationException, Configurable} +import org.apache.flume.sink.AbstractSink +import org.apache.flume.{FlumeException, Context} + +import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} +import org.slf4j.LoggerFactory + + class SparkSink extends AbstractSink with Configurable { private val LOG = LoggerFactory.getLogger(this.getClass) From d6fa3aa25e21be508c695067a858afd0d3ddbd64 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Jun 2014 22:27:19 -0700 Subject: [PATCH 11/32] SPARK-1729. New Flume-Spark integration. Made the Flume Sink considerably simpler. Added a lot of documentation. --- .../apache/spark/flume/sink/SparkSink.scala | 527 ++++++++++-------- .../flume/FlumePollingInputDStream.scala | 27 +- 2 files changed, 323 insertions(+), 231 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index 030396f12844a..e830b388c12b2 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -16,13 +16,11 @@ */ package org.apache.spark.flume.sink - import java.net.InetSocketAddress import java.nio.ByteBuffer import java.util import java.util.concurrent._ import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.locks.ReentrantLock import scala.util.control.Breaks @@ -33,63 +31,68 @@ import org.apache.commons.lang.RandomStringUtils import org.apache.flume.Sink.Status import org.apache.flume.conf.{ConfigurationException, Configurable} import org.apache.flume.sink.AbstractSink -import org.apache.flume.{FlumeException, Context} - -import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} +import org.apache.flume.{Channel, Transaction, FlumeException, Context} import org.slf4j.LoggerFactory - - +import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} +/** + * A sink that uses Avro RPC to run a server that can be polled by Spark's + * FlumePollingInputDStream. This sink has the following configuration parameters: + * + * hostname - The hostname to bind to. Default: 0.0.0.0 + * port - The port to bind to. (No default - mandatory) + * timeout - Time in seconds after which a transaction is rolled back, + * if an ACK is not received from Spark within that time + * threads - Number of threads to use to receive requests from Spark (Default: 10) + * + */ +// Flume forces transactions to be thread-local. So each transaction *must* be committed, or +// rolled back from the thread it was originally created in. So each getEvents call from Spark +// creates a TransactionProcessor which runs in a new thread, in which the transaction is created +// and events are pulled off the channel. Once the events are sent to spark, +// that thread is blocked and the TransactionProcessor is saved in a map, +// until an ACK or NACK comes back or the transaction times out (after the specified timeout). +// When the response comes, the TransactionProcessor is retrieved and then unblocked, +// at which point the transaction is committed or rolled back. class SparkSink extends AbstractSink with Configurable { - private val LOG = LoggerFactory.getLogger(this.getClass) - - // This sink will not persist sequence numbers and reuses them if it gets restarted. - // So it is possible to commit a transaction which may have been meant for the sink before the - // restart. - // Since the new txn may not have the same sequence number we must guard against accidentally - // committing - // a new transaction. To reduce the probability of that happening a random string is prepended - // to the sequence number. - // Does not change for life of sink - private val seqBase = RandomStringUtils.randomAlphanumeric(8) - // Incremented for each transaction - private val seqNum = new AtomicLong(0) - private var transactionExecutorOpt: Option[ExecutorService] = None + // Size of the pool to use for holding transaction processors. + private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS - private var numProcessors: Integer = SparkSinkConfig.DEFAULT_PROCESSOR_COUNT + // Timeout for each transaction. If spark does not respond in this much time, + // rollback the transaction private var transactionTimeout = SparkSinkConfig.DEFAULT_TRANSACTION_TIMEOUT - private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() - - private var processorManager: Option[TransactionProcessorManager] = None + // Address info to bind on private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME private var port: Int = 0 - private var maxThreads: Int = SparkSinkConfig.DEFAULT_MAX_THREADS + + // Handle to the server private var serverOpt: Option[NettyServer] = None + // The handler that handles the callback from Avro + private var handler: Option[SparkAvroCallbackHandler] = None + + // Latch that blocks off the Flume framework from wasting 1 thread. private val blockingLatch = new CountDownLatch(1) override def start() { - transactionExecutorOpt = Option(Executors.newFixedThreadPool(numProcessors, - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("Spark Sink, " + getName + " Processor Thread - %d").build())) - - processorManager = Option(new TransactionProcessorManager(numProcessors)) - - val responder = new SpecificResponder(classOf[SparkFlumeProtocol], new AvroCallbackHandler()) - + handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout)) + val responder = new SpecificResponder(classOf[SparkFlumeProtocol], handler.get) // Using the constructor that takes specific thread-pools requires bringing in netty // dependencies which are being excluded in the build. In practice, // Netty dependencies are already available on the JVM as Flume would have pulled them in. serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) - - serverOpt.map(server => server.start()) + serverOpt.map(server => { + server.start() + }) super.start() } override def stop() { - transactionExecutorOpt.map(executor => executor.shutdownNow()) + handler.map(callbackHandler => { + callbackHandler.shutdown() + }) serverOpt.map(server => { server.close() server.join() @@ -98,14 +101,16 @@ class SparkSink extends AbstractSink with Configurable { super.stop() } + /** + * @param ctx + */ override def configure(ctx: Context) { import SparkSinkConfig._ hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) port = Option(ctx.getInteger(CONF_PORT)). getOrElse(throw new ConfigurationException("The port to bind to must be specified")) - numProcessors = ctx.getInteger(PROCESSOR_COUNT, DEFAULT_PROCESSOR_COUNT) + poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) - maxThreads = ctx.getInteger(CONF_MAX_THREADS, DEFAULT_MAX_THREADS) } override def process(): Status = { @@ -115,233 +120,306 @@ class SparkSink extends AbstractSink with Configurable { blockingLatch.await() Status.BACKOFF } +} +/** + * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process + * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. + * @param threads Number of threads to use to process requests. + * @param channel The channel that the sink pulls events from + * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark + * is rolled back. + */ +private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, + val transactionTimeout: Int) extends SparkFlumeProtocol { + private val LOG = LoggerFactory.getLogger(classOf[SparkAvroCallbackHandler]) + val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("Spark Sink Processor Thread - %d").build())) + private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() + // This sink will not persist sequence numbers and reuses them if it gets restarted. + // So it is possible to commit a transaction which may have been meant for the sink before the + // restart. + // Since the new txn may not have the same sequence number we must guard against accidentally + // committing a new transaction. To reduce the probability of that happening a random string is + // prepended to the sequence number. Does not change for life of sink + private val seqBase = RandomStringUtils.randomAlphanumeric(8) + private val seqCounter = new AtomicLong(0) - // Object representing an empty batch returned by the txn processor due to some error. - case object ErrorEventBatch extends EventBatch + /** + * Returns a bunch of events to Spark over Avro RPC. + * @param n Maximum number of events to return in a batch + * @return [[EventBatch]] instance that has a sequence number and an array of at most n events + */ + override def getEventBatch(n: Int): EventBatch = { + val sequenceNumber = seqBase + seqCounter.incrementAndGet() + val processor = new TransactionProcessor(channel, sequenceNumber, + n, transactionTimeout, this) + transactionExecutorOpt.map(executor => { + executor.submit(processor) + }) + // Wait until a batch is available - can be null if some error was thrown + processor.getEventBatch match { + case ErrorEventBatch => throw new FlumeException("Something went wrong. No events" + + " retrieved from channel.") + case eventBatch: EventBatch => + processorMap.put(sequenceNumber, processor) + if (LOG.isDebugEnabled()) { + LOG.debug("Sent " + eventBatch.getEvents.size() + + " events with sequence number: " + eventBatch.getSequenceNumber) + } + eventBatch + } + } - private class AvroCallbackHandler extends SparkFlumeProtocol { + /** + * Called by Spark to indicate successful commit of a batch + * @param sequenceNumber The sequence number of the event batch that was successful + */ + override def ack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = true) + null + } - override def getEventBatch(n: Int): EventBatch = { - val processor = processorManager.get.checkOut(n) - transactionExecutorOpt.map(executor => executor.submit(processor)) - // Wait until a batch is available - can be null if some error was thrown - processor.eventQueue.take() match { - case ErrorEventBatch => throw new FlumeException("Something went wrong. No events" + - " retrieved from channel.") - case eventBatch: EventBatch => - processorMap.put(eventBatch.getSequenceNumber, processor) - if (LOG.isDebugEnabled) { - LOG.debug("Sent " + eventBatch.getEvents.size() + - " events with sequence number: " + eventBatch.getSequenceNumber) - } - eventBatch - } - } + /** + * Called by Spark to indicate failed commit of a batch + * @param sequenceNumber The sequence number of the event batch that failed + * @return + */ + override def nack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = false) + LOG.info("Spark failed to commit transaction. Will reattempt events.") + null + } - override def ack(sequenceNumber: CharSequence): Void = { - completeTransaction(sequenceNumber, success = true) - null - } + /** + * Helper method to commit or rollback a transaction. + * @param sequenceNumber The sequence number of the batch that was completed + * @param success Whether the batch was successful or not. + */ + private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { + Option(removeAndGetProcessor(sequenceNumber)).map(processor => { + processor.batchProcessed(success) + }) + } - override def nack(sequenceNumber: CharSequence): Void = { - completeTransaction(sequenceNumber, success = false) - LOG.info("Spark failed to commit transaction. Will reattempt events.") - null - } + /** + * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. + * @param sequenceNumber + * @return The transaction processor for the corresponding batch. Note that this instance is no + * longer tracked and the caller is responsible for that txn processor. + */ + private[flume] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { + processorMap.remove(sequenceNumber.toString) // The toString is required! + } - def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - val processorOpt = Option(processorMap.remove(sequenceNumber)) - if (processorOpt.isDefined) { - val processor = processorOpt.get - processor.resultQueueUpdateLock.lock() - try { - // Is the sequence number the same as the one the processor is processing? If not, - // don't update { - if (processor.eventBatch.getSequenceNumber.equals(sequenceNumber)) { - processor.resultQueue.put(success) - } - } finally { - processor.resultQueueUpdateLock.unlock() - } - } - } + /** + * Shuts down the executor used to process transactions. + */ + def shutdown() { + transactionExecutorOpt.map(executor => { + executor.shutdownNow() + }) } +} + +/** + * Object representing an empty batch returned by the txn processor due to some error. + */ +case object ErrorEventBatch extends EventBatch + +// Flume forces transactions to be thread-local (horrible, I know!) +// So the sink basically spawns a new thread to pull the events out within a transaction. +// The thread fills in the event batch object that is set before the thread is scheduled. +// After filling it in, the thread waits on a condition - which is released only +// when the success message comes back for the specific sequence number for that event batch. +/** + * This class represents a transaction on the Flume channel. This class runs a separate thread + * which owns the transaction. The thread is blocked until the success call for that transaction + * comes back with an ACK or NACK. + * @param channel The channel from which to pull events + * @param seqNum The sequence number to use for the transaction. Must be unique + * @param maxBatchSize The maximum number of events to process per batch + * @param transactionTimeout Time in seconds after which a transaction must be rolled back + * without waiting for an ACK from Spark + * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts + */ +private class TransactionProcessor(val channel: Channel, val seqNum: String, + var maxBatchSize: Int, val transactionTimeout: Int, + val parent: SparkAvroCallbackHandler) extends Callable[Void] { + + private val LOG = LoggerFactory.getLogger(classOf[TransactionProcessor]) + + // If a real batch is not returned, we always have to return an error batch. + @volatile private var eventBatch: EventBatch = ErrorEventBatch + + // Synchronization primitives + val batchGeneratedLatch = new CountDownLatch(1) + val batchAckLatch = new CountDownLatch(1) + + // Sanity check to ensure we don't loop like crazy + val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 + + // OK to use volatile, since the change would only make this true (otherwise it will be + // changed to false - we never apply a negation operation to this) - which means the transaction + // succeeded. + @volatile private var batchSuccess = false + + // The transaction that this processor would handle + var txOpt: Option[Transaction] = None - // Flume forces transactions to be thread-local (horrible, I know!) - // So the sink basically spawns a new thread to pull the events out within a transaction. - // The thread fills in the event batch object that is set before the thread is scheduled. - // After filling it in, the thread waits on a condition - which is released only - // when the success message comes back for the specific sequence number for that event batch. /** - * This class represents a transaction on the Flume channel. This class runs a separate thread - * which owns the transaction. It is blocked until the success call for that transaction comes - * back. - * @param maxBatchSize + * Get an event batch from the channel. This method will block until a batch of events is + * available from the channel. If no events are available after a large number of attempts of + * polling the channel, this method will return [[ErrorEventBatch]]. + * + * @return An [[EventBatch]] instance with sequence number set to [[seqNum]], filled with a + * maximum of [[maxBatchSize]] events */ - private class TransactionProcessor(var maxBatchSize: Int) extends Callable[Void] { - // Must be set to a new event batch before scheduling this!! - val eventBatch = new EventBatch("", new util.LinkedList[SparkSinkEvent]) - val eventQueue = new SynchronousQueue[EventBatch]() - val resultQueue = new SynchronousQueue[Boolean]() - val resultQueueUpdateLock = new ReentrantLock() - - object Zero { - val zero = "0" // Oh, I miss static finals - } + def getEventBatch: EventBatch = { + batchGeneratedLatch.await() + eventBatch + } + /** + * This method is to be called by the sink when it receives an ACK or NACK from Spark. This + * method is a no-op if it is called after [[transactionTimeout]] has expired since + * [[getEventBatch]] returned a batch of events. + * @param success True if an ACK was received and the transaction should be committed, else false. + */ + def batchProcessed(success: Boolean) { + if (LOG.isDebugEnabled) { + LOG.debug("Batch processed for sequence number: " + seqNum) + } + batchSuccess = success + batchAckLatch.countDown() + } - override def call(): Void = { - val tx = getChannel.getTransaction - tx.begin() - try { - eventBatch.setSequenceNumber(seqBase + seqNum.incrementAndGet()) - val events = eventBatch.getEvents - events.clear() + /** + * Populates events into the event batch. If the batch cannot be populated, + * this method will not set the event batch which will stay [[ErrorEventBatch]] + */ + private def populateEvents() { + try { + txOpt = Option(channel.getTransaction) + txOpt.map(tx => { + tx.begin() + val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) val loop = new Breaks var gotEventsInThisTxn = false + var loopCounter: Int = 0 loop.breakable { - while (events.size() < maxBatchSize) { - Option(getChannel.take()) match { + while (events.size() < maxBatchSize + && loopCounter < totalAttemptsToRemoveFromChannel) { + loopCounter += 1 + Option(channel.take()) match { case Some(event) => events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), ByteBuffer.wrap(event.getBody))) gotEventsInThisTxn = true case None => if (!gotEventsInThisTxn) { - Thread.sleep(500) + TimeUnit.MILLISECONDS.sleep(500) } else { loop.break() } } } } - // Make the data available to the sender thread - eventQueue.put(eventBatch) - - // Wait till timeout for the ack/nack - val maybeResult = Option(resultQueue.poll(transactionTimeout, TimeUnit.SECONDS)) - // There is a race condition here. - // 1. This times out. - // 2. The result is empty, so timeout exception is thrown. - // 3. The ack comes in before the finally block is entered - // 4. The thread with the ack has a handle to this processor, - // and another thread has the same processor checked out - // (since the finally block was executed and the processor checked back in) - // 5. The thread with the ack now updates the result queue, - // so the processor thinks it is the ack for the current batch. - // To avoid this - update the sequence number to "0" (with or without a result - does not - // matter). - // In the ack method, check if the seq number is the same as the processor's - - // if they are then update the result queue. Now if the - // processor updates the seq number first - the ack/nack never updates the result. If the - // ack/nack updates the - // result after the timeout but before the seq number is updated to "0" it does not - // matter - the processor would - // still timeout and the result is cleared before reusing the processor. - // Unfortunately, this needs to be done from within a lock - // to make sure that the new sequence number is actually visible to the ack thread - // (happens-before) - resultQueueUpdateLock.lock() + if (!gotEventsInThisTxn) { + throw new FlumeException("Tried too many times, didn't get any events from the channel") + } + // At this point, the events are available, so fill them into the event batch + eventBatch = new EventBatch(seqNum, events) + }) + } catch { + case e: Throwable => + LOG.error("Error while processing transaction.", e) try { - eventBatch.setSequenceNumber(Zero.zero) + txOpt.map(tx => { + rollbackAndClose(tx, close = true) + }) } finally { - resultQueueUpdateLock.unlock() + // Avro might serialize the exception and cause a NACK, + // so don't bother with the transaction + txOpt = None } - eventBatch.getEvents.clear() - // If the batch failed on spark side, throw a FlumeException - maybeResult.map(success => - if (!success) { - throw new - FlumeException("Spark could not accept events. The transaction will be retried.") - } - ) - // If the operation timed out, throw a TimeoutException - if (maybeResult.isEmpty) { - throw new TimeoutException("Spark did not respond within the timeout period of " + - transactionTimeout + "seconds. Transaction will be retried") - } - null - } catch { - case e: Throwable => - try { - LOG.warn("Error while attempting to remove events from the channel.", e) - tx.rollback() - } catch { - case e1: Throwable => LOG.error( - "Rollback failed while attempting to rollback due to commit failure.", e1) - } - null // No point rethrowing the exception - } finally { - // Must *always* release the caller thread - eventQueue.put(ErrorEventBatch) - // In the case of success coming after the timeout, but before resetting the seq number - // remove the event from the map and then clear the value - resultQueue.clear() - processorMap.remove(eventBatch.getSequenceNumber) - processorManager.get.checkIn(this) - tx.close() - } - } - - def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, - CharSequence] = { - val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) - charSeqMap.putAll(inMap) - charSeqMap + } finally { + batchGeneratedLatch.countDown() } } - private class TransactionProcessorManager(val maxInstances: Int) { - val queue = new scala.collection.mutable.Queue[TransactionProcessor] - val queueModificationLock = new ReentrantLock() - var currentSize = 0 - val waitForCheckIn = queueModificationLock.newCondition() - - def checkOut(n: Int): TransactionProcessor = { - def getProcessor = { - val processor = queue.dequeue() - processor.maxBatchSize = n - processor - } - queueModificationLock.lock() - try { - if (queue.size > 0) { - getProcessor - } - else { - if (currentSize < maxInstances) { - currentSize += 1 - new TransactionProcessor(n) - } else { - // No events in queue and cannot initialize more! - // Since currentSize never reduces, queue size increasing is the only hope - while (queue.size == 0 && currentSize >= maxInstances) { - waitForCheckIn.await() - } - getProcessor - } + /** + * Waits for upto [[transactionTimeout]] seconds for an ACK. If an ACK comes in, + * this method commits the transaction with the channel. If the ACK does not come in within + * that time or a NACK comes in, this method rolls back the transaction. + */ + private def processAckOrNack() { + batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) + txOpt.map(tx => { + if (batchSuccess) { + try { + tx.commit() + } catch { + case e: Throwable => + rollbackAndClose(tx, close = false) // tx will be closed later anyway + } finally { + tx.close() } - } finally { - queueModificationLock.unlock() + } else { + rollbackAndClose(tx, close = true) + // This might have been due to timeout or a NACK. Either way the following call does not + // cause issues. This is required to ensure the TransactionProcessor instance is not leaked + parent.removeAndGetProcessor(seqNum) } - } + }) + } - def checkIn(processor: TransactionProcessor) { - queueModificationLock.lock() - try { - queue.enqueue(processor) - waitForCheckIn.signal() - } finally { - queueModificationLock.unlock() + /** + * Helper method to rollback and optionally close a transaction + * @param tx The transaction to rollback + * @param close Whether the transaction should be closed or not after rolling back + */ + private def rollbackAndClose(tx: Transaction, close: Boolean) { + try { + tx.rollback() + LOG.warn("Spark was unable to successfully process the events. Transaction is being " + + "rolled back.") + } catch { + case e: Throwable => + LOG.error("Error rolling back transaction. Rollback may have failed!", e) + } finally { + if (close) { + tx.close() } } } + + /** + * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] + * @param inMap The map to be converted + * @return The converted map + */ + private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, + CharSequence] = { + val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) + charSeqMap.putAll(inMap) + charSeqMap + } + + override def call(): Void = { + populateEvents() + processAckOrNack() + null + } } +/** + * Configuration parameters and their defaults. + */ object SparkSinkConfig { - val PROCESSOR_COUNT = "processorCount" - val DEFAULT_PROCESSOR_COUNT = 10 + val THREADS = "threads" + val DEFAULT_THREADS = 10 val CONF_TRANSACTION_TIMEOUT = "timeout" val DEFAULT_TRANSACTION_TIMEOUT = 60 @@ -350,7 +428,4 @@ object SparkSinkConfig { val DEFAULT_HOSTNAME = "0.0.0.0" val CONF_PORT = "port" - - val CONF_MAX_THREADS = "maxThreads" - val DEFAULT_MAX_THREADS = 5 } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index ee337b5f5507f..324f9551287b1 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -23,7 +23,6 @@ import java.nio.ByteBuffer import java.util.concurrent.{TimeUnit, Executors} import scala.collection.JavaConversions._ -import scala.collection.mutable import scala.reflect.ClassTag import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -38,8 +37,16 @@ import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.receiver.Receiver - - +/** + * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running + * [[org.apache.spark.flume.sink.SparkSink]]s. + * @param ssc_ Streaming context that will execute this input stream + * @param addresses List of addresses at which SparkSinks are listening + * @param maxBatchSize Maximum size of a batch + * @param parallelism Number of parallel connections to open + * @param storageLevel The storage level to use. + * @tparam T Class type of the object of this stream + */ class FlumePollingInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, val addresses: Seq[InetSocketAddress], @@ -84,6 +91,7 @@ private[streaming] class FlumePollingReceiver( new FlumeConnection(transceiver, client) }).toArray + // Threads that pull data from Flume. val dataReceiver = new Runnable { override def run(): Unit = { var counter = 0 @@ -96,11 +104,14 @@ private[streaming] class FlumePollingReceiver( val events: java.util.List[SparkSinkEvent] = batch.getEvents logDebug("Received batch of " + events.size() + " events with sequence number: " + seq) try { + // Convert each Flume event to a serializable SparkPollingEvent events.foreach(event => store(SparkPollingEvent.fromSparkSinkEvent(event))) + // Send an ack to Flume so that Flume discards the events from its channels. client.ack(seq) } catch { case e: Throwable => try { + // Let Flume know that the events need to be pushed back into the channel. client.nack(seq) // If the agent is down, even this could fail and throw } catch { case e: Throwable => logError("Sending Nack also failed. A Flume agent is down.") @@ -111,6 +122,7 @@ private[streaming] class FlumePollingReceiver( } } } + // Create multiple threads and start all of them. for (i <- 0 until parallelism) { logInfo("Starting Flume Polling Receiver worker threads starting..") receiverExecutor.submit(dataReceiver) @@ -129,13 +141,18 @@ private[streaming] class FlumePollingReceiver( logInfo("Shutting down Flume Polling Receiver") receiverExecutor.shutdownNow() connections.map(connection => { - connection.tranceiver.close() + connection.transceiver.close() }) channelFactory.releaseExternalResources() } } -private class FlumeConnection(val tranceiver: NettyTransceiver, +/** + * A wrapper around the transceiver and the Avro IPC API. + * @param transceiver The transceiver to use for communication with Flume + * @param client The client that the callbacks are received on. + */ +private class FlumeConnection(val transceiver: NettyTransceiver, val client: SparkFlumeProtocol.Callback) private[streaming] object SparkPollingEvent { From 70bcc2ad5b117324652e41f0331eb974ab696966 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 9 Jun 2014 22:34:40 -0700 Subject: [PATCH 12/32] SPARK-1729. New Flume-Spark integration. Renamed the SparkPollingEvent to SparkFlumePollingEvent. --- .../flume/FlumePollingInputDStream.scala | 18 +++++++++--------- .../spark/streaming/flume/FlumeUtils.scala | 8 ++++---- .../flume/FlumePollingReceiverSuite.scala | 14 +++++++------- 3 files changed, 20 insertions(+), 20 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 324f9551287b1..dea5e0103ee26 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -53,13 +53,13 @@ class FlumePollingInputDStream[T: ClassTag]( val maxBatchSize: Int, val parallelism: Int, storageLevel: StorageLevel -) extends ReceiverInputDStream[SparkPollingEvent](ssc_) { +) extends ReceiverInputDStream[SparkFlumePollingEvent](ssc_) { /** * Gets the receiver object that will be sent to the worker nodes * to receive data. This method needs to defined by any specific implementation * of a NetworkInputDStream. */ - override def getReceiver(): Receiver[SparkPollingEvent] = { + override def getReceiver(): Receiver[SparkFlumePollingEvent] = { new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) } } @@ -69,7 +69,7 @@ private[streaming] class FlumePollingReceiver( maxBatchSize: Int, parallelism: Int, storageLevel: StorageLevel -) extends Receiver[SparkPollingEvent](storageLevel) with Logging { +) extends Receiver[SparkFlumePollingEvent](storageLevel) with Logging { lazy val channelFactoryExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). @@ -105,7 +105,7 @@ private[streaming] class FlumePollingReceiver( logDebug("Received batch of " + events.size() + " events with sequence number: " + seq) try { // Convert each Flume event to a serializable SparkPollingEvent - events.foreach(event => store(SparkPollingEvent.fromSparkSinkEvent(event))) + events.foreach(event => store(SparkFlumePollingEvent.fromSparkSinkEvent(event))) // Send an ack to Flume so that Flume discards the events from its channels. client.ack(seq) } catch { @@ -129,7 +129,7 @@ private[streaming] class FlumePollingReceiver( } } - override def store(dataItem: SparkPollingEvent) { + override def store(dataItem: SparkFlumePollingEvent) { // Not entirely sure store is thread-safe for all storage levels - so wrap it in synchronized // This takes a performance hit, since the parallelism is useful only for pulling data now. this.synchronized { @@ -155,9 +155,9 @@ private[streaming] class FlumePollingReceiver( private class FlumeConnection(val transceiver: NettyTransceiver, val client: SparkFlumeProtocol.Callback) -private[streaming] object SparkPollingEvent { - def fromSparkSinkEvent(in: SparkSinkEvent): SparkPollingEvent = { - val event = new SparkPollingEvent() +private[streaming] object SparkFlumePollingEvent { + def fromSparkSinkEvent(in: SparkSinkEvent): SparkFlumePollingEvent = { + val event = new SparkFlumePollingEvent() event.event = in event } @@ -167,7 +167,7 @@ private[streaming] object SparkPollingEvent { * SparkSinkEvent is identical to AvroFlumeEvent, we need to create a new class and a wrapper * around that to make it externalizable. */ -class SparkPollingEvent() extends Externalizable with Logging { +class SparkFlumePollingEvent() extends Externalizable with Logging { var event : SparkSinkEvent = new SparkSinkEvent() /* De-serialize from bytes. */ diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 2ff8ce2a77d2d..71bc364114f2e 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -89,8 +89,8 @@ object FlumeUtils { maxBatchSize: Int = 100, parallelism: Int = 5, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[SparkPollingEvent] = { - new FlumePollingInputDStream[SparkPollingEvent](ssc, addresses, maxBatchSize, + ): ReceiverInputDStream[SparkFlumePollingEvent] = { + new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, maxBatchSize, parallelism, storageLevel) } @@ -111,8 +111,8 @@ object FlumeUtils { maxBatchSize: Int = 100, parallelism: Int = 5, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): JavaReceiverInputDStream[SparkPollingEvent] = { - new FlumePollingInputDStream[SparkPollingEvent](ssc, addresses, maxBatchSize, + ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, maxBatchSize, parallelism, storageLevel) } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala index dc64405872a2d..87a9ae0797e00 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -42,11 +42,11 @@ class FlumePollingReceiverSuite extends TestSuiteBase { test("flume polling test") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream: ReceiverInputDStream[SparkPollingEvent] = + val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), 100, 5, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[SparkPollingEvent]] - with SynchronizedBuffer[Seq[SparkPollingEvent]] + val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] + with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) outputStream.register() @@ -73,12 +73,12 @@ class FlumePollingReceiverSuite extends TestSuiteBase { test("flume polling test multiple hosts") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream: ReceiverInputDStream[SparkPollingEvent] = + val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort), new InetSocketAddress("localhost", testPort + 1)), 100, 5, StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[SparkPollingEvent]] - with SynchronizedBuffer[Seq[SparkPollingEvent]] + val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] + with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) outputStream.register() @@ -114,7 +114,7 @@ class FlumePollingReceiverSuite extends TestSuiteBase { } def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, - outputBuffer: ArrayBuffer[Seq[SparkPollingEvent]]) { + outputBuffer: ArrayBuffer[Seq[SparkFlumePollingEvent]]) { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val executor = Executors.newCachedThreadPool() val executorCompletion = new ExecutorCompletionService[Void](executor) From 3c23c182fd8655e0f1a64cee64641f1cc803f7c2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 10 Jun 2014 16:20:40 -0700 Subject: [PATCH 13/32] SPARK-1729. New Spark-Flume integration. Minor formatting changes. --- .../scala/org/apache/spark/flume/sink/SparkSink.scala | 1 + .../streaming/flume/FlumePollingInputDStream.scala | 9 ++++++--- .../streaming/flume/FlumePollingReceiverSuite.scala | 11 +++++++---- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index e830b388c12b2..e430c0935e528 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -35,6 +35,7 @@ import org.apache.flume.{Channel, Transaction, FlumeException, Context} import org.slf4j.LoggerFactory import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} + /** * A sink that uses Avro RPC to run a server that can be polled by Spark's * FlumePollingInputDStream. This sink has the following configuration parameters: diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index dea5e0103ee26..3309b15604dd0 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -105,7 +105,9 @@ private[streaming] class FlumePollingReceiver( logDebug("Received batch of " + events.size() + " events with sequence number: " + seq) try { // Convert each Flume event to a serializable SparkPollingEvent - events.foreach(event => store(SparkFlumePollingEvent.fromSparkSinkEvent(event))) + events.foreach(event => { + store(SparkFlumePollingEvent.fromSparkSinkEvent(event)) + }) // Send an ack to Flume so that Flume discards the events from its channels. client.ack(seq) } catch { @@ -153,7 +155,7 @@ private[streaming] class FlumePollingReceiver( * @param client The client that the callbacks are received on. */ private class FlumeConnection(val transceiver: NettyTransceiver, - val client: SparkFlumeProtocol.Callback) + val client: SparkFlumeProtocol.Callback) private[streaming] object SparkFlumePollingEvent { def fromSparkSinkEvent(in: SparkSinkEvent): SparkFlumePollingEvent = { @@ -162,13 +164,14 @@ private[streaming] object SparkFlumePollingEvent { event } } + /* * Unfortunately Avro does not allow including pre-compiled classes - so even though * SparkSinkEvent is identical to AvroFlumeEvent, we need to create a new class and a wrapper * around that to make it externalizable. */ class SparkFlumePollingEvent() extends Externalizable with Logging { - var event : SparkSinkEvent = new SparkSinkEvent() + var event: SparkSinkEvent = new SparkSinkEvent() /* De-serialize from bytes. */ def readExternal(in: ObjectInput) { diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala index 87a9ae0797e00..7e6fe66052138 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -114,18 +114,19 @@ class FlumePollingReceiverSuite extends TestSuiteBase { } def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, - outputBuffer: ArrayBuffer[Seq[SparkFlumePollingEvent]]) { + outputBuffer: ArrayBuffer[Seq[SparkFlumePollingEvent]]) { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val executor = Executors.newCachedThreadPool() val executorCompletion = new ExecutorCompletionService[Void](executor) channels.map(channel => { executorCompletion.submit(new TxnSubmitter(channel, clock)) }) - for(i <- 0 until channels.size) { + for (i <- 0 until channels.size) { executorCompletion.take() } val startTime = System.currentTimeMillis() - while (outputBuffer.size < 5 * channels.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + while (outputBuffer.size < 5 * channels.size && + System.currentTimeMillis() - startTime < maxWaitTimeMillis) { logInfo("output.size = " + outputBuffer.size) Thread.sleep(100) } @@ -164,7 +165,8 @@ class FlumePollingReceiverSuite extends TestSuiteBase { val tx = channel.getTransaction tx.begin() for (j <- 0 until 5) { - channel.put(EventBuilder.withBody((channel.getName + " - " + String.valueOf(t)).getBytes("utf-8"), + channel.put(EventBuilder.withBody((channel.getName + " - " + String.valueOf(t)).getBytes( + "utf-8"), Map[String, String]("test-" + t.toString -> "header"))) t += 1 } @@ -176,4 +178,5 @@ class FlumePollingReceiverSuite extends TestSuiteBase { null } } + } From 0d69604ae319610b9fde1b3a77fd8130f70b4ec2 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 16 Jun 2014 12:44:12 -0700 Subject: [PATCH 14/32] FLUME-1729. Better Flume-Spark integration. Use readFully instead of read in EventTransformer. --- .../org/apache/spark/streaming/flume/EventTransformer.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala index 75b224afca39b..069a9a215675c 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -33,7 +33,7 @@ object EventTransformer extends Logging { Array[Byte]) = { val bodyLength = in.readInt() val bodyBuff = new Array[Byte](bodyLength) - in.read(bodyBuff) + in.readFully(bodyBuff) val numHeaders = in.readInt() val headers = new java.util.HashMap[CharSequence, CharSequence] @@ -41,12 +41,12 @@ object EventTransformer extends Logging { for (i <- 0 until numHeaders) { val keyLength = in.readInt() val keyBuff = new Array[Byte](keyLength) - in.read(keyBuff) + in.readFully(keyBuff) val key: String = Utils.deserialize(keyBuff) val valLength = in.readInt() val valBuff = new Array[Byte](valLength) - in.read(valBuff) + in.readFully(valBuff) val value: String = Utils.deserialize(valBuff) headers.put(key, value) From bda01fc18daae511603a526ca5fcd2ada97a3de4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 17 Jun 2014 15:15:36 -0700 Subject: [PATCH 15/32] FLUME-1729. Flume-Spark integration. Refactoring classes into new files and minor changes in protocol. --- .../apache/spark/flume/ErrorEventBatch.scala | 28 ++ .../flume/sink/SparkAvroCallbackHandler.scala | 118 +++++++ .../apache/spark/flume/sink/SparkSink.scala | 324 +----------------- .../flume/sink/TransactionProcessor.scala | 226 ++++++++++++ .../flume/FlumePollingInputDStream.scala | 44 ++- 5 files changed, 415 insertions(+), 325 deletions(-) create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala new file mode 100644 index 0000000000000..9bee61c71304c --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala @@ -0,0 +1,28 @@ +/* + * 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.flume + +import java.util + +/** + * Object representing an empty batch returned by the txn processor due to some error. + */ +case class ErrorEventBatch(var message: String) extends EventBatch { + // Make sure the internal data structures are initialized with non-null values. + setEvents(util.Collections.emptyList()) + setSequenceNumber("") +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala new file mode 100644 index 0000000000000..6a7c591455d5f --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala @@ -0,0 +1,118 @@ +/* + * 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.flume.sink + +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.{ConcurrentHashMap, Executors} + +import com.google.common.util.concurrent.ThreadFactoryBuilder + +import org.apache.commons.lang.RandomStringUtils +import org.apache.flume.Channel +import org.apache.spark.flume.{EventBatch, SparkFlumeProtocol} +import org.slf4j.LoggerFactory + +/** + * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process + * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. + * @param threads Number of threads to use to process requests. + * @param channel The channel that the sink pulls events from + * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark + * is rolled back. + */ +private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, + val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol { + private val LOG = LoggerFactory.getLogger(classOf[SparkAvroCallbackHandler]) + val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("Spark Sink Processor Thread - %d").build())) + private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() + // This sink will not persist sequence numbers and reuses them if it gets restarted. + // So it is possible to commit a transaction which may have been meant for the sink before the + // restart. + // Since the new txn may not have the same sequence number we must guard against accidentally + // committing a new transaction. To reduce the probability of that happening a random string is + // prepended to the sequence number. Does not change for life of sink + private val seqBase = RandomStringUtils.randomAlphanumeric(8) + private val seqCounter = new AtomicLong(0) + + /** + * Returns a bunch of events to Spark over Avro RPC. + * @param n Maximum number of events to return in a batch + * @return [[EventBatch]] instance that has a sequence number and an array of at most n events + */ + override def getEventBatch(n: Int): EventBatch = { + val sequenceNumber = seqBase + seqCounter.incrementAndGet() + val processor = new TransactionProcessor(channel, sequenceNumber, + n, transactionTimeout, backOffInterval, this) + transactionExecutorOpt.map(executor => { + executor.submit(processor) + }) + // Wait until a batch is available - will be an error if + processor.getEventBatch + } + + /** + * Called by Spark to indicate successful commit of a batch + * @param sequenceNumber The sequence number of the event batch that was successful + */ + override def ack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = true) + null + } + + /** + * Called by Spark to indicate failed commit of a batch + * @param sequenceNumber The sequence number of the event batch that failed + * @return + */ + override def nack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = false) + LOG.info("Spark failed to commit transaction. Will reattempt events.") + null + } + + /** + * Helper method to commit or rollback a transaction. + * @param sequenceNumber The sequence number of the batch that was completed + * @param success Whether the batch was successful or not. + */ + private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { + Option(removeAndGetProcessor(sequenceNumber)).map(processor => { + processor.batchProcessed(success) + }) + } + + /** + * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. + * @param sequenceNumber + * @return The transaction processor for the corresponding batch. Note that this instance is no + * longer tracked and the caller is responsible for that txn processor. + */ + private[flume] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { + processorMap.remove(sequenceNumber.toString) // The toString is required! + } + + /** + * Shuts down the executor used to process transactions. + */ + def shutdown() { + transactionExecutorOpt.map(executor => { + executor.shutdownNow() + }) + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index e430c0935e528..cf968ee39435f 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -17,24 +17,17 @@ package org.apache.spark.flume.sink import java.net.InetSocketAddress -import java.nio.ByteBuffer -import java.util import java.util.concurrent._ -import java.util.concurrent.atomic.AtomicLong -import scala.util.control.Breaks - -import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.avro.ipc.NettyServer import org.apache.avro.ipc.specific.SpecificResponder -import org.apache.commons.lang.RandomStringUtils import org.apache.flume.Sink.Status -import org.apache.flume.conf.{ConfigurationException, Configurable} +import org.apache.flume.conf.{Configurable, ConfigurationException} import org.apache.flume.sink.AbstractSink -import org.apache.flume.{Channel, Transaction, FlumeException, Context} +import org.apache.flume.Context import org.slf4j.LoggerFactory -import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} +import org.apache.spark.flume.SparkFlumeProtocol /** * A sink that uses Avro RPC to run a server that can be polled by Spark's @@ -57,6 +50,7 @@ import org.apache.spark.flume.{SparkSinkEvent, EventBatch, SparkFlumeProtocol} // at which point the transaction is committed or rolled back. class SparkSink extends AbstractSink with Configurable { + private val LOG = LoggerFactory.getLogger(classOf[SparkSink]) // Size of the pool to use for holding transaction processors. private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS @@ -68,6 +62,8 @@ class SparkSink extends AbstractSink with Configurable { private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME private var port: Int = 0 + private var backOffInterval: Int = 200 + // Handle to the server private var serverOpt: Option[NettyServer] = None @@ -78,23 +74,30 @@ class SparkSink extends AbstractSink with Configurable { private val blockingLatch = new CountDownLatch(1) override def start() { - handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout)) + LOG.info("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + + hostname + " with " + "pool size: " + poolSize + " and transaction timeout: " + + transactionTimeout + ".") + handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout, + backOffInterval)) val responder = new SpecificResponder(classOf[SparkFlumeProtocol], handler.get) // Using the constructor that takes specific thread-pools requires bringing in netty // dependencies which are being excluded in the build. In practice, // Netty dependencies are already available on the JVM as Flume would have pulled them in. serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) serverOpt.map(server => { + LOG.info("Starting Avro server for sink: " + getName) server.start() }) super.start() } override def stop() { + LOG.info("Stopping Spark Sink: " + getName) handler.map(callbackHandler => { callbackHandler.shutdown() }) serverOpt.map(server => { + LOG.info("Stopping Avro Server for sink: " + getName) server.close() server.join() }) @@ -102,9 +105,6 @@ class SparkSink extends AbstractSink with Configurable { super.stop() } - /** - * @param ctx - */ override def configure(ctx: Context) { import SparkSinkConfig._ hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) @@ -112,6 +112,7 @@ class SparkSink extends AbstractSink with Configurable { getOrElse(throw new ConfigurationException("The port to bind to must be specified")) poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) + backOffInterval = ctx.getInteger(CONF_BACKOFF_INTERVAL, DEFAULT_BACKOFF_INTERVAL) } override def process(): Status = { @@ -123,298 +124,6 @@ class SparkSink extends AbstractSink with Configurable { } } -/** - * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process - * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. - * @param threads Number of threads to use to process requests. - * @param channel The channel that the sink pulls events from - * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark - * is rolled back. - */ -private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, - val transactionTimeout: Int) extends SparkFlumeProtocol { - private val LOG = LoggerFactory.getLogger(classOf[SparkAvroCallbackHandler]) - val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat("Spark Sink Processor Thread - %d").build())) - private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() - // This sink will not persist sequence numbers and reuses them if it gets restarted. - // So it is possible to commit a transaction which may have been meant for the sink before the - // restart. - // Since the new txn may not have the same sequence number we must guard against accidentally - // committing a new transaction. To reduce the probability of that happening a random string is - // prepended to the sequence number. Does not change for life of sink - private val seqBase = RandomStringUtils.randomAlphanumeric(8) - private val seqCounter = new AtomicLong(0) - - /** - * Returns a bunch of events to Spark over Avro RPC. - * @param n Maximum number of events to return in a batch - * @return [[EventBatch]] instance that has a sequence number and an array of at most n events - */ - override def getEventBatch(n: Int): EventBatch = { - val sequenceNumber = seqBase + seqCounter.incrementAndGet() - val processor = new TransactionProcessor(channel, sequenceNumber, - n, transactionTimeout, this) - transactionExecutorOpt.map(executor => { - executor.submit(processor) - }) - // Wait until a batch is available - can be null if some error was thrown - processor.getEventBatch match { - case ErrorEventBatch => throw new FlumeException("Something went wrong. No events" + - " retrieved from channel.") - case eventBatch: EventBatch => - processorMap.put(sequenceNumber, processor) - if (LOG.isDebugEnabled()) { - LOG.debug("Sent " + eventBatch.getEvents.size() + - " events with sequence number: " + eventBatch.getSequenceNumber) - } - eventBatch - } - } - - /** - * Called by Spark to indicate successful commit of a batch - * @param sequenceNumber The sequence number of the event batch that was successful - */ - override def ack(sequenceNumber: CharSequence): Void = { - completeTransaction(sequenceNumber, success = true) - null - } - - /** - * Called by Spark to indicate failed commit of a batch - * @param sequenceNumber The sequence number of the event batch that failed - * @return - */ - override def nack(sequenceNumber: CharSequence): Void = { - completeTransaction(sequenceNumber, success = false) - LOG.info("Spark failed to commit transaction. Will reattempt events.") - null - } - - /** - * Helper method to commit or rollback a transaction. - * @param sequenceNumber The sequence number of the batch that was completed - * @param success Whether the batch was successful or not. - */ - private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - Option(removeAndGetProcessor(sequenceNumber)).map(processor => { - processor.batchProcessed(success) - }) - } - - /** - * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. - * @param sequenceNumber - * @return The transaction processor for the corresponding batch. Note that this instance is no - * longer tracked and the caller is responsible for that txn processor. - */ - private[flume] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { - processorMap.remove(sequenceNumber.toString) // The toString is required! - } - - /** - * Shuts down the executor used to process transactions. - */ - def shutdown() { - transactionExecutorOpt.map(executor => { - executor.shutdownNow() - }) - } -} - -/** - * Object representing an empty batch returned by the txn processor due to some error. - */ -case object ErrorEventBatch extends EventBatch - -// Flume forces transactions to be thread-local (horrible, I know!) -// So the sink basically spawns a new thread to pull the events out within a transaction. -// The thread fills in the event batch object that is set before the thread is scheduled. -// After filling it in, the thread waits on a condition - which is released only -// when the success message comes back for the specific sequence number for that event batch. -/** - * This class represents a transaction on the Flume channel. This class runs a separate thread - * which owns the transaction. The thread is blocked until the success call for that transaction - * comes back with an ACK or NACK. - * @param channel The channel from which to pull events - * @param seqNum The sequence number to use for the transaction. Must be unique - * @param maxBatchSize The maximum number of events to process per batch - * @param transactionTimeout Time in seconds after which a transaction must be rolled back - * without waiting for an ACK from Spark - * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts - */ -private class TransactionProcessor(val channel: Channel, val seqNum: String, - var maxBatchSize: Int, val transactionTimeout: Int, - val parent: SparkAvroCallbackHandler) extends Callable[Void] { - - private val LOG = LoggerFactory.getLogger(classOf[TransactionProcessor]) - - // If a real batch is not returned, we always have to return an error batch. - @volatile private var eventBatch: EventBatch = ErrorEventBatch - - // Synchronization primitives - val batchGeneratedLatch = new CountDownLatch(1) - val batchAckLatch = new CountDownLatch(1) - - // Sanity check to ensure we don't loop like crazy - val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 - - // OK to use volatile, since the change would only make this true (otherwise it will be - // changed to false - we never apply a negation operation to this) - which means the transaction - // succeeded. - @volatile private var batchSuccess = false - - // The transaction that this processor would handle - var txOpt: Option[Transaction] = None - - /** - * Get an event batch from the channel. This method will block until a batch of events is - * available from the channel. If no events are available after a large number of attempts of - * polling the channel, this method will return [[ErrorEventBatch]]. - * - * @return An [[EventBatch]] instance with sequence number set to [[seqNum]], filled with a - * maximum of [[maxBatchSize]] events - */ - def getEventBatch: EventBatch = { - batchGeneratedLatch.await() - eventBatch - } - - /** - * This method is to be called by the sink when it receives an ACK or NACK from Spark. This - * method is a no-op if it is called after [[transactionTimeout]] has expired since - * [[getEventBatch]] returned a batch of events. - * @param success True if an ACK was received and the transaction should be committed, else false. - */ - def batchProcessed(success: Boolean) { - if (LOG.isDebugEnabled) { - LOG.debug("Batch processed for sequence number: " + seqNum) - } - batchSuccess = success - batchAckLatch.countDown() - } - - /** - * Populates events into the event batch. If the batch cannot be populated, - * this method will not set the event batch which will stay [[ErrorEventBatch]] - */ - private def populateEvents() { - try { - txOpt = Option(channel.getTransaction) - txOpt.map(tx => { - tx.begin() - val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) - val loop = new Breaks - var gotEventsInThisTxn = false - var loopCounter: Int = 0 - loop.breakable { - while (events.size() < maxBatchSize - && loopCounter < totalAttemptsToRemoveFromChannel) { - loopCounter += 1 - Option(channel.take()) match { - case Some(event) => - events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), - ByteBuffer.wrap(event.getBody))) - gotEventsInThisTxn = true - case None => - if (!gotEventsInThisTxn) { - TimeUnit.MILLISECONDS.sleep(500) - } else { - loop.break() - } - } - } - } - if (!gotEventsInThisTxn) { - throw new FlumeException("Tried too many times, didn't get any events from the channel") - } - // At this point, the events are available, so fill them into the event batch - eventBatch = new EventBatch(seqNum, events) - }) - } catch { - case e: Throwable => - LOG.error("Error while processing transaction.", e) - try { - txOpt.map(tx => { - rollbackAndClose(tx, close = true) - }) - } finally { - // Avro might serialize the exception and cause a NACK, - // so don't bother with the transaction - txOpt = None - } - } finally { - batchGeneratedLatch.countDown() - } - } - - /** - * Waits for upto [[transactionTimeout]] seconds for an ACK. If an ACK comes in, - * this method commits the transaction with the channel. If the ACK does not come in within - * that time or a NACK comes in, this method rolls back the transaction. - */ - private def processAckOrNack() { - batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) - txOpt.map(tx => { - if (batchSuccess) { - try { - tx.commit() - } catch { - case e: Throwable => - rollbackAndClose(tx, close = false) // tx will be closed later anyway - } finally { - tx.close() - } - } else { - rollbackAndClose(tx, close = true) - // This might have been due to timeout or a NACK. Either way the following call does not - // cause issues. This is required to ensure the TransactionProcessor instance is not leaked - parent.removeAndGetProcessor(seqNum) - } - }) - } - - /** - * Helper method to rollback and optionally close a transaction - * @param tx The transaction to rollback - * @param close Whether the transaction should be closed or not after rolling back - */ - private def rollbackAndClose(tx: Transaction, close: Boolean) { - try { - tx.rollback() - LOG.warn("Spark was unable to successfully process the events. Transaction is being " + - "rolled back.") - } catch { - case e: Throwable => - LOG.error("Error rolling back transaction. Rollback may have failed!", e) - } finally { - if (close) { - tx.close() - } - } - } - - /** - * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] - * @param inMap The map to be converted - * @return The converted map - */ - private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, - CharSequence] = { - val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) - charSeqMap.putAll(inMap) - charSeqMap - } - - override def call(): Void = { - populateEvents() - processAckOrNack() - null - } -} - /** * Configuration parameters and their defaults. */ @@ -429,4 +138,7 @@ object SparkSinkConfig { val DEFAULT_HOSTNAME = "0.0.0.0" val CONF_PORT = "port" + + val CONF_BACKOFF_INTERVAL = "backoffInterval" + val DEFAULT_BACKOFF_INTERVAL = 200 } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala new file mode 100644 index 0000000000000..b1499cdbf54e1 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala @@ -0,0 +1,226 @@ +/* + * 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.flume.sink + +import java.nio.ByteBuffer +import java.util +import java.util.concurrent.{TimeUnit, CountDownLatch, Callable} + +import scala.util.control.Breaks + +import org.apache.flume.{Transaction, Channel} +import org.apache.spark.flume.{SparkSinkEvent, ErrorEventBatch, EventBatch} +import org.slf4j.LoggerFactory + + +// Flume forces transactions to be thread-local (horrible, I know!) +// So the sink basically spawns a new thread to pull the events out within a transaction. +// The thread fills in the event batch object that is set before the thread is scheduled. +// After filling it in, the thread waits on a condition - which is released only +// when the success message comes back for the specific sequence number for that event batch. +/** + * This class represents a transaction on the Flume channel. This class runs a separate thread + * which owns the transaction. The thread is blocked until the success call for that transaction + * comes back with an ACK or NACK. + * @param channel The channel from which to pull events + * @param seqNum The sequence number to use for the transaction. Must be unique + * @param maxBatchSize The maximum number of events to process per batch + * @param transactionTimeout Time in seconds after which a transaction must be rolled back + * without waiting for an ACK from Spark + * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts + */ +private class TransactionProcessor(val channel: Channel, val seqNum: String, + var maxBatchSize: Int, val transactionTimeout: Int, val backOffInterval: Int, + val parent: SparkAvroCallbackHandler) extends Callable[Void] { + + private val LOG = LoggerFactory.getLogger(classOf[TransactionProcessor]) + + // If a real batch is not returned, we always have to return an error batch. + @volatile private var eventBatch: EventBatch = new ErrorEventBatch("Unknown Error") + + // Synchronization primitives + val batchGeneratedLatch = new CountDownLatch(1) + val batchAckLatch = new CountDownLatch(1) + + // Sanity check to ensure we don't loop like crazy + val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 + + // OK to use volatile, since the change would only make this true (otherwise it will be + // changed to false - we never apply a negation operation to this) - which means the transaction + // succeeded. + @volatile private var batchSuccess = false + + // The transaction that this processor would handle + var txOpt: Option[Transaction] = None + + /** + * Get an event batch from the channel. This method will block until a batch of events is + * available from the channel. If no events are available after a large number of attempts of + * polling the channel, this method will return [[ErrorEventBatch]]. + * + * @return An [[EventBatch]] instance with sequence number set to seqNum, filled with a + * maximum of maxBatchSize events + */ + def getEventBatch: EventBatch = { + batchGeneratedLatch.await() + eventBatch + } + + /** + * This method is to be called by the sink when it receives an ACK or NACK from Spark. This + * method is a no-op if it is called after transactionTimeout has expired since + * getEventBatch returned a batch of events. + * @param success True if an ACK was received and the transaction should be committed, else false. + */ + def batchProcessed(success: Boolean) { + if (LOG.isDebugEnabled) { + LOG.debug("Batch processed for sequence number: " + seqNum) + } + batchSuccess = success + batchAckLatch.countDown() + } + + /** + * Populates events into the event batch. If the batch cannot be populated, + * this method will not set the event batch which will stay [[ErrorEventBatch]] + */ + private def populateEvents() { + try { + txOpt = Option(channel.getTransaction) + if(txOpt.isEmpty) { + assert(eventBatch.isInstanceOf[ErrorEventBatch]) + eventBatch.asInstanceOf[ErrorEventBatch].message = "Something went wrong. Channel was " + + "unable to create a transaction!" + eventBatch + } + txOpt.map(tx => { + tx.begin() + val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) + val loop = new Breaks + var gotEventsInThisTxn = false + var loopCounter: Int = 0 + loop.breakable { + while (events.size() < maxBatchSize + && loopCounter < totalAttemptsToRemoveFromChannel) { + loopCounter += 1 + Option(channel.take()) match { + case Some(event) => + events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), + ByteBuffer.wrap(event.getBody))) + gotEventsInThisTxn = true + case None => + if (!gotEventsInThisTxn) { + TimeUnit.MILLISECONDS.sleep(backOffInterval) + } else { + loop.break() + } + } + } + } + if (!gotEventsInThisTxn) { + val msg = "Tried several times, " + + "but did not get any events from the channel!" + LOG.warn(msg) + eventBatch.asInstanceOf[ErrorEventBatch].message = msg + } else { + // At this point, the events are available, so fill them into the event batch + eventBatch = new EventBatch(seqNum, events) + } + }) + } catch { + case e: Exception => + LOG.error("Error while processing transaction.", e) + eventBatch.asInstanceOf[ErrorEventBatch].message = e.getMessage + try { + txOpt.map(tx => { + rollbackAndClose(tx, close = true) + }) + } finally { + txOpt = None + } + } finally { + batchGeneratedLatch.countDown() + } + } + + /** + * Waits for upto transactionTimeout seconds for an ACK. If an ACK comes in + * this method commits the transaction with the channel. If the ACK does not come in within + * that time or a NACK comes in, this method rolls back the transaction. + */ + private def processAckOrNack() { + batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) + txOpt.map(tx => { + if (batchSuccess) { + try { + tx.commit() + } catch { + case e: Exception => + LOG.warn("Error while attempting to commit transaction. Transaction will be rolled " + + "back", e) + rollbackAndClose(tx, close = false) // tx will be closed later anyway + } finally { + tx.close() + } + } else { + LOG.warn("Spark could not commit transaction, NACK received. Rolling back transaction.") + rollbackAndClose(tx, close = true) + // This might have been due to timeout or a NACK. Either way the following call does not + // cause issues. This is required to ensure the TransactionProcessor instance is not leaked + parent.removeAndGetProcessor(seqNum) + } + }) + } + + /** + * Helper method to rollback and optionally close a transaction + * @param tx The transaction to rollback + * @param close Whether the transaction should be closed or not after rolling back + */ + private def rollbackAndClose(tx: Transaction, close: Boolean) { + try { + LOG.warn("Spark was unable to successfully process the events. Transaction is being " + + "rolled back.") + tx.rollback() + } catch { + case e: Exception => + LOG.error("Error rolling back transaction. Rollback may have failed!", e) + } finally { + if (close) { + tx.close() + } + } + } + + /** + * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] + * @param inMap The map to be converted + * @return The converted map + */ + private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, + CharSequence] = { + val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) + charSeqMap.putAll(inMap) + charSeqMap + } + + override def call(): Void = { + populateEvents() + processAckOrNack() + null + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 3309b15604dd0..90a0b7113e2f7 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -31,7 +31,7 @@ import org.apache.avro.ipc.specific.SpecificRequestor import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.apache.spark.Logging -import org.apache.spark.flume.{SparkSinkEvent, SparkFlumeProtocol} +import org.apache.spark.flume.{EventBatch, ErrorEventBatch, SparkSinkEvent, SparkFlumeProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.ReceiverInputDStream @@ -99,27 +99,33 @@ private[streaming] class FlumePollingReceiver( counter = counter % connections.size val client = connections(counter).client counter += 1 - val batch = client.getEventBatch(maxBatchSize) - val seq = batch.getSequenceNumber - val events: java.util.List[SparkSinkEvent] = batch.getEvents - logDebug("Received batch of " + events.size() + " events with sequence number: " + seq) - try { - // Convert each Flume event to a serializable SparkPollingEvent - events.foreach(event => { - store(SparkFlumePollingEvent.fromSparkSinkEvent(event)) - }) - // Send an ack to Flume so that Flume discards the events from its channels. - client.ack(seq) - } catch { - case e: Throwable => + client.getEventBatch(maxBatchSize) match { + case errorBatch: ErrorEventBatch => + logWarning("Error Event Batch received from Spark Sink. " + errorBatch.message) + case batch: EventBatch => + val seq = batch.getSequenceNumber + val events: java.util.List[SparkSinkEvent] = batch.getEvents + logDebug( + "Received batch of " + events.size() + " events with sequence number: " + seq) try { - // Let Flume know that the events need to be pushed back into the channel. - client.nack(seq) // If the agent is down, even this could fail and throw + // Convert each Flume event to a serializable SparkPollingEvent + events.foreach(event => { + store(SparkFlumePollingEvent.fromSparkSinkEvent(event)) + }) + // Send an ack to Flume so that Flume discards the events from its channels. + client.ack(seq) } catch { - case e: Throwable => logError("Sending Nack also failed. A Flume agent is down.") + case e: Throwable => + try { + // Let Flume know that the events need to be pushed back into the channel. + client.nack(seq) // If the agent is down, even this could fail and throw + } catch { + case e: Throwable => logError( + "Sending Nack also failed. A Flume agent is down.") + } + TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. + logWarning("Error while attempting to store events", e) } - TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. - logWarning("Error while attempting to store events", e) } } } From 4b0c7fcdf654023f56d3e85b8d52ee1d049d8c65 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 17 Jun 2014 22:47:49 -0700 Subject: [PATCH 16/32] FLUME-1729. New Flume-Spark integration. Avro does not support inheritance, so the error message needs to be part of the message itself. --- .../flume-sink/src/main/avro/sparkflume.avdl | 1 + .../apache/spark/flume/ErrorEventBatch.scala | 28 --------- .../flume/sink/TransactionProcessor.scala | 21 ++++--- .../flume/FlumePollingInputDStream.scala | 59 ++++++++++--------- 4 files changed, 42 insertions(+), 67 deletions(-) delete mode 100644 external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl index fa00b2310a17b..f8edd92f67e3e 100644 --- a/external/flume-sink/src/main/avro/sparkflume.avdl +++ b/external/flume-sink/src/main/avro/sparkflume.avdl @@ -27,6 +27,7 @@ protocol SparkFlumeProtocol { } record EventBatch { + string errorMsg = ""; // If this is empty it is a valid message, else it represents an error string sequenceNumber; array events; } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala deleted file mode 100644 index 9bee61c71304c..0000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/ErrorEventBatch.scala +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.flume - -import java.util - -/** - * Object representing an empty batch returned by the txn processor due to some error. - */ -case class ErrorEventBatch(var message: String) extends EventBatch { - // Make sure the internal data structures are initialized with non-null values. - setEvents(util.Collections.emptyList()) - setSequenceNumber("") -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala index b1499cdbf54e1..8c4860d364aee 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala @@ -23,7 +23,7 @@ import java.util.concurrent.{TimeUnit, CountDownLatch, Callable} import scala.util.control.Breaks import org.apache.flume.{Transaction, Channel} -import org.apache.spark.flume.{SparkSinkEvent, ErrorEventBatch, EventBatch} +import org.apache.spark.flume.{SparkSinkEvent, EventBatch} import org.slf4j.LoggerFactory @@ -50,7 +50,8 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, private val LOG = LoggerFactory.getLogger(classOf[TransactionProcessor]) // If a real batch is not returned, we always have to return an error batch. - @volatile private var eventBatch: EventBatch = new ErrorEventBatch("Unknown Error") + @volatile private var eventBatch: EventBatch = new EventBatch("Unknown Error", "", + util.Collections.emptyList()) // Synchronization primitives val batchGeneratedLatch = new CountDownLatch(1) @@ -70,7 +71,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, /** * Get an event batch from the channel. This method will block until a batch of events is * available from the channel. If no events are available after a large number of attempts of - * polling the channel, this method will return [[ErrorEventBatch]]. + * polling the channel, this method will return an [[EventBatch]] with a non-empty error message * * @return An [[EventBatch]] instance with sequence number set to seqNum, filled with a * maximum of maxBatchSize events @@ -96,16 +97,14 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, /** * Populates events into the event batch. If the batch cannot be populated, - * this method will not set the event batch which will stay [[ErrorEventBatch]] + * this method will not set the events into the event batch, but it sets an error message. */ private def populateEvents() { try { txOpt = Option(channel.getTransaction) if(txOpt.isEmpty) { - assert(eventBatch.isInstanceOf[ErrorEventBatch]) - eventBatch.asInstanceOf[ErrorEventBatch].message = "Something went wrong. Channel was " + - "unable to create a transaction!" - eventBatch + eventBatch.setErrorMsg("Something went wrong. Channel was " + + "unable to create a transaction!") } txOpt.map(tx => { tx.begin() @@ -135,16 +134,16 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, val msg = "Tried several times, " + "but did not get any events from the channel!" LOG.warn(msg) - eventBatch.asInstanceOf[ErrorEventBatch].message = msg + eventBatch.setErrorMsg(msg) } else { // At this point, the events are available, so fill them into the event batch - eventBatch = new EventBatch(seqNum, events) + eventBatch = new EventBatch("",seqNum, events) } }) } catch { case e: Exception => LOG.error("Error while processing transaction.", e) - eventBatch.asInstanceOf[ErrorEventBatch].message = e.getMessage + eventBatch.setErrorMsg(e.getMessage) try { txOpt.map(tx => { rollbackAndClose(tx, close = true) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 90a0b7113e2f7..ff6a5b5ce1d04 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -31,7 +31,7 @@ import org.apache.avro.ipc.specific.SpecificRequestor import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.apache.spark.Logging -import org.apache.spark.flume.{EventBatch, ErrorEventBatch, SparkSinkEvent, SparkFlumeProtocol} +import org.apache.spark.flume.{EventBatch, SparkSinkEvent, SparkFlumeProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.ReceiverInputDStream @@ -99,37 +99,40 @@ private[streaming] class FlumePollingReceiver( counter = counter % connections.size val client = connections(counter).client counter += 1 - client.getEventBatch(maxBatchSize) match { - case errorBatch: ErrorEventBatch => - logWarning("Error Event Batch received from Spark Sink. " + errorBatch.message) - case batch: EventBatch => - val seq = batch.getSequenceNumber - val events: java.util.List[SparkSinkEvent] = batch.getEvents - logDebug( - "Received batch of " + events.size() + " events with sequence number: " + seq) - try { - // Convert each Flume event to a serializable SparkPollingEvent - events.foreach(event => { - store(SparkFlumePollingEvent.fromSparkSinkEvent(event)) - }) - // Send an ack to Flume so that Flume discards the events from its channels. - client.ack(seq) - } catch { - case e: Throwable => - try { - // Let Flume know that the events need to be pushed back into the channel. - client.nack(seq) // If the agent is down, even this could fail and throw - } catch { - case e: Throwable => logError( - "Sending Nack also failed. A Flume agent is down.") - } - TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. - logWarning("Error while attempting to store events", e) - } + val eventBatch = client.getEventBatch(maxBatchSize) + val errorMsg = eventBatch.getErrorMsg + if (errorMsg.toString.equals("")) { // No error, proceed with processing data + val seq = eventBatch.getSequenceNumber + val events: java.util.List[SparkSinkEvent] = eventBatch.getEvents + logDebug( + "Received batch of " + events.size() + " events with sequence number: " + seq) + try { + // Convert each Flume event to a serializable SparkPollingEvent + events.foreach(event => { + store(SparkFlumePollingEvent.fromSparkSinkEvent(event)) + }) + // Send an ack to Flume so that Flume discards the events from its channels. + client.ack(seq) + } catch { + case e: Exception => + try { + // Let Flume know that the events need to be pushed back into the channel. + client.nack(seq) // If the agent is down, even this could fail and throw + } catch { + case e: Exception => logError( + "Sending Nack also failed. A Flume agent is down.") + } + TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. + logWarning("Error while attempting to store events", e) + } + } else { + logWarning("Did not receive events from Flume agent due to error on the Flume agent: " + + "" + errorMsg.toString) } } } } + // Create multiple threads and start all of them. for (i <- 0 until parallelism) { logInfo("Starting Flume Polling Receiver worker threads starting..") From 8136aa60068e9a526403453c525afb2641acf1df Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 14 Jul 2014 16:12:12 -0700 Subject: [PATCH 17/32] Adding TransactionProcessor to map on returning batch of data --- .../org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala index 6a7c591455d5f..1f6e60815dd06 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala @@ -62,6 +62,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, transactionExecutorOpt.map(executor => { executor.submit(processor) }) + processorMap.put(sequenceNumber, processor) // Wait until a batch is available - will be an error if processor.getEventBatch } From 9fd0da7885fb778edb9a1983d4922d304052fc95 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 14 Jul 2014 16:29:33 -0700 Subject: [PATCH 18/32] SPARK-1729. Use foreach instead of map for all Options. --- .../flume/sink/SparkAvroCallbackHandler.scala | 16 ++++++++++------ .../org/apache/spark/flume/sink/SparkSink.scala | 6 +++--- .../spark/flume/sink/TransactionProcessor.scala | 6 +++--- .../flume/FlumePollingInputDStream.scala | 2 +- 4 files changed, 17 insertions(+), 13 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala index 1f6e60815dd06..11d4805657803 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala @@ -59,12 +59,16 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, val sequenceNumber = seqBase + seqCounter.incrementAndGet() val processor = new TransactionProcessor(channel, sequenceNumber, n, transactionTimeout, backOffInterval, this) - transactionExecutorOpt.map(executor => { + transactionExecutorOpt.foreach(executor => { executor.submit(processor) }) - processorMap.put(sequenceNumber, processor) - // Wait until a batch is available - will be an error if - processor.getEventBatch + // Wait until a batch is available - will be an error if error message is non-empty + val batch = processor.getEventBatch + if (batch.getErrorMsg != null && !batch.getErrorMsg.equals("")) { + processorMap.put(sequenceNumber, processor) + } + + batch } /** @@ -93,7 +97,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, * @param success Whether the batch was successful or not. */ private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - Option(removeAndGetProcessor(sequenceNumber)).map(processor => { + Option(removeAndGetProcessor(sequenceNumber)).foreach(processor => { processor.batchProcessed(success) }) } @@ -112,7 +116,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, * Shuts down the executor used to process transactions. */ def shutdown() { - transactionExecutorOpt.map(executor => { + transactionExecutorOpt.foreach(executor => { executor.shutdownNow() }) } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index cf968ee39435f..265b37dd0b302 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -84,7 +84,7 @@ class SparkSink extends AbstractSink with Configurable { // dependencies which are being excluded in the build. In practice, // Netty dependencies are already available on the JVM as Flume would have pulled them in. serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) - serverOpt.map(server => { + serverOpt.foreach(server => { LOG.info("Starting Avro server for sink: " + getName) server.start() }) @@ -93,10 +93,10 @@ class SparkSink extends AbstractSink with Configurable { override def stop() { LOG.info("Stopping Spark Sink: " + getName) - handler.map(callbackHandler => { + handler.foreach(callbackHandler => { callbackHandler.shutdown() }) - serverOpt.map(server => { + serverOpt.foreach(server => { LOG.info("Stopping Avro Server for sink: " + getName) server.close() server.join() diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala index 8c4860d364aee..a4689cca5624a 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala @@ -106,7 +106,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, eventBatch.setErrorMsg("Something went wrong. Channel was " + "unable to create a transaction!") } - txOpt.map(tx => { + txOpt.foreach(tx => { tx.begin() val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) val loop = new Breaks @@ -145,7 +145,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, LOG.error("Error while processing transaction.", e) eventBatch.setErrorMsg(e.getMessage) try { - txOpt.map(tx => { + txOpt.foreach(tx => { rollbackAndClose(tx, close = true) }) } finally { @@ -163,7 +163,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, */ private def processAckOrNack() { batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) - txOpt.map(tx => { + txOpt.foreach(tx => { if (batchSuccess) { try { tx.commit() diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index ff6a5b5ce1d04..b8507f0a48a0e 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -151,7 +151,7 @@ private[streaming] class FlumePollingReceiver( override def onStop(): Unit = { logInfo("Shutting down Flume Polling Receiver") receiverExecutor.shutdownNow() - connections.map(connection => { + connections.foreach(connection => { connection.transceiver.close() }) channelFactory.releaseExternalResources() From 120e2a125f3bb8a828ae48332e12e28f6a5eebac Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 14 Jul 2014 22:33:08 -0700 Subject: [PATCH 19/32] SPARK-1729. Some test changes and changes to utils classes. --- external/flume-sink/pom.xml | 5 ++ .../flume/sink/SparkAvroCallbackHandler.scala | 5 +- .../spark/flume/sink/SparkSinkUtils.scala | 30 +++++++ .../flume/FlumePollingInputDStream.scala | 88 ++++++++++--------- .../spark/streaming/flume/FlumeUtils.scala | 87 +++++++++++++++--- .../flume/FlumePollingReceiverSuite.scala | 21 +++-- 6 files changed, 175 insertions(+), 61 deletions(-) create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d375013e7596a..d11129ce8d89d 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -64,6 +64,11 @@ + + org.scala-lang + scala-library + 2.10.4 + target/scala-${scala.binary.version}/classes diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala index 11d4805657803..3651041b41638 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala @@ -64,10 +64,9 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, }) // Wait until a batch is available - will be an error if error message is non-empty val batch = processor.getEventBatch - if (batch.getErrorMsg != null && !batch.getErrorMsg.equals("")) { - processorMap.put(sequenceNumber, processor) + if (!SparkSinkUtils.isErrorBatch(batch)) { + processorMap.put(sequenceNumber.toString, processor) } - batch } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala new file mode 100644 index 0000000000000..e7041c2895025 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala @@ -0,0 +1,30 @@ +/* + * 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.flume.sink + +import org.apache.spark.flume.EventBatch + +object SparkSinkUtils { + /** + * This method determines if this batch represents an error or not. + * @param batch - The batch to check + * @return - true if the batch represents an error + */ + def isErrorBatch(batch: EventBatch): Boolean = { + !batch.getErrorMsg.toString.equals("") //If there is an error message, it is an error batch. + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index b8507f0a48a0e..2f1e23031ea30 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -22,6 +22,8 @@ import java.net.InetSocketAddress import java.nio.ByteBuffer import java.util.concurrent.{TimeUnit, Executors} +import org.apache.spark.flume.sink.SparkSinkUtils + import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -91,52 +93,54 @@ private[streaming] class FlumePollingReceiver( new FlumeConnection(transceiver, client) }).toArray - // Threads that pull data from Flume. - val dataReceiver = new Runnable { - override def run(): Unit = { - var counter = 0 - while (true) { - counter = counter % connections.size - val client = connections(counter).client - counter += 1 - val eventBatch = client.getEventBatch(maxBatchSize) - val errorMsg = eventBatch.getErrorMsg - if (errorMsg.toString.equals("")) { // No error, proceed with processing data - val seq = eventBatch.getSequenceNumber - val events: java.util.List[SparkSinkEvent] = eventBatch.getEvents - logDebug( - "Received batch of " + events.size() + " events with sequence number: " + seq) - try { - // Convert each Flume event to a serializable SparkPollingEvent - events.foreach(event => { - store(SparkFlumePollingEvent.fromSparkSinkEvent(event)) - }) - // Send an ack to Flume so that Flume discards the events from its channels. - client.ack(seq) - } catch { - case e: Exception => - try { - // Let Flume know that the events need to be pushed back into the channel. - client.nack(seq) // If the agent is down, even this could fail and throw - } catch { - case e: Exception => logError( - "Sending Nack also failed. A Flume agent is down.") + for (i <- 0 until parallelism) { + logInfo("Starting Flume Polling Receiver worker threads starting..") + // Threads that pull data from Flume. + receiverExecutor.submit(new Runnable { + override def run(): Unit = { + var counter = i + while (true) { + counter = counter % (connections.length) + val client = connections(counter).client + counter += 1 + val eventBatch = client.getEventBatch(maxBatchSize) + if (!SparkSinkUtils.isErrorBatch(eventBatch)) { + // No error, proceed with processing data + val seq = eventBatch.getSequenceNumber + val events: java.util.List[SparkSinkEvent] = eventBatch.getEvents + logDebug( + "Received batch of " + events.size() + " events with sequence number: " + seq) + try { + // Convert each Flume event to a serializable SparkPollingEvent + var j = 0 + while (j < events.size()) { + store(SparkFlumePollingEvent.fromSparkSinkEvent(events(j))) + logDebug("Stored events with seq:" + seq) + j += 1 } - TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. - logWarning("Error while attempting to store events", e) + logInfo("Sending ack for: " +seq) + // Send an ack to Flume so that Flume discards the events from its channels. + client.ack(seq) + logDebug("Ack sent for sequence number: " + seq) + } catch { + case e: Exception => + try { + // Let Flume know that the events need to be pushed back into the channel. + client.nack(seq) // If the agent is down, even this could fail and throw + } catch { + case e: Exception => logError( + "Sending Nack also failed. A Flume agent is down.") + } + TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. + logWarning("Error while attempting to store events", e) + } + } else { + logWarning("Did not receive events from Flume agent due to error on the Flume " + + "agent: " + eventBatch.getErrorMsg) } - } else { - logWarning("Did not receive events from Flume agent due to error on the Flume agent: " + - "" + errorMsg.toString) } } - } - } - - // Create multiple threads and start all of them. - for (i <- 0 until parallelism) { - logInfo("Starting Flume Polling Receiver worker threads starting..") - receiverExecutor.submit(dataReceiver) + }) } } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 7364d9842bcbf..4e4fc3a612d4f 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -59,7 +59,7 @@ object FlumeUtils { ): ReceiverInputDStream[SparkFlumeEvent] = { val inputStream = new FlumeInputDStream[SparkFlumeEvent]( ssc, hostname, port, storageLevel, enableDecompression) - + inputStream } @@ -109,6 +109,39 @@ object FlumeUtils { createStream(jssc.ssc, hostname, port, storageLevel, enableDecompression) } + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 100 events and run 5 threads to pull data. + * @param host The address of the host on which the Spark Sink is running + * @param port The port that the host is listening on + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + ssc: StreamingContext, + host: String, + port: Int, + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumePollingEvent] = { + new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, + Seq(new InetSocketAddress(host, port)), 100, 5, storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 100 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream ( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumePollingEvent] = { + new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, 100, 5, storageLevel) + } + /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. @@ -123,14 +156,48 @@ object FlumeUtils { def createPollingStream ( ssc: StreamingContext, addresses: Seq[InetSocketAddress], - maxBatchSize: Int = 100, - parallelism: Int = 5, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + maxBatchSize: Int, + parallelism: Int, + storageLevel: StorageLevel ): ReceiverInputDStream[SparkFlumePollingEvent] = { new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, maxBatchSize, parallelism, storageLevel) } + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 100 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream ( + jssc: JavaStreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumePollingEvent] = { + new FlumePollingInputDStream[SparkFlumePollingEvent](jssc.ssc, addresses, 100, 5, + StorageLevel.MEMORY_AND_DISK_SER_2) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 100 events and run 5 threads to pull data. + * @param host The address of the host on which the Spark Sink is running + * @param port The port that the host is listening on + * @param storageLevel Storage level to use for storing the received objects + */ + def createPollingStream( + jssc: JavaStreamingContext, + host: String, + port: Int, + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumePollingEvent] = { + new FlumePollingInputDStream[SparkFlumePollingEvent](jssc.ssc, + Seq(new InetSocketAddress(host, port)), 100, 5, storageLevel) + } + /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. @@ -142,14 +209,14 @@ object FlumeUtils { * result in this stream using more threads * @param storageLevel Storage level to use for storing the received objects */ - def createJavaPollingStream ( - ssc: StreamingContext, + def createPollingStream ( + jssc: JavaStreamingContext, addresses: Seq[InetSocketAddress], - maxBatchSize: Int = 100, - parallelism: Int = 5, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + maxBatchSize: Int, + parallelism: Int, + storageLevel: StorageLevel ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { - new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, maxBatchSize, + new FlumePollingInputDStream[SparkFlumePollingEvent](jssc.ssc, addresses, maxBatchSize, parallelism, storageLevel) } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala index 7e6fe66052138..ff852e362386d 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.util.ManualClock import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} -class FlumePollingReceiverSuite extends TestSuiteBase { + class FlumePollingReceiverSuite extends TestSuiteBase { val testPort = 9999 @@ -43,7 +43,7 @@ class FlumePollingReceiverSuite extends TestSuiteBase { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = - FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), 100, 5, + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), 100, 1, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] @@ -66,6 +66,7 @@ class FlumePollingReceiverSuite extends TestSuiteBase { sink.start() ssc.start() writeAndVerify(Seq(channel), ssc, outputBuffer) + assertQueuesAreEmpty(channel) sink.stop() channel.stop() } @@ -75,7 +76,7 @@ class FlumePollingReceiverSuite extends TestSuiteBase { val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort), - new InetSocketAddress("localhost", testPort + 1)), 100, 5, + new InetSocketAddress("localhost", testPort + 1)), 100, 2, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] @@ -108,9 +109,10 @@ class FlumePollingReceiverSuite extends TestSuiteBase { sink2.start() ssc.start() writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) + assertQueuesAreEmpty(channel) + assertQueuesAreEmpty(channel2) sink.stop() channel.stop() - } def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, @@ -126,12 +128,12 @@ class FlumePollingReceiverSuite extends TestSuiteBase { } val startTime = System.currentTimeMillis() while (outputBuffer.size < 5 * channels.size && - System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + System.currentTimeMillis() - startTime < 15000) { logInfo("output.size = " + outputBuffer.size) Thread.sleep(100) } val timeTaken = System.currentTimeMillis() - startTime - assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") + assert(timeTaken < 15000, "Operation timed out after " + timeTaken + " ms") logInfo("Stopping context") ssc.stop() @@ -158,6 +160,13 @@ class FlumePollingReceiverSuite extends TestSuiteBase { assert(counter === 25 * channels.size) } + def assertQueuesAreEmpty(channel: MemoryChannel) = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining"); + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + assert(m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] === 5000) + } + private class TxnSubmitter(channel: MemoryChannel, clock: ManualClock) extends Callable[Void] { override def call(): Void = { var t = 0 From 393bd94c4668d09341c85d7391309275320a299b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 14 Jul 2014 23:39:16 -0700 Subject: [PATCH 20/32] SPARK-1729. Use LinkedBlockingQueue instead of ArrayBuffer to keep track of connections. --- .../flume/FlumePollingInputDStream.scala | 90 ++++++++++--------- .../flume/FlumePollingReceiverSuite.scala | 10 +-- project/SparkBuild.scala | 1 - 3 files changed, 52 insertions(+), 49 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 2f1e23031ea30..69baadcb27c28 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.flume import java.io.{ObjectOutput, ObjectInput, Externalizable} import java.net.InetSocketAddress import java.nio.ByteBuffer -import java.util.concurrent.{TimeUnit, Executors} +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, Executors} import org.apache.spark.flume.sink.SparkSinkUtils @@ -33,7 +33,7 @@ import org.apache.avro.ipc.specific.SpecificRequestor import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.apache.spark.Logging -import org.apache.spark.flume.{EventBatch, SparkSinkEvent, SparkFlumeProtocol} +import org.apache.spark.flume.{SparkSinkEvent, SparkFlumeProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.ReceiverInputDStream @@ -83,60 +83,64 @@ private[streaming] class FlumePollingReceiver( lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) - private var connections = Array.empty[FlumeConnection] // temporarily empty, filled in later + private val connections = new LinkedBlockingQueue[FlumeConnection]() override def onStart(): Unit = { // Create the connections to each Flume agent. - connections = addresses.map(host => { + addresses.foreach(host => { val transceiver = new NettyTransceiver(host, channelFactory) val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) - new FlumeConnection(transceiver, client) - }).toArray - + connections.add(new FlumeConnection(transceiver, client)) + }) for (i <- 0 until parallelism) { logInfo("Starting Flume Polling Receiver worker threads starting..") // Threads that pull data from Flume. receiverExecutor.submit(new Runnable { override def run(): Unit = { - var counter = i while (true) { - counter = counter % (connections.length) - val client = connections(counter).client - counter += 1 - val eventBatch = client.getEventBatch(maxBatchSize) - if (!SparkSinkUtils.isErrorBatch(eventBatch)) { - // No error, proceed with processing data - val seq = eventBatch.getSequenceNumber - val events: java.util.List[SparkSinkEvent] = eventBatch.getEvents - logDebug( - "Received batch of " + events.size() + " events with sequence number: " + seq) - try { - // Convert each Flume event to a serializable SparkPollingEvent - var j = 0 - while (j < events.size()) { - store(SparkFlumePollingEvent.fromSparkSinkEvent(events(j))) - logDebug("Stored events with seq:" + seq) - j += 1 - } - logInfo("Sending ack for: " +seq) - // Send an ack to Flume so that Flume discards the events from its channels. - client.ack(seq) - logDebug("Ack sent for sequence number: " + seq) - } catch { - case e: Exception => - try { - // Let Flume know that the events need to be pushed back into the channel. - client.nack(seq) // If the agent is down, even this could fail and throw - } catch { - case e: Exception => logError( - "Sending Nack also failed. A Flume agent is down.") + val connection = connections.poll() + val client = connection.client + try { + val eventBatch = client.getEventBatch(maxBatchSize) + if (!SparkSinkUtils.isErrorBatch(eventBatch)) { + // No error, proceed with processing data + val seq = eventBatch.getSequenceNumber + val events: java.util.List[SparkSinkEvent] = eventBatch.getEvents + logDebug( + "Received batch of " + events.size() + " events with sequence number: " + seq) + try { + // Convert each Flume event to a serializable SparkPollingEvent + var j = 0 + while (j < events.size()) { + store(SparkFlumePollingEvent.fromSparkSinkEvent(events(j))) + logDebug("Stored events with seq:" + seq) + j += 1 } - TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. - logWarning("Error while attempting to store events", e) + logDebug("Sending ack for: " +seq) + // Send an ack to Flume so that Flume discards the events from its channels. + client.ack(seq) + logDebug("Ack sent for sequence number: " + seq) + } catch { + case e: Exception => + try { + // Let Flume know that the events need to be pushed back into the channel. + client.nack(seq) // If the agent is down, even this could fail and throw + } catch { + case e: Exception => logError( + "Sending Nack also failed. A Flume agent is down.") + } + TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. + logWarning("Error while attempting to store events", e) + } + } else { + logWarning("Did not receive events from Flume agent due to error on the Flume " + + "agent: " + eventBatch.getErrorMsg) } - } else { - logWarning("Did not receive events from Flume agent due to error on the Flume " + - "agent: " + eventBatch.getErrorMsg) + } catch { + case e: Exception => + logWarning("Error while reading data from Flume", e) + } finally { + connections.add(connection) } } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala index ff852e362386d..d1c8042c68d1a 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala @@ -66,7 +66,7 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon sink.start() ssc.start() writeAndVerify(Seq(channel), ssc, outputBuffer) - assertQueuesAreEmpty(channel) + assertChannelIsEmpty(channel) sink.stop() channel.stop() } @@ -76,7 +76,7 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort), - new InetSocketAddress("localhost", testPort + 1)), 100, 2, + new InetSocketAddress("localhost", testPort + 1)), 100, 5, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] @@ -109,8 +109,8 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon sink2.start() ssc.start() writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) - assertQueuesAreEmpty(channel) - assertQueuesAreEmpty(channel2) + assertChannelIsEmpty(channel) + assertChannelIsEmpty(channel2) sink.stop() channel.stop() } @@ -160,7 +160,7 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon assert(counter === 25 * channels.size) } - def assertQueuesAreEmpty(channel: MemoryChannel) = { + def assertChannelIsEmpty(channel: MemoryChannel) = { val queueRemaining = channel.getClass.getDeclaredField("queueRemaining"); queueRemaining.setAccessible(true) val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 85b086e2c7808..c959ae99e3b4f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -122,7 +122,6 @@ object SparkBuild extends PomBuild { retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", - "spark-streaming-flume-sink", "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-twitter", "spark-streaming", "spark-mllib", "spark-bagel", "spark-graphx", "spark-core").map(versionArtifact(_).get intransitive()) From 8c00289dc78dea11e9ce4aed4326e43cf2b64c01 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 15 Jul 2014 00:02:31 -0700 Subject: [PATCH 21/32] More debug messages --- .../spark/streaming/flume/FlumePollingInputDStream.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 69baadcb27c28..5f6854afa0207 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -116,7 +116,7 @@ private[streaming] class FlumePollingReceiver( logDebug("Stored events with seq:" + seq) j += 1 } - logDebug("Sending ack for: " +seq) + logDebug("Sending ack for sequence number: " +seq) // Send an ack to Flume so that Flume discards the events from its channels. client.ack(seq) logDebug("Ack sent for sequence number: " + seq) @@ -124,7 +124,9 @@ private[streaming] class FlumePollingReceiver( case e: Exception => try { // Let Flume know that the events need to be pushed back into the channel. + logDebug("Sending nack for sequence number: " + seq) client.nack(seq) // If the agent is down, even this could fail and throw + logDebug("Nack sent for sequence number: " + seq) } catch { case e: Exception => logError( "Sending Nack also failed. A Flume agent is down.") From 1edc806ad8b44e695303f97d71fa7a325645eef8 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 15 Jul 2014 10:04:58 -0700 Subject: [PATCH 22/32] SPARK-1729. Update logging in Spark Sink. --- .../org/apache/spark/flume/sink/Logging.scala | 135 ++++++++++++++++++ .../flume/sink/SparkAvroCallbackHandler.scala | 9 +- .../apache/spark/flume/sink/SparkSink.scala | 15 +- .../spark/flume/sink/SparkSinkUtils.scala | 2 +- .../flume/sink/TransactionProcessor.scala | 23 ++- .../flume/FlumePollingInputDStream.scala | 2 +- 6 files changed, 163 insertions(+), 23 deletions(-) create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/flume/sink/Logging.scala diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/Logging.scala new file mode 100644 index 0000000000000..8b6453362a121 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/Logging.scala @@ -0,0 +1,135 @@ +/* + * 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.flume.sink +import org.apache.log4j.{LogManager, PropertyConfigurator} +import org.slf4j.{Logger, LoggerFactory} +import org.slf4j.impl.StaticLoggerBinder + +trait Logging { + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null + + // Method to get or create the logger for this object + protected def log: Logger = { + if (log_ == null) { + initializeIfNecessary() + var className = this.getClass.getName + // Ignore trailing $'s in the class names for Scala objects + if (className.endsWith("$")) { + className = className.substring(0, className.length - 1) + } + log_ = LoggerFactory.getLogger(className) + } + log_ + } + + // Log methods that take only a String + protected def logInfo(msg: => String) { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logDebug(msg: => String) { + if (log.isDebugEnabled) log.debug(msg) + } + + protected def logTrace(msg: => String) { + if (log.isTraceEnabled) log.trace(msg) + } + + protected def logWarning(msg: => String) { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String) { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable) { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logDebug(msg: => String, throwable: Throwable) { + if (log.isDebugEnabled) log.debug(msg, throwable) + } + + protected def logTrace(msg: => String, throwable: Throwable) { + if (log.isTraceEnabled) log.trace(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable) { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable) { + if (log.isErrorEnabled) log.error(msg, throwable) + } + + protected def isTraceEnabled(): Boolean = { + log.isTraceEnabled + } + + private def initializeIfNecessary() { + if (!Logging.initialized) { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging() + } + } + } + } + + private def initializeLogging() { + // If Log4j is being used, but is not initialized, load a default properties file + val binder = StaticLoggerBinder.getSingleton + val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory") + val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements + if (!log4jInitialized && usingLog4j) { + val defaultLogProps = "org/apache/spark/log4j-defaults.properties" + Option(getClass.getClassLoader.getResource(defaultLogProps)) match { + case Some(url) => + PropertyConfigurator.configure(url) + log.info(s"Using Spark's default log4j profile: $defaultLogProps") + case None => + System.err.println(s"Spark was unable to load $defaultLogProps") + } + } + Logging.initialized = true + + // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads + // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html + log + } +} + +private object Logging { + @volatile private var initialized = false + val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala index 3651041b41638..e4925b85c81ee 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala @@ -35,8 +35,7 @@ import org.slf4j.LoggerFactory * is rolled back. */ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, - val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol { - private val LOG = LoggerFactory.getLogger(classOf[SparkAvroCallbackHandler]) + val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, new ThreadFactoryBuilder().setDaemon(true) .setNameFormat("Spark Sink Processor Thread - %d").build())) @@ -56,6 +55,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, * @return [[EventBatch]] instance that has a sequence number and an array of at most n events */ override def getEventBatch(n: Int): EventBatch = { + logDebug("Got getEventBatch call from Spark.") val sequenceNumber = seqBase + seqCounter.incrementAndGet() val processor = new TransactionProcessor(channel, sequenceNumber, n, transactionTimeout, backOffInterval, this) @@ -66,6 +66,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, val batch = processor.getEventBatch if (!SparkSinkUtils.isErrorBatch(batch)) { processorMap.put(sequenceNumber.toString, processor) + logDebug("Sending event batch with sequence number: " + sequenceNumber) } batch } @@ -75,6 +76,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, * @param sequenceNumber The sequence number of the event batch that was successful */ override def ack(sequenceNumber: CharSequence): Void = { + logDebug("Received Ack for batch with sequence number: " + sequenceNumber) completeTransaction(sequenceNumber, success = true) null } @@ -86,7 +88,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, */ override def nack(sequenceNumber: CharSequence): Void = { completeTransaction(sequenceNumber, success = false) - LOG.info("Spark failed to commit transaction. Will reattempt events.") + logInfo("Spark failed to commit transaction. Will reattempt events.") null } @@ -115,6 +117,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, * Shuts down the executor used to process transactions. */ def shutdown() { + logInfo("Shutting down Spark Avro Callback Handler") transactionExecutorOpt.foreach(executor => { executor.shutdownNow() }) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala index 265b37dd0b302..35b766ff04d9f 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala @@ -48,9 +48,8 @@ import org.apache.spark.flume.SparkFlumeProtocol // until an ACK or NACK comes back or the transaction times out (after the specified timeout). // When the response comes, the TransactionProcessor is retrieved and then unblocked, // at which point the transaction is committed or rolled back. -class SparkSink extends AbstractSink with Configurable { +class SparkSink extends AbstractSink with Logging with Configurable { - private val LOG = LoggerFactory.getLogger(classOf[SparkSink]) // Size of the pool to use for holding transaction processors. private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS @@ -74,7 +73,7 @@ class SparkSink extends AbstractSink with Configurable { private val blockingLatch = new CountDownLatch(1) override def start() { - LOG.info("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + + logInfo("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + hostname + " with " + "pool size: " + poolSize + " and transaction timeout: " + transactionTimeout + ".") handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout, @@ -85,19 +84,19 @@ class SparkSink extends AbstractSink with Configurable { // Netty dependencies are already available on the JVM as Flume would have pulled them in. serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) serverOpt.foreach(server => { - LOG.info("Starting Avro server for sink: " + getName) + logInfo("Starting Avro server for sink: " + getName) server.start() }) super.start() } override def stop() { - LOG.info("Stopping Spark Sink: " + getName) + logInfo("Stopping Spark Sink: " + getName) handler.foreach(callbackHandler => { callbackHandler.shutdown() }) serverOpt.foreach(server => { - LOG.info("Stopping Avro Server for sink: " + getName) + logInfo("Stopping Avro Server for sink: " + getName) server.close() server.join() }) @@ -113,12 +112,16 @@ class SparkSink extends AbstractSink with Configurable { poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) backOffInterval = ctx.getInteger(CONF_BACKOFF_INTERVAL, DEFAULT_BACKOFF_INTERVAL) + logInfo("Configured Spark Sink with hostname: " + hostname + ", port: " + port + ", " + + "poolSize: " + poolSize + ", transactionTimeout: " + transactionTimeout + ", " + + "backoffInterval: " + backOffInterval) } override def process(): Status = { // This method is called in a loop by the Flume framework - block it until the sink is // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is // being shut down. + logInfo("Blocking Sink Runner, sink will continue to run..") blockingLatch.await() Status.BACKOFF } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala index e7041c2895025..8f16246d495a0 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala @@ -25,6 +25,6 @@ object SparkSinkUtils { * @return - true if the batch represents an error */ def isErrorBatch(batch: EventBatch): Boolean = { - !batch.getErrorMsg.toString.equals("") //If there is an error message, it is an error batch. + !batch.getErrorMsg.toString.equals("") // If there is an error message, it is an error batch. } } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala index a4689cca5624a..1d2dddfbf7ff0 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala @@ -45,9 +45,7 @@ import org.slf4j.LoggerFactory */ private class TransactionProcessor(val channel: Channel, val seqNum: String, var maxBatchSize: Int, val transactionTimeout: Int, val backOffInterval: Int, - val parent: SparkAvroCallbackHandler) extends Callable[Void] { - - private val LOG = LoggerFactory.getLogger(classOf[TransactionProcessor]) + val parent: SparkAvroCallbackHandler) extends Callable[Void] with Logging { // If a real batch is not returned, we always have to return an error batch. @volatile private var eventBatch: EventBatch = new EventBatch("Unknown Error", "", @@ -88,9 +86,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, * @param success True if an ACK was received and the transaction should be committed, else false. */ def batchProcessed(success: Boolean) { - if (LOG.isDebugEnabled) { - LOG.debug("Batch processed for sequence number: " + seqNum) - } + logDebug("Batch processed for sequence number: " + seqNum) batchSuccess = success batchAckLatch.countDown() } @@ -123,6 +119,8 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, gotEventsInThisTxn = true case None => if (!gotEventsInThisTxn) { + logDebug("Sleeping for " + backOffInterval + " millis as no events were read in" + + " the current transaction") TimeUnit.MILLISECONDS.sleep(backOffInterval) } else { loop.break() @@ -133,7 +131,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, if (!gotEventsInThisTxn) { val msg = "Tried several times, " + "but did not get any events from the channel!" - LOG.warn(msg) + logWarning(msg) eventBatch.setErrorMsg(msg) } else { // At this point, the events are available, so fill them into the event batch @@ -142,7 +140,7 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, }) } catch { case e: Exception => - LOG.error("Error while processing transaction.", e) + logWarning("Error while processing transaction.", e) eventBatch.setErrorMsg(e.getMessage) try { txOpt.foreach(tx => { @@ -166,17 +164,18 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, txOpt.foreach(tx => { if (batchSuccess) { try { + logDebug("Committing transaction") tx.commit() } catch { case e: Exception => - LOG.warn("Error while attempting to commit transaction. Transaction will be rolled " + + logWarning("Error while attempting to commit transaction. Transaction will be rolled " + "back", e) rollbackAndClose(tx, close = false) // tx will be closed later anyway } finally { tx.close() } } else { - LOG.warn("Spark could not commit transaction, NACK received. Rolling back transaction.") + logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") rollbackAndClose(tx, close = true) // This might have been due to timeout or a NACK. Either way the following call does not // cause issues. This is required to ensure the TransactionProcessor instance is not leaked @@ -192,12 +191,12 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, */ private def rollbackAndClose(tx: Transaction, close: Boolean) { try { - LOG.warn("Spark was unable to successfully process the events. Transaction is being " + + logWarning("Spark was unable to successfully process the events. Transaction is being " + "rolled back.") tx.rollback() } catch { case e: Exception => - LOG.error("Error rolling back transaction. Rollback may have failed!", e) + logError("Error rolling back transaction. Rollback may have failed!", e) } finally { if (close) { tx.close() diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 5f6854afa0207..66df20863037b 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -116,7 +116,7 @@ private[streaming] class FlumePollingReceiver( logDebug("Stored events with seq:" + seq) j += 1 } - logDebug("Sending ack for sequence number: " +seq) + logDebug("Sending ack for sequence number: " + seq) // Send an ack to Flume so that Flume discards the events from its channels. client.ack(seq) logDebug("Ack sent for sequence number: " + seq) From 10b6214c39ba4f41d428ae87cba4bfb00f89a497 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 17 Jul 2014 01:17:02 -0700 Subject: [PATCH 23/32] Changed public API, changed sink package, and added java unit test to make sure Java API is callable from Java. --- .../streaming/FlumePollingEventCount.scala | 67 +++++++++ .../flume-sink/src/main/avro/sparkflume.avdl | 3 +- .../{ => streaming}/flume/sink/Logging.scala | 26 ++-- .../flume/sink/SparkAvroCallbackHandler.scala | 15 +- .../flume/sink/SparkSink.scala | 9 +- .../flume/sink/SparkSinkUtils.scala | 6 +- .../flume/sink/TransactionProcessor.scala | 7 +- .../flume/FlumePollingInputDStream.scala | 52 +++---- .../spark/streaming/flume/FlumeUtils.scala | 136 ++++++++++-------- .../flume/JavaFlumePollingStreamSuite.java | 27 ++++ ...te.scala => FlumePollingStreamSuite.scala} | 14 +- 11 files changed, 226 insertions(+), 136 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala rename external/flume-sink/src/main/scala/org/apache/spark/{ => streaming}/flume/sink/Logging.scala (82%) rename external/flume-sink/src/main/scala/org/apache/spark/{ => streaming}/flume/sink/SparkAvroCallbackHandler.scala (94%) rename external/flume-sink/src/main/scala/org/apache/spark/{ => streaming}/flume/sink/SparkSink.scala (98%) rename external/flume-sink/src/main/scala/org/apache/spark/{ => streaming}/flume/sink/SparkSinkUtils.scala (91%) rename external/flume-sink/src/main/scala/org/apache/spark/{ => streaming}/flume/sink/TransactionProcessor.scala (97%) create mode 100644 external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java rename external/flume/src/test/scala/org/apache/spark/streaming/flume/{FlumePollingReceiverSuite.scala => FlumePollingStreamSuite.scala} (94%) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala new file mode 100644 index 0000000000000..1cc8c8d5c23b6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala @@ -0,0 +1,67 @@ +/* + * 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.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ +import org.apache.spark.streaming.flume._ +import org.apache.spark.util.IntParam +import java.net.InetSocketAddress + +/** + * Produces a count of events received from Flume. + * + * This should be used in conjunction with the Spark Sink running in a Flume agent. See + * the Spark Streaming programming guide for more details. + * + * Usage: FlumePollingEventCount + * `host` is the host on which the Spark Sink is running. + * `port` is the port at which the Spark Sink is listening. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.FlumePollingEventCount [host] [port] ` + */ +object FlumePollingEventCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println( + "Usage: FlumePollingEventCount ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + val Array(host, IntParam(port)) = args + + val batchInterval = Milliseconds(2000) + + // Create the context and set the batch size + val sparkConf = new SparkConf().setAppName("FlumePollingEventCount") + val ssc = new StreamingContext(sparkConf, batchInterval) + + // Create a flume stream that polls the Spark Sink running in a Flume agent + val stream = FlumeUtils.createPollingStream(ssc, host, port) + + // Print out the count of events received from this server in each batch + stream.count().map(cnt => "Received " + cnt + " flume events." ).print() + + ssc.start() + ssc.awaitTermination() + } +} diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl index f8edd92f67e3e..8806e863ac7c6 100644 --- a/external/flume-sink/src/main/avro/sparkflume.avdl +++ b/external/flume-sink/src/main/avro/sparkflume.avdl @@ -17,7 +17,7 @@ * under the License. */ -@namespace("org.apache.spark.flume") +@namespace("org.apache.spark.streaming.flume.sink") protocol SparkFlumeProtocol { @@ -37,5 +37,4 @@ protocol SparkFlumeProtocol { void ack (string sequenceNumber); void nack (string sequenceNumber); - } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala similarity index 82% rename from external/flume-sink/src/main/scala/org/apache/spark/flume/sink/Logging.scala rename to external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala index 8b6453362a121..81eb25f2ee6d7 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/Logging.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -14,12 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.flume.sink +package org.apache.spark.streaming.flume.sink + import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder -trait Logging { +/** + * Copy of the org.apache.spark.Logging for being used in the Spark Sink. + * The org.apache.spark.Logging is not used so that all of Spark is not brought + * in as a dependency. + */ +private[sink] trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine @transient private var log_ : Logger = null @@ -95,20 +101,6 @@ trait Logging { } private def initializeLogging() { - // If Log4j is being used, but is not initialized, load a default properties file - val binder = StaticLoggerBinder.getSingleton - val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory") - val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements - if (!log4jInitialized && usingLog4j) { - val defaultLogProps = "org/apache/spark/log4j-defaults.properties" - Option(getClass.getClassLoader.getResource(defaultLogProps)) match { - case Some(url) => - PropertyConfigurator.configure(url) - log.info(s"Using Spark's default log4j profile: $defaultLogProps") - case None => - System.err.println(s"Spark was unable to load $defaultLogProps") - } - } Logging.initialized = true // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads @@ -117,7 +109,7 @@ trait Logging { } } -private object Logging { +private[sink] object Logging { @volatile private var initialized = false val initLock = new Object() try { diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala similarity index 94% rename from external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala rename to external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index e4925b85c81ee..6249a3197d07d 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -14,17 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.flume.sink +package org.apache.spark.streaming.flume.sink -import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.{ConcurrentHashMap, Executors} +import java.util.concurrent.atomic.AtomicLong -import com.google.common.util.concurrent.ThreadFactoryBuilder - -import org.apache.commons.lang.RandomStringUtils import org.apache.flume.Channel -import org.apache.spark.flume.{EventBatch, SparkFlumeProtocol} -import org.slf4j.LoggerFactory +import org.apache.commons.lang.RandomStringUtils +import com.google.common.util.concurrent.ThreadFactoryBuilder /** * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process @@ -34,7 +31,7 @@ import org.slf4j.LoggerFactory * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark * is rolled back. */ -private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, +private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, new ThreadFactoryBuilder().setDaemon(true) @@ -109,7 +106,7 @@ private class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, * @return The transaction processor for the corresponding batch. Note that this instance is no * longer tracked and the caller is responsible for that txn processor. */ - private[flume] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { + private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { processorMap.remove(sequenceNumber.toString) // The toString is required! } diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala similarity index 98% rename from external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala rename to external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 35b766ff04d9f..d5afde0fae19d 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -14,20 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.flume.sink +package org.apache.spark.streaming.flume.sink import java.net.InetSocketAddress import java.util.concurrent._ import org.apache.avro.ipc.NettyServer import org.apache.avro.ipc.specific.SpecificResponder +import org.apache.flume.Context import org.apache.flume.Sink.Status import org.apache.flume.conf.{Configurable, ConfigurationException} import org.apache.flume.sink.AbstractSink -import org.apache.flume.Context -import org.slf4j.LoggerFactory - -import org.apache.spark.flume.SparkFlumeProtocol /** * A sink that uses Avro RPC to run a server that can be polled by Spark's @@ -48,6 +45,7 @@ import org.apache.spark.flume.SparkFlumeProtocol // until an ACK or NACK comes back or the transaction times out (after the specified timeout). // When the response comes, the TransactionProcessor is retrieved and then unblocked, // at which point the transaction is committed or rolled back. +private[flume] class SparkSink extends AbstractSink with Logging with Configurable { // Size of the pool to use for holding transaction processors. @@ -130,6 +128,7 @@ class SparkSink extends AbstractSink with Logging with Configurable { /** * Configuration parameters and their defaults. */ +private[flume] object SparkSinkConfig { val THREADS = "threads" val DEFAULT_THREADS = 10 diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala similarity index 91% rename from external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala rename to external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala index 8f16246d495a0..47c0e294d6b52 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/SparkSinkUtils.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala @@ -14,11 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.flume.sink +package org.apache.spark.streaming.flume.sink -import org.apache.spark.flume.EventBatch - -object SparkSinkUtils { +private[flume] object SparkSinkUtils { /** * This method determines if this batch represents an error or not. * @param batch - The batch to check diff --git a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala similarity index 97% rename from external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala rename to external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala index 1d2dddfbf7ff0..6f4e50b0f1d63 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -14,18 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.flume.sink +package org.apache.spark.streaming.flume.sink import java.nio.ByteBuffer import java.util -import java.util.concurrent.{TimeUnit, CountDownLatch, Callable} +import java.util.concurrent.{Callable, CountDownLatch, TimeUnit} import scala.util.control.Breaks import org.apache.flume.{Transaction, Channel} -import org.apache.spark.flume.{SparkSinkEvent, EventBatch} -import org.slf4j.LoggerFactory - // Flume forces transactions to be thread-local (horrible, I know!) // So the sink basically spawns a new thread to pull the events out within a transaction. diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 66df20863037b..6d7a74f7a9020 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -22,8 +22,6 @@ import java.net.InetSocketAddress import java.nio.ByteBuffer import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, Executors} -import org.apache.spark.flume.sink.SparkSinkUtils - import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -33,45 +31,44 @@ import org.apache.avro.ipc.specific.SpecificRequestor import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory import org.apache.spark.Logging -import org.apache.spark.flume.{SparkSinkEvent, SparkFlumeProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.flume.sink._ + /** * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running * [[org.apache.spark.flume.sink.SparkSink]]s. - * @param ssc_ Streaming context that will execute this input stream + * @param _ssc Streaming context that will execute this input stream * @param addresses List of addresses at which SparkSinks are listening * @param maxBatchSize Maximum size of a batch * @param parallelism Number of parallel connections to open * @param storageLevel The storage level to use. * @tparam T Class type of the object of this stream */ +private[streaming] class FlumePollingInputDStream[T: ClassTag]( - @transient ssc_ : StreamingContext, - val addresses: Seq[InetSocketAddress], - val maxBatchSize: Int, - val parallelism: Int, - storageLevel: StorageLevel -) extends ReceiverInputDStream[SparkFlumePollingEvent](ssc_) { - /** - * Gets the receiver object that will be sent to the worker nodes - * to receive data. This method needs to defined by any specific implementation - * of a NetworkInputDStream. - */ + @transient _ssc: StreamingContext, + val addresses: Seq[InetSocketAddress], + val maxBatchSize: Int, + val parallelism: Int, + storageLevel: StorageLevel + ) extends ReceiverInputDStream[SparkFlumePollingEvent](_ssc) { + override def getReceiver(): Receiver[SparkFlumePollingEvent] = { new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) } } -private[streaming] class FlumePollingReceiver( - addresses: Seq[InetSocketAddress], - maxBatchSize: Int, - parallelism: Int, - storageLevel: StorageLevel -) extends Receiver[SparkFlumePollingEvent](storageLevel) with Logging { +private[streaming] +class FlumePollingReceiver( + addresses: Seq[InetSocketAddress], + maxBatchSize: Int, + parallelism: Int, + storageLevel: StorageLevel + ) extends Receiver[SparkFlumePollingEvent](storageLevel) with Logging { lazy val channelFactoryExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). @@ -150,14 +147,6 @@ private[streaming] class FlumePollingReceiver( } } - override def store(dataItem: SparkFlumePollingEvent) { - // Not entirely sure store is thread-safe for all storage levels - so wrap it in synchronized - // This takes a performance hit, since the parallelism is useful only for pulling data now. - this.synchronized { - super.store(dataItem) - } - } - override def onStop(): Unit = { logInfo("Shutting down Flume Polling Receiver") receiverExecutor.shutdownNow() @@ -176,6 +165,9 @@ private[streaming] class FlumePollingReceiver( private class FlumeConnection(val transceiver: NettyTransceiver, val client: SparkFlumeProtocol.Callback) +/** + * Companion object of [[SparkFlumePollingEvent]] + */ private[streaming] object SparkFlumePollingEvent { def fromSparkSinkEvent(in: SparkSinkEvent): SparkFlumePollingEvent = { val event = new SparkFlumePollingEvent() @@ -189,7 +181,7 @@ private[streaming] object SparkFlumePollingEvent { * SparkSinkEvent is identical to AvroFlumeEvent, we need to create a new class and a wrapper * around that to make it externalizable. */ -class SparkFlumePollingEvent() extends Externalizable with Logging { +class SparkFlumePollingEvent extends Externalizable with Logging { var event: SparkSinkEvent = new SparkSinkEvent() /* De-serialize from bytes. */ diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 4e4fc3a612d4f..c754fe33738b8 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.flume import java.net.InetSocketAddress +import org.apache.spark.annotation.Experimental import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} @@ -26,6 +27,9 @@ import org.apache.spark.streaming.dstream.ReceiverInputDStream object FlumeUtils { + private val DEFAULT_POLLING_PARALLELISM = 5 + private val DEFAULT_POLLING_BATCH_SIZE = 1000 + /** * Create a input stream from a Flume source. * @param ssc StreamingContext object @@ -112,54 +116,57 @@ object FlumeUtils { /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 100 events and run 5 threads to pull data. - * @param host The address of the host on which the Spark Sink is running - * @param port The port that the host is listening on + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param host Address of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening * @param storageLevel Storage level to use for storing the received objects */ + @Experimental def createPollingStream( - ssc: StreamingContext, - host: String, - port: Int, - storageLevel: StorageLevel + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): ReceiverInputDStream[SparkFlumePollingEvent] = { - new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, - Seq(new InetSocketAddress(host, port)), 100, 5, storageLevel) + createPollingStream(ssc, Seq(new InetSocketAddress(hostname, port)), storageLevel) } /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 100 events and run 5 threads to pull data. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. * @param addresses List of InetSocketAddresses representing the hosts to connect to. * @param storageLevel Storage level to use for storing the received objects */ - def createPollingStream ( - ssc: StreamingContext, - addresses: Seq[InetSocketAddress], - storageLevel: StorageLevel - ): ReceiverInputDStream[SparkFlumePollingEvent] = { - new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, 100, 5, storageLevel) + @Experimental + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumePollingEvent] = { + createPollingStream(ssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) } /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. * @param addresses List of InetSocketAddresses representing the hosts to connect to. - * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a + * @param maxBatchSize Maximum number of events to be pulled from the Spark sink in a * single RPC call * @param parallelism Number of concurrent requests this stream should send to the sink. Note * that having a higher number of requests concurrently being pulled will * result in this stream using more threads * @param storageLevel Storage level to use for storing the received objects */ - def createPollingStream ( - ssc: StreamingContext, - addresses: Seq[InetSocketAddress], - maxBatchSize: Int, - parallelism: Int, - storageLevel: StorageLevel - ): ReceiverInputDStream[SparkFlumePollingEvent] = { + @Experimental + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): ReceiverInputDStream[SparkFlumePollingEvent] = { new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, maxBatchSize, parallelism, storageLevel) } @@ -167,56 +174,73 @@ object FlumeUtils { /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 100 events and run 5 threads to pull data. - * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int + ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + createPollingStream(jssc, hostname, port, StorageLevel.MEMORY_AND_DISK_SER_2) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening * @param storageLevel Storage level to use for storing the received objects */ - def createPollingStream ( - jssc: JavaStreamingContext, - addresses: Seq[InetSocketAddress], - storageLevel: StorageLevel - ): ReceiverInputDStream[SparkFlumePollingEvent] = { - new FlumePollingInputDStream[SparkFlumePollingEvent](jssc.ssc, addresses, 100, 5, - StorageLevel.MEMORY_AND_DISK_SER_2) + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + createPollingStream(jssc, Array(new InetSocketAddress(hostname, port)), storageLevel) } /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 100 events and run 5 threads to pull data. - * @param host The address of the host on which the Spark Sink is running - * @param port The port that the host is listening on + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running. * @param storageLevel Storage level to use for storing the received objects */ + @Experimental def createPollingStream( - jssc: JavaStreamingContext, - host: String, - port: Int, - storageLevel: StorageLevel - ): ReceiverInputDStream[SparkFlumePollingEvent] = { - new FlumePollingInputDStream[SparkFlumePollingEvent](jssc.ssc, - Seq(new InetSocketAddress(host, port)), 100, 5, storageLevel) + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + createPollingStream(jssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) } /** * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. - * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a * single RPC call - * @param parallelism Number of concurrent requests this stream should send to the sink. Note - * that having a higher number of requests concurrently being pulled will - * result in this stream using more threads + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads * @param storageLevel Storage level to use for storing the received objects */ - def createPollingStream ( - jssc: JavaStreamingContext, - addresses: Seq[InetSocketAddress], - maxBatchSize: Int, - parallelism: Int, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { - new FlumePollingInputDStream[SparkFlumePollingEvent](jssc.ssc, addresses, maxBatchSize, - parallelism, storageLevel) + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) } } diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java new file mode 100644 index 0000000000000..ef8622481d599 --- /dev/null +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java @@ -0,0 +1,27 @@ +package org.apache.spark.streaming.flume; + +import java.net.InetSocketAddress; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; + +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.junit.Test; + +public class JavaFlumePollingStreamSuite extends LocalJavaStreamingContext { + @Test + public void testFlumeStream() { + // tests the API, does not actually test data receiving + InetSocketAddress[] addresses = new InetSocketAddress[] { + new InetSocketAddress("localhost", 12345) + }; + JavaReceiverInputDStream test1 = + FlumeUtils.createPollingStream(ssc, "localhost", 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( + ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2(), 100, 5); + } +} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala similarity index 94% rename from external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala rename to external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index d1c8042c68d1a..3ff0cca523928 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingReceiverSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -29,13 +29,13 @@ import org.apache.flume.channel.MemoryChannel import org.apache.flume.conf.Configurables import org.apache.flume.event.EventBuilder -import org.apache.spark.flume.sink.{SparkSinkConfig, SparkSink} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.util.ManualClock import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} +import org.apache.spark.streaming.flume.sink._ - class FlumePollingReceiverSuite extends TestSuiteBase { + class FlumePollingStreamSuite extends TestSuiteBase { val testPort = 9999 @@ -43,8 +43,7 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = - FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), 100, 1, - StorageLevel.MEMORY_AND_DISK) + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), StorageLevel.MEMORY_AND_DISK, 100, 1) val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) @@ -65,6 +64,7 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon sink.setChannel(channel) sink.start() ssc.start() + writeAndVerify(Seq(channel), ssc, outputBuffer) assertChannelIsEmpty(channel) sink.stop() @@ -74,10 +74,9 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon test("flume polling test multiple hosts") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) + val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _)) val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = - FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort), - new InetSocketAddress("localhost", testPort + 1)), 100, 5, - StorageLevel.MEMORY_AND_DISK) + FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, 100, 5) val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) @@ -187,5 +186,4 @@ import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingCon null } } - } From 799509f28f81f9b8ede6f7d02753502ff44b2965 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 21 Jul 2014 15:04:20 -0700 Subject: [PATCH 24/32] Fix a compile issue. --- .../scala/org/apache/spark/streaming/flume/sink/Logging.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala index 81eb25f2ee6d7..17cbc6707b5ea 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -16,9 +16,7 @@ */ package org.apache.spark.streaming.flume.sink -import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} -import org.slf4j.impl.StaticLoggerBinder /** * Copy of the org.apache.spark.Logging for being used in the Spark Sink. From 35721801023785a9e86e44c3e705a395e2ae8f7c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 21 Jul 2014 15:55:20 -0700 Subject: [PATCH 25/32] Adding a license header, making Jenkins happy. --- .../streaming/flume/EventTransformer.scala | 2 +- .../flume/FlumePollingInputDStream.scala | 9 +++------ .../flume/JavaFlumePollingStreamSuite.java | 17 +++++++++++++++++ 3 files changed, 21 insertions(+), 7 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala index 069a9a215675c..dc629df4f4ac2 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -28,7 +28,7 @@ import org.apache.spark.Logging * A simple object that provides the implementation of readExternal and writeExternal for both * the wrapper classes for Flume-style Events. */ -object EventTransformer extends Logging { +private[streaming] object EventTransformer extends Logging { def readExternal(in: ObjectInput): (java.util.HashMap[CharSequence, CharSequence], Array[Byte]) = { val bodyLength = in.readInt() diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 6d7a74f7a9020..01de8781615a7 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -37,10 +37,9 @@ import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.streaming.flume.sink._ - /** * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running - * [[org.apache.spark.flume.sink.SparkSink]]s. + * [[org.apache.spark.streaming.flume.sink.SparkSink]]s. * @param _ssc Streaming context that will execute this input stream * @param addresses List of addresses at which SparkSinks are listening * @param maxBatchSize Maximum size of a batch @@ -48,8 +47,7 @@ import org.apache.spark.streaming.flume.sink._ * @param storageLevel The storage level to use. * @tparam T Class type of the object of this stream */ -private[streaming] -class FlumePollingInputDStream[T: ClassTag]( +private[streaming] class FlumePollingInputDStream[T: ClassTag]( @transient _ssc: StreamingContext, val addresses: Seq[InetSocketAddress], val maxBatchSize: Int, @@ -62,8 +60,7 @@ class FlumePollingInputDStream[T: ClassTag]( } } -private[streaming] -class FlumePollingReceiver( +private[streaming] class FlumePollingReceiver( addresses: Seq[InetSocketAddress], maxBatchSize: Int, parallelism: Int, diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java index ef8622481d599..a30157b94b972 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java @@ -1,3 +1,20 @@ +/* + * 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.flume; import java.net.InetSocketAddress; From e59cc205bccb4ddb01e9c026d2c7d0a1e5ec3825 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 22 Jul 2014 21:54:25 -0700 Subject: [PATCH 26/32] Use SparkFlumeEvent instead of the new type. Also, Flume Polling Receiver now uses the store(ArrayBuffer) method. --- .../streaming/flume/FlumeInputDStream.scala | 43 +++++++++++--- .../flume/FlumePollingInputDStream.scala | 57 +++++++------------ .../spark/streaming/flume/FlumeUtils.scala | 18 +++--- .../flume/JavaFlumePollingStreamSuite.java | 8 +-- .../flume/FlumePollingStreamSuite.scala | 17 +++--- 5 files changed, 76 insertions(+), 67 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 3efe6a0a5ae7b..4b2ea45fb81d0 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -30,9 +30,6 @@ import org.apache.flume.source.avro.AvroFlumeEvent import org.apache.flume.source.avro.Status import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer - -import org.apache.spark.util.Utils - import org.apache.spark.Logging import org.apache.spark.util.Utils import org.apache.spark.storage.StorageLevel @@ -42,11 +39,8 @@ import org.apache.spark.streaming.receiver.Receiver import org.jboss.netty.channel.ChannelPipelineFactory import org.jboss.netty.channel.Channels -import org.jboss.netty.channel.ChannelPipeline -import org.jboss.netty.channel.ChannelFactory import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.codec.compression._ -import org.jboss.netty.handler.execution.ExecutionHandler private[streaming] class FlumeInputDStream[T: ClassTag]( @@ -73,14 +67,47 @@ class SparkFlumeEvent() extends Externalizable { /* De-serialize from bytes. */ def readExternal(in: ObjectInput) { - val (headers, bodyBuff) = EventTransformer.readExternal(in) + val bodyLength = in.readInt() + val bodyBuff = new Array[Byte](bodyLength) + in.readFully(bodyBuff) + + val numHeaders = in.readInt() + val headers = new java.util.HashMap[CharSequence, CharSequence] + + for (i <- 0 until numHeaders) { + val keyLength = in.readInt() + val keyBuff = new Array[Byte](keyLength) + in.readFully(keyBuff) + val key : String = Utils.deserialize(keyBuff) + + val valLength = in.readInt() + val valBuff = new Array[Byte](valLength) + in.readFully(valBuff) + val value : String = Utils.deserialize(valBuff) + + headers.put(key, value) + } + event.setBody(ByteBuffer.wrap(bodyBuff)) event.setHeaders(headers) } /* Serialize to bytes. */ def writeExternal(out: ObjectOutput) { - EventTransformer.writeExternal(out, event.getHeaders, event.getBody.array()) + val body = event.getBody.array() + out.writeInt(body.length) + out.write(body) + + val numHeaders = event.getHeaders.size() + out.writeInt(numHeaders) + for ((k, v) <- event.getHeaders) { + val keyBuff = Utils.serialize(k.toString) + out.writeInt(keyBuff.length) + out.write(keyBuff) + val valBuff = Utils.serialize(v.toString) + out.writeInt(valBuff.length) + out.write(valBuff) + } } } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index 01de8781615a7..c93b7fee09f59 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -17,12 +17,11 @@ package org.apache.spark.streaming.flume -import java.io.{ObjectOutput, ObjectInput, Externalizable} import java.net.InetSocketAddress -import java.nio.ByteBuffer import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, Executors} import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import com.google.common.util.concurrent.ThreadFactoryBuilder @@ -53,9 +52,9 @@ private[streaming] class FlumePollingInputDStream[T: ClassTag]( val maxBatchSize: Int, val parallelism: Int, storageLevel: StorageLevel - ) extends ReceiverInputDStream[SparkFlumePollingEvent](_ssc) { + ) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { - override def getReceiver(): Receiver[SparkFlumePollingEvent] = { + override def getReceiver(): Receiver[SparkFlumeEvent] = { new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) } } @@ -65,7 +64,7 @@ private[streaming] class FlumePollingReceiver( maxBatchSize: Int, parallelism: Int, storageLevel: StorageLevel - ) extends Receiver[SparkFlumePollingEvent](storageLevel) with Logging { + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { lazy val channelFactoryExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). @@ -104,12 +103,13 @@ private[streaming] class FlumePollingReceiver( "Received batch of " + events.size() + " events with sequence number: " + seq) try { // Convert each Flume event to a serializable SparkPollingEvent + val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) var j = 0 while (j < events.size()) { - store(SparkFlumePollingEvent.fromSparkSinkEvent(events(j))) - logDebug("Stored events with seq:" + seq) + buffer += sparkSinkEventToSparkFlumeEvent(events(j)) j += 1 } + store(buffer) logDebug("Sending ack for sequence number: " + seq) // Send an ack to Flume so that Flume discards the events from its channels. client.ack(seq) @@ -152,6 +152,18 @@ private[streaming] class FlumePollingReceiver( }) channelFactory.releaseExternalResources() } + + /** + * Utility method to convert [[SparkSinkEvent]] to [[SparkFlumeEvent]] + * @param event - Event to convert to SparkFlumeEvent + * @return - The SparkSinkEvent generated from Spar + */ + private def sparkSinkEventToSparkFlumeEvent(event: SparkSinkEvent): SparkFlumeEvent = { + val sparkFlumeEvent = new SparkFlumeEvent() + sparkFlumeEvent.event.setBody(event.getBody) + sparkFlumeEvent.event.setHeaders(event.getHeaders) + sparkFlumeEvent + } } /** @@ -162,36 +174,5 @@ private[streaming] class FlumePollingReceiver( private class FlumeConnection(val transceiver: NettyTransceiver, val client: SparkFlumeProtocol.Callback) -/** - * Companion object of [[SparkFlumePollingEvent]] - */ -private[streaming] object SparkFlumePollingEvent { - def fromSparkSinkEvent(in: SparkSinkEvent): SparkFlumePollingEvent = { - val event = new SparkFlumePollingEvent() - event.event = in - event - } -} - -/* - * Unfortunately Avro does not allow including pre-compiled classes - so even though - * SparkSinkEvent is identical to AvroFlumeEvent, we need to create a new class and a wrapper - * around that to make it externalizable. - */ -class SparkFlumePollingEvent extends Externalizable with Logging { - var event: SparkSinkEvent = new SparkSinkEvent() - - /* De-serialize from bytes. */ - def readExternal(in: ObjectInput) { - val (headers, bodyBuff) = EventTransformer.readExternal(in) - event.setBody(ByteBuffer.wrap(bodyBuff)) - event.setHeaders(headers) - } - - /* Serialize to bytes. */ - def writeExternal(out: ObjectOutput) { - EventTransformer.writeExternal(out, event.getHeaders, event.getBody.array()) - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index c754fe33738b8..4b732c1592ab2 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -117,7 +117,7 @@ object FlumeUtils { * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. * This stream will poll the sink for data and will pull events as they are available. * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param host Address of the host on which the Spark Sink is running + * @param hostname Address of the host on which the Spark Sink is running * @param port Port of the host at which the Spark Sink is listening * @param storageLevel Storage level to use for storing the received objects */ @@ -127,7 +127,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[SparkFlumePollingEvent] = { + ): ReceiverInputDStream[SparkFlumeEvent] = { createPollingStream(ssc, Seq(new InetSocketAddress(hostname, port)), storageLevel) } @@ -143,7 +143,7 @@ object FlumeUtils { ssc: StreamingContext, addresses: Seq[InetSocketAddress], storageLevel: StorageLevel - ): ReceiverInputDStream[SparkFlumePollingEvent] = { + ): ReceiverInputDStream[SparkFlumeEvent] = { createPollingStream(ssc, addresses, storageLevel, DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) } @@ -166,8 +166,8 @@ object FlumeUtils { storageLevel: StorageLevel, maxBatchSize: Int, parallelism: Int - ): ReceiverInputDStream[SparkFlumePollingEvent] = { - new FlumePollingInputDStream[SparkFlumePollingEvent](ssc, addresses, maxBatchSize, + ): ReceiverInputDStream[SparkFlumeEvent] = { + new FlumePollingInputDStream[SparkFlumeEvent](ssc, addresses, maxBatchSize, parallelism, storageLevel) } @@ -183,7 +183,7 @@ object FlumeUtils { jssc: JavaStreamingContext, hostname: String, port: Int - ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createPollingStream(jssc, hostname, port, StorageLevel.MEMORY_AND_DISK_SER_2) } @@ -201,7 +201,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createPollingStream(jssc, Array(new InetSocketAddress(hostname, port)), storageLevel) } @@ -217,7 +217,7 @@ object FlumeUtils { jssc: JavaStreamingContext, addresses: Array[InetSocketAddress], storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createPollingStream(jssc, addresses, storageLevel, DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) } @@ -240,7 +240,7 @@ object FlumeUtils { storageLevel: StorageLevel, maxBatchSize: Int, parallelism: Int - ): JavaReceiverInputDStream[SparkFlumePollingEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) } } diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java index a30157b94b972..79c5b91654b42 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java @@ -32,13 +32,13 @@ public void testFlumeStream() { InetSocketAddress[] addresses = new InetSocketAddress[] { new InetSocketAddress("localhost", 12345) }; - JavaReceiverInputDStream test1 = + JavaReceiverInputDStream test1 = FlumeUtils.createPollingStream(ssc, "localhost", 12345); - JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( + JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( + JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( + JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2(), 100, 5); } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 3ff0cca523928..ec06b841d2321 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -42,10 +42,11 @@ import org.apache.spark.streaming.flume.sink._ test("flume polling test") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = - FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), StorageLevel.MEMORY_AND_DISK, 100, 1) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] - with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), + StorageLevel.MEMORY_AND_DISK, 100, 1) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) outputStream.register() @@ -75,10 +76,10 @@ import org.apache.spark.streaming.flume.sink._ // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _)) - val flumeStream: ReceiverInputDStream[SparkFlumePollingEvent] = + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, 100, 5) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumePollingEvent]] - with SynchronizedBuffer[Seq[SparkFlumePollingEvent]] + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) outputStream.register() @@ -115,7 +116,7 @@ import org.apache.spark.streaming.flume.sink._ } def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, - outputBuffer: ArrayBuffer[Seq[SparkFlumePollingEvent]]) { + outputBuffer: ArrayBuffer[Seq[SparkFlumeEvent]]) { val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val executor = Executors.newCachedThreadPool() val executorCompletion = new ExecutorCompletionService[Void](executor) From 65b76b4de8ebd9d89c9779e3205887ece0cbf663 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 22 Jul 2014 22:05:54 -0700 Subject: [PATCH 27/32] Fixing the unit test. --- .../apache/spark/streaming/flume/FlumePollingStreamSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index ec06b841d2321..739398c7ad92d 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -127,7 +127,7 @@ import org.apache.spark.streaming.flume.sink._ executorCompletion.take() } val startTime = System.currentTimeMillis() - while (outputBuffer.size < 5 * channels.size && + while (outputBuffer.size < 5 && System.currentTimeMillis() - startTime < 15000) { logInfo("output.size = " + outputBuffer.size) Thread.sleep(100) From 73d6f6d174013ce4067d07ecc791d70344d3dcc4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 24 Jul 2014 15:03:59 -0700 Subject: [PATCH 28/32] Cleaned up tests a bit. Added some docs in multiple places. --- .../flume/sink/SparkAvroCallbackHandler.scala | 9 +++++++ .../streaming/flume/sink/SparkSink.scala | 24 +++++++++++------ .../flume/sink/TransactionProcessor.scala | 7 +++++ .../flume/FlumePollingStreamSuite.scala | 27 +++++++++++-------- 4 files changed, 48 insertions(+), 19 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index 6249a3197d07d..7da8eb3e35912 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -31,6 +31,15 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark * is rolled back. */ +// Flume forces transactions to be thread-local. So each transaction *must* be committed, or +// rolled back from the thread it was originally created in. So each getEvents call from Spark +// creates a TransactionProcessor which runs in a new thread, in which the transaction is created +// and events are pulled off the channel. Once the events are sent to spark, +// that thread is blocked and the TransactionProcessor is saved in a map, +// until an ACK or NACK comes back or the transaction times out (after the specified timeout). +// When the response comes or a timeout is hit, the TransactionProcessor is retrieved and then +// unblocked, at which point the transaction is committed or rolled back. + private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index d5afde0fae19d..7b735133e3d14 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -36,15 +36,23 @@ import org.apache.flume.sink.AbstractSink * if an ACK is not received from Spark within that time * threads - Number of threads to use to receive requests from Spark (Default: 10) * + * This sink is unlike other Flume sinks in the sense that it does not push data, + * instead the process method in this sink simply blocks the SinkRunner the first time it is + * called. This sink starts up an Avro IPC server that uses the SparkFlumeProtocol. + * + * Each time a getEventBatch call comes, creates a transaction and reads events + * from the channel. When enough events are read, the events are sent to the Spark receiver and + * the thread itself is blocked and a reference to it saved off. + * + * When the ack for that batch is received, + * the thread which created the transaction is is retrieved and it commits the transaction with the + * channel from the same thread it was originally created in (since Flume transactions are + * thread local). If a nack is received instead, the sink rolls back the transaction. If no ack + * is received within the specified timeout, the transaction is rolled back too. If an ack comes + * after that, it is simply ignored and the events get re-sent. + * */ -// Flume forces transactions to be thread-local. So each transaction *must* be committed, or -// rolled back from the thread it was originally created in. So each getEvents call from Spark -// creates a TransactionProcessor which runs in a new thread, in which the transaction is created -// and events are pulled off the channel. Once the events are sent to spark, -// that thread is blocked and the TransactionProcessor is saved in a map, -// until an ACK or NACK comes back or the transaction times out (after the specified timeout). -// When the response comes, the TransactionProcessor is retrieved and then unblocked, -// at which point the transaction is committed or rolled back. + private[flume] class SparkSink extends AbstractSink with Logging with Configurable { diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala index 6f4e50b0f1d63..b9e3c786ebb3b 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -213,6 +213,13 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, charSeqMap } + /** + * When the thread is started it sets as many events as the batch size or less (if enough + * events aren't available) into the eventBatch and object and lets any threads waiting on the + * [[getEventBatch]] method to proceed. Then this thread waits for acks or nacks to come in, + * or for a specified timeout and commits or rolls back the transaction. + * @return + */ override def call(): Void = { populateEvents() processAckOrNack() diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 739398c7ad92d..47071d0cc4714 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -35,16 +35,20 @@ import org.apache.spark.streaming.util.ManualClock import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} import org.apache.spark.streaming.flume.sink._ - class FlumePollingStreamSuite extends TestSuiteBase { +class FlumePollingStreamSuite extends TestSuiteBase { val testPort = 9999 + val batchCount = 5 + val eventsPerBatch = 100 + val totalEventsPerChannel = batchCount * eventsPerBatch + val channelCapacity = 5000 test("flume polling test") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), - StorageLevel.MEMORY_AND_DISK, 100, 1) + StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) @@ -52,7 +56,7 @@ import org.apache.spark.streaming.flume.sink._ // Start the channel and sink. val context = new Context() - context.put("capacity", "5000") + context.put("capacity", channelCapacity.toString) context.put("transactionCapacity", "1000") context.put("keep-alive", "0") val channel = new MemoryChannel() @@ -77,7 +81,8 @@ import org.apache.spark.streaming.flume.sink._ val ssc = new StreamingContext(conf, batchDuration) val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _)) val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, 100, 5) + FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, + eventsPerBatch, 5) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) @@ -85,7 +90,7 @@ import org.apache.spark.streaming.flume.sink._ // Start the channel and sink. val context = new Context() - context.put("capacity", "5000") + context.put("capacity", channelCapacity.toString) context.put("transactionCapacity", "1000") context.put("keep-alive", "0") val channel = new MemoryChannel() @@ -127,7 +132,7 @@ import org.apache.spark.streaming.flume.sink._ executorCompletion.take() } val startTime = System.currentTimeMillis() - while (outputBuffer.size < 5 && + while (outputBuffer.size < batchCount * channels.size && System.currentTimeMillis() - startTime < 15000) { logInfo("output.size = " + outputBuffer.size) Thread.sleep(100) @@ -138,9 +143,9 @@ import org.apache.spark.streaming.flume.sink._ ssc.stop() val flattenedBuffer = outputBuffer.flatten - assert(flattenedBuffer.size === 25 * channels.size) + assert(flattenedBuffer.size === totalEventsPerChannel * channels.size) var counter = 0 - for (k <- 0 until channels.size; i <- 0 until 25) { + for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { val eventToVerify = EventBuilder.withBody((channels(k).getName + " - " + String.valueOf(i)).getBytes("utf-8"), Map[String, String]("test-" + i.toString -> "header")) @@ -157,7 +162,7 @@ import org.apache.spark.streaming.flume.sink._ j += 1 } } - assert(counter === 25 * channels.size) + assert(counter === totalEventsPerChannel * channels.size) } def assertChannelIsEmpty(channel: MemoryChannel) = { @@ -170,10 +175,10 @@ import org.apache.spark.streaming.flume.sink._ private class TxnSubmitter(channel: MemoryChannel, clock: ManualClock) extends Callable[Void] { override def call(): Void = { var t = 0 - for (i <- 0 until 5) { + for (i <- 0 until batchCount) { val tx = channel.getTransaction tx.begin() - for (j <- 0 until 5) { + for (j <- 0 until eventsPerBatch) { channel.put(EventBuilder.withBody((channel.getName + " - " + String.valueOf(t)).getBytes( "utf-8"), Map[String, String]("test-" + t.toString -> "header"))) From 1f47364aea322ef3747a16b5fac62754c9ea655c Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 25 Jul 2014 14:25:11 -0700 Subject: [PATCH 29/32] Minor fixes. --- .../streaming/flume/FlumePollingInputDStream.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala index c93b7fee09f59..148262bb6771e 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -76,7 +76,7 @@ private[streaming] class FlumePollingReceiver( lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) - private val connections = new LinkedBlockingQueue[FlumeConnection]() + private lazy val connections = new LinkedBlockingQueue[FlumeConnection]() override def onStart(): Unit = { // Create the connections to each Flume agent. @@ -102,11 +102,11 @@ private[streaming] class FlumePollingReceiver( logDebug( "Received batch of " + events.size() + " events with sequence number: " + seq) try { - // Convert each Flume event to a serializable SparkPollingEvent + // Convert each Flume event to a serializable SparkFlumeEvent val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) var j = 0 while (j < events.size()) { - buffer += sparkSinkEventToSparkFlumeEvent(events(j)) + buffer += toSparkFlumeEvent(events(j)) j += 1 } store(buffer) @@ -156,9 +156,9 @@ private[streaming] class FlumePollingReceiver( /** * Utility method to convert [[SparkSinkEvent]] to [[SparkFlumeEvent]] * @param event - Event to convert to SparkFlumeEvent - * @return - The SparkSinkEvent generated from Spar + * @return - The SparkFlumeEvent generated from SparkSinkEvent */ - private def sparkSinkEventToSparkFlumeEvent(event: SparkSinkEvent): SparkFlumeEvent = { + private def toSparkFlumeEvent(event: SparkSinkEvent): SparkFlumeEvent = { val sparkFlumeEvent = new SparkFlumeEvent() sparkFlumeEvent.event.setBody(event.getBody) sparkFlumeEvent.event.setHeaders(event.getHeaders) From 7a1bc6e3ad727a9c7378a90f4d81b8a4ff8db5d4 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 25 Jul 2014 14:51:37 -0700 Subject: [PATCH 30/32] Fix SparkBuild.scala --- project/SparkBuild.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 162bb8900cb90..21dcb40e9a22f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -31,10 +31,12 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, - streaming, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = + streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, + streamingZeromq) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "spark", "sql", "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", - "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) + "spark", "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", + "streaming-mqtt", "streaming-twitter", "streaming-zeromq"). + map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") From 5f212cebf63d79ee30598758d254ae8f0ed826f5 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 28 Jul 2014 13:20:58 -0700 Subject: [PATCH 31/32] Ignore Spark Sink from mima. --- project/SparkBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ae985fe549c85..7db88c42e120d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -159,8 +159,8 @@ object SparkBuild extends PomBuild { /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl).exists(x => x == y)). - foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) + allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl, streamingFlumeSink).exists(x => + x == y)).foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) From e48d785d57b74cea92267ea53f7b538a34addccb Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Mon, 28 Jul 2014 15:51:57 -0700 Subject: [PATCH 32/32] Documenting flume-sink being ignored for Mima checks. --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7db88c42e120d..860c0542253cd 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -157,7 +157,7 @@ object SparkBuild extends PomBuild { /* Enable tests settings for all projects except examples, assembly and tools */ (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) - /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ + /* Enable Mima for all projects except spark, hive, catalyst, sql, repl and flume-sink */ // TODO: Add Sql to mima checks allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl, streamingFlumeSink).exists(x => x == y)).foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x))