KAFKA-14563: RemoveClient-Side AddPartitionsToTxn Requests (#17698)

Removes the client side AddPartitionsToTxn/AddOffsetsToTxn calls so that the partition is implicitly added as part of KIP-890 part 2. 

This change also requires updating the valid state transitions. The client side can not know for certain if a partition has been added server side when the request times out (partial completion). Thus for TV2, the transition to PrepareAbort is now valid for Empty, CompleteCommit, and CompleteAbort. 

For readability, the V1 and V2 endTransaction methods have been separated. 

Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>, Ritika Reddy <rreddy@confluent.io>
This commit is contained in:
Calvin Liu 2024-12-06 09:00:04 -08:00 committed by GitHub
parent c920989205
commit 755adf8a56
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
38 changed files with 1411 additions and 472 deletions

View File

@ -297,7 +297,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
*/
public KafkaProducer(Map<String, Object> configs, Serializer<K> keySerializer, Serializer<V> valueSerializer) {
this(new ProducerConfig(ProducerConfig.appendSerializerToConfig(configs, keySerializer, valueSerializer)),
keySerializer, valueSerializer, null, null, null, Time.SYSTEM);
keySerializer, valueSerializer, null, null, null, new ApiVersions(), Time.SYSTEM);
}
/**
@ -351,6 +351,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
ProducerMetadata metadata,
KafkaClient kafkaClient,
ProducerInterceptors<K, V> interceptors,
ApiVersions apiVersions,
Time time) {
try {
this.producerConfig = config;
@ -423,7 +424,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG);
int deliveryTimeoutMs = configureDeliveryTimeout(config, log);
this.apiVersions = new ApiVersions();
this.apiVersions = apiVersions;
this.transactionManager = configureTransactionState(config, logContext);
// There is no need to do work required for adaptive partitioning, if we use a custom partitioner.
boolean enableAdaptivePartitioning = partitioner == null &&
@ -1509,6 +1510,11 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
return clientId;
}
// Visible for testing
TransactionManager getTransactionManager() {
return transactionManager;
}
private static class ClusterAndWaitTime {
final Cluster cluster;
final long waitedOnMetadataMs;

View File

@ -904,8 +904,15 @@ public class Sender implements Runnable {
}
String transactionalId = null;
// When we use transaction V1 protocol in transaction we set the request version upper limit to
// LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 so that the broker knows that we're using transaction protocol V1.
boolean useTransactionV1Version = false;
if (transactionManager != null && transactionManager.isTransactional()) {
transactionalId = transactionManager.transactionalId();
if (!transactionManager.isTransactionV2Enabled()) {
useTransactionV1Version = true;
}
}
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(minUsedMagic,
@ -913,7 +920,9 @@ public class Sender implements Runnable {
.setAcks(acks)
.setTimeoutMs(timeout)
.setTransactionalId(transactionalId)
.setTopicData(tpd));
.setTopicData(tpd),
useTransactionV1Version
);
RequestCompletionHandler callback = response -> handleProduceResponse(response, recordsByPartition, time.milliseconds());
String nodeId = Integer.toString(destination);

View File

@ -31,6 +31,7 @@ import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.ClusterAuthorizationException;
import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.InvalidPidMappingException;
import org.apache.kafka.common.errors.InvalidProducerEpochException;
import org.apache.kafka.common.errors.OutOfOrderSequenceException;
import org.apache.kafka.common.errors.ProducerFencedException;
@ -369,15 +370,22 @@ public class TransactionManager {
"(currentState= " + currentState + ")");
}
log.debug("Begin adding offsets {} for consumer group {} to transaction", offsets, groupMetadata);
AddOffsetsToTxnRequest.Builder builder = new AddOffsetsToTxnRequest.Builder(
new AddOffsetsToTxnRequestData()
.setTransactionalId(transactionalId)
.setProducerId(producerIdAndEpoch.producerId)
.setProducerEpoch(producerIdAndEpoch.epoch)
.setGroupId(groupMetadata.groupId())
);
AddOffsetsToTxnHandler handler = new AddOffsetsToTxnHandler(builder, offsets, groupMetadata);
// In transaction V2, the client will skip sending AddOffsetsToTxn before sending txnOffsetCommit.
TxnRequestHandler handler;
if (isTransactionV2Enabled()) {
log.debug("Begin adding offsets {} for consumer group {} to transaction with transaction protocol V2", offsets, groupMetadata);
handler = txnOffsetCommitHandler(null, offsets, groupMetadata);
} else {
log.debug("Begin adding offsets {} for consumer group {} to transaction", offsets, groupMetadata);
AddOffsetsToTxnRequest.Builder builder = new AddOffsetsToTxnRequest.Builder(
new AddOffsetsToTxnRequestData()
.setTransactionalId(transactionalId)
.setProducerId(producerIdAndEpoch.producerId)
.setProducerEpoch(producerIdAndEpoch.epoch)
.setGroupId(groupMetadata.groupId())
);
handler = new AddOffsetsToTxnHandler(builder, offsets, groupMetadata);
}
enqueueRequest(handler);
return handler.result;
@ -394,7 +402,10 @@ public class TransactionManager {
} else if (currentState != State.IN_TRANSACTION) {
throw new IllegalStateException("Cannot add partition " + topicPartition +
" to transaction while in state " + currentState);
} else if (isPartitionAdded(topicPartition) || isPartitionPendingAdd(topicPartition)) {
} else if (isTransactionV2Enabled()) {
txnPartitionMap.getOrCreate(topicPartition);
partitionsInTransaction.add(topicPartition);
} else if (transactionContainsPartition(topicPartition) || isPartitionPendingAdd(topicPartition)) {
return;
} else {
log.debug("Begin adding new partition {} to transaction", topicPartition);
@ -497,11 +508,6 @@ public class TransactionManager {
}
}
// visible for testing
synchronized boolean isPartitionAdded(TopicPartition partition) {
return partitionsInTransaction.contains(partition);
}
// visible for testing
synchronized boolean isPartitionPendingAdd(TopicPartition partition) {
return newPartitionsInTransaction.contains(partition) || pendingPartitionsInTransaction.contains(partition);
@ -695,7 +701,8 @@ public class TransactionManager {
if (exception instanceof ClusterAuthorizationException
|| exception instanceof TransactionalIdAuthorizationException
|| exception instanceof ProducerFencedException
|| exception instanceof UnsupportedVersionException) {
|| exception instanceof UnsupportedVersionException
|| exception instanceof InvalidPidMappingException) {
transitionToFatalError(exception);
} else if (isTransactional()) {
if (needToTriggerEpochBumpFromClient() && !isCompleting()) {
@ -830,7 +837,7 @@ public class TransactionManager {
return null;
}
if (nextRequestHandler.isEndTxn() && !transactionStarted) {
if (nextRequestHandler.isEndTxn() && (!isTransactionV2Enabled() && !transactionStarted)) {
nextRequestHandler.result.done();
if (currentState != State.FATAL_ERROR) {
log.debug("Not sending EndTxn for completed transaction since no partitions " +
@ -908,7 +915,7 @@ public class TransactionManager {
}
// visible for testing
synchronized boolean transactionContainsPartition(TopicPartition topicPartition) {
public synchronized boolean transactionContainsPartition(TopicPartition topicPartition) {
return partitionsInTransaction.contains(topicPartition);
}
@ -1142,8 +1149,13 @@ public class TransactionManager {
pendingTxnOffsetCommits,
groupMetadata.memberId(),
groupMetadata.generationId(),
groupMetadata.groupInstanceId()
groupMetadata.groupInstanceId(),
isTransactionV2Enabled()
);
if (result == null) {
// In this case, transaction V2 is in use.
return new TxnOffsetCommitHandler(builder);
}
return new TxnOffsetCommitHandler(result, builder);
}
@ -1741,6 +1753,11 @@ public class TransactionManager {
this.builder = builder;
}
private TxnOffsetCommitHandler(TxnOffsetCommitRequest.Builder builder) {
super("TxnOffsetCommitHandler");
this.builder = builder;
}
@Override
TxnOffsetCommitRequest.Builder requestBuilder() {
return builder;

View File

@ -40,8 +40,9 @@ import java.util.stream.Collectors;
import static org.apache.kafka.common.requests.ProduceResponse.INVALID_OFFSET;
public class ProduceRequest extends AbstractRequest {
public static final short LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 = 11;
public static Builder forMagic(byte magic, ProduceRequestData data) {
public static Builder forMagic(byte magic, ProduceRequestData data, boolean useTransactionV1Version) {
// Message format upgrades correspond with a bump in the produce request version. Older
// message format versions are generally not supported by the produce request versions
// following the bump.
@ -53,13 +54,15 @@ public class ProduceRequest extends AbstractRequest {
maxVersion = 2;
} else {
minVersion = 3;
maxVersion = ApiKeys.PRODUCE.latestVersion();
short latestVersion = ApiKeys.PRODUCE.latestVersion();
maxVersion = useTransactionV1Version ?
(short) Math.min(latestVersion, LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2) : latestVersion;
}
return new Builder(minVersion, maxVersion, data);
}
public static Builder forCurrentMagic(ProduceRequestData data) {
return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, data);
return forMagic(RecordBatch.CURRENT_MAGIC_VALUE, data, false);
}
public static class Builder extends AbstractRequest.Builder<ProduceRequest> {
@ -255,6 +258,10 @@ public class ProduceRequest extends AbstractRequest {
return new ProduceRequest(new ProduceRequestData(new ByteBufferAccessor(buffer), version), version);
}
public static boolean isTransactionV2Requested(short version) {
return version > LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2;
}
public static byte requiredMagicForVersion(short produceRequestVersion) {
if (produceRequestVersion < ApiKeys.PRODUCE.oldestVersion() || produceRequestVersion > ApiKeys.PRODUCE.latestVersion())
throw new IllegalArgumentException("Magic value to use for produce request version " +

View File

@ -39,19 +39,21 @@ import java.util.Optional;
import java.util.stream.Collectors;
public class TxnOffsetCommitRequest extends AbstractRequest {
public static final short LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 = 4;
private final TxnOffsetCommitRequestData data;
public static class Builder extends AbstractRequest.Builder<TxnOffsetCommitRequest> {
public final TxnOffsetCommitRequestData data;
public final boolean isTransactionV2Enabled;
public Builder(final String transactionalId,
final String consumerGroupId,
final long producerId,
final short producerEpoch,
final Map<TopicPartition, CommittedOffset> pendingTxnOffsetCommits) {
final Map<TopicPartition, CommittedOffset> pendingTxnOffsetCommits,
final boolean isTransactionV2Enabled) {
this(transactionalId,
consumerGroupId,
producerId,
@ -59,7 +61,8 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
pendingTxnOffsetCommits,
JoinGroupRequest.UNKNOWN_MEMBER_ID,
JoinGroupRequest.UNKNOWN_GENERATION_ID,
Optional.empty());
Optional.empty(),
isTransactionV2Enabled);
}
public Builder(final String transactionalId,
@ -69,22 +72,25 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
final Map<TopicPartition, CommittedOffset> pendingTxnOffsetCommits,
final String memberId,
final int generationId,
final Optional<String> groupInstanceId) {
final Optional<String> groupInstanceId,
final boolean isTransactionV2Enabled) {
super(ApiKeys.TXN_OFFSET_COMMIT);
this.isTransactionV2Enabled = isTransactionV2Enabled;
this.data = new TxnOffsetCommitRequestData()
.setTransactionalId(transactionalId)
.setGroupId(consumerGroupId)
.setProducerId(producerId)
.setProducerEpoch(producerEpoch)
.setTopics(getTopics(pendingTxnOffsetCommits))
.setMemberId(memberId)
.setGenerationId(generationId)
.setGroupInstanceId(groupInstanceId.orElse(null));
.setTransactionalId(transactionalId)
.setGroupId(consumerGroupId)
.setProducerId(producerId)
.setProducerEpoch(producerEpoch)
.setTopics(getTopics(pendingTxnOffsetCommits))
.setMemberId(memberId)
.setGenerationId(generationId)
.setGroupInstanceId(groupInstanceId.orElse(null));
}
public Builder(final TxnOffsetCommitRequestData data) {
super(ApiKeys.TXN_OFFSET_COMMIT);
this.data = data;
this.isTransactionV2Enabled = true;
}
@Override
@ -93,6 +99,9 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
throw new UnsupportedVersionException("Broker doesn't support group metadata commit API on version " + version
+ ", minimum supported request version is 3 which requires brokers to be on version 2.5 or above.");
}
if (!isTransactionV2Enabled) {
version = (short) Math.min(version, LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2);
}
return new TxnOffsetCommitRequest(data, version);
}

View File

@ -37,7 +37,12 @@
// Version 10 is the same as version 9 (KIP-951).
//
// Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890).
"validVersions": "0-11",
//
// Version 12 is the same as version 11 (KIP-890). Note when produce requests are used in transaction, if
// transaction V2 (KIP_890 part 2) is enabled, the produce request will also include the function for a
// AddPartitionsToTxn call. If V2 is disabled, the client can't use produce request version higher than 11 within
// a transaction.
"validVersions": "0-12",
"deprecatedVersions": "0-6",
"flexibleVersions": "9+",
"fields": [

View File

@ -36,7 +36,9 @@
// Version 10 adds 'CurrentLeader' and 'NodeEndpoints' as tagged fields (KIP-951)
//
// Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890).
"validVersions": "0-11",
//
// Version 12 is the same as version 10 (KIP-890).
"validVersions": "0-12",
"flexibleVersions": "9+",
"fields": [
{ "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+",

View File

@ -25,7 +25,12 @@
// Version 3 adds the member.id, group.instance.id and generation.id.
//
// Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890).
"validVersions": "0-4",
//
// Version 5 is the same as version 4 (KIP-890). Note when TxnOffsetCommit requests are used in transaction, if
// transaction V2 (KIP_890 part 2) is enabled, the TxnOffsetCommit request will also include the function for a
// AddOffsetsToTxn call. If V2 is disabled, the client can't use TxnOffsetCommit request version higher than 4 within
// a transaction.
"validVersions": "0-5",
"flexibleVersions": "3+",
"fields": [
{ "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",

View File

@ -24,7 +24,9 @@
// Version 3 adds illegal generation, fenced instance id, and unknown member id errors.
//
// Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890).
"validVersions": "0-4",
//
// Version 5 is the same with version 3 (KIP-890).
"validVersions": "0-5",
"flexibleVersions": "3+",
"fields": [
{ "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.clients.producer;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.KafkaClient;
import org.apache.kafka.clients.LeastLoadedNode;
@ -50,6 +51,7 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.message.AddOffsetsToTxnResponseData;
import org.apache.kafka.common.message.ApiVersionsResponseData;
import org.apache.kafka.common.message.EndTxnResponseData;
import org.apache.kafka.common.message.InitProducerIdResponseData;
import org.apache.kafka.common.message.TxnOffsetCommitRequestData;
@ -195,7 +197,7 @@ public class KafkaProducerTest {
ProducerInterceptors<K, V> interceptors,
Time time) {
return new KafkaProducer<>(new ProducerConfig(ProducerConfig.appendSerializerToConfig(configs, keySerializer, valueSerializer)),
keySerializer, valueSerializer, metadata, kafkaClient, interceptors, time);
keySerializer, valueSerializer, metadata, kafkaClient, interceptors, new ApiVersions(), time);
}
@BeforeEach
@ -757,7 +759,7 @@ public class KafkaProducerTest {
return new KafkaProducer<>(
new ProducerConfig(ProducerConfig.appendSerializerToConfig(configs, new StringSerializer(), new StringSerializer())),
new StringSerializer(), new StringSerializer(), metadata, mockClient, null, time) {
new StringSerializer(), new StringSerializer(), metadata, mockClient, null, new ApiVersions(), time) {
@Override
Sender newSender(LogContext logContext, KafkaClient kafkaClient, ProducerMetadata metadata) {
// give Sender its own Metadata instance so that we can isolate Metadata calls from KafkaProducer
@ -1609,6 +1611,123 @@ public class KafkaProducerTest {
}
}
@Test
public void testSendTxnOffsetsWithGroupIdTransactionV2() {
Properties properties = new Properties();
properties.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "some.id");
properties.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000);
properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000");
properties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
properties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1));
ProducerMetadata metadata = newMetadata(0, 0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
client.updateMetadata(initialUpdateResponse);
Node node = metadata.fetch().nodes().get(0);
client.setNodeApiVersions(NodeApiVersions.create());
NodeApiVersions nodeApiVersions = new NodeApiVersions(NodeApiVersions.create().allSupportedApiVersions().values(),
Arrays.asList(new ApiVersionsResponseData.SupportedFeatureKey()
.setName("transaction.version")
.setMaxVersion((short) 2)
.setMinVersion((short) 0)),
false,
Arrays.asList(new ApiVersionsResponseData.FinalizedFeatureKey()
.setName("transaction.version")
.setMaxVersionLevel((short) 2)
.setMinVersionLevel((short) 2)),
0);
client.setNodeApiVersions(nodeApiVersions);
ApiVersions apiVersions = new ApiVersions();
apiVersions.update(NODE.idString(), nodeApiVersions);
client.throttle(node, 5000);
client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", NODE));
client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE));
client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some.id", NODE));
String groupId = "group";
client.prepareResponse(request ->
((TxnOffsetCommitRequest) request).data().groupId().equals(groupId),
txnOffsetsCommitResponse(Collections.singletonMap(
new TopicPartition("topic", 0), Errors.NONE)));
client.prepareResponse(endTxnResponse(Errors.NONE));
try (KafkaProducer<String, String> producer = new KafkaProducer<>(
new ProducerConfig(properties), new StringSerializer(), new StringSerializer(), metadata, client,
new ProducerInterceptors<>(Collections.emptyList()), apiVersions, time)) {
producer.initTransactions();
producer.beginTransaction();
producer.sendOffsetsToTransaction(Collections.singletonMap(
new TopicPartition("topic", 0),
new OffsetAndMetadata(5L)),
new ConsumerGroupMetadata(groupId));
producer.commitTransaction();
}
}
@Test
public void testTransactionV2Produce() throws Exception {
StringSerializer serializer = new StringSerializer();
KafkaProducerTestContext<String> ctx = new KafkaProducerTestContext<>(testInfo, serializer);
String topic = "foo";
TopicPartition topicPartition = new TopicPartition(topic, 0);
Cluster cluster = TestUtils.singletonCluster(topic, 1);
when(ctx.sender.isRunning()).thenReturn(true);
when(ctx.metadata.fetch()).thenReturn(cluster);
long timestamp = ctx.time.milliseconds();
ProducerRecord<String, String> record = new ProducerRecord<>(topic, 0, timestamp, "key", "value");
Properties props = new Properties();
props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
props.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "some-txn");
props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
ProducerConfig config = new ProducerConfig(props);
Time time = new MockTime(1);
MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap(topic, 1));
ProducerMetadata metadata = newMetadata(0, 0, Long.MAX_VALUE);
MockClient client = new MockClient(time, metadata);
client.updateMetadata(initialUpdateResponse);
NodeApiVersions nodeApiVersions = new NodeApiVersions(NodeApiVersions.create().allSupportedApiVersions().values(),
Arrays.asList(new ApiVersionsResponseData.SupportedFeatureKey()
.setName("transaction.version")
.setMaxVersion((short) 2)
.setMinVersion((short) 0)),
false,
Arrays.asList(new ApiVersionsResponseData.FinalizedFeatureKey()
.setName("transaction.version")
.setMaxVersionLevel((short) 2)
.setMinVersionLevel((short) 2)),
0);
client.setNodeApiVersions(nodeApiVersions);
ApiVersions apiVersions = new ApiVersions();
apiVersions.update(NODE.idString(), nodeApiVersions);
ProducerInterceptors<String, String> interceptor = new ProducerInterceptors<>(Collections.emptyList());
client.prepareResponse(FindCoordinatorResponse.prepareResponse(Errors.NONE, "some-txn", NODE));
client.prepareResponse(initProducerIdResponse(1L, (short) 5, Errors.NONE));
client.prepareResponse(produceResponse(topicPartition, 1L, Errors.NONE, 0, 1));
client.prepareResponse(endTxnResponse(Errors.NONE));
try (KafkaProducer<String, String> producer = new KafkaProducer<>(
config, new StringSerializer(), new StringSerializer(), metadata, client, interceptor, apiVersions, time)
) {
producer.initTransactions();
producer.beginTransaction();
producer.send(record).get();
producer.commitTransaction();
}
}
private void assertDurationAtLeast(KafkaProducer<?, ?> producer, String name, double floor) {
getAndAssertDurationAtLeast(producer, name, floor);
}
@ -2095,7 +2214,7 @@ public class KafkaProducerTest {
assertTrue(config.unused().contains(SslConfigs.SSL_PROTOCOL_CONFIG));
try (KafkaProducer<byte[], byte[]> producer = new KafkaProducer<>(config, null, null,
null, null, null, Time.SYSTEM)) {
null, null, null, null, Time.SYSTEM)) {
assertTrue(config.unused().contains(SslConfigs.SSL_PROTOCOL_CONFIG));
}
}
@ -2549,6 +2668,12 @@ public class KafkaProducerTest {
assertDoesNotThrow(() -> new KafkaProducer<>(configs, new StringSerializer(), new StringSerializer()).close());
}
@SuppressWarnings("deprecation")
private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs, int logStartOffset) {
ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP, logStartOffset);
Map<TopicPartition, ProduceResponse.PartitionResponse> partResp = singletonMap(tp, resp);
return new ProduceResponse(partResp, throttleTimeMs);
}
@Test
public void testSubscribingCustomMetricsDoesntAffectProducerMetrics() {
@ -2655,7 +2780,4 @@ public class KafkaProducerTest {
KafkaMetric streamClientMetricTwo = new KafkaMetric(lock, metricNameTwo, (Measurable) (m, now) -> 2.0, metricConfig, Time.SYSTEM);
return Map.of(metricNameOne, streamClientMetricOne, metricNameTwo, streamClientMetricTwo);
}
}

View File

@ -1004,7 +1004,7 @@ public class RecordAccumulatorTest {
ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(12345L, (short) 5);
Mockito.when(transactionManager.producerIdAndEpoch()).thenReturn(producerIdAndEpoch);
Mockito.when(transactionManager.isSendToPartitionAllowed(tp1)).thenReturn(true);
Mockito.when(transactionManager.isPartitionAdded(tp1)).thenReturn(true);
Mockito.when(transactionManager.transactionContainsPartition(tp1)).thenReturn(true);
Mockito.when(transactionManager.firstInFlightSequence(tp1)).thenReturn(0);
// Initially, the transaction is still in progress, so we should respect the linger.

View File

@ -2870,7 +2870,7 @@ public class SenderTest {
private void addPartitionToTxn(Sender sender, TransactionManager txnManager, TopicPartition tp) {
txnManager.maybeAddPartition(tp);
client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tp, Errors.NONE)));
runUntil(sender, () -> txnManager.isPartitionAdded(tp));
runUntil(sender, () -> txnManager.transactionContainsPartition(tp));
assertFalse(txnManager.hasInFlightRequest());
}

View File

@ -172,7 +172,11 @@ public class TransactionManagerTest {
new ApiVersion()
.setApiKey(ApiKeys.PRODUCE.id)
.setMinVersion((short) 0)
.setMaxVersion((short) 7)),
.setMaxVersion(transactionV2Enabled ? ApiKeys.PRODUCE.latestVersion() : (short) 11),
new ApiVersion()
.setApiKey(ApiKeys.TXN_OFFSET_COMMIT.id)
.setMinVersion((short) 0)
.setMaxVersion(transactionV2Enabled ? ApiKeys.TXN_OFFSET_COMMIT.latestVersion() : (short) 4)),
Arrays.asList(new ApiVersionsResponseData.SupportedFeatureKey()
.setName("transaction.version")
.setMaxVersion(transactionV2Enabled ? (short) 2 : (short) 1)
@ -230,7 +234,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxn(tp0, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
transactionManager.beginCommit();
assertNull(transactionManager.nextRequest(true));
@ -293,7 +297,7 @@ public class TransactionManagerTest {
runUntil(transactionManager::hasOngoingTransaction);
prepareAddPartitionsToTxn(partition, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(partition));
runUntil(() -> transactionManager.transactionContainsPartition(partition));
transactionManager.beginAbort();
assertTrue(transactionManager.hasOngoingTransaction());
@ -317,7 +321,7 @@ public class TransactionManagerTest {
assertTrue(transactionManager.hasOngoingTransaction());
prepareAddPartitionsToTxn(partition, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(partition));
runUntil(() -> transactionManager.transactionContainsPartition(partition));
transactionManager.beginCommit();
assertTrue(transactionManager.hasOngoingTransaction());
@ -341,7 +345,7 @@ public class TransactionManagerTest {
assertTrue(transactionManager.hasOngoingTransaction());
prepareAddPartitionsToTxn(partition, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(partition));
runUntil(() -> transactionManager.transactionContainsPartition(partition));
transactionManager.transitionToAbortableError(new KafkaException());
assertTrue(transactionManager.hasOngoingTransaction());
@ -368,7 +372,7 @@ public class TransactionManagerTest {
assertTrue(transactionManager.hasOngoingTransaction());
prepareAddPartitionsToTxn(partition, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(partition));
runUntil(() -> transactionManager.transactionContainsPartition(partition));
transactionManager.transitionToFatalError(new KafkaException());
assertFalse(transactionManager.hasOngoingTransaction());
@ -382,20 +386,40 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(partition);
assertTrue(transactionManager.hasPartitionsToAdd());
assertFalse(transactionManager.isPartitionAdded(partition));
assertFalse(transactionManager.transactionContainsPartition(partition));
assertTrue(transactionManager.isPartitionPendingAdd(partition));
prepareAddPartitionsToTxn(partition, Errors.NONE);
assertTrue(transactionManager.hasPartitionsToAdd());
runUntil(() -> transactionManager.isPartitionAdded(partition));
runUntil(() -> transactionManager.transactionContainsPartition(partition));
assertFalse(transactionManager.hasPartitionsToAdd());
assertFalse(transactionManager.isPartitionPendingAdd(partition));
// adding the partition again should not have any effect
transactionManager.maybeAddPartition(partition);
assertFalse(transactionManager.hasPartitionsToAdd());
assertTrue(transactionManager.isPartitionAdded(partition));
assertTrue(transactionManager.transactionContainsPartition(partition));
assertFalse(transactionManager.isPartitionPendingAdd(partition));
}
@Test
public void testMaybeAddPartitionToTransactionInTransactionV2() {
initializeTransactionManager(Optional.of(transactionalId), true);
TopicPartition partition = new TopicPartition("foo", 0);
doInitTransactions();
transactionManager.beginTransaction();
transactionManager.maybeAddPartition(partition);
// In V2, the maybeAddPartition should not add the partition to the pending list.
assertFalse(transactionManager.hasPartitionsToAdd());
assertTrue(transactionManager.transactionContainsPartition(partition));
assertFalse(transactionManager.isPartitionPendingAdd(partition));
// Adding the partition again should not have any effect
transactionManager.maybeAddPartition(partition);
assertFalse(transactionManager.hasPartitionsToAdd());
assertTrue(transactionManager.transactionContainsPartition(partition));
assertFalse(transactionManager.isPartitionPendingAdd(partition));
}
@ -407,7 +431,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(partition);
assertTrue(transactionManager.hasPartitionsToAdd());
assertFalse(transactionManager.isPartitionAdded(partition));
assertFalse(transactionManager.transactionContainsPartition(partition));
assertTrue(transactionManager.isPartitionPendingAdd(partition));
prepareAddPartitionsToTxn(partition, Errors.CONCURRENT_TRANSACTIONS);
@ -426,7 +450,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(partition);
assertTrue(transactionManager.hasPartitionsToAdd());
assertFalse(transactionManager.isPartitionAdded(partition));
assertFalse(transactionManager.transactionContainsPartition(partition));
assertTrue(transactionManager.isPartitionPendingAdd(partition));
prepareAddPartitionsToTxn(partition, Errors.COORDINATOR_NOT_AVAILABLE);
@ -445,11 +469,11 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(partition);
assertTrue(transactionManager.hasPartitionsToAdd());
assertFalse(transactionManager.isPartitionAdded(partition));
assertFalse(transactionManager.transactionContainsPartition(partition));
assertTrue(transactionManager.isPartitionPendingAdd(partition));
prepareAddPartitionsToTxn(partition, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(partition));
runUntil(() -> transactionManager.transactionContainsPartition(partition));
TopicPartition otherPartition = new TopicPartition("foo", 1);
transactionManager.maybeAddPartition(otherPartition);
@ -919,7 +943,7 @@ public class TransactionManagerTest {
assertTrue(transactionManager.hasOngoingTransaction());
prepareAddPartitionsToTxn(tp1, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(tp1));
runUntil(() -> transactionManager.transactionContainsPartition(tp1));
TransactionalRequestResult retryResult = transactionManager.beginCommit();
assertTrue(transactionManager.hasOngoingTransaction());
@ -937,6 +961,104 @@ public class TransactionManagerTest {
assertTrue(transactionManager.isTransactionV2Enabled());
}
@Test
public void testTransactionV2AddPartitionAndOffsets() throws InterruptedException {
initializeTransactionManager(Optional.of(transactionalId), true);
doInitTransactions();
transactionManager.beginTransaction();
Future<RecordMetadata> responseFuture = appendToAccumulator(tp0);
assertFalse(responseFuture.isDone());
prepareProduceResponse(Errors.NONE, producerId, epoch);
transactionManager.maybeAddPartition(tp0);
assertTrue(transactionManager.transactionContainsPartition(tp0));
assertTrue(transactionManager.isSendToPartitionAllowed(tp0));
assertFalse(responseFuture.isDone());
runUntil(responseFuture::isDone);
// Now, test adding the offsets.
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
offsets.put(tp1, new OffsetAndMetadata(1));
TransactionalRequestResult addOffsetsResult = transactionManager.sendOffsetsToTransaction(
offsets, new ConsumerGroupMetadata(consumerGroupId));
assertTrue(transactionManager.hasPendingOffsetCommits());
// the result doesn't complete until TxnOffsetCommit returns
assertFalse(addOffsetsResult.isCompleted());
Map<TopicPartition, Errors> txnOffsetCommitResponse = new HashMap<>();
txnOffsetCommitResponse.put(tp1, Errors.NONE);
prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId);
prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, txnOffsetCommitResponse);
assertNull(transactionManager.coordinator(CoordinatorType.GROUP));
runUntil(() -> transactionManager.coordinator(CoordinatorType.GROUP) != null);
assertTrue(transactionManager.hasPendingOffsetCommits());
runUntil(() -> !transactionManager.hasPendingOffsetCommits());
// We should only be done after both RPCs complete.
assertTrue(addOffsetsResult.isCompleted());
transactionManager.beginCommit();
prepareEndTxnResponse(Errors.NONE, TransactionResult.COMMIT, producerId, epoch, producerId, (short) (epoch + 1), false);
runUntil(() -> !transactionManager.hasOngoingTransaction());
assertFalse(transactionManager.isCompleting());
}
@Test
public void testTransactionManagerDisablesV2() {
Metrics metrics = new Metrics(time);
apiVersions.update("0", new NodeApiVersions(Arrays.asList(
new ApiVersion()
.setApiKey(ApiKeys.INIT_PRODUCER_ID.id)
.setMinVersion((short) 0)
.setMaxVersion((short) 3),
new ApiVersion()
.setApiKey(ApiKeys.PRODUCE.id)
.setMinVersion((short) 5)
.setMaxVersion((short) (ProduceRequest.LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 + 1)),
new ApiVersion()
.setApiKey(ApiKeys.TXN_OFFSET_COMMIT.id)
.setMinVersion((short) 1)
.setMaxVersion((short) (TxnOffsetCommitRequest.LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2 + 1))),
Arrays.asList(new ApiVersionsResponseData.SupportedFeatureKey()
.setName("transaction.version")
.setMaxVersion((short) 1)
.setMinVersion((short) 0)),
false,
Arrays.asList(new ApiVersionsResponseData.FinalizedFeatureKey()
.setName("transaction.version")
.setMaxVersionLevel((short) 1)
.setMinVersionLevel((short) 1)),
0));
this.transactionManager = new TransactionManager(logContext, transactionalId,
transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, apiVersions);
int batchSize = 16 * 1024;
int deliveryTimeoutMs = 3000;
long totalSize = 1024 * 1024;
String metricGrpName = "producer-metrics";
this.brokerNode = new Node(0, "localhost", 2211);
this.accumulator = new RecordAccumulator(logContext, batchSize, Compression.NONE, 0, 0L, 0L,
deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
this.sender = new Sender(logContext, this.client, this.metadata, this.accumulator, true,
MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(metrics), this.time, REQUEST_TIMEOUT,
50, transactionManager, apiVersions);
doInitTransactions();
assertFalse(transactionManager.isTransactionV2Enabled());
}
@Test
public void testDisconnectAndRetry() {
// This is called from the initTransactions method in the producer as the first order of business.
@ -1291,6 +1413,22 @@ public class TransactionManagerTest {
assertAbortableError(GroupAuthorizationException.class);
}
@Test
public void testFatalErrorWhenProduceResponseWithInvalidPidMapping() throws InterruptedException {
initializeTransactionManager(Optional.of(transactionalId), true);
doInitTransactions();
transactionManager.beginTransaction();
Future<RecordMetadata> responseFuture = appendToAccumulator(tp0);
transactionManager.maybeAddPartition(tp0);
assertFalse(responseFuture.isDone());
prepareProduceResponse(Errors.INVALID_PRODUCER_ID_MAPPING, producerId, epoch);
assertFalse(responseFuture.isDone());
runUntil(responseFuture::isDone);
assertTrue(transactionManager.hasFatalError());
}
@Test
public void testTransactionalIdAuthorizationFailureInAddOffsetsToTxn() {
final TopicPartition tp = new TopicPartition("foo", 0);
@ -1380,8 +1518,8 @@ public class TransactionManagerTest {
assertInstanceOf(TopicAuthorizationException.class, transactionManager.lastError());
assertFalse(transactionManager.isPartitionPendingAdd(tp0));
assertFalse(transactionManager.isPartitionPendingAdd(tp1));
assertFalse(transactionManager.isPartitionAdded(tp0));
assertFalse(transactionManager.isPartitionAdded(tp1));
assertFalse(transactionManager.transactionContainsPartition(tp0));
assertFalse(transactionManager.transactionContainsPartition(tp1));
assertFalse(transactionManager.hasPartitionsToAdd());
TopicAuthorizationException exception = (TopicAuthorizationException) transactionManager.lastError();
@ -1476,7 +1614,7 @@ public class TransactionManagerTest {
responseFuture = appendToAccumulator(tp0);
prepareAddPartitionsToTxn(singletonMap(tp0, Errors.NONE));
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
assertFalse(transactionManager.hasPartitionsToAdd());
transactionManager.beginCommit();
@ -1497,7 +1635,7 @@ public class TransactionManagerTest {
prepareAddPartitionsToTxn(tp0, Errors.NONE);
appendToAccumulator(tp0);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
TransactionalRequestResult result = transactionManager.beginAbort();
assertThrows(TimeoutException.class, () -> result.await(0, TimeUnit.MILLISECONDS));
@ -1531,7 +1669,7 @@ public class TransactionManagerTest {
prepareProduceResponse(Errors.NONE, producerId, epoch);
appendToAccumulator(tp0);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
TransactionalRequestResult result = transactionManager.beginCommit();
assertThrows(TimeoutException.class, () -> result.await(0, TimeUnit.MILLISECONDS));
@ -1597,14 +1735,14 @@ public class TransactionManagerTest {
prepareAddPartitionsToTxn(tp0, Errors.NONE);
Future<RecordMetadata> authorizedTopicProduceFuture = appendToAccumulator(unauthorizedPartition);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
transactionManager.maybeAddPartition(unauthorizedPartition);
Future<RecordMetadata> unauthorizedTopicProduceFuture = appendToAccumulator(unauthorizedPartition);
prepareAddPartitionsToTxn(singletonMap(unauthorizedPartition, Errors.TOPIC_AUTHORIZATION_FAILED));
runUntil(transactionManager::hasAbortableError);
assertTrue(transactionManager.isPartitionAdded(tp0));
assertFalse(transactionManager.isPartitionAdded(unauthorizedPartition));
assertTrue(transactionManager.transactionContainsPartition(tp0));
assertFalse(transactionManager.transactionContainsPartition(unauthorizedPartition));
assertFalse(authorizedTopicProduceFuture.isDone());
assertFalse(unauthorizedTopicProduceFuture.isDone());
@ -1627,7 +1765,7 @@ public class TransactionManagerTest {
FutureRecordMetadata nextTransactionFuture = appendToAccumulator(tp0);
prepareAddPartitionsToTxn(singletonMap(tp0, Errors.NONE));
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
assertFalse(transactionManager.hasPartitionsToAdd());
transactionManager.beginCommit();
@ -1650,7 +1788,7 @@ public class TransactionManagerTest {
prepareAddPartitionsToTxn(tp0, Errors.NONE);
Future<RecordMetadata> authorizedTopicProduceFuture = appendToAccumulator(tp0);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
accumulator.beginFlush();
prepareProduceResponse(Errors.REQUEST_TIMED_OUT, producerId, epoch);
@ -1662,8 +1800,8 @@ public class TransactionManagerTest {
Future<RecordMetadata> unauthorizedTopicProduceFuture = appendToAccumulator(unauthorizedPartition);
prepareAddPartitionsToTxn(singletonMap(unauthorizedPartition, Errors.TOPIC_AUTHORIZATION_FAILED));
runUntil(transactionManager::hasAbortableError);
assertTrue(transactionManager.isPartitionAdded(tp0));
assertFalse(transactionManager.isPartitionAdded(unauthorizedPartition));
assertTrue(transactionManager.transactionContainsPartition(tp0));
assertFalse(transactionManager.transactionContainsPartition(unauthorizedPartition));
assertFalse(authorizedTopicProduceFuture.isDone());
prepareProduceResponse(Errors.NONE, producerId, epoch);
@ -1691,7 +1829,7 @@ public class TransactionManagerTest {
FutureRecordMetadata nextTransactionFuture = appendToAccumulator(tp0);
prepareAddPartitionsToTxn(singletonMap(tp0, Errors.NONE));
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
assertFalse(transactionManager.hasPartitionsToAdd());
transactionManager.beginCommit();
@ -2821,7 +2959,7 @@ public class TransactionManagerTest {
@Test
public void testTransitionToFatalErrorWhenRetriedBatchIsExpired() throws InterruptedException {
apiVersions.update("0", NodeApiVersions.create(Arrays.asList(
apiVersions.update("0", new NodeApiVersions(Arrays.asList(
new ApiVersion()
.setApiKey(ApiKeys.INIT_PRODUCER_ID.id)
.setMinVersion((short) 0)
@ -2829,7 +2967,11 @@ public class TransactionManagerTest {
new ApiVersion()
.setApiKey(ApiKeys.PRODUCE.id)
.setMinVersion((short) 0)
.setMaxVersion((short) 7))));
.setMaxVersion((short) 7)),
Collections.emptyList(),
false,
Collections.emptyList(),
0));
doInitTransactions();
@ -3006,7 +3148,7 @@ public class TransactionManagerTest {
transactionManager.beginTransaction();
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
ProducerBatch b1 = writeTransactionalBatchWithValue(transactionManager, tp0, "1");
assertEquals(Integer.valueOf(1), transactionManager.sequenceNumber(tp0));
@ -3037,8 +3179,6 @@ public class TransactionManagerTest {
transactionManager.beginTransaction();
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
// Append record with initial producer ID and epoch.
Future<RecordMetadata> responseFuture = appendToAccumulator(tp0);
@ -3065,12 +3205,9 @@ public class TransactionManagerTest {
doInitTransactions(producerId, epoch);
transactionManager.beginTransaction();
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
// Append record with initial producer ID and epoch
Future<RecordMetadata> responseFuture = appendToAccumulator(tp0);
transactionManager.maybeAddPartition(tp0);
prepareProduceResponse(Errors.NONE, producerId, epoch);
runUntil(responseFuture::isDone);
@ -3113,7 +3250,7 @@ public class TransactionManagerTest {
@Test
public void testAbortTransactionAndReuseSequenceNumberOnError() throws InterruptedException {
apiVersions.update("0", NodeApiVersions.create(Arrays.asList(
apiVersions.update("0", new NodeApiVersions(Arrays.asList(
new ApiVersion()
.setApiKey(ApiKeys.INIT_PRODUCER_ID.id)
.setMinVersion((short) 0)
@ -3125,8 +3262,11 @@ public class TransactionManagerTest {
new ApiVersion()
.setApiKey(ApiKeys.PRODUCE.id)
.setMinVersion((short) 0)
.setMaxVersion((short) 7)
)));
.setMaxVersion((short) 7)),
Collections.emptyList(),
false,
Collections.emptyList(),
0));
doInitTransactions();
@ -3136,7 +3276,7 @@ public class TransactionManagerTest {
Future<RecordMetadata> responseFuture0 = appendToAccumulator(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId);
prepareProduceResponse(Errors.NONE, producerId, epoch);
runUntil(() -> transactionManager.isPartitionAdded(tp0)); // Send AddPartitionsRequest
runUntil(() -> transactionManager.transactionContainsPartition(tp0)); // Send AddPartitionsRequest
runUntil(responseFuture0::isDone);
Future<RecordMetadata> responseFuture1 = appendToAccumulator(tp0);
@ -3160,7 +3300,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0)); // Send AddPartitionsRequest
runUntil(() -> transactionManager.transactionContainsPartition(tp0)); // Send AddPartitionsRequest
assertEquals(2, transactionManager.sequenceNumber(tp0));
}
@ -3171,7 +3311,7 @@ public class TransactionManagerTest {
// where the sequence number is reset on an UnknownProducerId error, allowing subsequent transactions to
// append to the log successfully
// Set the EndTxn version such that sequence is not reset on every end txn.
apiVersions.update("0", NodeApiVersions.create(Arrays.asList(
apiVersions.update("0", new NodeApiVersions(Arrays.asList(
new ApiVersion()
.setApiKey(ApiKeys.INIT_PRODUCER_ID.id)
.setMinVersion((short) 0)
@ -3183,8 +3323,11 @@ public class TransactionManagerTest {
new ApiVersion()
.setApiKey(ApiKeys.END_TXN.id)
.setMinVersion((short) 0)
.setMaxVersion((short) 4)
)));
.setMaxVersion((short) 4)),
Collections.emptyList(),
false,
Collections.emptyList(),
0));
doInitTransactions();
@ -3195,14 +3338,14 @@ public class TransactionManagerTest {
prepareAddPartitionsToTxnResponse(Errors.NONE, tp1, epoch, producerId);
prepareProduceResponse(Errors.NONE, producerId, epoch, tp1);
runUntil(successPartitionResponseFuture::isDone);
assertTrue(transactionManager.isPartitionAdded(tp1));
assertTrue(transactionManager.transactionContainsPartition(tp1));
transactionManager.maybeAddPartition(tp0);
Future<RecordMetadata> responseFuture0 = appendToAccumulator(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId);
prepareProduceResponse(Errors.NONE, producerId, epoch);
runUntil(responseFuture0::isDone);
assertTrue(transactionManager.isPartitionAdded(tp0));
assertTrue(transactionManager.transactionContainsPartition(tp0));
Future<RecordMetadata> responseFuture1 = appendToAccumulator(tp0);
prepareProduceResponse(Errors.NONE, producerId, epoch);
@ -3226,7 +3369,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
assertEquals(0, transactionManager.sequenceNumber(tp0));
assertEquals(1, transactionManager.sequenceNumber(tp1));
@ -3244,7 +3387,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, initialEpoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
Future<RecordMetadata> responseFuture0 = appendToAccumulator(tp0);
prepareProduceResponse(Errors.NONE, producerId, initialEpoch);
@ -3274,7 +3417,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, bumpedEpoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
assertEquals(0, transactionManager.sequenceNumber(tp0));
}
@ -3290,7 +3433,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, initialEpoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
Future<RecordMetadata> responseFuture0 = appendToAccumulator(tp0);
prepareProduceResponse(Errors.NONE, producerId, initialEpoch);
@ -3321,7 +3464,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, bumpedEpoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
assertEquals(0, transactionManager.sequenceNumber(tp0));
}
@ -3337,7 +3480,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, initialEpoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
Future<RecordMetadata> responseFuture0 = appendToAccumulator(tp0);
prepareProduceResponse(Errors.NONE, producerId, initialEpoch);
@ -3380,7 +3523,7 @@ public class TransactionManagerTest {
transactionManager.maybeAddPartition(tp0);
prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, bumpedEpoch, producerId);
runUntil(() -> transactionManager.isPartitionAdded(tp0));
runUntil(() -> transactionManager.transactionContainsPartition(tp0));
assertEquals(0, transactionManager.sequenceNumber(tp0));
}
@ -3697,7 +3840,7 @@ public class TransactionManagerTest {
assertTrue(transactionManager.hasOngoingTransaction());
prepareAddPartitionsToTxn(tp1, Errors.NONE);
runUntil(() -> transactionManager.isPartitionAdded(tp1));
runUntil(() -> transactionManager.transactionContainsPartition(tp1));
TransactionalRequestResult retryResult = transactionManager.beginCommit();
assertTrue(transactionManager.hasOngoingTransaction());

View File

@ -50,18 +50,18 @@ public class ProduceRequestTest {
@Test
public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() {
final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, Compression.NONE, 1L,
(short) 1, 1, 1, simpleRecord);
(short) 1, 1, 1, simpleRecord);
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("topic")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(1)
.setRecords(memoryRecords)))).iterator()))
.setAcks((short) -1)
.setTimeoutMs(10)).build();
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("topic")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(1)
.setRecords(memoryRecords)))).iterator()))
.setAcks((short) -1)
.setTimeoutMs(10)).build();
assertTrue(RequestUtils.hasTransactionalRecords(request));
}
@ -80,17 +80,17 @@ public class ProduceRequestTest {
@Test
public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() {
final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, Compression.NONE, 1L,
(short) 1, 1, 1, simpleRecord);
(short) 1, 1, 1, simpleRecord);
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("topic")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(1)
.setRecords(memoryRecords)))).iterator()))
.setAcks((short) -1)
.setTimeoutMs(10)).build();
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("topic")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(1)
.setRecords(memoryRecords)))).iterator()))
.setAcks((short) -1)
.setTimeoutMs(10)).build();
assertTrue(RequestTestUtils.hasIdempotentRecords(request));
}
@ -98,7 +98,7 @@ public class ProduceRequestTest {
public void testBuildWithOldMessageFormat() {
ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, Compression.NONE,
TimestampType.CREATE_TIME, 0L);
TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V1,
new ProduceRequestData()
@ -107,7 +107,8 @@ public class ProduceRequestTest {
new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
.setTimeoutMs(5000),
true);
assertEquals(2, requestBuilder.oldestAllowedVersion());
assertEquals(2, requestBuilder.latestAllowedVersion());
}
@ -116,16 +117,17 @@ public class ProduceRequestTest {
public void testBuildWithCurrentMessageFormat() {
ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
Compression.NONE, TimestampType.CREATE_TIME, 0L);
Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData().setName("test").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData().setName("test").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(9).setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000),
false);
assertEquals(3, requestBuilder.oldestAllowedVersion());
assertEquals(ApiKeys.PRODUCE.latestVersion(), requestBuilder.latestAllowedVersion());
}
@ -145,30 +147,30 @@ public class ProduceRequestTest {
buffer.flip();
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(MemoryRecords.readableRecords(buffer))))).iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(MemoryRecords.readableRecords(buffer))))).iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
assertThrowsForAllVersions(requestBuilder, InvalidRecordException.class);
}
@Test
public void testV3AndAboveCannotHaveNoRecordBatches() {
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(MemoryRecords.EMPTY)))).iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(MemoryRecords.EMPTY)))).iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
assertThrowsForAllVersions(requestBuilder, InvalidRecordException.class);
}
@ -176,19 +178,19 @@ public class ProduceRequestTest {
public void testV3AndAboveCannotUseMagicV0() {
ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, Compression.NONE,
TimestampType.NO_TIMESTAMP_TYPE, 0L);
TimestampType.NO_TIMESTAMP_TYPE, 0L);
builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(builder.build())))).iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(builder.build())))).iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
assertThrowsForAllVersions(requestBuilder, InvalidRecordException.class);
}
@ -196,19 +198,19 @@ public class ProduceRequestTest {
public void testV3AndAboveCannotUseMagicV1() {
ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, Compression.NONE,
TimestampType.CREATE_TIME, 0L);
TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000));
assertThrowsForAllVersions(requestBuilder, InvalidRecordException.class);
}
@ -220,15 +222,15 @@ public class ProduceRequestTest {
builder.append(10L, null, "a".getBytes());
ProduceRequestData produceData = new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(1000);
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("test")
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(0)
.setRecords(builder.build()))))
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(1000);
// Can't create ProduceRequest instance with version within [3, 7)
for (short version = 3; version < 7; version++) {
@ -247,20 +249,21 @@ public class ProduceRequestTest {
final int sequence = 10;
final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes()));
new SimpleRecord("foo".getBytes()));
final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId,
producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(0).setRecords(txnRecords))),
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonTxnRecords))))
.iterator()))
.setAcks((short) -1)
.setTimeoutMs(5000));
new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(0).setRecords(txnRecords))),
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonTxnRecords))))
.iterator()))
.setAcks((short) -1)
.setTimeoutMs(5000),
true);
final ProduceRequest request = builder.build();
assertTrue(RequestUtils.hasTransactionalRecords(request));
assertTrue(RequestTestUtils.hasIdempotentRecords(request));
@ -273,20 +276,21 @@ public class ProduceRequestTest {
final int sequence = 10;
final MemoryRecords nonIdempotentRecords = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes()));
new SimpleRecord("foo".getBytes()));
final MemoryRecords idempotentRecords = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId,
producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordVersion.current().value,
new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(0).setRecords(idempotentRecords))),
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonIdempotentRecords))))
.iterator()))
.setAcks((short) -1)
.setTimeoutMs(5000));
new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Arrays.asList(
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(0).setRecords(idempotentRecords))),
new ProduceRequestData.TopicProduceData().setName("foo").setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData().setIndex(1).setRecords(nonIdempotentRecords))))
.iterator()))
.setAcks((short) -1)
.setTimeoutMs(5000),
true);
final ProduceRequest request = builder.build();
assertFalse(RequestUtils.hasTransactionalRecords(request));
@ -301,14 +305,14 @@ public class ProduceRequestTest {
private ProduceRequest createNonIdempotentNonTransactionalRecords() {
return ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("topic")
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(1)
.setRecords(MemoryRecords.withRecords(Compression.NONE, simpleRecord)))))
.iterator()))
.setAcks((short) -1)
.setTimeoutMs(10)).build();
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName("topic")
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(1)
.setRecords(MemoryRecords.withRecords(Compression.NONE, simpleRecord)))))
.iterator()))
.setAcks((short) -1)
.setTimeoutMs(10)).build();
}
}

View File

@ -488,7 +488,8 @@ public class RequestResponseTest {
.iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000)
.setTransactionalId("transactionalId"))
.setTransactionalId("transactionalId"),
true)
.build((short) 3);
assertEquals(2, request.partitionSizes().size());
assertEquals(records0.sizeInBytes(), (int) request.partitionSizes().get(tp0));
@ -2583,7 +2584,8 @@ public class RequestResponseTest {
.setRecords(records)))).iterator()))
.setAcks((short) 1)
.setTimeoutMs(5000)
.setTransactionalId(version >= 3 ? "transactionalId" : null))
.setTransactionalId(version >= 3 ? "transactionalId" : null),
true)
.build(version);
}
@ -3108,7 +3110,18 @@ public class RequestResponseTest {
"groupId",
21L,
(short) 42,
offsets).build();
offsets,
false).build();
} else if (version < 5) {
return new TxnOffsetCommitRequest.Builder("transactionalId",
"groupId",
21L,
(short) 42,
offsets,
"member",
2,
Optional.of("instance"),
false).build(version);
} else {
return new TxnOffsetCommitRequest.Builder("transactionalId",
"groupId",
@ -3117,7 +3130,8 @@ public class RequestResponseTest {
offsets,
"member",
2,
Optional.of("instance")).build(version);
Optional.of("instance"),
true).build(version);
}
}
@ -3135,7 +3149,8 @@ public class RequestResponseTest {
offsets,
"member",
2,
Optional.of("instance")).build();
Optional.of("instance"),
false).build();
}
private TxnOffsetCommitResponse createTxnOffsetCommitResponse() {

View File

@ -70,7 +70,8 @@ public class TxnOffsetCommitRequestTest extends OffsetCommitRequestTest {
groupId,
producerId,
producerEpoch,
OFFSETS
OFFSETS,
true
);
int generationId = 5;
@ -82,14 +83,14 @@ public class TxnOffsetCommitRequestTest extends OffsetCommitRequestTest {
OFFSETS,
memberId,
generationId,
Optional.of(groupInstanceId)
Optional.of(groupInstanceId),
true
);
}
@Test
@Override
public void testConstructor() {
Map<TopicPartition, Errors> errorsMap = new HashMap<>();
errorsMap.put(new TopicPartition(topicOne, partitionOne), Errors.NOT_COORDINATOR);
errorsMap.put(new TopicPartition(topicTwo, partitionTwo), Errors.NOT_COORDINATOR);

View File

@ -46,4 +46,25 @@ public class TestUtil {
false
);
}
public static RequestContext requestContext(
ApiKeys apiKey,
Short version
) {
return new RequestContext(
new RequestHeader(
apiKey,
version,
"client",
0
),
"1",
InetAddress.getLoopbackAddress(),
KafkaPrincipal.ANONYMOUS,
ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
SecurityProtocol.PLAINTEXT,
ClientInformation.EMPTY,
false
);
}
}

View File

@ -17,7 +17,7 @@
package kafka.coordinator.group
import kafka.cluster.PartitionListener
import kafka.server.{ReplicaManager, defaultError, genericError}
import kafka.server.{AddPartitionsToTxnManager, ReplicaManager}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
@ -110,7 +110,7 @@ class CoordinatorPartitionWriter(
producerEpoch: Short,
apiVersion: Short
): CompletableFuture[VerificationGuard] = {
val transactionSupportedOperation = if (apiVersion >= 4) genericError else defaultError
val transactionSupportedOperation = AddPartitionsToTxnManager.txnOffsetCommitRequestVersionToTransactionSupportedOperation(apiVersion)
val future = new CompletableFuture[VerificationGuard]()
replicaManager.maybeStartTransactionVerificationForPartition(
topicPartition = tp,

View File

@ -931,7 +931,7 @@ private[group] class GroupCoordinator(
offsetTopicPartition, offsetMetadata, newRequestLocal, responseCallback, Some(verificationGuard))
}
}
val transactionSupportedOperation = if (apiVersion >= 4) genericError else defaultError
val transactionSupportedOperation = AddPartitionsToTxnManager.txnOffsetCommitRequestVersionToTransactionSupportedOperation(apiVersion)
groupManager.replicaManager.maybeStartTransactionVerificationForPartition(
topicPartition = offsetTopicPartition,
transactionalId,

View File

@ -16,8 +16,6 @@
*/
package kafka.coordinator.transaction
import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import kafka.server.{KafkaConfig, MetadataCache, ReplicaManager}
import kafka.utils.Logging
import org.apache.kafka.common.TopicPartition
@ -33,6 +31,8 @@ import org.apache.kafka.coordinator.transaction.ProducerIdManager
import org.apache.kafka.server.common.{RequestLocal, TransactionVersion}
import org.apache.kafka.server.util.Scheduler
import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import scala.jdk.CollectionConverters._
object TransactionCoordinator {
@ -476,7 +476,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
private def logInvalidStateTransitionAndReturnError(transactionalId: String,
transactionState: TransactionState,
transactionResult: TransactionResult) = {
debug(s"TransactionalId: $transactionalId's state is $transactionState, but received transaction " +
warn(s"TransactionalId: $transactionalId's state is $transactionState, but received transaction " +
s"marker result to send: $transactionResult")
Left(Errors.INVALID_TXN_STATE)
}
@ -498,20 +498,28 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
requestLocal)
}
private def endTransaction(transactionalId: String,
/**
* Handling the endTxn request under the Transaction Version 1.
*
* @param transactionalId The transaction ID from the endTxn request
* @param producerId The producer ID from the endTxn request
* @param producerEpoch The producer epoch from the endTxn request
* @param txnMarkerResult To commit or abort the transaction
* @param isFromClient Is the request from client
* @param responseCallback The response callback
* @param requestLocal The request local object
*/
private def endTransactionWithTV1(transactionalId: String,
producerId: Long,
producerEpoch: Short,
txnMarkerResult: TransactionResult,
isFromClient: Boolean,
clientTransactionVersion: TransactionVersion,
responseCallback: EndTxnCallback,
requestLocal: RequestLocal): Unit = {
var isEpochFence = false
if (transactionalId == null || transactionalId.isEmpty)
responseCallback(Errors.INVALID_REQUEST, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
else {
var producerIdCopy = RecordBatch.NO_PRODUCER_ID
var producerEpochCopy = RecordBatch.NO_PRODUCER_EPOCH
val preAppendResult: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap {
case None =>
Left(Errors.INVALID_PRODUCER_ID_MAPPING)
@ -521,39 +529,10 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
val coordinatorEpoch = epochAndTxnMetadata.coordinatorEpoch
txnMetadata.inLock {
producerIdCopy = txnMetadata.producerId
producerEpochCopy = txnMetadata.producerEpoch
// PrepareEpochFence has slightly different epoch bumping logic so don't include it here.
val currentTxnMetadataIsAtLeastTransactionsV2 = !txnMetadata.pendingState.contains(PrepareEpochFence) && txnMetadata.clientTransactionVersion.supportsEpochBump()
// True if the client used TV_2 and retried a request that had overflowed the epoch, and a new producer ID is stored in the txnMetadata
val retryOnOverflow = currentTxnMetadataIsAtLeastTransactionsV2 &&
txnMetadata.previousProducerId == producerId && producerEpoch == Short.MaxValue - 1 && txnMetadata.producerEpoch == 0
// True if the client used TV_2 and retried an endTxn request, and the bumped producer epoch is stored in the txnMetadata.
val retryOnEpochBump = endTxnEpochBumped(txnMetadata, producerEpoch)
val isValidEpoch = {
if (currentTxnMetadataIsAtLeastTransactionsV2) {
// With transactions V2, state + same epoch is not sufficient to determine if a retry transition is valid. If the epoch is the
// same it actually indicates the next endTransaction call. Instead, we want to check the epoch matches with the epoch in the retry conditions.
// Return producer fenced even in the cases where the epoch is higher and could indicate an invalid state transition.
// Use the following criteria to determine if a v2 retry is valid:
txnMetadata.state match {
case Ongoing | Empty | Dead | PrepareEpochFence =>
producerEpoch == txnMetadata.producerEpoch
case PrepareCommit | PrepareAbort =>
retryOnEpochBump
case CompleteCommit | CompleteAbort =>
retryOnEpochBump || retryOnOverflow
}
} else {
// For transactions V1 strict equality is enforced on the client side requests, as they shouldn't bump the producer epoch without server knowledge.
(!isFromClient || producerEpoch == txnMetadata.producerEpoch) && producerEpoch >= txnMetadata.producerEpoch
}
}
if (txnMetadata.producerId != producerId && !retryOnOverflow)
if (txnMetadata.producerId != producerId)
Left(Errors.INVALID_PRODUCER_ID_MAPPING)
else if (!isValidEpoch)
// Strict equality is enforced on the client side requests, as they shouldn't bump the producer epoch.
else if ((isFromClient && producerEpoch != txnMetadata.producerEpoch) || producerEpoch < txnMetadata.producerEpoch)
Left(Errors.PRODUCER_FENCED)
else if (txnMetadata.pendingTransitionInProgress && txnMetadata.pendingState.get != PrepareEpochFence)
Left(Errors.CONCURRENT_TRANSACTIONS)
@ -564,18 +543,6 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
else
PrepareAbort
// Maybe allocate new producer ID if we are bumping epoch and epoch is exhausted
val nextProducerIdOrErrors =
if (clientTransactionVersion.supportsEpochBump() && !txnMetadata.pendingState.contains(PrepareEpochFence) && txnMetadata.isProducerEpochExhausted) {
try {
Right(producerIdManager.generateProducerId())
} catch {
case e: Exception => Left(Errors.forException(e))
}
} else {
Right(RecordBatch.NO_PRODUCER_ID)
}
if (nextState == PrepareAbort && txnMetadata.pendingState.contains(PrepareEpochFence)) {
// We should clear the pending state to make way for the transition to PrepareAbort and also bump
// the epoch in the transaction metadata we are about to append.
@ -585,10 +552,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
txnMetadata.lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH
}
nextProducerIdOrErrors.flatMap {
nextProducerId =>
Right(coordinatorEpoch, txnMetadata.prepareAbortOrCommit(nextState, clientTransactionVersion, nextProducerId.asInstanceOf[Long], time.milliseconds()))
}
Right(coordinatorEpoch, txnMetadata.prepareAbortOrCommit(nextState, TransactionVersion.fromFeatureLevel(0), RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false))
case CompleteCommit =>
if (txnMarkerResult == TransactionResult.COMMIT)
Left(Errors.NONE)
@ -616,6 +580,295 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
s"This is illegal as we should never have transitioned to this state."
fatal(errorMsg)
throw new IllegalStateException(errorMsg)
}
}
}
preAppendResult match {
case Left(err) =>
debug(s"Aborting append of $txnMarkerResult to transaction log with coordinator and returning $err error to client for $transactionalId's EndTransaction request")
responseCallback(err, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
case Right((coordinatorEpoch, newMetadata)) =>
def sendTxnMarkersCallback(error: Errors): Unit = {
if (error == Errors.NONE) {
val preSendResult: ApiResult[(TransactionMetadata, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap {
case None =>
val errorMsg = s"The coordinator still owns the transaction partition for $transactionalId, but there is " +
s"no metadata in the cache; this is not expected"
fatal(errorMsg)
throw new IllegalStateException(errorMsg)
case Some(epochAndMetadata) =>
if (epochAndMetadata.coordinatorEpoch == coordinatorEpoch) {
val txnMetadata = epochAndMetadata.transactionMetadata
txnMetadata.inLock {
if (txnMetadata.producerId != producerId)
Left(Errors.INVALID_PRODUCER_ID_MAPPING)
else if (txnMetadata.producerEpoch != producerEpoch)
Left(Errors.PRODUCER_FENCED)
else if (txnMetadata.pendingTransitionInProgress)
Left(Errors.CONCURRENT_TRANSACTIONS)
else txnMetadata.state match {
case Empty| Ongoing | CompleteCommit | CompleteAbort =>
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
case PrepareCommit =>
if (txnMarkerResult != TransactionResult.COMMIT)
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
else
Right(txnMetadata, txnMetadata.prepareComplete(time.milliseconds()))
case PrepareAbort =>
if (txnMarkerResult != TransactionResult.ABORT)
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
else
Right(txnMetadata, txnMetadata.prepareComplete(time.milliseconds()))
case Dead | PrepareEpochFence =>
val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " +
s"This is illegal as we should never have transitioned to this state."
fatal(errorMsg)
throw new IllegalStateException(errorMsg)
}
}
} else {
debug(s"The transaction coordinator epoch has changed to ${epochAndMetadata.coordinatorEpoch} after $txnMarkerResult was " +
s"successfully appended to the log for $transactionalId with old epoch $coordinatorEpoch")
Left(Errors.NOT_COORDINATOR)
}
}
preSendResult match {
case Left(err) =>
info(s"Aborting sending of transaction markers after appended $txnMarkerResult to transaction log and returning $err error to client for $transactionalId's EndTransaction request")
responseCallback(err, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
case Right((txnMetadata, newPreSendMetadata)) =>
// we can respond to the client immediately and continue to write the txn markers if
// the log append was successful
responseCallback(Errors.NONE, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
txnMarkerChannelManager.addTxnMarkersToSend(coordinatorEpoch, txnMarkerResult, txnMetadata, newPreSendMetadata)
}
} else {
info(s"Aborting sending of transaction markers and returning $error error to client for $transactionalId's EndTransaction request of $txnMarkerResult, " +
s"since appending $newMetadata to transaction log with coordinator epoch $coordinatorEpoch failed")
if (isEpochFence) {
txnManager.getTransactionState(transactionalId).foreach {
case None =>
warn(s"The coordinator still owns the transaction partition for $transactionalId, but there is " +
s"no metadata in the cache; this is not expected")
case Some(epochAndMetadata) =>
if (epochAndMetadata.coordinatorEpoch == coordinatorEpoch) {
// This was attempted epoch fence that failed, so mark this state on the metadata
epochAndMetadata.transactionMetadata.hasFailedEpochFence = true
warn(s"The coordinator failed to write an epoch fence transition for producer $transactionalId to the transaction log " +
s"with error $error. The epoch was increased to ${newMetadata.producerEpoch} but not returned to the client")
}
}
}
responseCallback(error, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
}
}
txnManager.appendTransactionToLog(transactionalId, coordinatorEpoch, newMetadata,
sendTxnMarkersCallback, requestLocal = requestLocal)
}
}
}
/*
Here is the table to demonstrate the state transition for Empty, CompleteAbort, CompleteCommit in Transaction V2.
Note:
PF = PRODUCER_FENCED
ITS = INVALID_TXN_STATE
NONE = No error and no epoch bump
EB = No error and epoch bump
Retry => producerEpoch = txnState.ProducerEpoch - 1
Current => producerEpoch = txnState.ProducerEpoch
------------------------------------------------------
With transaction V1, Retry is not allowed, PRODUCER_FENCED will be returned if the epoch does not match.
Empty does not accept Abort and Commit.
CompleteAbort only accepts Abort.
CompleteCommit only accepts Commit.
For all invalid cases, INVALID_TXN_STATE is returned.
------------------------------------------------------
With transaction V2.
+----------------+-----------------+-----------------+
| | Abort | Commit |
+----------------+-------+---------+-------+---------+
| | Retry | Current | Retry | Current |
+----------------+-------+---------+-------+---------+
| Empty | PF | EB | PF | ITS |
+----------------+-------+---------+-------+---------+
| CompleteAbort | NONE | EB | ITS | ITS |
+----------------+-------+---------+-------+---------+
| CompleteCommit | ITS | EB | NONE | ITS |
+----------------+-------+---------+-------+---------+
*/
/**
* Handling the endTxn request above the Transaction Version 2.
*
* @param transactionalId The transaction ID from the endTxn request
* @param producerId The producer ID from the endTxn request
* @param producerEpoch The producer epoch from the endTxn request
* @param txnMarkerResult To commit or abort the transaction
* @param isFromClient Is the request from client
* @param clientTransactionVersion The transaction version for the endTxn request
* @param responseCallback The response callback
* @param requestLocal The request local object
*/
private def endTransaction(transactionalId: String,
producerId: Long,
producerEpoch: Short,
txnMarkerResult: TransactionResult,
isFromClient: Boolean,
clientTransactionVersion: TransactionVersion,
responseCallback: EndTxnCallback,
requestLocal: RequestLocal): Unit = {
if (!clientTransactionVersion.supportsEpochBump()) {
endTransactionWithTV1(transactionalId, producerId, producerEpoch, txnMarkerResult, isFromClient, responseCallback, requestLocal)
return
}
var isEpochFence = false
if (transactionalId == null || transactionalId.isEmpty)
responseCallback(Errors.INVALID_REQUEST, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
else {
var producerIdCopy = RecordBatch.NO_PRODUCER_ID
var producerEpochCopy = RecordBatch.NO_PRODUCER_EPOCH
val preAppendResult: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap {
case None =>
Left(Errors.INVALID_PRODUCER_ID_MAPPING)
case Some(epochAndTxnMetadata) =>
val txnMetadata = epochAndTxnMetadata.transactionMetadata
val coordinatorEpoch = epochAndTxnMetadata.coordinatorEpoch
txnMetadata.inLock {
producerIdCopy = txnMetadata.producerId
producerEpochCopy = txnMetadata.producerEpoch
// PrepareEpochFence has slightly different epoch bumping logic so don't include it here.
// Note that, it can only happen when the current state is Ongoing.
isEpochFence = txnMetadata.pendingState.contains(PrepareEpochFence)
// True if the client retried a request that had overflowed the epoch, and a new producer ID is stored in the txnMetadata
val retryOnOverflow = !isEpochFence && txnMetadata.previousProducerId == producerId &&
producerEpoch == Short.MaxValue - 1 && txnMetadata.producerEpoch == 0
// True if the client retried an endTxn request, and the bumped producer epoch is stored in the txnMetadata.
val retryOnEpochBump = !isEpochFence && txnMetadata.producerEpoch == producerEpoch + 1
val isValidEpoch = {
if (!isEpochFence) {
// With transactions V2, state + same epoch is not sufficient to determine if a retry transition is valid. If the epoch is the
// same it actually indicates the next endTransaction call. Instead, we want to check the epoch matches with the epoch in the retry conditions.
// Return producer fenced even in the cases where the epoch is higher and could indicate an invalid state transition.
// Use the following criteria to determine if a v2 retry is valid:
txnMetadata.state match {
case Ongoing | Empty | Dead | PrepareEpochFence =>
producerEpoch == txnMetadata.producerEpoch
case PrepareCommit | PrepareAbort =>
retryOnEpochBump
case CompleteCommit | CompleteAbort =>
retryOnEpochBump || retryOnOverflow || producerEpoch == txnMetadata.producerEpoch
}
} else {
// If the epoch is going to be fenced, it bumps the epoch differently with TV2.
(!isFromClient || producerEpoch == txnMetadata.producerEpoch) && producerEpoch >= txnMetadata.producerEpoch
}
}
val isRetry = retryOnEpochBump || retryOnOverflow
def generateTxnTransitMetadataForTxnCompletion(nextState: TransactionState, noPartitionAdded: Boolean): ApiResult[(Int, TxnTransitMetadata)] = {
// Maybe allocate new producer ID if we are bumping epoch and epoch is exhausted
val nextProducerIdOrErrors =
if (!isEpochFence && txnMetadata.isProducerEpochExhausted) {
try {
Right(producerIdManager.generateProducerId())
} catch {
case e: Exception => Left(Errors.forException(e))
}
} else {
Right(RecordBatch.NO_PRODUCER_ID)
}
if (nextState == PrepareAbort && isEpochFence) {
// We should clear the pending state to make way for the transition to PrepareAbort and also bump
// the epoch in the transaction metadata we are about to append.
txnMetadata.pendingState = None
txnMetadata.producerEpoch = producerEpoch
txnMetadata.lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH
}
nextProducerIdOrErrors.flatMap {
nextProducerId =>
Right(coordinatorEpoch, txnMetadata.prepareAbortOrCommit(nextState, clientTransactionVersion, nextProducerId.asInstanceOf[Long], time.milliseconds(), noPartitionAdded))
}
}
if (txnMetadata.producerId != producerId && !retryOnOverflow)
Left(Errors.INVALID_PRODUCER_ID_MAPPING)
else if (!isValidEpoch)
Left(Errors.PRODUCER_FENCED)
else if (txnMetadata.pendingTransitionInProgress && txnMetadata.pendingState.get != PrepareEpochFence)
Left(Errors.CONCURRENT_TRANSACTIONS)
else txnMetadata.state match {
case Ongoing =>
val nextState = if (txnMarkerResult == TransactionResult.COMMIT)
PrepareCommit
else
PrepareAbort
generateTxnTransitMetadataForTxnCompletion(nextState, false)
case CompleteCommit =>
if (txnMarkerResult == TransactionResult.COMMIT) {
if (isRetry)
Left(Errors.NONE)
else
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
} else {
// Abort.
if (isRetry)
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
else
generateTxnTransitMetadataForTxnCompletion(PrepareAbort, true)
}
case CompleteAbort =>
if (txnMarkerResult == TransactionResult.ABORT) {
if (isRetry)
Left(Errors.NONE)
else
generateTxnTransitMetadataForTxnCompletion(PrepareAbort, true)
} else {
// Commit.
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
}
case PrepareCommit =>
if (txnMarkerResult == TransactionResult.COMMIT)
Left(Errors.CONCURRENT_TRANSACTIONS)
else
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
case PrepareAbort =>
if (txnMarkerResult == TransactionResult.ABORT)
Left(Errors.CONCURRENT_TRANSACTIONS)
else
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
case Empty =>
if (txnMarkerResult == TransactionResult.ABORT) {
generateTxnTransitMetadataForTxnCompletion(PrepareAbort, true)
} else {
logInvalidStateTransitionAndReturnError(transactionalId, txnMetadata.state, txnMarkerResult)
}
case Dead | PrepareEpochFence =>
val errorMsg = s"Found transactionalId $transactionalId with state ${txnMetadata.state}. " +
s"This is illegal as we should never have transitioned to this state."
fatal(errorMsg)
throw new IllegalStateException(errorMsg)
}
}
@ -626,8 +879,8 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
if (err == Errors.NONE) {
responseCallback(err, producerIdCopy, producerEpochCopy)
} else {
debug(s"Aborting append of $txnMarkerResult to transaction log with coordinator and returning $err error to client for $transactionalId's EndTransaction request")
responseCallback(err, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
debug(s"Aborting append of $txnMarkerResult to transaction log with coordinator and returning $err error to client for $transactionalId's EndTransaction request")
responseCallback(err, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH)
}
case Right((coordinatorEpoch, newMetadata)) =>
@ -646,7 +899,7 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
txnMetadata.inLock {
if (txnMetadata.producerId != producerId)
Left(Errors.INVALID_PRODUCER_ID_MAPPING)
else if (txnMetadata.producerEpoch != producerEpoch && !endTxnEpochBumped(txnMetadata, producerEpoch))
else if (txnMetadata.producerEpoch != producerEpoch && txnMetadata.producerEpoch != producerEpoch + 1)
Left(Errors.PRODUCER_FENCED)
else if (txnMetadata.pendingTransitionInProgress)
Left(Errors.CONCURRENT_TRANSACTIONS)
@ -720,14 +973,6 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
}
}
// When a client and server support V2, every endTransaction call bumps the producer epoch. When checking epoch, we want to
// check epoch + 1. Epoch bumps from PrepareEpochFence state are handled separately, so this method should not be used to check that case.
// Returns true if the transaction state epoch is the specified producer epoch + 1 and epoch bump on every transaction is expected.
private def endTxnEpochBumped(txnMetadata: TransactionMetadata, producerEpoch: Short): Boolean = {
!txnMetadata.pendingState.contains(PrepareEpochFence) && txnMetadata.clientTransactionVersion.supportsEpochBump() &&
txnMetadata.producerEpoch == producerEpoch + 1
}
def transactionTopicConfigs: Properties = txnManager.transactionTopicConfigs
def partitionFor(transactionalId: String): Int = txnManager.partitionFor(transactionalId)

View File

@ -75,6 +75,7 @@ private[transaction] sealed trait TransactionState {
*
* transition: received AddPartitionsToTxnRequest => Ongoing
* received AddOffsetsToTxnRequest => Ongoing
* received EndTxnRequest with abort and TransactionV2 enabled => PrepareAbort
*/
private[transaction] case object Empty extends TransactionState {
val id: Byte = 0
@ -112,11 +113,14 @@ private[transaction] case object PrepareCommit extends TransactionState {
* Group is preparing to abort
*
* transition: received acks from all partitions => CompleteAbort
*
* Note, In transaction v2, we allow Empty, CompleteCommit, CompleteAbort to transition to PrepareAbort. because the
* client may not know the txn state on the server side, it needs to send endTxn request when uncertain.
*/
private[transaction] case object PrepareAbort extends TransactionState {
val id: Byte = 3
val name: String = "PrepareAbort"
val validPreviousStates: Set[TransactionState] = Set(Ongoing, PrepareEpochFence)
val validPreviousStates: Set[TransactionState] = Set(Ongoing, PrepareEpochFence, Empty, CompleteCommit, CompleteAbort)
}
/**
@ -329,7 +333,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String,
(topicPartitions ++ addedTopicPartitions).toSet, newTxnStartTimestamp, updateTimestamp)
}
def prepareAbortOrCommit(newState: TransactionState, clientTransactionVersion: TransactionVersion, nextProducerId: Long, updateTimestamp: Long): TxnTransitMetadata = {
def prepareAbortOrCommit(newState: TransactionState, clientTransactionVersion: TransactionVersion, nextProducerId: Long, updateTimestamp: Long, noPartitionAdded: Boolean): TxnTransitMetadata = {
val (updatedProducerEpoch, updatedLastProducerEpoch) = if (clientTransactionVersion.supportsEpochBump()) {
// We already ensured that we do not overflow here. MAX_SHORT is the highest possible value.
((producerEpoch + 1).toShort, producerEpoch)
@ -337,8 +341,11 @@ private[transaction] class TransactionMetadata(val transactionalId: String,
(producerEpoch, lastProducerEpoch)
}
// With transaction V2, it is allowed to abort the transaction without adding any partitions. Then, the transaction
// start time is uncertain but it is still required. So we can use the update time as the transaction start time.
val newTxnStartTimestamp = if (noPartitionAdded) updateTimestamp else txnStartTimestamp
prepareTransitionTo(newState, producerId, nextProducerId, updatedProducerEpoch, updatedLastProducerEpoch, txnTimeoutMs, topicPartitions.toSet,
txnStartTimestamp, updateTimestamp, clientTransactionVersion)
newTxnStartTimestamp, updateTimestamp, clientTransactionVersion)
}
def prepareComplete(updateTimestamp: Long): TxnTransitMetadata = {
@ -477,23 +484,27 @@ private[transaction] class TransactionMetadata(val transactionalId: String,
}
case PrepareAbort | PrepareCommit => // from endTxn
// In V2, we allow state transits from Empty, CompleteCommit and CompleteAbort to PrepareAbort. It is possible
// their updated start time is not equal to the current start time.
val allowedEmptyAbort = toState == PrepareAbort && transitMetadata.clientTransactionVersion.supportsEpochBump() &&
(state == Empty || state == CompleteCommit || state == CompleteAbort)
val validTimestamp = txnStartTimestamp == transitMetadata.txnStartTimestamp || allowedEmptyAbort
if (!validProducerEpoch(transitMetadata) ||
!topicPartitions.toSet.equals(transitMetadata.topicPartitions) ||
txnTimeoutMs != transitMetadata.txnTimeoutMs ||
txnStartTimestamp != transitMetadata.txnStartTimestamp) {
txnTimeoutMs != transitMetadata.txnTimeoutMs || !validTimestamp) {
throwStateTransitionFailure(transitMetadata)
} else if (transitMetadata.clientTransactionVersion.supportsEpochBump()) {
producerEpoch = transitMetadata.producerEpoch
lastProducerEpoch = transitMetadata.lastProducerEpoch
nextProducerId = transitMetadata.nextProducerId
txnStartTimestamp = transitMetadata.txnStartTimestamp
}
case CompleteAbort | CompleteCommit => // from write markers
if (!validProducerEpoch(transitMetadata) ||
txnTimeoutMs != transitMetadata.txnTimeoutMs ||
transitMetadata.txnStartTimestamp == -1) {
throwStateTransitionFailure(transitMetadata)
} else {
txnStartTimestamp = transitMetadata.txnStartTimestamp
@ -593,9 +604,10 @@ private[transaction] class TransactionMetadata(val transactionalId: String,
"TransactionMetadata(" +
s"transactionalId=$transactionalId, " +
s"producerId=$producerId, " +
s"previousProducerId=$previousProducerId, "
s"nextProducerId=$nextProducerId, "
s"previousProducerId=$previousProducerId, " +
s"nextProducerId=$nextProducerId, " +
s"producerEpoch=$producerEpoch, " +
s"lastProducerEpoch=$lastProducerEpoch, " +
s"txnTimeoutMs=$txnTimeoutMs, " +
s"state=$state, " +
s"pendingState=$pendingState, " +

View File

@ -603,7 +603,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
* Creation starts the verification process. Otherwise return the sentinel VerificationGuard.
*/
def maybeStartTransactionVerification(producerId: Long, sequence: Int, epoch: Short): VerificationGuard = lock synchronized {
if (hasOngoingTransaction(producerId))
if (hasOngoingTransaction(producerId, epoch))
VerificationGuard.SENTINEL
else
maybeCreateVerificationGuard(producerId, sequence, epoch)
@ -629,10 +629,11 @@ class UnifiedLog(@volatile var logStartOffset: Long,
/**
* Return true if the given producer ID has a transaction ongoing.
* Note, if the incoming producer epoch is newer than the stored one, the transaction may have finished.
*/
def hasOngoingTransaction(producerId: Long): Boolean = lock synchronized {
def hasOngoingTransaction(producerId: Long, producerEpoch: Short): Boolean = lock synchronized {
val entry = producerStateManager.activeProducers.get(producerId)
entry != null && entry.currentTxnFirstOffset.isPresent
entry != null && entry.currentTxnFirstOffset.isPresent && entry.producerEpoch() == producerEpoch
}
/**
@ -1061,7 +1062,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
// transaction is completed or aborted. We can guarantee the transaction coordinator knows about the transaction given step 1 and that the transaction is still
// ongoing. If the transaction is expected to be ongoing, we will not set a VerificationGuard. If the transaction is aborted, hasOngoingTransaction is false and
// requestVerificationGuard is the sentinel, so we will throw an error. A subsequent produce request (retry) should create verification state and return to phase 1.
if (batch.isTransactional && !hasOngoingTransaction(batch.producerId) && batchMissingRequiredVerification(batch, requestVerificationGuard))
if (batch.isTransactional && !hasOngoingTransaction(batch.producerId, batch.producerEpoch()) && batchMissingRequiredVerification(batch, requestVerificationGuard))
throw new InvalidTxnStateException("Record was not part of an ongoing transaction")
}

View File

@ -39,17 +39,37 @@ object AddPartitionsToTxnManager {
val VerificationFailureRateMetricName = "VerificationFailureRate"
val VerificationTimeMsMetricName = "VerificationTimeMs"
def produceRequestVersionToTransactionSupportedOperation(version: Short): TransactionSupportedOperation = {
if (version > 11) {
addPartition
} else if (version > 10) {
genericErrorSupported
} else {
defaultError
}
}
def txnOffsetCommitRequestVersionToTransactionSupportedOperation(version: Short): TransactionSupportedOperation = {
if (version > 4) {
addPartition
} else if (version > 3) {
genericErrorSupported
} else {
defaultError
}
}
}
/**
* This is an enum which handles the Partition Response based on the Request Version and the exact operation
* defaultError: This is the default workflow which maps to cases when the Produce Request Version or the Txn_offset_commit request was lower than the first version supporting the new Error Class
* genericError: This maps to the case when the clients are updated to handle the TransactionAbortableException
* addPartition: This is a WIP. To be updated as a part of KIP-890 Part 2
* defaultError: This is the default workflow which maps to cases when the Produce Request Version or the Txn_offset_commit request was lower than the first version supporting the new Error Class
* genericErrorSupported: This maps to the case when the clients are updated to handle the TransactionAbortableException
* addPartition: This allows the partition to be added to the transactions inflight with the Produce and TxnOffsetCommit requests. Plus the behaviors in genericErrorSupported.
*/
sealed trait TransactionSupportedOperation
case object defaultError extends TransactionSupportedOperation
case object genericError extends TransactionSupportedOperation
case object genericErrorSupported extends TransactionSupportedOperation
case object addPartition extends TransactionSupportedOperation
/*
@ -85,7 +105,7 @@ class AddPartitionsToTxnManager(
private val verificationFailureRate = metricsGroup.newMeter(VerificationFailureRateMetricName, "failures", TimeUnit.SECONDS)
private val verificationTimeMs = metricsGroup.newHistogram(VerificationTimeMsMetricName)
def verifyTransaction(
def addOrVerifyTransaction(
transactionalId: String,
producerId: Long,
producerEpoch: Short,
@ -108,7 +128,7 @@ class AddPartitionsToTxnManager(
.setTransactionalId(transactionalId)
.setProducerId(producerId)
.setProducerEpoch(producerEpoch)
.setVerifyOnly(true)
.setVerifyOnly(transactionSupportedOperation != addPartition)
.setTopics(topicCollection)
addTxnData(coordinatorNode.get, transactionData, callback, transactionSupportedOperation)
@ -225,7 +245,8 @@ class AddPartitionsToTxnManager(
val code =
if (partitionResult.partitionErrorCode == Errors.PRODUCER_FENCED.code)
Errors.INVALID_PRODUCER_EPOCH.code
else if (partitionResult.partitionErrorCode() == Errors.TRANSACTION_ABORTABLE.code && transactionDataAndCallbacks.transactionSupportedOperation != genericError) // For backward compatibility with clients.
else if (partitionResult.partitionErrorCode() == Errors.TRANSACTION_ABORTABLE.code
&& transactionDataAndCallbacks.transactionSupportedOperation == defaultError) // For backward compatibility with clients.
Errors.INVALID_TXN_STATE.code
else
partitionResult.partitionErrorCode

View File

@ -741,7 +741,7 @@ class KafkaApis(val requestChannel: RequestChannel,
sendResponseCallback(Map.empty)
else {
val internalTopicsAllowed = request.header.clientId == AdminUtils.ADMIN_CLIENT_ID
val transactionSupportedOperation = if (request.header.apiVersion > 10) genericError else defaultError
val transactionSupportedOperation = AddPartitionsToTxnManager.produceRequestVersionToTransactionSupportedOperation(request.header.apiVersion())
// call the replica manager to append messages to the replicas
replicaManager.handleProduceAppend(
timeout = produceRequest.timeout.toLong,

View File

@ -1141,7 +1141,7 @@ class ReplicaManager(val config: KafkaConfig,
callback((errors ++ verificationErrors, verificationGuards.toMap))
}
addPartitionsToTxnManager.foreach(_.verifyTransaction(
addPartitionsToTxnManager.foreach(_.addOrVerifyTransaction(
transactionalId = transactionalId,
producerId = producerId,
producerEpoch = producerEpoch,

View File

@ -95,11 +95,16 @@ public class AdminFenceProducersTest {
producer.beginTransaction();
ExecutionException exceptionDuringSend = assertThrows(
ExecutionException.class,
() -> producer.send(RECORD).get(), "expected ProducerFencedException"
() -> producer.send(RECORD).get(), "expected InvalidProducerEpochException"
);
assertInstanceOf(ProducerFencedException.class, exceptionDuringSend.getCause());
assertThrows(ProducerFencedException.class, producer::commitTransaction);
// In Transaction V2, the ProducerFencedException will be converted to InvalidProducerEpochException when
// coordinator handles AddPartitionRequest.
assertInstanceOf(InvalidProducerEpochException.class, exceptionDuringSend.getCause());
// InvalidProducerEpochException is treated as fatal error. The commitTransaction will return this last
// fatal error.
assertThrows(InvalidProducerEpochException.class, producer::commitTransaction);
}
}

View File

@ -2161,7 +2161,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
producer.initTransactions()
producer.beginTransaction()
removeAllClientAcls()
assertThrows(classOf[TransactionalIdAuthorizationException], () => {
// In transaction V2, the server receives the offset commit request first, so the error is GroupAuthorizationException
// instead of TransactionalIdAuthorizationException.
assertThrows(classOf[GroupAuthorizationException], () => {
val offsets = Map(tp -> new OffsetAndMetadata(1L)).asJava
producer.sendOffsetsToTransaction(offsets, new ConsumerGroupMetadata(group))
producer.commitTransaction()

View File

@ -17,9 +17,9 @@
package kafka.api
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.{consumeRecords, waitUntilTrue}
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerGroupMetadata, ConsumerRecord, OffsetAndMetadata}
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFencedException, TimeoutException}
@ -30,8 +30,7 @@ import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, Server
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import org.junit.jupiter.params.provider.CsvSource
import org.junit.jupiter.params.provider.{CsvSource, MethodSource}
import java.lang.{Long => JLong}
import java.nio.charset.StandardCharsets
@ -39,8 +38,8 @@ import java.time.Duration
import java.util
import java.util.concurrent.TimeUnit
import java.util.{Optional, Properties}
import scala.collection.{Seq, mutable}
import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import scala.collection.{Seq, mutable}
import scala.concurrent.ExecutionException
import scala.jdk.CollectionConverters._
@ -394,6 +393,7 @@ class TransactionsTest extends IntegrationTestHarness {
producer1.beginTransaction()
producer1.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "1", "1", willBeCommitted = false))
producer1.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic2, null, "3", "3", willBeCommitted = false))
producer1.flush()
producer2.initTransactions() // ok, will abort the open transaction.
producer2.beginTransaction()
@ -431,7 +431,7 @@ class TransactionsTest extends IntegrationTestHarness {
producer2.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "4", willBeCommitted = true))
producer2.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic2, null, "2", "4", willBeCommitted = true))
assertThrows(classOf[ProducerFencedException], () => producer1.sendOffsetsToTransaction(Map(new TopicPartition("foobartopic", 0)
assertThrows(classOf[ProducerFencedException], () => producer1.sendOffsetsToTransaction(Map(new TopicPartition(topic1, 0)
-> new OffsetAndMetadata(110L)).asJava, new ConsumerGroupMetadata("foobarGroup")))
producer2.commitTransaction() // ok
@ -561,6 +561,8 @@ class TransactionsTest extends IntegrationTestHarness {
val consumer = transactionalConsumers(0)
consumer.subscribe(List(topic1, topic2).asJava)
TestUtils.waitUntilLeaderIsKnown(brokers, new TopicPartition(topic1, 0))
TestUtils.waitUntilLeaderIsKnown(brokers, new TopicPartition(topic2, 0))
producer1.initTransactions()
producer1.beginTransaction()
@ -579,15 +581,21 @@ class TransactionsTest extends IntegrationTestHarness {
producer1.beginTransaction()
val result = producer1.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "1", "5", willBeCommitted = false))
val recordMetadata = result.get()
error(s"Missed a producer fenced exception when writing to ${recordMetadata.topic}-${recordMetadata.partition}. Grab the logs!!")
error(s"Missed an exception when writing to ${recordMetadata.topic}-${recordMetadata.partition}. Grab the logs!!")
brokers.foreach { broker =>
error(s"log dirs: ${broker.logManager.liveLogDirs.map(_.getAbsolutePath).head}")
}
fail("Should not be able to send messages from a fenced producer.")
} catch {
case _: ProducerFencedException =>
case e: ExecutionException =>
assertTrue(e.getCause.isInstanceOf[ProducerFencedException])
case _: InvalidProducerEpochException =>
case e: ExecutionException => {
if (quorum == "zk") {
assertTrue(e.getCause.isInstanceOf[ProducerFencedException])
} else {
// In kraft mode, transactionV2 is used.
assertTrue(e.getCause.isInstanceOf[InvalidProducerEpochException])
}
}
case e: Exception =>
throw new AssertionError("Got an unexpected exception from a fenced producer.", e)
}
@ -615,14 +623,27 @@ class TransactionsTest extends IntegrationTestHarness {
// Wait for the expiration cycle to kick in.
Thread.sleep(600)
try {
// Now that the transaction has expired, the second send should fail with a ProducerFencedException.
producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "2", willBeCommitted = false)).get()
fail("should have raised a ProducerFencedException since the transaction has expired")
} catch {
case _: ProducerFencedException =>
case e: ExecutionException =>
assertTrue(e.getCause.isInstanceOf[ProducerFencedException])
if (quorum == "zk") {
// In zk mode, transaction v1 is used.
try {
// Now that the transaction has expired, the second send should fail with a ProducerFencedException.
producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "2", willBeCommitted = false)).get()
fail("should have raised a ProducerFencedException since the transaction has expired")
} catch {
case _: ProducerFencedException =>
case e: ExecutionException =>
assertTrue(e.getCause.isInstanceOf[ProducerFencedException])
}
} else {
try {
// Now that the transaction has expired, the second send should fail with a InvalidProducerEpochException.
producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "2", willBeCommitted = false)).get()
fail("should have raised a InvalidProducerEpochException since the transaction has expired")
} catch {
case _: InvalidProducerEpochException =>
case e: ExecutionException =>
assertTrue(e.getCause.isInstanceOf[InvalidProducerEpochException])
}
}
// Verify that the first message was aborted and the second one was never written at all.
@ -789,6 +810,8 @@ class TransactionsTest extends IntegrationTestHarness {
val producerId = producerStateEntry.producerId
var previousProducerEpoch = producerStateEntry.producerEpoch
Thread.sleep(3000) // Wait for the markers to be persisted and the transaction state to be updated.
// Second transaction: abort
producer.beginTransaction()
producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "2", willBeCommitted = false))
@ -899,7 +922,8 @@ class TransactionsTest extends IntegrationTestHarness {
producer1.close()
}
val producer3 = createTransactionalProducer("transactional-producer", maxBlockMs = 5000)
// Make sure to leave this producer enough time before request timeout. The broker restart can take some time.
val producer3 = createTransactionalProducer("transactional-producer")
producer3.initTransactions()
producer3.beginTransaction()

View File

@ -30,9 +30,9 @@ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.mockito.{ArgumentCaptor, ArgumentMatchers}
import org.mockito.ArgumentMatchers.{any, anyInt}
import org.mockito.Mockito.{mock, times, verify, when}
import org.mockito.Mockito._
import org.mockito.{ArgumentCaptor, ArgumentMatchers}
import scala.collection.mutable
import scala.jdk.CollectionConverters._
@ -464,30 +464,104 @@ class TransactionCoordinatorTest {
}
@ParameterizedTest
@ValueSource(shorts = Array(0, 2))
def shouldReturnOkOnEndTxnWhenStatusIsCompleteCommitAndResultIsCommit(transactionVersion: Short): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion)
@ValueSource(booleans = Array(false, true))
def testEndTxnWhenStatusIsCompleteCommitAndResultIsCommitInV1(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch,
new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, producerEpoch,
(producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)))))
coordinator.handleEndTransaction(transactionalId, producerId, requestEpoch(clientTransactionVersion), TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback)
assertEquals(Errors.NONE, error)
val epoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback)
if (isRetry) {
assertEquals(Errors.PRODUCER_FENCED, error)
} else {
assertEquals(Errors.NONE, error)
verify(transactionManager, never()).appendTransactionToLog(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ParameterizedTest
@ValueSource(shorts = Array(0, 2))
def shouldReturnOkOnEndTxnWhenStatusIsCompleteAbortAndResultIsAbort(transactionVersion: Short): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion)
@ValueSource(booleans = Array(false, true))
def testEndTxnWhenStatusIsCompleteCommitAndResultIsCommitInV2(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch,
new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID, producerEpoch,
(producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)))))
val epoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback)
if (isRetry) {
assertEquals(Errors.NONE, error)
} else {
assertEquals(Errors.INVALID_TXN_STATE, error)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ParameterizedTest
@ValueSource(booleans = Array(false, true))
def testEndTxnWhenStatusIsCompleteAbortAndResultIsAbortInV1(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0)
val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
producerEpoch, (producerEpoch - 1).toShort, 1, CompleteAbort, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
coordinator.handleEndTransaction(transactionalId, producerId, requestEpoch(clientTransactionVersion), TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
val nextProducerEpoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, nextProducerEpoch.toShort , TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
if (isRetry) {
assertEquals(Errors.PRODUCER_FENCED, error)
} else {
assertEquals(Errors.NONE, error)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ParameterizedTest
@ValueSource(booleans = Array(false, true))
def shouldReturnOkOnEndTxnWhenStatusIsCompleteAbortAndResultIsAbortInV2(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2)
val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
producerEpoch, (producerEpoch - 1).toShort, 1, CompleteAbort, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
val nextProducerEpoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, nextProducerEpoch.toShort , TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
assertEquals(Errors.NONE, error)
if (isRetry) {
verify(transactionManager, never()).appendTransactionToLog(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
)
} else {
val newMetadata = ArgumentCaptor.forClass(classOf[TxnTransitMetadata]);
verify(transactionManager).appendTransactionToLog(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.any(),
newMetadata.capture(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
)
assertEquals(producerEpoch + 1, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].producerEpoch, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].toString)
assertEquals(time.milliseconds(), newMetadata.getValue.asInstanceOf[TxnTransitMetadata].txnStartTimestamp, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].toString)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ -505,20 +579,68 @@ class TransactionCoordinatorTest {
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ParameterizedTest
@ValueSource(shorts = Array(0, 2))
def shouldReturnInvalidTxnRequestOnEndTxnRequestWhenStatusIsCompleteCommitAndResultIsNotCommit(transactionVersion: Short): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(transactionVersion)
@Test
def shouldReturnInvalidTxnRequestOnEndTxnRequestWhenStatusIsCompleteCommitAndResultIsNotCommit(): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0)
val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
producerEpoch, (producerEpoch - 1).toShort,1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
coordinator.handleEndTransaction(transactionalId, producerId, requestEpoch(clientTransactionVersion), TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
assertEquals(Errors.INVALID_TXN_STATE, error)
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ParameterizedTest
@ValueSource(booleans = Array(false, true))
def testEndTxnRequestWhenStatusIsCompleteCommitAndResultIsAbortInV1(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0)
val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
producerEpoch, (producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
val epoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
if (isRetry) {
assertEquals(Errors.PRODUCER_FENCED, error)
} else {
assertEquals(Errors.INVALID_TXN_STATE, error)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ParameterizedTest
@ValueSource(booleans = Array(false, true))
def testEndTxnRequestWhenStatusIsCompleteCommitAndResultIsAbortInV2(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2)
val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
producerEpoch, (producerEpoch - 1).toShort, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
val epoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
if (isRetry) {
assertEquals(Errors.INVALID_TXN_STATE, error)
} else {
assertEquals(Errors.NONE, error)
val newMetadata = ArgumentCaptor.forClass(classOf[TxnTransitMetadata]);
verify(transactionManager).appendTransactionToLog(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.any(),
newMetadata.capture(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
)
assertEquals(producerEpoch + 1, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].producerEpoch, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].toString)
assertEquals(time.milliseconds(), newMetadata.getValue.asInstanceOf[TxnTransitMetadata].txnStartTimestamp, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].toString)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ParameterizedTest
@ValueSource(shorts = Array(0, 2))
def shouldReturnConcurrentTransactionsOnEndTxnRequestWhenStatusIsPrepareCommit(transactionVersion: Short): Unit = {
@ -546,51 +668,62 @@ class TransactionCoordinatorTest {
}
@Test
def shouldReturnWhenTransactionVersionDowngraded(): Unit = {
// State was written when transactions V2
def TestEndTxnRequestWhenEmptyTransactionStateForAbortInV1(): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(0)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId,
RecordBatch.NO_PRODUCER_ID, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, 1, PrepareCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2)))))
RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, Empty, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)))))
// Return CONCURRENT_TRANSACTIONS as the transaction is still completing
coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, TV_0, endTxnCallback)
assertEquals(Errors.CONCURRENT_TRANSACTIONS, error)
assertEquals(RecordBatch.NO_PRODUCER_ID, newProducerId)
assertEquals(RecordBatch.NO_PRODUCER_EPOCH, newEpoch)
coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
assertEquals(Errors.INVALID_TXN_STATE, error)
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
// Recognize the retry and return NONE
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId,
RecordBatch.NO_PRODUCER_ID, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2)))))
coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, TV_0, endTxnCallback)
assertEquals(Errors.NONE, error)
assertEquals(producerId, newProducerId)
assertEquals((producerEpoch + 1).toShort, newEpoch) // epoch is bumped since we started as V2
verify(transactionManager, times(2)).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@Test
def shouldReturnCorrectlyWhenTransactionVersionUpgraded(): Unit = {
// State was written when transactions V0
@ParameterizedTest
@ValueSource(booleans = Array(false, true))
def TestEndTxnRequestWhenEmptyTransactionStateForAbortInV2(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId,
RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, PrepareCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_0)))))
RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, Empty, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)))))
// Transactions V0 throws the concurrent transactions error here.
coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, TV_2, endTxnCallback)
assertEquals(Errors.CONCURRENT_TRANSACTIONS, error)
val epoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.ABORT, clientTransactionVersion, endTxnCallback)
if (isRetry) {
assertEquals(Errors.PRODUCER_FENCED, error)
} else {
assertEquals(Errors.NONE, error)
val newMetadata = ArgumentCaptor.forClass(classOf[TxnTransitMetadata]);
verify(transactionManager).appendTransactionToLog(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.any(),
newMetadata.capture(),
ArgumentMatchers.any(),
ArgumentMatchers.any(),
ArgumentMatchers.any()
)
assertEquals(producerEpoch + 1, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].producerEpoch, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].toString)
assertEquals(time.milliseconds(), newMetadata.getValue.asInstanceOf[TxnTransitMetadata].txnStartTimestamp, newMetadata.getValue.asInstanceOf[TxnTransitMetadata].toString)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
// When the transaction is completed, return and do not throw an error.
@ParameterizedTest
@ValueSource(booleans = Array(false, true))
def TestEndTxnRequestWhenEmptyTransactionStateForCommitInV2(isRetry: Boolean): Unit = {
val clientTransactionVersion = TransactionVersion.fromFeatureLevel(2)
when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId,
RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_0)))))
coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, TV_2, endTxnCallback)
assertEquals(Errors.NONE, error)
assertEquals(producerId, newProducerId)
assertEquals(producerEpoch, newEpoch) // epoch is not bumped since this started as V1
verify(transactionManager, times(2)).getTransactionState(ArgumentMatchers.eq(transactionalId))
RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, Empty, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), clientTransactionVersion)))))
val epoch = if (isRetry) producerEpoch - 1 else producerEpoch
coordinator.handleEndTransaction(transactionalId, producerId, epoch.toShort, TransactionResult.COMMIT, clientTransactionVersion, endTxnCallback)
if (isRetry) {
assertEquals(Errors.PRODUCER_FENCED, error)
} else {
assertEquals(Errors.INVALID_TXN_STATE, error)
}
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@Test
@ -608,9 +741,9 @@ class TransactionCoordinatorTest {
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, new TransactionMetadata(transactionalId, producerId, producerId,
RecordBatch.NO_PRODUCER_ID, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, 1, CompleteCommit, collection.mutable.Set.empty[TopicPartition], 0, time.milliseconds(), TV_2)))))
// If producerEpoch is the same, this is not a retry of the EndTxnRequest, but the next EndTxnRequest. Return PRODUCER_FENCED.
// If producerEpoch is the same, this is not a retry of the EndTxnRequest, but the next EndTxnRequest. Return INVALID_TXN_STATE.
coordinator.handleEndTransaction(transactionalId, producerId, producerEpoch, TransactionResult.COMMIT, TV_2, endTxnCallback)
assertEquals(Errors.PRODUCER_FENCED, error)
assertEquals(Errors.INVALID_TXN_STATE, error)
verify(transactionManager, times(2)).getTransactionState(ArgumentMatchers.eq(transactionalId))
}
@ -799,7 +932,7 @@ class TransactionCoordinatorTest {
verify(transactionManager).appendTransactionToLog(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(coordinatorEpoch),
ArgumentMatchers.eq(originalMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds())),
ArgumentMatchers.eq(originalMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)),
any(),
any(),
any())
@ -820,6 +953,7 @@ class TransactionCoordinatorTest {
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, bumpedTxnMetadata))))
when(transactionManager.transactionVersionLevel()).thenReturn(TV_0)
coordinator.handleInitProducerId(transactionalId, txnTimeoutMs, None, initProducerIdMockCallback)
assertEquals(InitProducerIdResult(-1, -1, Errors.PRODUCER_FENCED), result)
@ -846,7 +980,7 @@ class TransactionCoordinatorTest {
val originalMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_ID,
(producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0)
val txnTransitMetadata = originalMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds())
val txnTransitMetadata = originalMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)
when(transactionManager.appendTransactionToLog(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(coordinatorEpoch),
@ -1230,7 +1364,7 @@ class TransactionCoordinatorTest {
def shouldNotAbortExpiredTransactionsThatHaveAPendingStateTransition(): Unit = {
val metadata = new TransactionMetadata(transactionalId, producerId, producerId,
RecordBatch.NO_PRODUCER_EPOCH, producerEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0)
metadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds())
metadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)
when(transactionManager.timedOutTransactions())
.thenReturn(List(TransactionalIdAndProducerIdEpoch(transactionalId, producerId, producerEpoch)))
@ -1297,7 +1431,7 @@ class TransactionCoordinatorTest {
def shouldNotBumpEpochWithPendingTransaction(): Unit = {
val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, RecordBatch.NO_PRODUCER_EPOCH, producerEpoch,
RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Ongoing, partitions, time.milliseconds(), time.milliseconds(), TV_0)
txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds())
txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)
when(transactionManager.validateTransactionTimeoutMs(anyInt()))
.thenReturn(true)

View File

@ -105,6 +105,81 @@ class TransactionMetadataTest {
None, time.milliseconds()))
}
@Test
def testTransitFromEmptyToPrepareAbortInV2(): Unit = {
val producerEpoch = 735.toShort
val txnMetadata = new TransactionMetadata(
transactionalId = transactionalId,
producerId = producerId,
previousProducerId = RecordBatch.NO_PRODUCER_ID,
nextProducerId = RecordBatch.NO_PRODUCER_ID,
producerEpoch = producerEpoch,
lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH,
txnTimeoutMs = 30000,
state = Empty,
topicPartitions = mutable.Set.empty,
txnStartTimestamp = -1,
txnLastUpdateTimestamp = time.milliseconds(),
clientTransactionVersion = TV_2)
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(producerId, txnMetadata.producerId)
assertEquals(producerEpoch + 1, txnMetadata.producerEpoch)
assertEquals(time.milliseconds() + 1, txnMetadata.txnStartTimestamp)
}
@Test
def testTransitFromCompleteAbortToPrepareAbortInV2(): Unit = {
val producerEpoch = 735.toShort
val txnMetadata = new TransactionMetadata(
transactionalId = transactionalId,
producerId = producerId,
previousProducerId = RecordBatch.NO_PRODUCER_ID,
nextProducerId = RecordBatch.NO_PRODUCER_ID,
producerEpoch = producerEpoch,
lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH,
txnTimeoutMs = 30000,
state = CompleteAbort,
topicPartitions = mutable.Set.empty,
txnStartTimestamp = time.milliseconds() - 1,
txnLastUpdateTimestamp = time.milliseconds(),
clientTransactionVersion = TV_2)
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(producerId, txnMetadata.producerId)
assertEquals(producerEpoch + 1, txnMetadata.producerEpoch)
assertEquals(time.milliseconds() + 1, txnMetadata.txnStartTimestamp)
}
@Test
def testTransitFromCompleteCommitToPrepareAbortInV2(): Unit = {
val producerEpoch = 735.toShort
val txnMetadata = new TransactionMetadata(
transactionalId = transactionalId,
producerId = producerId,
previousProducerId = RecordBatch.NO_PRODUCER_ID,
nextProducerId = RecordBatch.NO_PRODUCER_ID,
producerEpoch = producerEpoch,
lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH,
txnTimeoutMs = 30000,
state = CompleteCommit,
topicPartitions = mutable.Set.empty,
txnStartTimestamp = time.milliseconds() - 1,
txnLastUpdateTimestamp = time.milliseconds(),
clientTransactionVersion = TV_2)
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() + 1, true)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(producerId, txnMetadata.producerId)
assertEquals(producerEpoch + 1, txnMetadata.producerEpoch)
assertEquals(time.milliseconds() + 1, txnMetadata.txnStartTimestamp)
}
@Test
def testTolerateUpdateTimeShiftDuringEpochBump(): Unit = {
val producerEpoch: Short = 1
@ -216,7 +291,7 @@ class TransactionMetadataTest {
clientTransactionVersion = TV_0)
// let new time be smaller
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1)
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(PrepareCommit, txnMetadata.state)
assertEquals(producerId, txnMetadata.producerId)
@ -244,7 +319,7 @@ class TransactionMetadataTest {
clientTransactionVersion = TV_0)
// let new time be smaller
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1)
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(PrepareAbort, txnMetadata.state)
assertEquals(producerId, txnMetadata.producerId)
@ -346,11 +421,53 @@ class TransactionMetadataTest {
// We should reset the pending state to make way for the abort transition.
txnMetadata.pendingState = None
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds())
val transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareAbort, TV_0, RecordBatch.NO_PRODUCER_ID, time.milliseconds(), false)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(producerId, transitMetadata.producerId)
}
@Test
def testInvalidTransitionFromCompleteCommitToFence(): Unit = {
val producerEpoch = (Short.MaxValue - 1).toShort
val txnMetadata = new TransactionMetadata(
transactionalId = transactionalId,
producerId = producerId,
previousProducerId = RecordBatch.NO_PRODUCER_ID,
nextProducerId = RecordBatch.NO_PRODUCER_ID,
producerEpoch = producerEpoch,
lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH,
txnTimeoutMs = 30000,
state = CompleteCommit,
topicPartitions = mutable.Set.empty,
txnLastUpdateTimestamp = time.milliseconds(),
clientTransactionVersion = TV_0)
assertTrue(txnMetadata.isProducerEpochExhausted)
assertThrows(classOf[IllegalStateException], () => txnMetadata.prepareFenceProducerEpoch())
}
@Test
def testInvalidTransitionFromCompleteAbortToFence(): Unit = {
val producerEpoch = (Short.MaxValue - 1).toShort
val txnMetadata = new TransactionMetadata(
transactionalId = transactionalId,
producerId = producerId,
previousProducerId = RecordBatch.NO_PRODUCER_ID,
nextProducerId = RecordBatch.NO_PRODUCER_ID,
producerEpoch = producerEpoch,
lastProducerEpoch = RecordBatch.NO_PRODUCER_EPOCH,
txnTimeoutMs = 30000,
state = CompleteAbort,
topicPartitions = mutable.Set.empty,
txnLastUpdateTimestamp = time.milliseconds(),
clientTransactionVersion = TV_0)
assertTrue(txnMetadata.isProducerEpochExhausted)
assertThrows(classOf[IllegalStateException], () => txnMetadata.prepareFenceProducerEpoch())
}
@Test
def testFenceProducerNotAllowedIfItWouldOverflow(): Unit = {
val producerEpoch = Short.MaxValue
@ -416,7 +533,7 @@ class TransactionMetadataTest {
txnLastUpdateTimestamp = time.milliseconds(),
clientTransactionVersion = TV_2)
var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1)
var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_2, RecordBatch.NO_PRODUCER_ID, time.milliseconds() - 1, false)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(producerId, txnMetadata.producerId)
assertEquals((producerEpoch + 1).toShort, txnMetadata.producerEpoch)
@ -450,7 +567,7 @@ class TransactionMetadataTest {
assertTrue(txnMetadata.isProducerEpochExhausted)
val newProducerId = 9893L
var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_2, newProducerId, time.milliseconds() - 1)
var transitMetadata = txnMetadata.prepareAbortOrCommit(PrepareCommit, TV_2, newProducerId, time.milliseconds() - 1, false)
txnMetadata.completeTransitionTo(transitMetadata)
assertEquals(producerId, txnMetadata.producerId)
assertEquals(Short.MaxValue, txnMetadata.producerEpoch)

View File

@ -3899,7 +3899,7 @@ class UnifiedLogTest {
var sequence = if (appendOrigin == AppendOrigin.CLIENT) 3 else 0
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5)
val log = createLog(logDir, logConfig, producerStateManagerConfig = producerStateManagerConfig)
assertFalse(log.hasOngoingTransaction(producerId))
assertFalse(log.hasOngoingTransaction(producerId, producerEpoch))
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
assertFalse(log.verificationGuard(producerId).verify(VerificationGuard.SENTINEL))
@ -3929,7 +3929,7 @@ class UnifiedLogTest {
assertNotEquals(VerificationGuard.SENTINEL, verificationGuard)
log.appendAsLeader(idempotentRecords, origin = appendOrigin, leaderEpoch = 0)
assertFalse(log.hasOngoingTransaction(producerId))
assertFalse(log.hasOngoingTransaction(producerId, producerEpoch))
// Since we wrote idempotent records, we keep VerificationGuard.
assertEquals(verificationGuard, log.verificationGuard(producerId))
@ -3937,7 +3937,7 @@ class UnifiedLogTest {
// Now write the transactional records
assertTrue(log.verificationGuard(producerId).verify(verificationGuard))
log.appendAsLeader(transactionalRecords, origin = appendOrigin, leaderEpoch = 0, verificationGuard = verificationGuard)
assertTrue(log.hasOngoingTransaction(producerId))
assertTrue(log.hasOngoingTransaction(producerId, producerEpoch))
// VerificationGuard should be cleared now.
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
@ -3951,7 +3951,7 @@ class UnifiedLogTest {
)
log.appendAsLeader(endTransactionMarkerRecord, origin = AppendOrigin.COORDINATOR, leaderEpoch = 0)
assertFalse(log.hasOngoingTransaction(producerId))
assertFalse(log.hasOngoingTransaction(producerId, producerEpoch))
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
if (appendOrigin == AppendOrigin.CLIENT)
@ -3983,7 +3983,7 @@ class UnifiedLogTest {
)
log.appendAsLeader(endTransactionMarkerRecord, origin = AppendOrigin.COORDINATOR, leaderEpoch = 0)
assertFalse(log.hasOngoingTransaction(producerId))
assertFalse(log.hasOngoingTransaction(producerId, producerEpoch))
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
}
@ -4011,7 +4011,7 @@ class UnifiedLogTest {
)
log.appendAsLeader(transactionalRecords, leaderEpoch = 0)
assertTrue(log.hasOngoingTransaction(producerId))
assertTrue(log.hasOngoingTransaction(producerId, producerEpoch))
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
}
@ -4036,14 +4036,14 @@ class UnifiedLogTest {
new SimpleRecord("2".getBytes)
)
assertThrows(classOf[InvalidTxnStateException], () => log.appendAsLeader(transactionalRecords, leaderEpoch = 0))
assertFalse(log.hasOngoingTransaction(producerId))
assertFalse(log.hasOngoingTransaction(producerId, producerEpoch))
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
val verificationGuard = log.maybeStartTransactionVerification(producerId, sequence, producerEpoch)
assertNotEquals(VerificationGuard.SENTINEL, verificationGuard)
log.appendAsLeader(transactionalRecords, leaderEpoch = 0, verificationGuard = verificationGuard)
assertTrue(log.hasOngoingTransaction(producerId))
assertTrue(log.hasOngoingTransaction(producerId, producerEpoch))
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
}
@ -4056,7 +4056,7 @@ class UnifiedLogTest {
val sequence = 3
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5)
val log = createLog(logDir, logConfig, producerStateManagerConfig = producerStateManagerConfig)
assertFalse(log.hasOngoingTransaction(producerId))
assertFalse(log.hasOngoingTransaction(producerId, producerEpoch))
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
val transactionalRecords = MemoryRecords.withTransactionalRecords(

View File

@ -34,6 +34,8 @@ import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.util.RequestAndCompletionHandler
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.mockito.ArgumentMatchers
import org.mockito.ArgumentMatchers.{any, anyLong, anyString}
import org.mockito.MockedConstruction.Context
@ -70,7 +72,7 @@ class AddPartitionsToTxnManagerTest {
private val authenticationErrorResponse = clientResponse(null, authException = new SaslAuthenticationException(""))
private val versionMismatchResponse = clientResponse(null, mismatchException = new UnsupportedVersionException(""))
private val disconnectedResponse = clientResponse(null, disconnected = true)
private val transactionSupportedOperation = genericError
private val transactionSupportedOperation = genericErrorSupported
private val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:2181"))
@ -94,8 +96,10 @@ class AddPartitionsToTxnManagerTest {
callbackErrors.foreachEntry(errors.put)
}
@Test
def testAddTxnData(): Unit = {
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testAddTxnData(isAddPartition: Boolean): Unit = {
val transactionSupportedOperation = if (isAddPartition) addPartition else genericErrorSupported
when(partitionFor.apply(transactionalId1)).thenReturn(0)
when(partitionFor.apply(transactionalId2)).thenReturn(1)
when(partitionFor.apply(transactionalId3)).thenReturn(0)
@ -106,9 +110,9 @@ class AddPartitionsToTxnManagerTest {
val transaction2Errors = mutable.Map[TopicPartition, Errors]()
val transaction3Errors = mutable.Map[TopicPartition, Errors]()
addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), transactionSupportedOperation)
addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), transactionSupportedOperation)
addPartitionsToTxnManager.verifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transaction3Errors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transaction3Errors), transactionSupportedOperation)
// We will try to add transaction1 3 more times (retries). One will have the same epoch, one will have a newer epoch, and one will have an older epoch than the new one we just added.
val transaction1RetryWithSameEpochErrors = mutable.Map[TopicPartition, Errors]()
@ -116,17 +120,17 @@ class AddPartitionsToTxnManagerTest {
val transaction1RetryWithOldEpochErrors = mutable.Map[TopicPartition, Errors]()
// Trying to add more transactional data for the same transactional ID, producer ID, and epoch should simply replace the old data and send a retriable response.
addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithSameEpochErrors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithSameEpochErrors), transactionSupportedOperation)
val expectedNetworkErrors = topicPartitions.map(_ -> Errors.NETWORK_EXCEPTION).toMap
assertEquals(expectedNetworkErrors, transaction1Errors)
// Trying to add more transactional data for the same transactional ID and producer ID, but new epoch should replace the old data and send an error response for it.
addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 1, topicPartitions, setErrors(transaction1RetryWithNewerEpochErrors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 1, topicPartitions, setErrors(transaction1RetryWithNewerEpochErrors), transactionSupportedOperation)
val expectedEpochErrors = topicPartitions.map(_ -> Errors.INVALID_PRODUCER_EPOCH).toMap
assertEquals(expectedEpochErrors, transaction1RetryWithSameEpochErrors)
// Trying to add more transactional data for the same transactional ID and producer ID, but an older epoch should immediately return with error and keep the old data queued to send.
addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithOldEpochErrors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1RetryWithOldEpochErrors), transactionSupportedOperation)
assertEquals(expectedEpochErrors, transaction1RetryWithOldEpochErrors)
val requestsAndHandlers = addPartitionsToTxnManager.generateRequests().asScala
@ -136,45 +140,44 @@ class AddPartitionsToTxnManagerTest {
assertEquals(
AddPartitionsToTxnRequest.Builder.forBroker(
new AddPartitionsToTxnTransactionCollection(Seq(
transactionData(transactionalId3, producerId3),
transactionData(transactionalId1, producerId1, producerEpoch = 1)
transactionData(transactionalId3, producerId3, verifyOnly = !isAddPartition),
transactionData(transactionalId1, producerId1, producerEpoch = 1, verifyOnly = !isAddPartition)
).iterator.asJava)
).data,
requestAndHandler.request.asInstanceOf[AddPartitionsToTxnRequest.Builder].data // insertion order
)
} else {
verifyRequest(node1, transactionalId2, producerId2, requestAndHandler)
verifyRequest(node1, transactionalId2, producerId2, !isAddPartition, requestAndHandler)
}
}
}
@Test
def testGenerateRequests(): Unit = {
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testGenerateRequests(isAddPartition: Boolean): Unit = {
when(partitionFor.apply(transactionalId1)).thenReturn(0)
when(partitionFor.apply(transactionalId2)).thenReturn(1)
when(partitionFor.apply(transactionalId3)).thenReturn(2)
mockTransactionStateMetadata(0, 0, Some(node0))
mockTransactionStateMetadata(1, 1, Some(node1))
mockTransactionStateMetadata(2, 2, Some(node2))
val transactionSupportedOperation = if (isAddPartition) addPartition else genericErrorSupported
val transactionErrors = mutable.Map[TopicPartition, Errors]()
addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
val requestsAndHandlers = addPartitionsToTxnManager.generateRequests().asScala
assertEquals(2, requestsAndHandlers.size)
// Note: handlers are tested in testAddPartitionsToTxnHandlerErrorHandling
requestsAndHandlers.foreach { requestAndHandler =>
if (requestAndHandler.destination == node0) {
verifyRequest(node0, transactionalId1, producerId1, requestAndHandler)
} else {
verifyRequest(node1, transactionalId2, producerId2, requestAndHandler)
}
if (requestAndHandler.destination == node0) verifyRequest(node0, transactionalId1, producerId1, !isAddPartition, requestAndHandler)
else verifyRequest(node1, transactionalId2, producerId2, !isAddPartition, requestAndHandler)
}
addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsToTxnManager.verifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId3, producerId3, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
// Test creationTimeMs increases too.
time.sleep(10)
@ -183,7 +186,7 @@ class AddPartitionsToTxnManagerTest {
// The request for node1 should not be added because one request is already inflight.
assertEquals(1, requestsAndHandlers2.size)
requestsAndHandlers2.foreach { requestAndHandler =>
verifyRequest(node2, transactionalId3, producerId3, requestAndHandler)
verifyRequest(node2, transactionalId3, producerId3, !isAddPartition, requestAndHandler)
}
// Complete the request for node1 so the new one can go through.
@ -191,7 +194,7 @@ class AddPartitionsToTxnManagerTest {
val requestsAndHandlers3 = addPartitionsToTxnManager.generateRequests().asScala
assertEquals(1, requestsAndHandlers3.size)
requestsAndHandlers3.foreach { requestAndHandler =>
verifyRequest(node1, transactionalId2, producerId2, requestAndHandler)
verifyRequest(node1, transactionalId2, producerId2, !isAddPartition, requestAndHandler)
}
}
@ -202,7 +205,7 @@ class AddPartitionsToTxnManagerTest {
def checkError(): Unit = {
val errors = mutable.Map[TopicPartition, Errors]()
addPartitionsToTxnManager.verifyTransaction(
addPartitionsToTxnManager.addOrVerifyTransaction(
transactionalId1,
producerId1,
producerEpoch = 0,
@ -241,16 +244,16 @@ class AddPartitionsToTxnManagerTest {
transaction1Errors.clear()
transaction2Errors.clear()
addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), transactionSupportedOperation)
addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), transactionSupportedOperation)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), transactionSupportedOperation)
}
def addTransactionsToVerifyRequestVersion(operationExpected: TransactionSupportedOperation): Unit = {
transaction1Errors.clear()
transaction2Errors.clear()
addPartitionsToTxnManager.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), operationExpected)
addPartitionsToTxnManager.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), operationExpected)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transaction1Errors), operationExpected)
addPartitionsToTxnManager.addOrVerifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transaction2Errors), operationExpected)
}
val expectedAuthErrors = topicPartitions.map(_ -> Errors.SASL_AUTHENTICATION_FAILED).toMap
@ -319,7 +322,7 @@ class AddPartitionsToTxnManagerTest {
assertEquals(expectedTransactionAbortableErrorsTxn1LowerVersion, transaction1Errors)
assertEquals(expectedTransactionAbortableErrorsTxn2LowerVersion, transaction2Errors)
addTransactionsToVerifyRequestVersion(genericError)
addTransactionsToVerifyRequestVersion(genericErrorSupported)
receiveResponse(mixedAbortableErrorsResponse)
assertEquals(expectedTransactionAbortableErrorsTxn1HigherVersion, transaction1Errors)
assertEquals(expectedTransactionAbortableErrorsTxn2HigherVersion, transaction2Errors)
@ -360,8 +363,8 @@ class AddPartitionsToTxnManagerTest {
)
try {
addPartitionsManagerWithMockedMetrics.verifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsManagerWithMockedMetrics.verifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsManagerWithMockedMetrics.addOrVerifyTransaction(transactionalId1, producerId1, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
addPartitionsManagerWithMockedMetrics.addOrVerifyTransaction(transactionalId2, producerId2, producerEpoch = 0, topicPartitions, setErrors(transactionErrors), transactionSupportedOperation)
time.sleep(100)
@ -424,13 +427,14 @@ class AddPartitionsToTxnManagerTest {
private def transactionData(
transactionalId: String,
producerId: Long,
producerEpoch: Short = 0
producerEpoch: Short = 0,
verifyOnly: Boolean,
): AddPartitionsToTxnTransaction = {
new AddPartitionsToTxnTransaction()
.setTransactionalId(transactionalId)
.setProducerId(producerId)
.setProducerEpoch(producerEpoch)
.setVerifyOnly(true)
.setVerifyOnly(verifyOnly)
.setTopics(new AddPartitionsToTxnTopicCollection(
Seq(new AddPartitionsToTxnTopic()
.setName(topic)
@ -445,6 +449,7 @@ class AddPartitionsToTxnManagerTest {
expectedDestination: Node,
transactionalId: String,
producerId: Long,
verifyOnly: Boolean,
requestAndHandler: RequestAndCompletionHandler
): Unit = {
assertEquals(time.milliseconds(), requestAndHandler.creationTimeMs)
@ -452,7 +457,7 @@ class AddPartitionsToTxnManagerTest {
assertEquals(
AddPartitionsToTxnRequest.Builder.forBroker(
new AddPartitionsToTxnTransactionCollection(
Seq(transactionData(transactionalId, producerId)).iterator.asJava
Seq(transactionData(transactionalId, producerId, verifyOnly = verifyOnly)).iterator.asJava
)
).data,
requestAndHandler.request.asInstanceOf[AddPartitionsToTxnRequest.Builder].data

View File

@ -1871,6 +1871,7 @@ class KafkaApisTest extends Logging {
15L,
0.toShort,
Map(invalidTopicPartition -> partitionOffsetCommitData).asJava,
true
).build()
val request = buildRequest(offsetCommitRequest)
when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](),
@ -2152,6 +2153,7 @@ class KafkaApisTest extends Logging {
producerId,
epoch,
Map(topicPartition -> partitionOffsetCommitData).asJava,
version >= TxnOffsetCommitRequest.LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2
).build(version)
val request = buildRequest(offsetCommitRequest)

View File

@ -25,7 +25,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.message.ProduceRequestData
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse}
import org.apache.kafka.server.metrics.KafkaYammerMetrics
@ -54,17 +54,18 @@ class ProduceRequestTest extends BaseRequestTest {
def sendAndCheck(memoryRecords: MemoryRecords, expectedOffset: Long): Unit = {
val topicPartition = new TopicPartition("topic", partition)
val produceResponse = sendProduceRequest(leader,
ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName(topicPartition.topic())
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(topicPartition.partition())
.setRecords(memoryRecords)))).iterator))
.setAcks((-1).toShort)
.setTimeoutMs(3000)
.setTransactionalId(null)).build())
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
new ProduceRequestData.TopicProduceData()
.setName(topicPartition.topic())
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(topicPartition.partition())
.setRecords(memoryRecords)))).iterator))
.setAcks((-1).toShort)
.setTimeoutMs(3000)
.setTransactionalId(null)).build()
assertEquals(ApiKeys.PRODUCE.latestVersion(), produceRequest.version())
val produceResponse = sendProduceRequest(leader, produceRequest)
assertEquals(1, produceResponse.data.responses.size)
val topicProduceResponse = produceResponse.data.responses.asScala.head
assertEquals(1, topicProduceResponse.partitionResponses.size)

View File

@ -120,7 +120,7 @@ class ReplicaManagerTest {
private var mockRemoteLogManager: RemoteLogManager = _
private var addPartitionsToTxnManager: AddPartitionsToTxnManager = _
private var brokerTopicStats: BrokerTopicStats = _
private val transactionSupportedOperation = genericError
private val transactionSupportedOperation = genericErrorSupported
private val quotaExceededThrottleTime = 1000
private val quotaAvailableThrottleTime = 0
@ -150,7 +150,7 @@ class ReplicaManagerTest {
addPartitionsToTxnManager = mock(classOf[AddPartitionsToTxnManager])
// Anytime we try to verify, just automatically run the callback as though the transaction was verified.
when(addPartitionsToTxnManager.verifyTransaction(any(), any(), any(), any(), any(), any())).thenAnswer { invocationOnMock =>
when(addPartitionsToTxnManager.addOrVerifyTransaction(any(), any(), any(), any(), any(), any())).thenAnswer { invocationOnMock =>
val callback = invocationOnMock.getArgument(4, classOf[AddPartitionsToTxnManager.AppendCallback])
callback(Map.empty[TopicPartition, Errors].toMap)
}
@ -2275,7 +2275,7 @@ class ReplicaManagerTest {
val idempotentRecords = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence,
new SimpleRecord("message".getBytes))
handleProduceAppend(replicaManager, tp0, idempotentRecords, transactionalId = null)
verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any[AddPartitionsToTxnManager.AppendCallback](), any())
verify(addPartitionsToTxnManager, times(0)).addOrVerifyTransaction(any(), any(), any(), any(), any[AddPartitionsToTxnManager.AppendCallback](), any())
assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp0, producerId))
// If we supply a transactional ID and some transactional and some idempotent records, we should only verify the topic partition with transactional records.
@ -2285,7 +2285,7 @@ class ReplicaManagerTest {
val idempotentRecords2 = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence,
new SimpleRecord("message".getBytes))
handleProduceAppendToMultipleTopics(replicaManager, Map(tp0 -> transactionalRecords, tp1 -> idempotentRecords2), transactionalId)
verify(addPartitionsToTxnManager, times(1)).verifyTransaction(
verify(addPartitionsToTxnManager, times(1)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),
@ -2322,7 +2322,7 @@ class ReplicaManagerTest {
// We should add these partitions to the manager to verify.
val result = handleProduceAppend(replicaManager, tp0, transactionalRecords, origin = appendOrigin, transactionalId = transactionalId)
val appendCallback = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
verify(addPartitionsToTxnManager, times(1)).verifyTransaction(
verify(addPartitionsToTxnManager, times(1)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),
@ -2342,7 +2342,7 @@ class ReplicaManagerTest {
// This time verification is successful.
handleProduceAppend(replicaManager, tp0, transactionalRecords, origin = appendOrigin, transactionalId = transactionalId)
val appendCallback2 = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
verify(addPartitionsToTxnManager, times(2)).verifyTransaction(
verify(addPartitionsToTxnManager, times(2)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),
@ -2355,7 +2355,7 @@ class ReplicaManagerTest {
val callback2: AddPartitionsToTxnManager.AppendCallback = appendCallback2.getValue()
callback2(Map.empty[TopicPartition, Errors].toMap)
assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp0, producerId))
assertTrue(replicaManager.localLog(tp0).get.hasOngoingTransaction(producerId))
assertTrue(replicaManager.localLog(tp0).get.hasOngoingTransaction(producerId, producerEpoch))
} finally {
replicaManager.shutdown(checkpointHW = false)
}
@ -2382,7 +2382,7 @@ class ReplicaManagerTest {
// We should add these partitions to the manager to verify.
val result = handleProduceAppend(replicaManager, tp0, transactionalRecords, transactionalId = transactionalId)
val appendCallback = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
verify(addPartitionsToTxnManager, times(1)).verifyTransaction(
verify(addPartitionsToTxnManager, times(1)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),
@ -2405,7 +2405,7 @@ class ReplicaManagerTest {
val result2 = handleProduceAppend(replicaManager, tp0, transactionalRecords2, transactionalId = transactionalId)
val appendCallback2 = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
verify(addPartitionsToTxnManager, times(2)).verifyTransaction(
verify(addPartitionsToTxnManager, times(2)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),
@ -2489,7 +2489,7 @@ class ReplicaManagerTest {
assertThrows(classOf[InvalidPidMappingException],
() => handleProduceAppendToMultipleTopics(replicaManager, transactionalRecords, transactionalId = transactionalId))
// We should not add these partitions to the manager to verify.
verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any())
verify(addPartitionsToTxnManager, times(0)).addOrVerifyTransaction(any(), any(), any(), any(), any(), any())
} finally {
replicaManager.shutdown(checkpointHW = false)
}
@ -2513,7 +2513,7 @@ class ReplicaManagerTest {
handleProduceAppend(replicaManager, tp0, transactionalRecords, transactionalId = transactionalId).onFire { response =>
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, response.error)
}
verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any())
verify(addPartitionsToTxnManager, times(0)).addOrVerifyTransaction(any(), any(), any(), any(), any(), any())
} finally {
replicaManager.shutdown(checkpointHW = false)
}
@ -2547,7 +2547,7 @@ class ReplicaManagerTest {
assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp, producerId))
// We should not add these partitions to the manager to verify.
verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any())
verify(addPartitionsToTxnManager, times(0)).addOrVerifyTransaction(any(), any(), any(), any(), any(), any())
// Dynamically enable verification.
config.dynamicConfig.initialize(None, None)
@ -2561,9 +2561,9 @@ class ReplicaManagerTest {
new SimpleRecord("message".getBytes))
handleProduceAppend(replicaManager, tp, moreTransactionalRecords, transactionalId = transactionalId)
verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any(), any())
verify(addPartitionsToTxnManager, times(0)).addOrVerifyTransaction(any(), any(), any(), any(), any(), any())
assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp, producerId))
assertTrue(replicaManager.localLog(tp).get.hasOngoingTransaction(producerId))
assertTrue(replicaManager.localLog(tp).get.hasOngoingTransaction(producerId, producerEpoch))
} finally {
replicaManager.shutdown(checkpointHW = false)
}
@ -2590,7 +2590,7 @@ class ReplicaManagerTest {
// We should add these partitions to the manager to verify.
val result = handleProduceAppend(replicaManager, tp0, transactionalRecords, transactionalId = transactionalId)
val appendCallback = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
verify(addPartitionsToTxnManager, times(1)).verifyTransaction(
verify(addPartitionsToTxnManager, times(1)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),
@ -2616,9 +2616,9 @@ class ReplicaManagerTest {
// This time we do not verify
handleProduceAppend(replicaManager, tp0, transactionalRecords, transactionalId = transactionalId)
verify(addPartitionsToTxnManager, times(1)).verifyTransaction(any(), any(), any(), any(), any(), any())
verify(addPartitionsToTxnManager, times(1)).addOrVerifyTransaction(any(), any(), any(), any(), any(), any())
assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp0, producerId))
assertTrue(replicaManager.localLog(tp0).get.hasOngoingTransaction(producerId))
assertTrue(replicaManager.localLog(tp0).get.hasOngoingTransaction(producerId, producerEpoch))
} finally {
replicaManager.shutdown(checkpointHW = false)
}
@ -2655,7 +2655,7 @@ class ReplicaManagerTest {
val expectedMessage = s"Unable to verify the partition has been added to the transaction. Underlying error: ${error.toString}"
val result = handleProduceAppend(replicaManager, tp0, transactionalRecords, transactionalId = transactionalId)
val appendCallback = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
verify(addPartitionsToTxnManager, times(1)).verifyTransaction(
verify(addPartitionsToTxnManager, times(1)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),
@ -2686,7 +2686,7 @@ class ReplicaManagerTest {
try {
val result = maybeStartTransactionVerificationForPartition(replicaManager, tp0, transactionalId, producerId, producerEpoch)
val appendCallback = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
verify(addPartitionsToTxnManager, times(0)).verifyTransaction(
verify(addPartitionsToTxnManager, times(0)).addOrVerifyTransaction(
ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(producerId),
ArgumentMatchers.eq(producerEpoch),

View File

@ -487,7 +487,8 @@ class RequestQuotaTest extends BaseRequestTest {
"test-txn-group",
2,
0,
Map.empty[TopicPartition, TxnOffsetCommitRequest.CommittedOffset].asJava
Map.empty[TopicPartition, TxnOffsetCommitRequest.CommittedOffset].asJava,
true
)
case ApiKeys.DESCRIBE_ACLS =>

View File

@ -1932,8 +1932,9 @@ public class GroupCoordinatorServiceTest {
);
}
@Test
public void testCommitTransactionalOffsets() throws ExecutionException, InterruptedException {
@ParameterizedTest
@ValueSource(shorts = {4, 5})
public void testCommitTransactionalOffsets(Short txnOffsetCommitVersion) throws ExecutionException, InterruptedException {
CoordinatorRuntime<GroupCoordinatorShard, CoordinatorRecord> runtime = mockRuntime();
GroupCoordinatorService service = new GroupCoordinatorService(
new LogContext(),
@ -1976,7 +1977,7 @@ public class GroupCoordinatorServiceTest {
)).thenReturn(CompletableFuture.completedFuture(response));
CompletableFuture<TxnOffsetCommitResponseData> future = service.commitTransactionalOffsets(
requestContext(ApiKeys.TXN_OFFSET_COMMIT),
requestContext(ApiKeys.TXN_OFFSET_COMMIT, txnOffsetCommitVersion),
request,
BufferSupplier.NO_CACHING
);

View File

@ -68,7 +68,7 @@ public class ProducerRequestBenchmark {
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(TOPIC_PRODUCE_DATA.iterator()));
private static ProduceRequest request() {
return ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, PRODUCE_REQUEST_DATA).build();
return ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, PRODUCE_REQUEST_DATA, false).build();
}
private static final ProduceRequest REQUEST = request();