-
Notifications
You must be signed in to change notification settings - Fork 28.9k
SPARK-1478: Upgrade FlumeInputDStream's FlumeReceiver to support FLUME-1915 #566
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -36,17 +36,27 @@ import org.apache.spark.streaming.StreamingContext | |
| import org.apache.spark.streaming.dstream._ | ||
| import org.apache.spark.Logging | ||
| import org.apache.spark.streaming.receiver.Receiver | ||
| import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory | ||
| import org.jboss.netty.channel.ChannelPipelineFactory | ||
| import java.util.concurrent.Executors | ||
| import org.jboss.netty.channel.Channels | ||
| import org.jboss.netty.handler.codec.compression.ZlibDecoder | ||
| import org.jboss.netty.handler.codec.compression.ZlibEncoder | ||
| import org.jboss.netty.channel.ChannelPipeline | ||
| import org.jboss.netty.channel.ChannelFactory | ||
| import org.jboss.netty.handler.execution.ExecutionHandler | ||
|
|
||
| private[streaming] | ||
| class FlumeInputDStream[T: ClassTag]( | ||
| @transient ssc_ : StreamingContext, | ||
| host: String, | ||
| port: Int, | ||
| storageLevel: StorageLevel | ||
| storageLevel: StorageLevel, | ||
| enableDecompression: Boolean | ||
| ) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { | ||
|
|
||
| override def getReceiver(): Receiver[SparkFlumeEvent] = { | ||
| new FlumeReceiver(host, port, storageLevel) | ||
| new FlumeReceiver(host, port, storageLevel, enableDecompression) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -134,22 +144,64 @@ private[streaming] | |
| class FlumeReceiver( | ||
| host: String, | ||
| port: Int, | ||
| storageLevel: StorageLevel | ||
| storageLevel: StorageLevel, | ||
| enableDecompression: Boolean | ||
| ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { | ||
|
|
||
| lazy val responder = new SpecificResponder( | ||
| classOf[AvroSourceProtocol], new FlumeEventServer(this)) | ||
| lazy val server = new NettyServer(responder, new InetSocketAddress(host, port)) | ||
| var server: NettyServer = null | ||
|
|
||
| private def initServer() = { | ||
| if (enableDecompression) { | ||
| val channelFactory = new NioServerSocketChannelFactory | ||
| (Executors.newCachedThreadPool(), Executors.newCachedThreadPool()); | ||
| val channelPipelieFactory = new CompressionChannelPipelineFactory() | ||
|
|
||
| new NettyServer( | ||
| responder, | ||
| new InetSocketAddress(host, port), | ||
| channelFactory, | ||
| channelPipelieFactory, | ||
| null) | ||
| } else { | ||
| new NettyServer(responder, new InetSocketAddress(host, port)) | ||
| } | ||
| } | ||
|
|
||
| def onStart() { | ||
| server.start() | ||
| 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() { | ||
| server.close() | ||
| synchronized { | ||
| if (server != null) { | ||
| server.close() | ||
| server = null | ||
| } | ||
| } | ||
| logInfo("Flume receiver stopped") | ||
| } | ||
|
|
||
| override def preferredLocation = Some(host) | ||
| } | ||
|
|
||
| private[streaming] | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you add comments to this class, explaining what this class does and why it is necessary? |
||
| class CompressionChannelPipelineFactory extends ChannelPipelineFactory { | ||
|
|
||
| def getPipeline() = { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just a line of comment saying what pipeline does this return. For Flume noob's like me ;)
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Cool will do before the weekend is done. Thanks |
||
| val pipeline = Channels.pipeline() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Formatting issue. 2 space indents required. |
||
| val encoder = new ZlibEncoder(6) | ||
| pipeline.addFirst("deflater", encoder) | ||
| pipeline.addFirst("inflater", new ZlibDecoder()) | ||
| pipeline | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
please dedup, and sort. see import style in https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide