1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
|
/*
* 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
import java.util.concurrent.{Executors, LinkedBlockingQueue, TimeUnit}
import scala.collection.JavaConverters._
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.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
import org.apache.spark.internal.Logging
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.dstream.ReceiverInputDStream
import org.apache.spark.streaming.flume.sink._
import org.apache.spark.streaming.receiver.Receiver
/**
* A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running
* [[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
* @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](
_ssc: StreamingContext,
val addresses: Seq[InetSocketAddress],
val maxBatchSize: Int,
val parallelism: Int,
storageLevel: StorageLevel
) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) {
override def getReceiver(): Receiver[SparkFlumeEvent] = {
new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel)
}
}
private[streaming] class FlumePollingReceiver(
addresses: Seq[InetSocketAddress],
maxBatchSize: Int,
parallelism: Int,
storageLevel: StorageLevel
) extends Receiver[SparkFlumeEvent](storageLevel) with Logging {
lazy val channelFactoryExecutor =
Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true).
setNameFormat("Flume Receiver Channel Thread - %d").build())
lazy val channelFactory =
new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor)
lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism,
new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build())
private lazy val connections = new LinkedBlockingQueue[FlumeConnection]()
override def onStart(): Unit = {
// Create the connections to each Flume agent.
addresses.foreach { host =>
val transceiver = new NettyTransceiver(host, channelFactory)
val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver)
connections.add(new FlumeConnection(transceiver, client))
}
for (i <- 0 until parallelism) {
logInfo("Starting Flume Polling Receiver worker threads..")
// Threads that pull data from Flume.
receiverExecutor.submit(new FlumeBatchFetcher(this))
}
}
override def onStop(): Unit = {
logInfo("Shutting down Flume Polling Receiver")
receiverExecutor.shutdown()
// Wait upto a minute for the threads to die
if (!receiverExecutor.awaitTermination(60, TimeUnit.SECONDS)) {
receiverExecutor.shutdownNow()
}
connections.asScala.foreach(_.transceiver.close())
channelFactory.releaseExternalResources()
}
private[flume] def getConnections: LinkedBlockingQueue[FlumeConnection] = {
this.connections
}
private[flume] def getMaxBatchSize: Int = {
this.maxBatchSize
}
}
/**
* 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[flume] class FlumeConnection(val transceiver: NettyTransceiver,
val client: SparkFlumeProtocol.Callback)
|