@@ -34,10 +34,8 @@ import org.apache.avro.ipc.NettyServer
34
34
import org .apache .avro .ipc .specific .SpecificResponder
35
35
import java .net .InetSocketAddress
36
36
37
- class SparkSink () extends AbstractSink with Configurable {
37
+ class SparkSink extends AbstractSink with Configurable {
38
38
private val LOG = LoggerFactory .getLogger(this .getClass)
39
- private val lock = new ReentrantLock ()
40
- private val blockingCondition = lock.newCondition()
41
39
42
40
// This sink will not persist sequence numbers and reuses them if it gets restarted.
43
41
// 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 {
58
56
59
57
private val processorMap = new ConcurrentHashMap [CharSequence , TransactionProcessor ]()
60
58
61
- private var processorFactory : Option [SparkHandlerFactory ] = None
59
+ private var processorManager : Option [TransactionProcessorManager ] = None
62
60
private var hostname : String = SparkSinkConfig .DEFAULT_HOSTNAME
63
61
private var port : Int = 0
64
62
private var maxThreads : Int = SparkSinkConfig .DEFAULT_MAX_THREADS
65
63
private var serverOpt : Option [NettyServer ] = None
66
- private var running = false
64
+
65
+ private val blockingLatch = new CountDownLatch (1 )
67
66
68
67
override def start () {
69
68
transactionExecutorOpt = Option (Executors .newFixedThreadPool(numProcessors,
70
69
new ThreadFactoryBuilder ().setDaemon(true )
71
70
.setNameFormat(" Spark Sink, " + getName + " Processor Thread - %d" ).build()))
72
71
73
- processorFactory = Option (new SparkHandlerFactory (numProcessors))
72
+ processorManager = Option (new TransactionProcessorManager (numProcessors))
74
73
75
74
val responder = new SpecificResponder (classOf [SparkFlumeProtocol ], new AvroCallbackHandler ())
76
75
@@ -80,12 +79,6 @@ class SparkSink() extends AbstractSink with Configurable {
80
79
serverOpt = Option (new NettyServer (responder, new InetSocketAddress (hostname, port)))
81
80
82
81
serverOpt.map(server => server.start())
83
- lock.lock()
84
- try {
85
- running = true
86
- } finally {
87
- lock.unlock()
88
- }
89
82
super .start()
90
83
}
91
84
@@ -95,65 +88,48 @@ class SparkSink() extends AbstractSink with Configurable {
95
88
server.close()
96
89
server.join()
97
90
})
98
- lock.lock()
99
- try {
100
- running = false
101
- blockingCondition.signalAll()
102
- } finally {
103
- lock.unlock()
104
- }
91
+ blockingLatch.countDown()
92
+ super .stop()
105
93
}
106
94
107
95
override def configure (ctx : Context ) {
108
96
import SparkSinkConfig ._
109
97
hostname = ctx.getString(CONF_HOSTNAME , DEFAULT_HOSTNAME )
110
- val portOpt = Option (ctx.getInteger(CONF_PORT ))
111
- if (portOpt.isDefined) {
112
- port = portOpt.get
113
- } else {
114
- throw new ConfigurationException (" The Port to bind must be specified" )
115
- }
98
+ port = Option (ctx.getInteger(CONF_PORT )).
99
+ getOrElse(throw new ConfigurationException (" The port to bind to must be specified" ))
116
100
numProcessors = ctx.getInteger(PROCESSOR_COUNT , DEFAULT_PROCESSOR_COUNT )
117
101
transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT , DEFAULT_TRANSACTION_TIMEOUT )
118
102
maxThreads = ctx.getInteger(CONF_MAX_THREADS , DEFAULT_MAX_THREADS )
119
103
}
120
104
121
105
override def process (): Status = {
122
106
// This method is called in a loop by the Flume framework - block it until the sink is
123
- // stopped to save CPU resources
124
- lock.lock()
125
- try {
126
- while (running) {
127
- blockingCondition.await()
128
- }
129
- } finally {
130
- lock.unlock()
131
- }
107
+ // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is
108
+ // being shut down.
109
+ blockingLatch.await()
132
110
Status .BACKOFF
133
111
}
134
112
135
113
136
114
// Object representing an empty batch returned by the txn processor due to some error.
137
115
case object ErrorEventBatch extends EventBatch
138
116
139
- private class AvroCallbackHandler () extends SparkFlumeProtocol {
117
+ private class AvroCallbackHandler extends SparkFlumeProtocol {
140
118
141
119
override def getEventBatch (n : Int ): EventBatch = {
142
- val processor = processorFactory .get.checkOut(n)
120
+ val processor = processorManager .get.checkOut(n)
143
121
transactionExecutorOpt.map(executor => executor.submit(processor))
144
122
// Wait until a batch is available - can be null if some error was thrown
145
- val eventBatch = processor.eventQueue.take()
146
- eventBatch match {
123
+ processor.eventQueue.take() match {
147
124
case ErrorEventBatch => throw new FlumeException (" Something went wrong. No events" +
148
125
" retrieved from channel." )
149
- case events => {
150
- processorMap.put(events .getSequenceNumber, processor)
126
+ case eventBatch : EventBatch =>
127
+ processorMap.put(eventBatch .getSequenceNumber, processor)
151
128
if (LOG .isDebugEnabled) {
152
- LOG .debug(" Sent " + events.getEventBatch .size() +
153
- " events with sequence number: " + events .getSequenceNumber)
129
+ LOG .debug(" Sent " + eventBatch.getEvents .size() +
130
+ " events with sequence number: " + eventBatch .getSequenceNumber)
154
131
}
155
- events
156
- }
132
+ eventBatch
157
133
}
158
134
}
159
135
@@ -214,41 +190,23 @@ class SparkSink() extends AbstractSink with Configurable {
214
190
tx.begin()
215
191
try {
216
192
eventBatch.setSequenceNumber(seqBase + seqNum.incrementAndGet())
217
- val events = eventBatch.getEventBatch
193
+ val events = eventBatch.getEvents
218
194
events.clear()
219
195
val loop = new Breaks
220
196
var gotEventsInThisTxn = false
221
197
loop.breakable {
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
227
- val eventOpt = Option (getChannel.take())
228
- eventOpt.map(event => {
229
- events.add(new SparkSinkEvent (toCharSequenceMap(event
230
- .getHeaders),
231
- ByteBuffer .wrap(event.getBody)))
232
- gotEventsInThisTxn = true
233
- })
234
- if (eventOpt.isEmpty) {
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!" )
198
+ while (events.size() < maxBatchSize) {
199
+ Option (getChannel.take()) match {
200
+ case Some (event) =>
201
+ events.add(new SparkSinkEvent (toCharSequenceMap(event.getHeaders),
202
+ ByteBuffer .wrap(event.getBody)))
203
+ gotEventsInThisTxn = true
204
+ case None =>
205
+ if (! gotEventsInThisTxn) {
206
+ Thread .sleep(500 )
207
+ } else {
208
+ loop.break()
246
209
}
247
- maxBatchSize += 1
248
- Thread .sleep(500 )
249
- } else {
250
- loop.break()
251
- }
252
210
}
253
211
}
254
212
}
@@ -284,7 +242,7 @@ class SparkSink() extends AbstractSink with Configurable {
284
242
} finally {
285
243
resultQueueUpdateLock.unlock()
286
244
}
287
- eventBatch.getEventBatch .clear()
245
+ eventBatch.getEvents .clear()
288
246
// If the batch failed on spark side, throw a FlumeException
289
247
maybeResult.map(success =>
290
248
if (! success) {
@@ -315,7 +273,7 @@ class SparkSink() extends AbstractSink with Configurable {
315
273
// remove the event from the map and then clear the value
316
274
resultQueue.clear()
317
275
processorMap.remove(eventBatch.getSequenceNumber)
318
- processorFactory .get.checkIn(this )
276
+ processorManager .get.checkIn(this )
319
277
tx.close()
320
278
}
321
279
}
@@ -328,7 +286,7 @@ class SparkSink() extends AbstractSink with Configurable {
328
286
}
329
287
}
330
288
331
- private class SparkHandlerFactory (val maxInstances : Int ) {
289
+ private class TransactionProcessorManager (val maxInstances : Int ) {
332
290
val queue = new scala.collection.mutable.Queue [TransactionProcessor ]
333
291
val queueModificationLock = new ReentrantLock ()
334
292
var currentSize = 0
0 commit comments