* the next even if errbacks for the current txn get called while
* the next one is being processed.
*
*/
if (!open) {
throw new EventDeliveryException("Sink was never opened. " +
"Please fix the configuration.");
}
AtomicBoolean txnFail = new AtomicBoolean(false);
AtomicInteger callbacksReceived = new AtomicInteger(0);
AtomicInteger callbacksExpected = new AtomicInteger(0);
final Lock lock = new ReentrantLock();
final Condition condition = lock.newCondition();
if (incrementBuffer != null) {
incrementBuffer.clear();
}
/*
* Callbacks can be reused per transaction, since they share the same
* locks and conditions.
*/
Callback<Object, Object> putSuccessCallback =
new SuccessCallback<Object, Object>(
lock, callbacksReceived, condition);
Callback<Object, Object> putFailureCallback =
new FailureCallback<Object, Object>(
lock, callbacksReceived, txnFail, condition);
Callback<Long, Long> incrementSuccessCallback =
new SuccessCallback<Long, Long>(
lock, callbacksReceived, condition);
Callback<Long, Long> incrementFailureCallback =
new FailureCallback<Long, Long>(
lock, callbacksReceived, txnFail, condition);
Status status = Status.READY;
Channel channel = getChannel();
int i = 0;
try {
txn = channel.getTransaction();
txn.begin();
for (; i < batchSize; i++) {
Event event = channel.take();
if (event == null) {
status = Status.BACKOFF;
if (i == 0) {
sinkCounter.incrementBatchEmptyCount();
} else {
sinkCounter.incrementBatchUnderflowCount();
}
break;
} else {
serializer.setEvent(event);
List<PutRequest> actions = serializer.getActions();
List<AtomicIncrementRequest> increments = serializer.getIncrements();
callbacksExpected.addAndGet(actions.size());
if (!batchIncrements) {
callbacksExpected.addAndGet(increments.size());
}
for (PutRequest action : actions) {
action.setDurable(enableWal);
client.put(action).addCallbacks(putSuccessCallback, putFailureCallback);
}
for (AtomicIncrementRequest increment : increments) {
if (batchIncrements) {
CellIdentifier identifier = new CellIdentifier(increment.key(),
increment.qualifier());
AtomicIncrementRequest request
= incrementBuffer.get(identifier);
if (request == null) {
incrementBuffer.put(identifier, increment);
} else {
request.setAmount(request.getAmount() + increment.getAmount());
}
} else {
client.atomicIncrement(increment).addCallbacks(
incrementSuccessCallback, incrementFailureCallback);
}
}
}
}
if (batchIncrements) {
Collection<AtomicIncrementRequest> increments = incrementBuffer.values();
for (AtomicIncrementRequest increment : increments) {
client.atomicIncrement(increment).addCallbacks(
incrementSuccessCallback, incrementFailureCallback);
}
callbacksExpected.addAndGet(increments.size());
}
client.flush();
} catch (Throwable e) {
this.handleTransactionFailure(txn);
this.checkIfChannelExceptionAndThrow(e);
}
if (i == batchSize) {
sinkCounter.incrementBatchCompleteCount();
}
sinkCounter.addToEventDrainAttemptCount(i);
lock.lock();
long startTime = System.nanoTime();
long timeRemaining;
try {
while ((callbacksReceived.get() < callbacksExpected.get())
&& !txnFail.get()) {
timeRemaining = timeout - (System.nanoTime() - startTime);
timeRemaining = (timeRemaining >= 0) ? timeRemaining : 0;
try {
if (!condition.await(timeRemaining, TimeUnit.NANOSECONDS)) {
txnFail.set(true);
logger.warn("HBase callbacks timed out. "
+ "Transaction will be rolled back.");
}
} catch (Exception ex) {
logger.error("Exception while waiting for callbacks from HBase.");
this.handleTransactionFailure(txn);
Throwables.propagate(ex);
}
}
} finally {
lock.unlock();
}
if (isCoalesceTest) {
totalCallbacksReceived += callbacksReceived.get();
}
/*
* At this point, either the txn has failed
* or all callbacks received and txn is successful.
*
* This need not be in the monitor, since all callbacks for this txn
* have been received. So txnFail will not be modified any more(even if
* it is, it is set from true to true only - false happens only
* in the next process call).
*
*/
if (txnFail.get()) {
this.handleTransactionFailure(txn);
throw new EventDeliveryException("Could not write events to Hbase. " +
"Transaction failed, and rolled back.");
} else {
try {
txn.commit();
txn.close();