aboutsummaryrefslogtreecommitdiff
path: root/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala
blob: 44b27edf85ce87d94ceb34e9a44b28726005fdd0 (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
/*
 * 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.sink

import java.net.InetSocketAddress
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.{TimeUnit, CountDownLatch, Executors}

import scala.collection.JavaConversions._
import scala.concurrent.{ExecutionContext, Future}
import scala.util.{Failure, Success}

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

class SparkSinkSuite extends TestSuiteBase {
  val eventsPerBatch = 1000
  val channelCapacity = 5000

  test("Success") {
    val (channel, sink) = initializeChannelAndSink()
    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()
    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(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") {
    testMultipleConsumers(failSome = false)
  }

  test("Multiple consumers with some failures") {
    testMultipleConsumers(failSome = true)
  }

  def testMultipleConsumers(failSome: Boolean): Unit = {
    implicit val executorContext = ExecutionContext
      .fromExecutorService(Executors.newFixedThreadPool(5))
    val (channel, sink) = initializeChannelAndSink()
    channel.start()
    sink.start()
    (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch))
    val port = sink.getPort
    val address = new InetSocketAddress("0.0.0.0", port)
    val transceiversAndClients = getTransceiverAndClient(address, 5)
    val batchCounter = new CountDownLatch(5)
    val counter = new AtomicInteger(0)
    transceiversAndClients.foreach(x => {
      Future {
        val client = x._2
        val events = client.getEventBatch(1000)
        if (!failSome || counter.getAndIncrement() % 2 == 0) {
          client.ack(events.getSequenceNumber)
        } else {
          client.nack(events.getSequenceNumber)
          throw new RuntimeException("Sending NACK for failure!")
        }
        events
      }.onComplete {
        case Success(events) =>
          assert(events.getEvents.size() === 1000)
          batchCounter.countDown()
        case Failure(t) =>
          // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout
          batchCounter.countDown()
      }
    })
    batchCounter.await()
    TimeUnit.SECONDS.sleep(1) // Allow the sink to commit the transactions.
    executorContext.shutdown()
    if(failSome) {
      assert(availableChannelSlots(channel) === 3000)
    } else {
      assertChannelIsEmpty(channel)
    }
    sink.stop()
    channel.stop()
    transceiversAndClients.foreach(x => x._1.close())
  }

  private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty): (MemoryChannel,
    SparkSink) = {
    val channel = new MemoryChannel()
    val channelContext = new Context()

    channelContext.put("capacity", channelCapacity.toString)
    channelContext.put("transactionCapacity", 1000.toString)
    channelContext.put("keep-alive", 0.toString)
    channelContext.putAll(overrides)
    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.configure(sinkContext)
    sink.setChannel(channel)
    (channel, sink)
  }

  private def putEvents(ch: MemoryChannel, count: Int): Unit = {
    val tx = ch.getTransaction
    tx.begin()
    (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes)))
    tx.commit()
    tx.close()
  }

  private def getTransceiverAndClient(address: InetSocketAddress,
    count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = {

    (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): Unit = {
    assert(availableChannelSlots(channel) === 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]
  }
}