-
Notifications
You must be signed in to change notification settings - Fork 0
Clone kafka 19082 client side 2 pc changes pt1 #46
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: trunk
Are you sure you want to change the base?
Changes from all commits
7c6e0cf
5af2d8c
3652d2b
eda67cf
f99b31f
b1b0072
a17217c
5ccb444
4618b63
4a08713
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -142,7 +142,7 @@ public MockProducer() { | |||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| @Override | ||||||||||||||||||||||||||||||||||
| public void initTransactions() { | ||||||||||||||||||||||||||||||||||
| public void initTransactions(boolean keepPreparedTxn) { | ||||||||||||||||||||||||||||||||||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Missing Default ImplementationMockProducer implements parameterized initTransactions but removes default no-argument version. Interface provides default delegation to initTransactions(false), but MockProducer override breaks this delegation pattern. This creates API inconsistency where MockProducer behavior differs from KafkaProducer for default initTransactions() calls. Commitable Suggestion
Suggested change
Standards
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Missing Default ImplementationMockProducer overrides initTransactions(boolean) but removes default initTransactions() method implementation. Producer interface defines default initTransactions() calling initTransactions(false), but MockProducer signature change breaks this contract. This creates inconsistent behavior between real and mock implementations. Commitable Suggestion
Suggested change
Standards
|
||||||||||||||||||||||||||||||||||
| verifyNotClosed(); | ||||||||||||||||||||||||||||||||||
| verifyNotFenced(); | ||||||||||||||||||||||||||||||||||
| if (this.transactionInitialized) { | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -42,7 +42,14 @@ public interface Producer<K, V> extends Closeable { | |||||||||||||||||||||||||||||||||
| /** | ||||||||||||||||||||||||||||||||||
| * See {@link KafkaProducer#initTransactions()} | ||||||||||||||||||||||||||||||||||
| */ | ||||||||||||||||||||||||||||||||||
| void initTransactions(); | ||||||||||||||||||||||||||||||||||
| default void initTransactions() { | ||||||||||||||||||||||||||||||||||
| initTransactions(false); | ||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||
|
Comment on lines
+45
to
+47
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🧩 Analysis chainVerify that breaking change for external implementations is acceptable. Converting Run the following script to identify any external implementations that might be affected: 🏁 Script executed: #!/bin/bash
# Description: Search for external Producer interface implementations to assess breaking change impact
# Search for classes implementing Producer interface
rg -n --type=java -A 5 -B 2 'class\s+\w+.*implements.*Producer<'Length of output: 1948 Avoid breaking change to external Producer implementations |
||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| /** | ||||||||||||||||||||||||||||||||||
| * See {@link KafkaProducer#initTransactions(boolean)} | ||||||||||||||||||||||||||||||||||
| */ | ||||||||||||||||||||||||||||||||||
| void initTransactions(boolean keepPreparedTxn); | ||||||||||||||||||||||||||||||||||
|
Comment on lines
+45
to
+52
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Suggestion: Revert the delegation direction to preserve compatibility: keep the existing no-arg
Comment on lines
+49
to
+52
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🛠️ Refactor suggestion | 🟠 Major Expand Javadoc to explain the keepPreparedTxn parameter. The Javadoc merely references Consider adding detailed documentation similar to this: /**
- * See {@link KafkaProducer#initTransactions(boolean)}
+ * Initializes transactions for this producer, optionally preserving prepared transactions
+ * during recovery.
+ *
+ * @param keepPreparedTxn If true, any prepared (not yet committed or aborted) transactions
+ * are preserved during initialization, enabling two-phase commit
+ * recovery flows. If false (default), prepared transactions are
+ * aborted during initialization.
+ *
+ * @see KafkaProducer#initTransactions(boolean)
*/
void initTransactions(boolean keepPreparedTxn);📝 Committable suggestion
Suggested change
🤖 Prompt for AI Agents |
||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| /** | ||||||||||||||||||||||||||||||||||
| * See {@link KafkaProducer#beginTransaction()} | ||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -355,6 +355,11 @@ public class ProducerConfig extends AbstractConfig { | |
| "By default the TransactionId is not configured, which means transactions cannot be used. " + | ||
| "Note that, by default, transactions require a cluster of at least three brokers which is the recommended setting for production; for development you can change this, by adjusting broker setting <code>transaction.state.log.replication.factor</code>."; | ||
|
|
||
| /** <code> transaction.two.phase.commit.enable </code> */ | ||
| public static final String TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG = "transaction.two.phase.commit.enable"; | ||
| private static final String TRANSACTION_TWO_PHASE_COMMIT_ENABLE_DOC = "If set to true, then the broker is informed that the client is participating in " + | ||
| "two phase commit protocol and transactions that this client starts never expire."; | ||
|
|
||
| /** | ||
| * <code>security.providers</code> | ||
| */ | ||
|
|
@@ -526,6 +531,11 @@ public class ProducerConfig extends AbstractConfig { | |
| new ConfigDef.NonEmptyString(), | ||
| Importance.LOW, | ||
| TRANSACTIONAL_ID_DOC) | ||
| .define(TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, | ||
| Type.BOOLEAN, | ||
| false, | ||
| Importance.LOW, | ||
| TRANSACTION_TWO_PHASE_COMMIT_ENABLE_DOC) | ||
|
Comment on lines
+534
to
+538
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hardcoded Configuration ValuesConfiguration value hardcoded as 'false' default violates organization guideline against hardcoding variables. This creates inflexibility in deployment environments where different defaults may be required. Consider using environment-based configuration or making the default configurable through build-time parameters. Standards
Comment on lines
+534
to
+538
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hardcoded Configuration ValuesConfiguration parameter uses hardcoded default value 'false' instead of externalized configuration. This violates organization guideline against hardcoding variables and creates maintenance burden when default behavior needs modification. Consider using environment-based or profile-based configuration defaults to improve flexibility and maintainability. Standards
|
||
| .define(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, | ||
| Type.STRING, | ||
| CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY, | ||
|
|
@@ -609,6 +619,20 @@ private void postProcessAndValidateIdempotenceConfigs(final Map<String, Object> | |
| if (!idempotenceEnabled && userConfiguredTransactions) { | ||
| throw new ConfigException("Cannot set a " + ProducerConfig.TRANSACTIONAL_ID_CONFIG + " without also enabling idempotence."); | ||
| } | ||
|
|
||
| // Validate that transaction.timeout.ms is not set when transaction.two.phase.commit.enable is true | ||
| // In standard Kafka transactions, the broker enforces transaction.timeout.ms and aborts any | ||
| // transaction that isn't completed in time. With two-phase commit (2PC), an external coordinator | ||
| // decides when to finalize, so broker-side timeouts don't apply. Disallow using both. | ||
| boolean enable2PC = this.getBoolean(TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG); | ||
| boolean userConfiguredTransactionTimeout = originalConfigs.containsKey(TRANSACTION_TIMEOUT_CONFIG); | ||
| if (enable2PC && userConfiguredTransactionTimeout) { | ||
| throw new ConfigException( | ||
| "Cannot set " + ProducerConfig.TRANSACTION_TIMEOUT_CONFIG + | ||
| " when " + ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG + | ||
| " is set to true. Transactions will not expire with two-phase commit enabled." | ||
| ); | ||
| } | ||
|
Comment on lines
+622
to
+635
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🧩 Analysis chainAdd validation that 2PC requires transactional.id. The validation correctly prevents using Add this validation after line 621: if (!idempotenceEnabled && userConfiguredTransactions) {
throw new ConfigException("Cannot set a " + ProducerConfig.TRANSACTIONAL_ID_CONFIG + " without also enabling idempotence.");
}
+
+ // Validate that transaction.two.phase.commit.enable requires transactional.id
+ boolean enable2PC = this.getBoolean(TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG);
+ if (enable2PC && !userConfiguredTransactions) {
+ throw new ConfigException(
+ "Cannot enable " + ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG +
+ " without setting a " + ProducerConfig.TRANSACTIONAL_ID_CONFIG + ". " +
+ "Two-phase commit requires transactional producers."
+ );
+ }Then move the existing 2PC validation to check - // Validate that transaction.timeout.ms is not set when transaction.two.phase.commit.enable is true
- // In standard Kafka transactions, the broker enforces transaction.timeout.ms and aborts any
- // transaction that isn't completed in time. With two-phase commit (2PC), an external coordinator
- // decides when to finalize, so broker-side timeouts don't apply. Disallow using both.
- boolean enable2PC = this.getBoolean(TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG);
- boolean userConfiguredTransactionTimeout = originalConfigs.containsKey(TRANSACTION_TIMEOUT_CONFIG);
- if (enable2PC && userConfiguredTransactionTimeout) {
+ // Validate that transaction.timeout.ms is not explicitly set when 2PC is enabled
+ if (enable2PC && originalConfigs.containsKey(TRANSACTION_TIMEOUT_CONFIG)) {
throw new ConfigException(
"Cannot set " + ProducerConfig.TRANSACTION_TIMEOUT_CONFIG +
" when " + ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG +
" is set to true. Transactions will not expire with two-phase commit enabled."
);
}Clarify behavior of default transaction.timeout.ms with 2PC enabled. The validation only prevents explicitly configured
If the default timeout should be ignored with 2PC, consider adding a comment explaining this behavior to prevent future confusion. 🏁 Script executed: #!/bin/bash
set -e
# Extract lines around idempotence and transactions validation
sed -n '600,650p' clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
# Extract lines around 2PC validation
sed -n '650,700p' clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
# Locate userConfiguredTransactions definition
grep -R -n "userConfiguredTransactions" clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
# Search for any existing check requiring transactional.id when 2PC is enabled
grep -R -n "TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG" clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.javaLength of output: 4804 Validate that
|
||
| } | ||
|
|
||
| private static String parseAcks(String acksString) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||||||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -138,6 +138,7 @@ public class TransactionManager { | |||||||||||||||||||||||||||||||||||
| * | ||||||||||||||||||||||||||||||||||||
| * <ul> | ||||||||||||||||||||||||||||||||||||
| * <li>{@link Producer#initTransactions()} calls {@link #initializeTransactions()}</li> | ||||||||||||||||||||||||||||||||||||
| * <li>{@link Producer#initTransactions(boolean)} calls {@link #initializeTransactions(boolean)}</li> | ||||||||||||||||||||||||||||||||||||
| * <li>{@link Producer#beginTransaction()} calls {@link #beginTransaction()}</li> | ||||||||||||||||||||||||||||||||||||
| * <li>{@link Producer#commitTransaction()}} calls {@link #beginCommit()}</li> | ||||||||||||||||||||||||||||||||||||
| * <li>{@link Producer#abortTransaction()} calls {@link #beginAbort()} | ||||||||||||||||||||||||||||||||||||
|
|
@@ -195,6 +196,7 @@ public class TransactionManager { | |||||||||||||||||||||||||||||||||||
| private volatile boolean clientSideEpochBumpRequired = false; | ||||||||||||||||||||||||||||||||||||
| private volatile long latestFinalizedFeaturesEpoch = -1; | ||||||||||||||||||||||||||||||||||||
| private volatile boolean isTransactionV2Enabled = false; | ||||||||||||||||||||||||||||||||||||
| private final boolean enable2PC; | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| private enum State { | ||||||||||||||||||||||||||||||||||||
| UNINITIALIZED, | ||||||||||||||||||||||||||||||||||||
|
|
@@ -255,7 +257,8 @@ public TransactionManager(final LogContext logContext, | |||||||||||||||||||||||||||||||||||
| final String transactionalId, | ||||||||||||||||||||||||||||||||||||
| final int transactionTimeoutMs, | ||||||||||||||||||||||||||||||||||||
| final long retryBackoffMs, | ||||||||||||||||||||||||||||||||||||
| final ApiVersions apiVersions) { | ||||||||||||||||||||||||||||||||||||
| final ApiVersions apiVersions, | ||||||||||||||||||||||||||||||||||||
| final boolean enable2PC) { | ||||||||||||||||||||||||||||||||||||
|
Comment on lines
257
to
+261
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Missing Constructor ValidationConstructor accepts enable2PC parameter without validation against transactionTimeoutMs consistency. Configuration validation occurs in ProducerConfig but constructor allows inconsistent state creation. Runtime failures possible when enable2PC=true with non-zero transactionTimeoutMs creating conflicting timeout behaviors. Commitable Suggestion
Suggested change
Standards
Comment on lines
257
to
+261
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Constructor Parameter ExplosionConstructor now has 6 parameters violating Clean Code parameter limit guidelines. Adding enable2PC parameter increases coupling and makes instantiation complex. This pattern will continue growing as more transaction features are added, reducing maintainability and testability. Standards
|
||||||||||||||||||||||||||||||||||||
| this.producerIdAndEpoch = ProducerIdAndEpoch.NONE; | ||||||||||||||||||||||||||||||||||||
| this.transactionalId = transactionalId; | ||||||||||||||||||||||||||||||||||||
| this.log = logContext.logger(TransactionManager.class); | ||||||||||||||||||||||||||||||||||||
|
|
@@ -273,17 +276,29 @@ public TransactionManager(final LogContext logContext, | |||||||||||||||||||||||||||||||||||
| this.retryBackoffMs = retryBackoffMs; | ||||||||||||||||||||||||||||||||||||
| this.txnPartitionMap = new TxnPartitionMap(logContext); | ||||||||||||||||||||||||||||||||||||
| this.apiVersions = apiVersions; | ||||||||||||||||||||||||||||||||||||
| this.enable2PC = enable2PC; | ||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| void setPoisonStateOnInvalidTransition(boolean shouldPoisonState) { | ||||||||||||||||||||||||||||||||||||
| shouldPoisonStateOnInvalidTransition.set(shouldPoisonState); | ||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| public synchronized TransactionalRequestResult initializeTransactions() { | ||||||||||||||||||||||||||||||||||||
| return initializeTransactions(ProducerIdAndEpoch.NONE); | ||||||||||||||||||||||||||||||||||||
| return initializeTransactions(ProducerIdAndEpoch.NONE, false); | ||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| synchronized TransactionalRequestResult initializeTransactions(ProducerIdAndEpoch producerIdAndEpoch) { | ||||||||||||||||||||||||||||||||||||
| return initializeTransactions(producerIdAndEpoch, false); | ||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| public synchronized TransactionalRequestResult initializeTransactions(boolean keepPreparedTxn) { | ||||||||||||||||||||||||||||||||||||
| return initializeTransactions(ProducerIdAndEpoch.NONE, keepPreparedTxn); | ||||||||||||||||||||||||||||||||||||
|
Comment on lines
286
to
+295
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Method Overloading ChainFour overloaded methods create a complex delegation chain with boolean parameters. This telescoping constructor anti-pattern makes the API confusing and error-prone. The multiple overloads with different parameter combinations reduce code clarity and increase maintenance burden. Standards
|
||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| synchronized TransactionalRequestResult initializeTransactions( | ||||||||||||||||||||||||||||||||||||
| ProducerIdAndEpoch producerIdAndEpoch, | ||||||||||||||||||||||||||||||||||||
| boolean keepPreparedTxn | ||||||||||||||||||||||||||||||||||||
| ) { | ||||||||||||||||||||||||||||||||||||
|
Comment on lines
286
to
+301
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Method Overload ChainMultiple overloaded methods create a complex delegation chain that reduces readability and increases maintenance burden. The four-method overload pattern makes it difficult to understand the primary implementation. Consider consolidating into fewer methods with optional parameters or using a configuration object. Standards
|
||||||||||||||||||||||||||||||||||||
| maybeFailWithError(); | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| boolean isEpochBump = producerIdAndEpoch != ProducerIdAndEpoch.NONE; | ||||||||||||||||||||||||||||||||||||
|
|
@@ -292,14 +307,20 @@ synchronized TransactionalRequestResult initializeTransactions(ProducerIdAndEpoc | |||||||||||||||||||||||||||||||||||
| if (!isEpochBump) { | ||||||||||||||||||||||||||||||||||||
| transitionTo(State.INITIALIZING); | ||||||||||||||||||||||||||||||||||||
| log.info("Invoking InitProducerId for the first time in order to acquire a producer ID"); | ||||||||||||||||||||||||||||||||||||
| if (keepPreparedTxn) { | ||||||||||||||||||||||||||||||||||||
| log.info("Invoking InitProducerId with keepPreparedTxn set to true for 2PC transactions"); | ||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||
| } else { | ||||||||||||||||||||||||||||||||||||
| log.info("Invoking InitProducerId with current producer ID and epoch {} in order to bump the epoch", producerIdAndEpoch); | ||||||||||||||||||||||||||||||||||||
| } | ||||||||||||||||||||||||||||||||||||
| InitProducerIdRequestData requestData = new InitProducerIdRequestData() | ||||||||||||||||||||||||||||||||||||
| .setTransactionalId(transactionalId) | ||||||||||||||||||||||||||||||||||||
| .setTransactionTimeoutMs(transactionTimeoutMs) | ||||||||||||||||||||||||||||||||||||
| .setProducerId(producerIdAndEpoch.producerId) | ||||||||||||||||||||||||||||||||||||
| .setProducerEpoch(producerIdAndEpoch.epoch); | ||||||||||||||||||||||||||||||||||||
| .setProducerEpoch(producerIdAndEpoch.epoch) | ||||||||||||||||||||||||||||||||||||
| .setEnable2Pc(enable2PC) | ||||||||||||||||||||||||||||||||||||
| .setKeepPreparedTxn(keepPreparedTxn); | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| InitProducerIdHandler handler = new InitProducerIdHandler(new InitProducerIdRequest.Builder(requestData), | ||||||||||||||||||||||||||||||||||||
| isEpochBump); | ||||||||||||||||||||||||||||||||||||
| enqueueRequest(handler); | ||||||||||||||||||||||||||||||||||||
|
|
||||||||||||||||||||||||||||||||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -73,6 +73,7 @@ | |
| import org.apache.kafka.common.requests.EndTxnResponse; | ||
| import org.apache.kafka.common.requests.FindCoordinatorRequest; | ||
| import org.apache.kafka.common.requests.FindCoordinatorResponse; | ||
| import org.apache.kafka.common.requests.InitProducerIdRequest; | ||
| import org.apache.kafka.common.requests.InitProducerIdResponse; | ||
| import org.apache.kafka.common.requests.JoinGroupRequest; | ||
| import org.apache.kafka.common.requests.MetadataResponse; | ||
|
|
@@ -102,6 +103,7 @@ | |
| import org.junit.jupiter.api.Test; | ||
| import org.junit.jupiter.api.TestInfo; | ||
| import org.junit.jupiter.params.ParameterizedTest; | ||
| import org.junit.jupiter.params.provider.CsvSource; | ||
| import org.junit.jupiter.params.provider.ValueSource; | ||
| import org.mockito.MockedStatic; | ||
| import org.mockito.Mockito; | ||
|
|
@@ -1289,7 +1291,7 @@ public void testInitTransactionsResponseAfterTimeout() throws Exception { | |
| ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), | ||
| FindCoordinatorResponse.prepareResponse(Errors.NONE, "bad-transaction", NODE)); | ||
|
|
||
| Future<?> future = executor.submit(producer::initTransactions); | ||
| Future<?> future = executor.submit(() -> producer.initTransactions()); | ||
| TestUtils.waitForCondition(client::hasInFlightRequests, | ||
| "Timed out while waiting for expected `InitProducerId` request to be sent"); | ||
|
|
||
|
|
@@ -1364,6 +1366,59 @@ public void testInitTransactionWhileThrottled() { | |
| } | ||
| } | ||
|
|
||
| @ParameterizedTest | ||
| @CsvSource({ | ||
| "true, false", | ||
| "true, true", | ||
| "false, true" | ||
| }) | ||
|
Comment on lines
+1370
to
+1374
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To enhance test completeness, consider adding the @CsvSource({
"true, false",
"true, true",
"false, true",
"false, false"
}) |
||
| public void testInitTransactionsWithKeepPreparedTxnAndTwoPhaseCommit(boolean keepPreparedTxn, boolean enable2PC) { | ||
| Map<String, Object> configs = new HashMap<>(); | ||
| configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "test-txn-id"); | ||
| configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); | ||
| configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); | ||
| if (enable2PC) { | ||
| configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, true); | ||
| } | ||
|
|
||
| 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); | ||
|
|
||
| // Capture flags from the InitProducerIdRequest | ||
| boolean[] requestFlags = new boolean[2]; // [keepPreparedTxn, enable2Pc] | ||
|
|
||
| client.prepareResponse( | ||
| request -> request instanceof FindCoordinatorRequest && | ||
| ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), | ||
| FindCoordinatorResponse.prepareResponse(Errors.NONE, "test-txn-id", NODE)); | ||
|
|
||
| client.prepareResponse( | ||
| request -> { | ||
| if (request instanceof InitProducerIdRequest) { | ||
| InitProducerIdRequest initRequest = (InitProducerIdRequest) request; | ||
| requestFlags[0] = initRequest.data().keepPreparedTxn(); | ||
| requestFlags[1] = initRequest.data().enable2Pc(); | ||
| return true; | ||
| } | ||
| return false; | ||
| }, | ||
| initProducerIdResponse(1L, (short) 5, Errors.NONE)); | ||
|
|
||
| try (Producer<String, String> producer = kafkaProducer(configs, new StringSerializer(), | ||
| new StringSerializer(), metadata, client, null, time)) { | ||
| producer.initTransactions(keepPreparedTxn); | ||
|
|
||
| // Verify request flags match expected values | ||
| assertEquals(keepPreparedTxn, requestFlags[0], | ||
| "keepPreparedTxn flag should match input parameter"); | ||
| assertEquals(enable2PC, requestFlags[1], | ||
| "enable2Pc flag should match producer configuration"); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testClusterAuthorizationFailure() throws Exception { | ||
| int maxBlockMs = 500; | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Magic Boolean Parameter
Boolean literal parameter reduces code readability and makes the intent unclear at call sites. The 'false' value requires developers to check method documentation to understand behavior. Consider using an enum or named constants to make the intent explicit and improve self-documenting code.
Standards