aboutsummaryrefslogtreecommitdiff
path: root/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala
blob: 7dc960691398b1a8529cc6adffb9055d714f0468 (plain) (blame)
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
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
/*
 * 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.{Externalizable, ObjectInput, ObjectOutput}
import java.net.InetSocketAddress
import java.nio.ByteBuffer
import java.util.concurrent.Executors

import scala.collection.JavaConverters._
import scala.reflect.ClassTag

import org.apache.avro.ipc.NettyServer
import org.apache.avro.ipc.specific.SpecificResponder
import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol, Status}
import org.jboss.netty.channel.{ChannelPipeline, ChannelPipelineFactory, Channels}
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
import org.jboss.netty.handler.codec.compression._

import org.apache.spark.internal.Logging
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming.StreamingContext
import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.receiver.Receiver
import org.apache.spark.util.Utils

private[streaming]
class FlumeInputDStream[T: ClassTag](
  _ssc: StreamingContext,
  host: String,
  port: Int,
  storageLevel: StorageLevel,
  enableDecompression: Boolean
) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) {

  override def getReceiver(): Receiver[SparkFlumeEvent] = {
    new FlumeReceiver(host, port, storageLevel, enableDecompression)
  }
}

/**
 * A wrapper class for AvroFlumeEvent's with a custom serialization format.
 *
 * This is necessary because AvroFlumeEvent uses inner data structures
 * which are not serializable.
 */
class SparkFlumeEvent() extends Externalizable {
  var event: AvroFlumeEvent = new AvroFlumeEvent()

  /* De-serialize from bytes. */
  def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
    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): Unit = Utils.tryOrIOException {
    val body = event.getBody
    out.writeInt(body.remaining())
    Utils.writeByteBuffer(body, out)

    val numHeaders = event.getHeaders.size()
    out.writeInt(numHeaders)
    for ((k, v) <- event.getHeaders.asScala) {
      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)
    }
  }
}

private[streaming] object SparkFlumeEvent {
  def fromAvroFlumeEvent(in: AvroFlumeEvent): SparkFlumeEvent = {
    val event = new SparkFlumeEvent
    event.event = in
    event
  }
}

/** A simple server that implements Flume's Avro protocol. */
private[streaming]
class FlumeEventServer(receiver: FlumeReceiver) extends AvroSourceProtocol {
  override def append(event: AvroFlumeEvent): Status = {
    receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))
    Status.OK
  }

  override def appendBatch(events: java.util.List[AvroFlumeEvent]): Status = {
    events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)))
    Status.OK
  }
}

/** A NetworkReceiver which listens for events using the
  * Flume Avro interface. */
private[streaming]
class FlumeReceiver(
    host: String,
    port: Int,
    storageLevel: StorageLevel,
    enableDecompression: Boolean
  ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging {

  lazy val responder = new SpecificResponder(
    classOf[AvroSourceProtocol], new FlumeEventServer(this))
  var server: NettyServer = null

  private def initServer() = {
    if (enableDecompression) {
      val channelFactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(),
                                                             Executors.newCachedThreadPool())
      val channelPipelineFactory = new CompressionChannelPipelineFactory()

      new NettyServer(
        responder,
        new InetSocketAddress(host, port),
        channelFactory,
        channelPipelineFactory,
        null)
    } else {
      new NettyServer(responder, new InetSocketAddress(host, port))
    }
  }

  def onStart() {
    synchronized {
      if (server == null) {
        server = initServer()
        server.start()
      } else {
        logWarning("Flume receiver being asked to start more then once with out close")
      }
    }
    logInfo("Flume receiver started")
  }

  def onStop() {
    synchronized {
      if (server != null) {
        server.close()
        server = null
      }
    }
    logInfo("Flume receiver stopped")
  }

  override def preferredLocation: Option[String] = Option(host)

  /** A Netty Pipeline factory that will decompress incoming data from
    * and the Netty client and compress data going back to the client.
    *
    * The compression on the return is required because Flume requires
    * a successful response to indicate it can remove the event/batch
    * from the configured channel
    */
  private[streaming]
  class CompressionChannelPipelineFactory extends ChannelPipelineFactory {
    def getPipeline(): ChannelPipeline = {
      val pipeline = Channels.pipeline()
      val encoder = new ZlibEncoder(6)
      pipeline.addFirst("deflater", encoder)
      pipeline.addFirst("inflater", new ZlibDecoder())
      pipeline
    }
  }
}