aboutsummaryrefslogtreecommitdiff
path: root/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala
blob: 7da8eb3e35912347354933061daf5d393eb47fd0 (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
/*
 * 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.util.concurrent.{ConcurrentHashMap, Executors}
import java.util.concurrent.atomic.AtomicLong

import org.apache.flume.Channel
import org.apache.commons.lang.RandomStringUtils
import com.google.common.util.concurrent.ThreadFactoryBuilder

/**
 * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process
 * requests. Each getEvents, ack and nack call is forwarded to an instance of this class.
 * @param threads Number of threads to use to process requests.
 * @param channel The channel that the sink pulls events from
 * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark
 *                           is rolled back.
 */
// Flume forces transactions to be thread-local. So each transaction *must* be committed, or
// rolled back from the thread it was originally created in. So each getEvents call from Spark
// creates a TransactionProcessor which runs in a new thread, in which the transaction is created
// and events are pulled off the channel. Once the events are sent to spark,
// that thread is blocked and the TransactionProcessor is saved in a map,
// until an ACK or NACK comes back or the transaction times out (after the specified timeout).
// When the response comes or a timeout is hit, the TransactionProcessor is retrieved and then
// unblocked, at which point the transaction is committed or rolled back.

private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel,
  val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging {
  val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads,
    new ThreadFactoryBuilder().setDaemon(true)
      .setNameFormat("Spark Sink Processor Thread - %d").build()))
  private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]()
  // This sink will not persist sequence numbers and reuses them if it gets restarted.
  // So it is possible to commit a transaction which may have been meant for the sink before the
  // restart.
  // Since the new txn may not have the same sequence number we must guard against accidentally
  // committing a new transaction. To reduce the probability of that happening a random string is
  // prepended to the sequence number. Does not change for life of sink
  private val seqBase = RandomStringUtils.randomAlphanumeric(8)
  private val seqCounter = new AtomicLong(0)

  /**
   * Returns a bunch of events to Spark over Avro RPC.
   * @param n Maximum number of events to return in a batch
   * @return [[EventBatch]] instance that has a sequence number and an array of at most n events
   */
  override def getEventBatch(n: Int): EventBatch = {
    logDebug("Got getEventBatch call from Spark.")
    val sequenceNumber = seqBase + seqCounter.incrementAndGet()
    val processor = new TransactionProcessor(channel, sequenceNumber,
      n, transactionTimeout, backOffInterval, this)
    transactionExecutorOpt.foreach(executor => {
      executor.submit(processor)
    })
    // Wait until a batch is available - will be an error if error message is non-empty
    val batch = processor.getEventBatch
    if (!SparkSinkUtils.isErrorBatch(batch)) {
      processorMap.put(sequenceNumber.toString, processor)
      logDebug("Sending event batch with sequence number: " + sequenceNumber)
    }
    batch
  }

  /**
   * Called by Spark to indicate successful commit of a batch
   * @param sequenceNumber The sequence number of the event batch that was successful
   */
  override def ack(sequenceNumber: CharSequence): Void = {
    logDebug("Received Ack for batch with sequence number: " + sequenceNumber)
    completeTransaction(sequenceNumber, success = true)
    null
  }

  /**
   * Called by Spark to indicate failed commit of a batch
   * @param sequenceNumber The sequence number of the event batch that failed
   * @return
   */
  override def nack(sequenceNumber: CharSequence): Void = {
    completeTransaction(sequenceNumber, success = false)
    logInfo("Spark failed to commit transaction. Will reattempt events.")
    null
  }

  /**
   * Helper method to commit or rollback a transaction.
   * @param sequenceNumber The sequence number of the batch that was completed
   * @param success Whether the batch was successful or not.
   */
  private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) {
    Option(removeAndGetProcessor(sequenceNumber)).foreach(processor => {
      processor.batchProcessed(success)
    })
  }

  /**
   * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak.
   * @param sequenceNumber
   * @return The transaction processor for the corresponding batch. Note that this instance is no
   *         longer tracked and the caller is responsible for that txn processor.
   */
  private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = {
    processorMap.remove(sequenceNumber.toString) // The toString is required!
  }

  /**
   * Shuts down the executor used to process transactions.
   */
  def shutdown() {
    logInfo("Shutting down Spark Avro Callback Handler")
    transactionExecutorOpt.foreach(executor => {
      executor.shutdownNow()
    })
  }
}