@ -18,7 +18,6 @@ package kafka.coordinator.transaction
@@ -18,7 +18,6 @@ package kafka.coordinator.transaction
import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantReadWriteLock
import kafka.server. { DelayedOperationPurgatory , KafkaConfig , MetadataCache , ReplicaManager }
import kafka.utils. { Logging , Scheduler , ZkUtils }
@ -28,8 +27,6 @@ import org.apache.kafka.common.protocol.Errors
@@ -28,8 +27,6 @@ import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.requests.TransactionResult
import org.apache.kafka.common.utils.Time
import kafka.utils.CoreUtils.inWriteLock
object TransactionCoordinator {
@ -253,7 +250,7 @@ class TransactionCoordinator(brokerId: Int,
@@ -253,7 +250,7 @@ class TransactionCoordinator(brokerId: Int,
}
def handleTxnImmigration ( transactionStateTopicPartitionId : Int , coordinatorEpoch : Int ) {
txnManager . loadTransactionsForPartition ( transactionStateTopicPartitionId , coordinatorEpoch )
txnManager . loadTransactionsForPartition ( transactionStateTopicPartitionId , coordinatorEpoch , writeTxnMarkers )
}
def handleTxnEmigration ( transactionStateTopicPartitionId : Int ) {
@ -322,61 +319,66 @@ class TransactionCoordinator(brokerId: Int,
@@ -322,61 +319,66 @@ class TransactionCoordinator(brokerId: Int,
if ( errors == Errors . NONE )
txnManager . coordinatorEpochFor ( transactionalId ) match {
case Some ( coordinatorEpoch ) =>
def completionCallback ( error : Errors ) : Unit = {
error match {
case Errors . NONE =>
txnManager . getTransactionState ( transactionalId ) match {
case Some ( preparedCommitMetadata ) =>
val completedState = if ( nextState == PrepareCommit ) CompleteCommit else CompleteAbort
val committedMetadata = new TransactionMetadata ( pid ,
epoch ,
preparedCommitMetadata . txnTimeoutMs ,
completedState ,
preparedCommitMetadata . topicPartitions ,
preparedCommitMetadata . transactionStartTime ,
time . milliseconds ( ) )
preparedCommitMetadata . prepareTransitionTo ( completedState )
def writeCommittedTransactionCallback ( error : Errors ) : Unit =
error match {
case Errors . NONE =>
trace ( s" completed txn for transactionalId: $transactionalId state after commit: ${ txnManager . getTransactionState ( transactionalId ) } " )
txnMarkerChannelManager . removeCompleted ( txnManager . partitionFor ( transactionalId ) , pid )
case Errors . NOT_COORDINATOR =>
// this one should be completed by the new coordinator
warn ( s" no longer the coordinator for transactionalId: $transactionalId " )
case _ =>
warn ( s" error: $error caught for transactionalId: $transactionalId when appending state: $completedState . retrying " )
// retry until success
txnManager . appendTransactionToLog ( transactionalId , committedMetadata , writeCommittedTransactionCallback )
}
txnManager . appendTransactionToLog ( transactionalId , committedMetadata , writeCommittedTransactionCallback )
case None =>
// this one should be completed by the new coordinator
warn ( s" no longer the coordinator for transactionalId: $transactionalId " )
}
case Errors . NOT_COORDINATOR =>
warn ( s" no longer the coordinator for transactionalId: $transactionalId " )
case _ =>
warn ( s" error: $error caught when writing transaction markers for transactionalId: $transactionalId . retrying " )
txnMarkerChannelManager . addTxnMarkerRequest ( txnManager . partitionFor ( transactionalId ) ,
newMetadata ,
coordinatorEpoch ,
completionCallback )
}
}
txnMarkerChannelManager . addTxnMarkerRequest ( txnManager . partitionFor ( transactionalId ) , newMetadata , coordinatorEpoch , completionCallback )
writeTxnMarkers ( WriteTxnMarkerArgs ( transactionalId , pid , epoch , nextState , newMetadata , coordinatorEpoch ) )
case None =>
// this one should be completed by the new coordinator
warn ( s" no longer the coordinator for transactionalId: $transactionalId " )
}
}
txnManager . appendTransactionToLog ( transactionalId , newMetadata , logAppendCallback )
}
private def writeTxnMarkers ( markerArgs : WriteTxnMarkerArgs ) : Unit = {
def completionCallback ( error : Errors ) : Unit = {
error match {
case Errors . NONE =>
txnManager . getTransactionState ( markerArgs . transactionalId ) match {
case Some ( preparedCommitMetadata ) =>
val completedState = if ( markerArgs . nextState == PrepareCommit ) CompleteCommit else CompleteAbort
val committedMetadata = new TransactionMetadata ( markerArgs . pid ,
markerArgs . epoch ,
preparedCommitMetadata . txnTimeoutMs ,
completedState ,
preparedCommitMetadata . topicPartitions ,
preparedCommitMetadata . transactionStartTime ,
time . milliseconds ( ) )
preparedCommitMetadata . prepareTransitionTo ( completedState )
def writeCommittedTransactionCallback ( error : Errors ) : Unit = {
error match {
case Errors . NONE =>
txnMarkerChannelManager . removeCompleted ( txnManager . partitionFor ( markerArgs . transactionalId ) ,
markerArgs . pid )
case Errors . NOT_COORDINATOR =>
// this one should be completed by the new coordinator
warn ( s" no longer the coordinator for transactionalId: ${ markerArgs . transactionalId } " )
case _ =>
warn ( s" error: $error caught for transactionalId: ${ markerArgs . transactionalId } when appending state: $completedState . Retrying. " )
// retry until success
txnManager . appendTransactionToLog ( markerArgs . transactionalId , committedMetadata , writeCommittedTransactionCallback )
}
}
txnManager . appendTransactionToLog ( markerArgs . transactionalId , committedMetadata , writeCommittedTransactionCallback )
case None =>
// this one should be completed by the new coordinator
warn ( s" no longer the coordinator for transactionalId: ${ markerArgs . transactionalId } " )
}
case Errors . NOT_COORDINATOR =>
warn ( s" no longer the coordinator for transactionalId: ${ markerArgs . transactionalId } " )
case _ =>
warn ( s" error: $error caught when writing transaction markers for transactionalId: ${ markerArgs . transactionalId } . retrying " )
txnMarkerChannelManager . addTxnMarkerRequest ( txnManager . partitionFor ( markerArgs . transactionalId ) ,
markerArgs . newMetadata ,
markerArgs . coordinatorEpoch ,
completionCallback )
}
}
txnMarkerChannelManager . addTxnMarkerRequest ( txnManager . partitionFor ( markerArgs . transactionalId ) ,
markerArgs . newMetadata ,
markerArgs . coordinatorEpoch ,
completionCallback )
}
def transactionTopicConfigs : Properties = txnManager . transactionTopicConfigs
def partitionFor ( transactionalId : String ) : Int = txnManager . partitionFor ( transactionalId )
@ -408,4 +410,10 @@ class TransactionCoordinator(brokerId: Int,
@@ -408,4 +410,10 @@ class TransactionCoordinator(brokerId: Int,
}
}
case class InitPidResult ( pid : Long , epoch : Short , error : Errors )
case class InitPidResult ( pid : Long , epoch : Short , error : Errors )
case class WriteTxnMarkerArgs ( transactionalId : String ,
pid : Long ,
epoch : Short ,
nextState : TransactionState ,
newMetadata : TransactionMetadata ,
coordinatorEpoch : Int )