|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.streaming.flume |
| 19 | + |
| 20 | +import java.util.concurrent._ |
| 21 | +import java.util.{List => JList, Map => JMap} |
| 22 | + |
| 23 | +import scala.collection.JavaConversions._ |
| 24 | +import scala.collection.mutable.ArrayBuffer |
| 25 | + |
| 26 | +import com.google.common.base.Charsets.UTF_8 |
| 27 | +import org.apache.flume.event.EventBuilder |
| 28 | +import org.apache.flume.Context |
| 29 | +import org.apache.flume.channel.MemoryChannel |
| 30 | +import org.apache.flume.conf.Configurables |
| 31 | + |
| 32 | +import org.apache.spark.streaming.flume.sink.{SparkSinkConfig, SparkSink} |
| 33 | + |
| 34 | +/** |
| 35 | + * Share codes for Scala and Python unit tests |
| 36 | + */ |
| 37 | +private[flume] class PollingFlumeTestUtils { |
| 38 | + |
| 39 | + private val batchCount = 5 |
| 40 | + private val eventsPerBatch = 100 |
| 41 | + private val totalEventsPerChannel = batchCount * eventsPerBatch |
| 42 | + private val channelCapacity = 5000 |
| 43 | + |
| 44 | + def getEventsPerBatch: Int = eventsPerBatch |
| 45 | + |
| 46 | + def getTotalEvents: Int = totalEventsPerChannel * channels.size |
| 47 | + |
| 48 | + private val channels = new ArrayBuffer[MemoryChannel] |
| 49 | + private val sinks = new ArrayBuffer[SparkSink] |
| 50 | + |
| 51 | + /** |
| 52 | + * Start a sink and return the port of this sink |
| 53 | + */ |
| 54 | + def startSingleSink(): Int = { |
| 55 | + channels.clear() |
| 56 | + sinks.clear() |
| 57 | + |
| 58 | + // Start the channel and sink. |
| 59 | + val context = new Context() |
| 60 | + context.put("capacity", channelCapacity.toString) |
| 61 | + context.put("transactionCapacity", "1000") |
| 62 | + context.put("keep-alive", "0") |
| 63 | + val channel = new MemoryChannel() |
| 64 | + Configurables.configure(channel, context) |
| 65 | + |
| 66 | + val sink = new SparkSink() |
| 67 | + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") |
| 68 | + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) |
| 69 | + Configurables.configure(sink, context) |
| 70 | + sink.setChannel(channel) |
| 71 | + sink.start() |
| 72 | + |
| 73 | + channels += (channel) |
| 74 | + sinks += sink |
| 75 | + |
| 76 | + sink.getPort() |
| 77 | + } |
| 78 | + |
| 79 | + /** |
| 80 | + * Start 2 sinks and return the ports |
| 81 | + */ |
| 82 | + def startMultipleSinks(): JList[Int] = { |
| 83 | + channels.clear() |
| 84 | + sinks.clear() |
| 85 | + |
| 86 | + // Start the channel and sink. |
| 87 | + val context = new Context() |
| 88 | + context.put("capacity", channelCapacity.toString) |
| 89 | + context.put("transactionCapacity", "1000") |
| 90 | + context.put("keep-alive", "0") |
| 91 | + val channel = new MemoryChannel() |
| 92 | + Configurables.configure(channel, context) |
| 93 | + |
| 94 | + val channel2 = new MemoryChannel() |
| 95 | + Configurables.configure(channel2, context) |
| 96 | + |
| 97 | + val sink = new SparkSink() |
| 98 | + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") |
| 99 | + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) |
| 100 | + Configurables.configure(sink, context) |
| 101 | + sink.setChannel(channel) |
| 102 | + sink.start() |
| 103 | + |
| 104 | + val sink2 = new SparkSink() |
| 105 | + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") |
| 106 | + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) |
| 107 | + Configurables.configure(sink2, context) |
| 108 | + sink2.setChannel(channel2) |
| 109 | + sink2.start() |
| 110 | + |
| 111 | + sinks += sink |
| 112 | + sinks += sink2 |
| 113 | + channels += channel |
| 114 | + channels += channel2 |
| 115 | + |
| 116 | + sinks.map(_.getPort()) |
| 117 | + } |
| 118 | + |
| 119 | + /** |
| 120 | + * Send data and wait until all data has been received |
| 121 | + */ |
| 122 | + def sendDatAndEnsureAllDataHasBeenReceived(): Unit = { |
| 123 | + val executor = Executors.newCachedThreadPool() |
| 124 | + val executorCompletion = new ExecutorCompletionService[Void](executor) |
| 125 | + |
| 126 | + val latch = new CountDownLatch(batchCount * channels.size) |
| 127 | + sinks.foreach(_.countdownWhenBatchReceived(latch)) |
| 128 | + |
| 129 | + channels.foreach(channel => { |
| 130 | + executorCompletion.submit(new TxnSubmitter(channel)) |
| 131 | + }) |
| 132 | + |
| 133 | + for (i <- 0 until channels.size) { |
| 134 | + executorCompletion.take() |
| 135 | + } |
| 136 | + |
| 137 | + latch.await(15, TimeUnit.SECONDS) // Ensure all data has been received. |
| 138 | + } |
| 139 | + |
| 140 | + /** |
| 141 | + * A Python-friendly method to assert the output |
| 142 | + */ |
| 143 | + def assertOutput( |
| 144 | + outputHeaders: JList[JMap[String, String]], outputBodies: JList[String]): Unit = { |
| 145 | + require(outputHeaders.size == outputBodies.size) |
| 146 | + val eventSize = outputHeaders.size |
| 147 | + if (eventSize != totalEventsPerChannel * channels.size) { |
| 148 | + throw new AssertionError( |
| 149 | + s"Expected ${totalEventsPerChannel * channels.size} events, but was $eventSize") |
| 150 | + } |
| 151 | + var counter = 0 |
| 152 | + for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { |
| 153 | + val eventBodyToVerify = s"${channels(k).getName}-$i" |
| 154 | + val eventHeaderToVerify: JMap[String, String] = Map[String, String](s"test-$i" -> "header") |
| 155 | + var found = false |
| 156 | + var j = 0 |
| 157 | + while (j < eventSize && !found) { |
| 158 | + if (eventBodyToVerify == outputBodies.get(j) && |
| 159 | + eventHeaderToVerify == outputHeaders.get(j)) { |
| 160 | + found = true |
| 161 | + counter += 1 |
| 162 | + } |
| 163 | + j += 1 |
| 164 | + } |
| 165 | + } |
| 166 | + if (counter != totalEventsPerChannel * channels.size) { |
| 167 | + throw new AssertionError( |
| 168 | + s"111 Expected ${totalEventsPerChannel * channels.size} events, but was $counter") |
| 169 | + } |
| 170 | + } |
| 171 | + |
| 172 | + def assertChannelsAreEmpty(): Unit = { |
| 173 | + channels.foreach(assertChannelIsEmpty) |
| 174 | + } |
| 175 | + |
| 176 | + private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { |
| 177 | + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") |
| 178 | + queueRemaining.setAccessible(true) |
| 179 | + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") |
| 180 | + if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != 5000) { |
| 181 | + throw new AssertionError(s"Channel ${channel.getName} is not empty") |
| 182 | + } |
| 183 | + } |
| 184 | + |
| 185 | + def close(): Unit = { |
| 186 | + sinks.foreach(_.stop()) |
| 187 | + sinks.clear() |
| 188 | + channels.foreach(_.stop()) |
| 189 | + channels.clear() |
| 190 | + } |
| 191 | + |
| 192 | + private class TxnSubmitter(channel: MemoryChannel) extends Callable[Void] { |
| 193 | + override def call(): Void = { |
| 194 | + var t = 0 |
| 195 | + for (i <- 0 until batchCount) { |
| 196 | + val tx = channel.getTransaction |
| 197 | + tx.begin() |
| 198 | + for (j <- 0 until eventsPerBatch) { |
| 199 | + channel.put(EventBuilder.withBody(s"${channel.getName}-$t".getBytes(UTF_8), |
| 200 | + Map[String, String](s"test-$t" -> "header"))) |
| 201 | + t += 1 |
| 202 | + } |
| 203 | + tx.commit() |
| 204 | + tx.close() |
| 205 | + Thread.sleep(500) // Allow some time for the events to reach |
| 206 | + } |
| 207 | + null |
| 208 | + } |
| 209 | + } |
| 210 | + |
| 211 | +} |
0 commit comments