-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-3054][STREAMING] Add unit tests for Spark Sink. #1958
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 2 commits
c86d615
a24aac8
f2c56c9
7b9b649
abc20cb
7fedc5a
c9190d1
4df5be6
120b81e
e3110b9
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 |
|---|---|---|
| @@ -0,0 +1,208 @@ | ||
| package org.apache.spark.streaming.flume.sink | ||
|
|
||
| import java.net.InetSocketAddress | ||
| import java.util.concurrent.atomic.AtomicInteger | ||
| import java.util.concurrent.{CountDownLatch, Executors} | ||
|
|
||
| import scala.collection.JavaConversions._ | ||
| import scala.concurrent.{Promise, Future} | ||
| import scala.util.{Failure, Success, Try} | ||
|
|
||
| import com.google.common.util.concurrent.ThreadFactoryBuilder | ||
| import org.apache.avro.ipc.NettyTransceiver | ||
| import org.apache.avro.ipc.specific.SpecificRequestor | ||
| import org.apache.flume.Context | ||
| import org.apache.flume.channel.MemoryChannel | ||
| import org.apache.flume.event.EventBuilder | ||
| import org.apache.spark.streaming.TestSuiteBase | ||
| import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory | ||
|
|
||
|
|
||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
|
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. Hey Hari, ASF header should be at the top of file :).
Contributor
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. Thanks! Done. |
||
| * 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. | ||
| */ | ||
| class SparkSinkSuite extends TestSuiteBase { | ||
| val eventsPerBatch = 1000 | ||
| val channelCapacity = 5000 | ||
|
|
||
| test("Success test") { | ||
|
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. Maybe keep the naming consistent. If this is called "Success Test", then name the next ones as "Failure Test" and "Timeout Test". You can even remove the "Test" |
||
| val (channel, sink) = initializeChannelAndSink(None) | ||
| channel.start() | ||
| sink.start() | ||
|
|
||
| putEvents(channel, eventsPerBatch) | ||
|
|
||
| val port = sink.getPort | ||
| val address = new InetSocketAddress("0.0.0.0", port) | ||
|
|
||
| val (transceiver, client) = getTransceiverAndClient(address, 1)(0) | ||
| val events = client.getEventBatch(1000) | ||
| client.ack(events.getSequenceNumber) | ||
| assert(events.getEvents.size() === 1000) | ||
| assertChannelIsEmpty(channel) | ||
| sink.stop() | ||
| channel.stop() | ||
| transceiver.close() | ||
| } | ||
|
|
||
| test("Nack") { | ||
| val (channel, sink) = initializeChannelAndSink(None) | ||
| channel.start() | ||
| sink.start() | ||
| putEvents(channel, eventsPerBatch) | ||
|
|
||
| val port = sink.getPort | ||
| val address = new InetSocketAddress("0.0.0.0", port) | ||
|
|
||
| val (transceiver, client) = getTransceiverAndClient(address, 1)(0) | ||
| val events = client.getEventBatch(1000) | ||
| assert(events.getEvents.size() === 1000) | ||
| client.nack(events.getSequenceNumber) | ||
| assert(availableChannelSlots(channel) === 4000) | ||
| sink.stop() | ||
| channel.stop() | ||
| transceiver.close() | ||
| } | ||
|
|
||
| test("Timeout") { | ||
| val (channel, sink) = initializeChannelAndSink(Option(Map(SparkSinkConfig | ||
| .CONF_TRANSACTION_TIMEOUT -> 1.toString))) | ||
| channel.start() | ||
| sink.start() | ||
| putEvents(channel, eventsPerBatch) | ||
| val port = sink.getPort | ||
| val address = new InetSocketAddress("0.0.0.0", port) | ||
|
|
||
| val (transceiver, client) = getTransceiverAndClient(address, 1)(0) | ||
| val events = client.getEventBatch(1000) | ||
| assert(events.getEvents.size() === 1000) | ||
| Thread.sleep(1000) | ||
| assert(availableChannelSlots(channel) === 4000) | ||
| sink.stop() | ||
| channel.stop() | ||
| transceiver.close() | ||
| } | ||
|
|
||
| test("Multiple consumers") { | ||
| multipleClients(failSome = false) | ||
| } | ||
|
|
||
| test("Multiple consumers With Some Failures") { | ||
|
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. nit: inconsistent capitalization, lets rather have "Multple consumers with some failures" |
||
| multipleClients(failSome = true) | ||
| } | ||
|
|
||
| def multipleClients(failSome: Boolean): Unit = { | ||
|
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. nit: this should be named "testMultipleClients" |
||
| import scala.concurrent.ExecutionContext.Implicits.global | ||
| val (channel, sink) = initializeChannelAndSink(None) | ||
| channel.start() | ||
| sink.start() | ||
| (1 to 5).map(_ => putEvents(channel, eventsPerBatch)) | ||
| val port = sink.getPort | ||
| val address = new InetSocketAddress("0.0.0.0", port) | ||
|
|
||
| val transAndClient = getTransceiverAndClient(address, 5) | ||
|
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. nit: transAndClients |
||
| val batchCounter = new CountDownLatch(5) | ||
| val counter = new AtomicInteger(0) | ||
| transAndClient.foreach(x => { | ||
| val promise = Promise[EventBatch]() | ||
| val future = promise.future | ||
| Future { | ||
|
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. Why make one Future, than another Promise + Future combination? Just the single Future is sufficient. Also, I question the use of ExecutionContext, we dont know the # of threads in that context, so not sure what parallelism we achieve here. Its best to create an ExecutionContext from a Executors.newFixedThreadPool().
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. super nit, no change necessary. A simpler implementation is possible. No latch necessary. |
||
| val client = x._2 | ||
| var events: EventBatch = null | ||
| Try { | ||
| events = client.getEventBatch(1000) | ||
| if(!failSome || counter.incrementAndGet() % 2 == 0) { | ||
| client.ack(events.getSequenceNumber) | ||
| } else { | ||
| client.nack(events.getSequenceNumber) | ||
| } | ||
| }.map(_ => promise.success(events)).recover({ | ||
|
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 think it should be |
||
| case e => promise.failure(e) | ||
| }) | ||
| } | ||
| future.onComplete { | ||
| case Success(events) => assert(events.getEvents.size() === 1000) | ||
|
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. for multiple lines, its started on the next line. |
||
| batchCounter.countDown() | ||
| case Failure(t) => | ||
| batchCounter.countDown() | ||
| throw t | ||
| } | ||
| }) | ||
| batchCounter.await() | ||
| if(failSome) { | ||
| assert(availableChannelSlots(channel) === 2000) | ||
| } else { | ||
| assertChannelIsEmpty(channel) | ||
| } | ||
| sink.stop() | ||
| channel.stop() | ||
| transAndClient.foreach(x => x._1.close()) | ||
| } | ||
|
|
||
| def initializeChannelAndSink(overrides: Option[Map[String, String]]): | ||
| (MemoryChannel, SparkSink) = { | ||
|
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. Wrong Indentation |
||
| val channel = new MemoryChannel() | ||
| val channelContext = new Context() | ||
|
|
||
| channelContext.put("capacity", channelCapacity.toString) | ||
| channelContext.put("transactionCapacity", 1000.toString) | ||
| channelContext.put("keep-alive", 0.toString) | ||
| overrides.foreach(channelContext.putAll(_)) | ||
| channel.configure(channelContext) | ||
|
|
||
| val sink = new SparkSink() | ||
| val sinkContext = new Context() | ||
| sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") | ||
| sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) | ||
| sink.setChannel(channel) | ||
| (channel, sink) | ||
| } | ||
|
|
||
| private def putEvents(ch: MemoryChannel, count: Int): Unit = { | ||
| val tx = ch.getTransaction | ||
| tx.begin() | ||
| (1 to count).map(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) | ||
| tx.commit() | ||
| tx.close() | ||
| } | ||
|
|
||
| private def getTransceiverAndClient(address: InetSocketAddress, count: Int): | ||
| Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { | ||
|
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. Incorrect indentation. This line should have 2 space indent. |
||
|
|
||
| (1 to count).map(_ => { | ||
| lazy val channelFactoryExecutor = | ||
| Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). | ||
| setNameFormat("Flume Receiver Channel Thread - %d").build()) | ||
| lazy val channelFactory = | ||
| new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) | ||
| val transceiver = new NettyTransceiver(address, channelFactory) | ||
| val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) | ||
| (transceiver, client) | ||
| }) | ||
| } | ||
|
|
||
| private def assertChannelIsEmpty(channel: MemoryChannel) = { | ||
|
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 make this as "Unit = {" |
||
| assert(availableChannelSlots(channel) === 5000) | ||
|
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 equaled to "channelCapacity" |
||
| } | ||
|
|
||
| private def availableChannelSlots(channel: MemoryChannel): Int = { | ||
| val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") | ||
| queueRemaining.setAccessible(true) | ||
| val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") | ||
| m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] | ||
| } | ||
|
|
||
|
||
| } | ||
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.
Why was this removed? We dont want to expose this as a public class as this class will then appear in the Scala docs.
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.
Since this class would be called from Flume. Flume will create an instance of this class to run the sink - so theoretically it should not be private to this package.
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.
In that case, can you add a line right at the top that this class is not intended to be used inside Spark application. Just in case it appears in the scala docs / java docs. I will try to see how to eliminate this module from appearing in the docs.