-
Notifications
You must be signed in to change notification settings - Fork 28.9k
SPARK-1478 #405
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
SPARK-1478 #405
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 |
|---|---|---|
|
|
@@ -29,14 +29,38 @@ object FlumeUtils { | |
| * @param hostname Hostname of the slave machine to which the flume data will be sent | ||
| * @param port Port of the slave machine to which the flume data will be sent | ||
| * @param storageLevel Storage level to use for storing the received objects | ||
| * @param enableCompression Should Netty Server decode input stream from client | ||
|
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. I am a little confused. What does "should netty server decode input stream" have to do with "compression" ? Maybe you wanted to say "should netty server decompress input stream" ?
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. Also, this parameter does not exist in this version of the createStream method! |
||
| */ | ||
| def createStream ( | ||
| ssc: StreamingContext, | ||
| hostname: String, | ||
| port: Int, | ||
| storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 | ||
| ): DStream[SparkFlumeEvent] = { | ||
| val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) | ||
| createStream(ssc, hostname, port, storageLevel, false); | ||
| } | ||
|
|
||
| /** | ||
| * Create a input stream from a Flume source. | ||
| * @param ssc StreamingContext object | ||
| * @param hostname Hostname of the slave machine to which the flume data will be sent | ||
| * @param port Port of the slave machine to which the flume data will be sent | ||
| * @param storageLevel Storage level to use for storing the received objects | ||
| * @param enableCompression Should Netty Server decode input stream from client | ||
|
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. Same comment as above, for enableCompression. |
||
| */ | ||
| def createStream ( | ||
| ssc: StreamingContext, | ||
| hostname: String, | ||
| port: Int, | ||
| storageLevel: StorageLevel, | ||
| enableCompression: Boolean | ||
| ): DStream[SparkFlumeEvent] = { | ||
| val inputStream = new FlumeInputDStream[SparkFlumeEvent]( | ||
| ssc, | ||
| hostname, | ||
| port, | ||
| storageLevel, | ||
| enableCompression) | ||
| inputStream | ||
| } | ||
|
|
||
|
|
@@ -66,6 +90,28 @@ object FlumeUtils { | |
| port: Int, | ||
| storageLevel: StorageLevel | ||
| ): JavaDStream[SparkFlumeEvent] = { | ||
| createStream(jssc.ssc, hostname, port, storageLevel) | ||
| createStream(jssc.ssc, hostname, port, storageLevel, false) | ||
| } | ||
|
|
||
| /** | ||
| * Creates a input stream from a Flume source. | ||
| * @param hostname Hostname of the slave machine to which the flume data will be sent | ||
| * @param port Port of the slave machine to which the flume data will be sent | ||
| * @param storageLevel Storage level to use for storing the received objects | ||
| * @param enableCompression Should Netty Server decode input stream from client | ||
|
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. same comment as above, for enableCompression. |
||
| */ | ||
| def createStream( | ||
| jssc: JavaStreamingContext, | ||
| hostname: String, | ||
| port: Int, | ||
| storageLevel: StorageLevel, | ||
| enableCompression: Boolean | ||
| ): JavaDStream[SparkFlumeEvent] = { | ||
| createStream( | ||
| jssc.ssc, | ||
| hostname, | ||
| port, | ||
| storageLevel, | ||
| enableCompression) | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,27 +19,41 @@ package org.apache.spark.streaming.flume | |
|
|
||
| import scala.collection.JavaConversions._ | ||
| import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} | ||
|
|
||
| import java.net.InetSocketAddress | ||
| import java.nio.ByteBuffer | ||
| import java.nio.charset.Charset | ||
|
|
||
| import org.apache.avro.ipc.NettyTransceiver | ||
| import org.apache.avro.ipc.specific.SpecificRequestor | ||
| import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} | ||
|
|
||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} | ||
| import org.apache.spark.streaming.util.ManualClock | ||
| import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory | ||
| import org.jboss.netty.channel.ChannelPipeline | ||
| import org.jboss.netty.handler.codec.compression.ZlibEncoder | ||
| import org.jboss.netty.handler.codec.compression.ZlibDecoder | ||
| import org.jboss.netty.channel.socket.SocketChannel | ||
|
|
||
| class FlumeStreamSuite extends TestSuiteBase { | ||
|
|
||
| val testPort = 9999 | ||
|
|
||
| test("flume input stream") { | ||
| runFlumeStreamTest(false, 9999) | ||
| } | ||
|
|
||
| test("flume input compressed stream") { | ||
| runFlumeStreamTest(true, 9998) | ||
| } | ||
|
|
||
| def runFlumeStreamTest(enableCompression: Boolean, testPort: Int) { | ||
| // Set up the streaming context and input streams | ||
| val ssc = new StreamingContext(conf, batchDuration) | ||
| val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) | ||
| val flumeStream = FlumeUtils.createStream( | ||
| ssc, | ||
| "localhost", | ||
| testPort, | ||
| StorageLevel.MEMORY_AND_DISK, | ||
| enableCompression) | ||
|
|
||
| val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] | ||
| with SynchronizedBuffer[Seq[SparkFlumeEvent]] | ||
| val outputStream = new TestOutputStream(flumeStream, outputBuffer) | ||
|
|
@@ -49,9 +63,20 @@ class FlumeStreamSuite extends TestSuiteBase { | |
| val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] | ||
| val input = Seq(1, 2, 3, 4, 5) | ||
| Thread.sleep(1000) | ||
| val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) | ||
| val client = SpecificRequestor.getClient( | ||
| classOf[AvroSourceProtocol], transceiver) | ||
|
|
||
| var client: AvroSourceProtocol = null; | ||
|
|
||
| if (enableCompression) { | ||
|
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. The Scala way of writing this would be |
||
| client = SpecificRequestor.getClient( | ||
| classOf[AvroSourceProtocol], | ||
| new NettyTransceiver(new InetSocketAddress("localhost", testPort), | ||
| new CompressionChannelFactory(6))); | ||
| } else { | ||
| client = SpecificRequestor.getClient( | ||
| classOf[AvroSourceProtocol], | ||
| new NettyTransceiver(new InetSocketAddress("localhost", testPort))); | ||
| } | ||
|
|
||
|
|
||
| for (i <- 0 until input.size) { | ||
| val event = new AvroFlumeEvent | ||
|
|
@@ -83,4 +108,16 @@ class FlumeStreamSuite extends TestSuiteBase { | |
| assert(outputBuffer(i).head.event.getHeaders.get("test") === "header") | ||
| } | ||
| } | ||
|
|
||
| class CompressionChannelFactory(compressionLevel: Int) extends | ||
| NioClientSocketChannelFactory { | ||
|
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. No need to wrap the line here. Usually, it is either or |
||
|
|
||
| override def newChannel(pipeline:ChannelPipeline) : SocketChannel = { | ||
| var encoder : ZlibEncoder = new ZlibEncoder(compressionLevel); | ||
| pipeline.addFirst("deflater", encoder); | ||
| pipeline.addFirst("inflater", new ZlibDecoder()); | ||
| super.newChannel(pipeline); | ||
| } | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -556,7 +556,7 @@ object SparkBuild extends Build { | |
| name := "spark-streaming-flume", | ||
| previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), | ||
| libraryDependencies ++= Seq( | ||
| "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty) | ||
| "org.apache.flume" % "flume-ng-sdk" % "1.3.0" % "compile" excludeAll(excludeNetty) | ||
|
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. This should be 1.4 if I am not mistaken, please merge with master. |
||
| ) | ||
| ) | ||
|
|
||
|
|
||
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.
Still 1.3? Have you merged master branch with your branch?