private FlinkKafkaProducer<byte[], byte[]> initProducer(boolean registerMetrics) { FlinkKafkaProducer<byte[], byte[]> producer = new FlinkKafkaProducer<>(this.producerConfig); Map<MetricName, ? extends Metric> metrics = producer.metrics();
@Override protected void abort(KafkaTransactionState transaction) { if (transaction.isTransactional()) { transaction.producer.abortTransaction(); recycleTransactionalProducer(transaction.producer); } }
@Override protected KafkaTransactionState beginTransaction() throws FlinkKafka011Exception { switch (semantic) { case EXACTLY_ONCE: FlinkKafkaProducer<byte[], byte[]> producer = createTransactionalProducer(); producer.beginTransaction(); return new KafkaTransactionState(producer.getTransactionalId(), producer); case AT_LEAST_ONCE: case NONE: // Do not create new producer on each beginTransaction() if it is not necessary final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction != null && currentTransaction.producer != null) { return new KafkaTransactionState(currentTransaction.producer); } return new KafkaTransactionState(initNonTransactionalProducer(true)); default: throw new UnsupportedOperationException("Not implemented semantic"); } }
KafkaTransactionState(String transactionalId, FlinkKafkaProducer<byte[], byte[]> producer) { this(transactionalId, producer.getProducerId(), producer.getEpoch(), producer); }
/** * Instead of obtaining producerId and epoch from the transaction coordinator, re-use previously obtained ones, * so that we can resume transaction after a restart. Implementation of this method is based on * {@link org.apache.kafka.clients.producer.KafkaProducer#initTransactions}. */ public void resumeTransaction(long producerId, short epoch) { Preconditions.checkState(producerId >= 0 && epoch >= 0, "Incorrect values for producerId {} and epoch {}", producerId, epoch); LOG.info("Attempting to resume transaction {} with producerId {} and epoch {}", transactionalId, producerId, epoch); Object transactionManager = getValue(kafkaProducer, "transactionManager"); synchronized (transactionManager) { Object sequenceNumbers = getValue(transactionManager, "sequenceNumbers"); invoke(transactionManager, "transitionTo", getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.INITIALIZING")); invoke(sequenceNumbers, "clear"); Object producerIdAndEpoch = getValue(transactionManager, "producerIdAndEpoch"); setValue(producerIdAndEpoch, "producerId", producerId); setValue(producerIdAndEpoch, "epoch", epoch); invoke(transactionManager, "transitionTo", getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.READY")); invoke(transactionManager, "transitionTo", getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.IN_TRANSACTION")); setValue(transactionManager, "transactionStarted", true); } }
/** * Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction} is also adding new * partitions to the transaction. flushNewPartitions method is moving this logic to pre-commit/flush, to make * resumeTransaction simpler. Otherwise resumeTransaction would require to restore state of the not yet added/"in-flight" * partitions. */ private void flushNewPartitions() { LOG.info("Flushing new partitions"); TransactionalRequestResult result = enqueueNewPartitions(); Object sender = getValue(kafkaProducer, "sender"); invoke(sender, "wakeup"); result.await(); }
private void recycleTransactionalProducer(FlinkKafkaProducer<byte[], byte[]> producer) { availableTransactionalIds.add(producer.getTransactionalId()); producer.close(); }
@Override protected void recoverAndCommit(KafkaTransactionState transaction) { if (transaction.isTransactional()) { try ( FlinkKafkaProducer<byte[], byte[]> producer = initTransactionalProducer(transaction.transactionalId, false)) { producer.resumeTransaction(transaction.producerId, transaction.epoch); producer.commitTransaction(); } catch (InvalidTxnStateException | ProducerFencedException ex) { // That means we have committed this transaction before. LOG.warn("Encountered error {} while recovering transaction {}. " + "Presumably this transaction has been already committed before", ex, transaction); } } }
@Override public void flush() { kafkaProducer.flush(); if (transactionalId != null) { flushNewPartitions(); } }
@Override protected void commit(KafkaTransactionState transaction) { if (transaction.isTransactional()) { try { transaction.producer.commitTransaction(); } finally { recycleTransactionalProducer(transaction.producer); } } }
/** * Flush pending records. * @param transaction */ private void flush(KafkaTransactionState transaction) throws FlinkKafka011Exception { if (transaction.producer != null) { transaction.producer.flush(); } long pendingRecordsCount = pendingRecords.get(); if (pendingRecordsCount != 0) { throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount); } // if the flushed requests has errors, we should propagate it also and fail the checkpoint checkErroneous(); }
@Override public void close() throws FlinkKafka011Exception { final KafkaTransactionState currentTransaction = currentTransaction(); if (currentTransaction != null) { // to avoid exceptions on aborting transactions with some pending records flush(currentTransaction); // normal abort for AT_LEAST_ONCE and NONE do not clean up resources because of producer reusing, thus // we need to close it manually switch (semantic) { case EXACTLY_ONCE: break; case AT_LEAST_ONCE: case NONE: currentTransaction.producer.close(); break; } } try { super.close(); } catch (Exception e) { asyncException = ExceptionUtils.firstOrSuppressed(e, asyncException); } // make sure we propagate pending errors checkErroneous(); pendingTransactions().forEach(transaction -> IOUtils.closeQuietly(transaction.getValue().producer) ); }
/** * Instead of obtaining producerId and epoch from the transaction coordinator, re-use previously obtained ones, * so that we can resume transaction after a restart. Implementation of this method is based on * {@link org.apache.kafka.clients.producer.KafkaProducer#initTransactions}. */ public void resumeTransaction(long producerId, short epoch) { Preconditions.checkState(producerId >= 0 && epoch >= 0, "Incorrect values for producerId {} and epoch {}", producerId, epoch); LOG.info("Attempting to resume transaction {} with producerId {} and epoch {}", transactionalId, producerId, epoch); Object transactionManager = getValue(kafkaProducer, "transactionManager"); synchronized (transactionManager) { Object sequenceNumbers = getValue(transactionManager, "sequenceNumbers"); invoke(transactionManager, "transitionTo", getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.INITIALIZING")); invoke(sequenceNumbers, "clear"); Object producerIdAndEpoch = getValue(transactionManager, "producerIdAndEpoch"); setValue(producerIdAndEpoch, "producerId", producerId); setValue(producerIdAndEpoch, "epoch", epoch); invoke(transactionManager, "transitionTo", getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.READY")); invoke(transactionManager, "transitionTo", getEnum("org.apache.kafka.clients.producer.internals.TransactionManager$State.IN_TRANSACTION")); setValue(transactionManager, "transactionStarted", true); } }
/** * Besides committing {@link org.apache.kafka.clients.producer.KafkaProducer#commitTransaction} is also adding new * partitions to the transaction. flushNewPartitions method is moving this logic to pre-commit/flush, to make * resumeTransaction simpler. Otherwise resumeTransaction would require to restore state of the not yet added/"in-flight" * partitions. */ private void flushNewPartitions() { LOG.info("Flushing new partitions"); TransactionalRequestResult result = enqueueNewPartitions(); Object sender = getValue(kafkaProducer, "sender"); invoke(sender, "wakeup"); result.await(); }
private void recycleTransactionalProducer(FlinkKafkaProducer<byte[], byte[]> producer) { availableTransactionalIds.add(producer.getTransactionalId()); producer.close(); }
KafkaTransactionState(String transactionalId, FlinkKafkaProducer<byte[], byte[]> producer) { this(transactionalId, producer.getProducerId(), producer.getEpoch(), producer); }
@Override protected void recoverAndCommit(KafkaTransactionState transaction) { switch (semantic) { case EXACTLY_ONCE: try (FlinkKafkaProducer<byte[], byte[]> producer = initTransactionalProducer(transaction.transactionalId, false)) { producer.resumeTransaction(transaction.producerId, transaction.epoch); producer.commitTransaction(); } catch (InvalidTxnStateException | ProducerFencedException ex) { // That means we have committed this transaction before. LOG.warn("Encountered error {} while recovering transaction {}. " + "Presumably this transaction has been already committed before", ex, transaction); } break; case AT_LEAST_ONCE: case NONE: break; default: throw new UnsupportedOperationException("Not implemented semantic"); } }
@Override public void flush() { kafkaProducer.flush(); if (transactionalId != null) { flushNewPartitions(); } }
@Override protected void commit(KafkaTransactionState transaction) { switch (semantic) { case EXACTLY_ONCE: transaction.producer.commitTransaction(); recycleTransactionalProducer(transaction.producer); break; case AT_LEAST_ONCE: case NONE: break; default: throw new UnsupportedOperationException("Not implemented semantic"); } }
/** * Flush pending records. * @param transaction */ private void flush(KafkaTransactionState transaction) throws FlinkKafka011Exception { if (transaction.producer != null) { transaction.producer.flush(); } long pendingRecordsCount = pendingRecords.get(); if (pendingRecordsCount != 0) { throw new IllegalStateException("Pending record count must be zero at this point: " + pendingRecordsCount); } // if the flushed requests has errors, we should propagate it also and fail the checkpoint checkErroneous(); }