|
| 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 | +package org.apache.spark.streaming.flume |
| 18 | + |
| 19 | +import scala.collection.JavaConversions._ |
| 20 | +import scala.collection.mutable.ArrayBuffer |
| 21 | + |
| 22 | +import com.google.common.base.Throwables |
| 23 | + |
| 24 | +import org.apache.spark.Logging |
| 25 | +import org.apache.spark.streaming.flume.sink._ |
| 26 | + |
| 27 | +/** |
| 28 | + * This class implements the core functionality of [[FlumePollingReceiver]]. When started it |
| 29 | + * pulls data from Flume, stores it to Spark and then sends an Ack or Nack. This class should be |
| 30 | + * run via an [[java.util.concurrent.Executor]] as this implements [[Runnable]] |
| 31 | + * |
| 32 | + * @param receiver The receiver that owns this instance. |
| 33 | + */ |
| 34 | + |
| 35 | +private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends Runnable with |
| 36 | + Logging { |
| 37 | + |
| 38 | + def run(): Unit = { |
| 39 | + while (!receiver.isStopped()) { |
| 40 | + val connection = receiver.getConnections.poll() |
| 41 | + val client = connection.client |
| 42 | + var batchReceived = false |
| 43 | + var seq: CharSequence = null |
| 44 | + try { |
| 45 | + getBatch(client) match { |
| 46 | + case Some(eventBatch) => |
| 47 | + batchReceived = true |
| 48 | + seq = eventBatch.getSequenceNumber |
| 49 | + val events = toSparkFlumeEvents(eventBatch.getEvents) |
| 50 | + if (store(events)) { |
| 51 | + sendAck(client, seq) |
| 52 | + } else { |
| 53 | + sendNack(batchReceived, client, seq) |
| 54 | + } |
| 55 | + case None => |
| 56 | + } |
| 57 | + } catch { |
| 58 | + case e: Exception => |
| 59 | + Throwables.getRootCause(e) match { |
| 60 | + // If the cause was an InterruptedException, then check if the receiver is stopped - |
| 61 | + // if yes, just break out of the loop. Else send a Nack and log a warning. |
| 62 | + // In the unlikely case, the cause was not an Exception, |
| 63 | + // then just throw it out and exit. |
| 64 | + case interrupted: InterruptedException => |
| 65 | + if (!receiver.isStopped()) { |
| 66 | + logWarning("Interrupted while receiving data from Flume", interrupted) |
| 67 | + sendNack(batchReceived, client, seq) |
| 68 | + } |
| 69 | + case exception: Exception => |
| 70 | + logWarning("Error while receiving data from Flume", exception) |
| 71 | + sendNack(batchReceived, client, seq) |
| 72 | + } |
| 73 | + } finally { |
| 74 | + receiver.getConnections.add(connection) |
| 75 | + } |
| 76 | + } |
| 77 | + } |
| 78 | + |
| 79 | + /** |
| 80 | + * Gets a batch of events from the specified client. This method does not handle any exceptions |
| 81 | + * which will be propogated to the caller. |
| 82 | + * @param client Client to get events from |
| 83 | + * @return [[Some]] which contains the event batch if Flume sent any events back, else [[None]] |
| 84 | + */ |
| 85 | + private def getBatch(client: SparkFlumeProtocol.Callback): Option[EventBatch] = { |
| 86 | + val eventBatch = client.getEventBatch(receiver.getMaxBatchSize) |
| 87 | + if (!SparkSinkUtils.isErrorBatch(eventBatch)) { |
| 88 | + // No error, proceed with processing data |
| 89 | + logDebug(s"Received batch of ${eventBatch.getEvents.size} events with sequence " + |
| 90 | + s"number: ${eventBatch.getSequenceNumber}") |
| 91 | + Some(eventBatch) |
| 92 | + } else { |
| 93 | + logWarning("Did not receive events from Flume agent due to error on the Flume agent: " + |
| 94 | + eventBatch.getErrorMsg) |
| 95 | + None |
| 96 | + } |
| 97 | + } |
| 98 | + |
| 99 | + /** |
| 100 | + * Store the events in the buffer to Spark. This method will not propogate any exceptions, |
| 101 | + * but will propogate any other errors. |
| 102 | + * @param buffer The buffer to store |
| 103 | + * @return true if the data was stored without any exception being thrown, else false |
| 104 | + */ |
| 105 | + private def store(buffer: ArrayBuffer[SparkFlumeEvent]): Boolean = { |
| 106 | + try { |
| 107 | + receiver.store(buffer) |
| 108 | + true |
| 109 | + } catch { |
| 110 | + case e: Exception => |
| 111 | + logWarning("Error while attempting to store data received from Flume", e) |
| 112 | + false |
| 113 | + } |
| 114 | + } |
| 115 | + |
| 116 | + /** |
| 117 | + * Send an ack to the client for the sequence number. This method does not handle any exceptions |
| 118 | + * which will be propagated to the caller. |
| 119 | + * @param client client to send the ack to |
| 120 | + * @param seq sequence number of the batch to be ack-ed. |
| 121 | + * @return |
| 122 | + */ |
| 123 | + private def sendAck(client: SparkFlumeProtocol.Callback, seq: CharSequence): Unit = { |
| 124 | + logDebug("Sending ack for sequence number: " + seq) |
| 125 | + client.ack(seq) |
| 126 | + logDebug("Ack sent for sequence number: " + seq) |
| 127 | + } |
| 128 | + |
| 129 | + /** |
| 130 | + * This method sends a Nack if a batch was received to the client with the given sequence |
| 131 | + * number. Any exceptions thrown by the RPC call is simply thrown out as is - no effort is made |
| 132 | + * to handle it. |
| 133 | + * @param batchReceived true if a batch was received. If this is false, no nack is sent |
| 134 | + * @param client The client to which the nack should be sent |
| 135 | + * @param seq The sequence number of the batch that is being nack-ed. |
| 136 | + */ |
| 137 | + private def sendNack(batchReceived: Boolean, client: SparkFlumeProtocol.Callback, |
| 138 | + seq: CharSequence): Unit = { |
| 139 | + if (batchReceived) { |
| 140 | + // Let Flume know that the events need to be pushed back into the channel. |
| 141 | + logDebug("Sending nack for sequence number: " + seq) |
| 142 | + client.nack(seq) // If the agent is down, even this could fail and throw |
| 143 | + logDebug("Nack sent for sequence number: " + seq) |
| 144 | + } |
| 145 | + } |
| 146 | + |
| 147 | + /** |
| 148 | + * Utility method to convert [[SparkSinkEvent]]s to [[SparkFlumeEvent]]s |
| 149 | + * @param events - Events to convert to SparkFlumeEvents |
| 150 | + * @return - The SparkFlumeEvent generated from SparkSinkEvent |
| 151 | + */ |
| 152 | + private def toSparkFlumeEvents(events: java.util.List[SparkSinkEvent]): |
| 153 | + ArrayBuffer[SparkFlumeEvent] = { |
| 154 | + // Convert each Flume event to a serializable SparkFlumeEvent |
| 155 | + val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) |
| 156 | + var j = 0 |
| 157 | + while (j < events.size()) { |
| 158 | + val event = events(j) |
| 159 | + val sparkFlumeEvent = new SparkFlumeEvent() |
| 160 | + sparkFlumeEvent.event.setBody(event.getBody) |
| 161 | + sparkFlumeEvent.event.setHeaders(event.getHeaders) |
| 162 | + buffer += sparkFlumeEvent |
| 163 | + j += 1 |
| 164 | + } |
| 165 | + buffer |
| 166 | + } |
| 167 | +} |
0 commit comments