@@ -30,9 +30,8 @@ import java.util.concurrent._
3030import java .util
3131import org .apache .flume .conf .{ConfigurationException , Configurable }
3232import com .google .common .util .concurrent .ThreadFactoryBuilder
33- import org .apache .avro .ipc .{ NettyTransceiver , NettyServer }
33+ import org .apache .avro .ipc .NettyServer
3434import org .apache .avro .ipc .specific .SpecificResponder
35- import org .jboss .netty .channel .socket .nio .NioServerSocketChannelFactory
3635import java .net .InetSocketAddress
3736
3837class SparkSink () extends AbstractSink with Configurable {
@@ -75,12 +74,10 @@ class SparkSink() extends AbstractSink with Configurable {
7574
7675 val responder = new SpecificResponder (classOf [SparkFlumeProtocol ], new AvroCallbackHandler ())
7776
78- serverOpt = Option (new NettyServer (responder, new InetSocketAddress (hostname, port),
79- new NioServerSocketChannelFactory (
80- Executors .newCachedThreadPool(new ThreadFactoryBuilder ().setNameFormat(
81- " Spark Sink " + classOf [NettyTransceiver ].getSimpleName + " Boss-%d" ).build),
82- Executors .newFixedThreadPool(maxThreads, new ThreadFactoryBuilder ().setNameFormat(
83- " Spark Sink " + classOf [NettyTransceiver ].getSimpleName + " I/O Worker-%d" ).build))))
77+ // Using the constructor that takes specific thread-pools requires bringing in netty
78+ // dependencies which are being excluded in the build. In practice,
79+ // Netty dependencies are already available on the JVM as Flume would have pulled them in.
80+ serverOpt = Option (new NettyServer (responder, new InetSocketAddress (hostname, port)))
8481
8582 serverOpt.map(server => server.start())
8683 lock.lock()
@@ -93,10 +90,14 @@ class SparkSink() extends AbstractSink with Configurable {
9390 }
9491
9592 override def stop () {
93+ transactionExecutorOpt.map(executor => executor.shutdownNow())
94+ serverOpt.map(server => {
95+ server.close()
96+ server.join()
97+ })
9698 lock.lock()
9799 try {
98100 running = false
99- transactionExecutorOpt.map(executor => executor.shutdownNow())
100101 blockingCondition.signalAll()
101102 } finally {
102103 lock.unlock()
@@ -131,23 +132,28 @@ class SparkSink() extends AbstractSink with Configurable {
131132 Status .BACKOFF
132133 }
133134
135+
136+ // Object representing an empty batch returned by the txn processor due to some error.
137+ case object ErrorEventBatch extends EventBatch
138+
134139 private class AvroCallbackHandler () extends SparkFlumeProtocol {
135140
136141 override def getEventBatch (n : Int ): EventBatch = {
137142 val processor = processorFactory.get.checkOut(n)
138143 transactionExecutorOpt.map(executor => executor.submit(processor))
139144 // Wait until a batch is available - can be null if some error was thrown
140- val eventBatch = Option (processor.eventQueue.take())
141- if (eventBatch.isDefined) {
142- val eventsToBeSent = eventBatch.get
143- processorMap.put(eventsToBeSent.getSequenceNumber, processor)
144- if (LOG .isDebugEnabled) {
145- LOG .debug(" Sent " + eventsToBeSent.getEventBatch.size() +
146- " events with sequence number: " + eventsToBeSent.getSequenceNumber)
145+ val eventBatch = processor.eventQueue.take()
146+ eventBatch match {
147+ case ErrorEventBatch => throw new FlumeException (" Something went wrong. No events" +
148+ " retrieved from channel." )
149+ case events => {
150+ processorMap.put(events.getSequenceNumber, processor)
151+ if (LOG .isDebugEnabled) {
152+ LOG .debug(" Sent " + events.getEventBatch.size() +
153+ " events with sequence number: " + events.getSequenceNumber)
154+ }
155+ events
147156 }
148- eventsToBeSent
149- } else {
150- throw new FlumeException (" Error while trying to retrieve events from the channel." )
151157 }
152158 }
153159
@@ -211,17 +217,38 @@ class SparkSink() extends AbstractSink with Configurable {
211217 val events = eventBatch.getEventBatch
212218 events.clear()
213219 val loop = new Breaks
220+ var gotEventsInThisTxn = false
214221 loop.breakable {
215- for (i <- 0 until maxBatchSize) {
222+ var i = 0
223+ // Using for here causes the maxBatchSize change to be ineffective as the Range gets
224+ // pregenerated
225+ while (i < maxBatchSize) {
226+ i += 1
216227 val eventOpt = Option (getChannel.take())
217-
218228 eventOpt.map(event => {
219229 events.add(new SparkSinkEvent (toCharSequenceMap(event
220230 .getHeaders),
221231 ByteBuffer .wrap(event.getBody)))
232+ gotEventsInThisTxn = true
222233 })
223234 if (eventOpt.isEmpty) {
224- loop.break()
235+ if (! gotEventsInThisTxn) {
236+ // To avoid sending empty batches, we wait till events are available backing off
237+ // between attempts to get events. Each attempt to get an event though causes one
238+ // iteration to be lost. To ensure that we still send back maxBatchSize number of
239+ // events, we cheat and increase the maxBatchSize by 1 to account for the lost
240+ // iteration. Even throwing an exception is expensive as Avro will serialize it
241+ // and send it over the wire, which is useless. Before incrementing though,
242+ // ensure that we are not anywhere near INT_MAX.
243+ if (maxBatchSize >= Int .MaxValue / 2 ) {
244+ // Random sanity check
245+ throw new RuntimeException (" Safety exception - polled too many times, no events!" )
246+ }
247+ maxBatchSize += 1
248+ Thread .sleep(500 )
249+ } else {
250+ loop.break()
251+ }
225252 }
226253 }
227254 }
@@ -283,7 +310,7 @@ class SparkSink() extends AbstractSink with Configurable {
283310 null // No point rethrowing the exception
284311 } finally {
285312 // Must *always* release the caller thread
286- eventQueue.put(null )
313+ eventQueue.put(ErrorEventBatch )
287314 // In the case of success coming after the timeout, but before resetting the seq number
288315 // remove the event from the map and then clear the value
289316 resultQueue.clear()
0 commit comments