mirror of https://github.com/apache/kafka.git
				
				
				
			Merge branch 'trunk' into jbodTiered2
This commit is contained in:
		
						commit
						51d8967bcf
					
				|  | @ -0,0 +1,44 @@ | |||
| # Licensed to the Apache Software Foundation (ASF) under one or more | ||||
| # contributor license agreements.  See the NOTICE file distributed with | ||||
| # this work for additional information regarding copyright ownership. | ||||
| # The ASF licenses this file to You under the Apache License, Version 2.0 | ||||
| # (the "License"); you may not use this file except in compliance with | ||||
| # the License.  You may obtain a copy of the License at | ||||
| # | ||||
| #    http://www.apache.org/licenses/LICENSE-2.0 | ||||
| # | ||||
| # Unless required by applicable law or agreed to in writing, software | ||||
| # distributed under the License is distributed on an "AS IS" BASIS, | ||||
| # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||||
| # See the License for the specific language governing permissions and | ||||
| # limitations under the License. | ||||
| 
 | ||||
| name: Docker Image CVE Scanner | ||||
| on: | ||||
|   schedule: | ||||
|     # This job will run at 3:30 UTC daily | ||||
|     - cron: '30 3 * * *' | ||||
|   workflow_dispatch: | ||||
| jobs: | ||||
|   scan_jvm: | ||||
|     runs-on: ubuntu-latest | ||||
|     strategy: | ||||
|       matrix: | ||||
|         # This is an array of supported tags. Make sure this array only contains the supported tags | ||||
|         supported_image_tag: ['latest', '3.7.0'] | ||||
|     steps: | ||||
|       - name: Run CVE scan | ||||
|         uses: aquasecurity/trivy-action@master | ||||
|         if: always() | ||||
|         with: | ||||
|           image-ref: apache/kafka:${{ matrix.supported_image_tag }} | ||||
|           format: 'table' | ||||
|           severity: 'CRITICAL,HIGH' | ||||
|           output: scan_report_jvm_${{ matrix.supported_image_tag }}.txt | ||||
|           exit-code: '1' | ||||
|       - name: Upload CVE scan report | ||||
|         if: always() | ||||
|         uses: actions/upload-artifact@v3 | ||||
|         with: | ||||
|           name: scan_report_jvm_${{ matrix.supported_image_tag }}.txt | ||||
|           path: scan_report_jvm_${{ matrix.supported_image_tag }}.txt | ||||
							
								
								
									
										19
									
								
								README.md
								
								
								
								
							
							
						
						
									
										19
									
								
								README.md
								
								
								
								
							|  | @ -54,9 +54,14 @@ Follow instructions in https://kafka.apache.org/quickstart | |||
|     ./gradlew clients:test --tests org.apache.kafka.clients.MetadataTest.testTimeToNextUpdate | ||||
| 
 | ||||
| ### Running a particular unit/integration test with log4j output ### | ||||
| Change the log4j setting in either `clients/src/test/resources/log4j.properties` or `core/src/test/resources/log4j.properties` | ||||
| By default, there will be only small number of logs output while testing. You can adjust it by changing the `log4j.properties` file in the module's `src/test/resources` directory. | ||||
| 
 | ||||
|     ./gradlew clients:test --tests RequestResponseTest | ||||
| For example, if you want to see more logs for clients project tests, you can modify [the line](https://github.com/apache/kafka/blob/trunk/clients/src/test/resources/log4j.properties#L21) in `clients/src/test/resources/log4j.properties`  | ||||
| to `log4j.logger.org.apache.kafka=INFO` and then run: | ||||
|      | ||||
|     ./gradlew cleanTest clients:test --tests NetworkClientTest    | ||||
| 
 | ||||
| And you should see `INFO` level logs in the file under the `clients/build/test-results/test` directory. | ||||
| 
 | ||||
| ### Specifying test retries ### | ||||
| By default, each failed test is retried once up to a maximum of five retries per test run. Tests are retried at the end of the test task. Adjust these parameters in the following way: | ||||
|  | @ -87,15 +92,25 @@ fail due to code changes. You can just run: | |||
| 
 | ||||
| ### Running a Kafka broker in KRaft mode | ||||
| 
 | ||||
| Using compiled files: | ||||
| 
 | ||||
|     KAFKA_CLUSTER_ID="$(./bin/kafka-storage.sh random-uuid)" | ||||
|     ./bin/kafka-storage.sh format -t $KAFKA_CLUSTER_ID -c config/kraft/server.properties | ||||
|     ./bin/kafka-server-start.sh config/kraft/server.properties | ||||
| 
 | ||||
| Using docker image: | ||||
| 
 | ||||
|     docker run -p 9092:9092 apache/kafka:3.7.0 | ||||
| 
 | ||||
| ### Running a Kafka broker in ZooKeeper mode | ||||
| 
 | ||||
| Using compiled files: | ||||
| 
 | ||||
|     ./bin/zookeeper-server-start.sh config/zookeeper.properties | ||||
|     ./bin/kafka-server-start.sh config/server.properties | ||||
| 
 | ||||
| >Since ZooKeeper mode is already deprecated and planned to be removed in Apache Kafka 4.0, the docker image only supports running in KRaft mode | ||||
| 
 | ||||
| ### Cleaning the build ### | ||||
|     ./gradlew clean | ||||
| 
 | ||||
|  |  | |||
|  | @ -14,4 +14,4 @@ | |||
| # See the License for the specific language governing permissions and | ||||
| # limitations under the License. | ||||
| 
 | ||||
| exec $(dirname $0)/kafka-run-class.sh kafka.admin.ConsumerGroupCommand "$@" | ||||
| exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.tools.consumer.group.ConsumerGroupCommand "$@" | ||||
|  |  | |||
|  | @ -14,4 +14,4 @@ rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | |||
| rem See the License for the specific language governing permissions and | ||||
| rem limitations under the License. | ||||
| 
 | ||||
| "%~dp0kafka-run-class.bat" kafka.admin.ConsumerGroupCommand %* | ||||
| "%~dp0kafka-run-class.bat" org.apache.kafka.tools.consumer.group.ConsumerGroupCommand %* | ||||
|  |  | |||
							
								
								
									
										43
									
								
								build.gradle
								
								
								
								
							
							
						
						
									
										43
									
								
								build.gradle
								
								
								
								
							|  | @ -305,6 +305,24 @@ subprojects { | |||
|           } else { | ||||
|             apply plugin: 'com.github.johnrengelman.shadow' | ||||
|             project.shadow.component(mavenJava) | ||||
| 
 | ||||
|             // Fix for avoiding inclusion of runtime dependencies marked as 'shadow' in MANIFEST Class-Path. | ||||
|             // https://github.com/johnrengelman/shadow/issues/324 | ||||
|             afterEvaluate { | ||||
|               pom.withXml { xml -> | ||||
|                 if (xml.asNode().get('dependencies') == null) { | ||||
|                   xml.asNode().appendNode('dependencies') | ||||
|                 } | ||||
|                 def dependenciesNode = xml.asNode().get('dependencies').get(0) | ||||
|                 project.configurations.shadowed.allDependencies.each { | ||||
|                   def dependencyNode = dependenciesNode.appendNode('dependency') | ||||
|                   dependencyNode.appendNode('groupId', it.group) | ||||
|                   dependencyNode.appendNode('artifactId', it.name) | ||||
|                   dependencyNode.appendNode('version', it.version) | ||||
|                   dependencyNode.appendNode('scope', 'runtime') | ||||
|                 } | ||||
|               } | ||||
|             } | ||||
|           } | ||||
| 
 | ||||
|           afterEvaluate { | ||||
|  | @ -748,7 +766,7 @@ subprojects { | |||
|   } | ||||
| 
 | ||||
|   if (userEnableTestCoverage) { | ||||
|     def coverageGen = it.path == ':core' ? 'reportScoverage' : 'jacocoTestReport' | ||||
|     def coverageGen = it.path == ':core' ? 'reportTestScoverage' : 'jacocoTestReport' | ||||
|     tasks.register('reportCoverage').configure { dependsOn(coverageGen) } | ||||
|   } | ||||
| 
 | ||||
|  | @ -829,6 +847,7 @@ project(':server') { | |||
|     implementation project(':transaction-coordinator') | ||||
|     implementation project(':raft') | ||||
|     implementation libs.metrics | ||||
|     implementation libs.jacksonDatabind | ||||
| 
 | ||||
|     implementation libs.slf4jApi | ||||
| 
 | ||||
|  | @ -1402,6 +1421,7 @@ project(':clients') { | |||
| 
 | ||||
|   configurations { | ||||
|     generator | ||||
|     shadowed | ||||
|   } | ||||
| 
 | ||||
|   dependencies { | ||||
|  | @ -1412,10 +1432,10 @@ project(':clients') { | |||
|     implementation libs.opentelemetryProto | ||||
| 
 | ||||
|     // libraries which should be added as runtime dependencies in generated pom.xml should be defined here: | ||||
|     shadow libs.zstd | ||||
|     shadow libs.lz4 | ||||
|     shadow libs.snappy | ||||
|     shadow libs.slf4jApi | ||||
|     shadowed libs.zstd | ||||
|     shadowed libs.lz4 | ||||
|     shadowed libs.snappy | ||||
|     shadowed libs.slf4jApi | ||||
| 
 | ||||
|     compileOnly libs.jacksonDatabind // for SASL/OAUTHBEARER bearer token parsing | ||||
|     compileOnly libs.jacksonJDK8Datatypes | ||||
|  | @ -1468,10 +1488,9 @@ project(':clients') { | |||
| 
 | ||||
|     // dependencies excluded from the final jar, since they are declared as runtime dependencies | ||||
|     dependencies { | ||||
|       exclude(dependency(libs.snappy)) | ||||
|       exclude(dependency(libs.zstd)) | ||||
|       exclude(dependency(libs.lz4)) | ||||
|       exclude(dependency(libs.slf4jApi)) | ||||
|       project.configurations.shadowed.allDependencies.each { | ||||
|         exclude(dependency(it.group + ':' + it.name + ':' + it.version)) | ||||
|       } | ||||
|       // exclude proto files from the jar | ||||
|       exclude "**/opentelemetry/proto/**/*.proto" | ||||
|       exclude "**/google/protobuf/*.proto" | ||||
|  | @ -1980,6 +1999,9 @@ project(':tools') { | |||
|     implementation project(':log4j-appender') | ||||
|     implementation project(':tools:tools-api') | ||||
|     implementation libs.argparse4j | ||||
|     implementation libs.jacksonDatabind | ||||
|     implementation libs.jacksonDataformatCsv | ||||
|     implementation libs.jacksonJDK8Datatypes | ||||
|     implementation libs.slf4jApi | ||||
|     implementation libs.log4j | ||||
|     implementation libs.joptSimple | ||||
|  | @ -2157,6 +2179,7 @@ project(':streams') { | |||
|     testImplementation project(':storage') | ||||
|     testImplementation project(':server-common') | ||||
|     testImplementation project(':server-common').sourceSets.test.output | ||||
|     testImplementation project(':server') | ||||
|     testImplementation libs.log4j | ||||
|     testImplementation libs.junitJupiter | ||||
|     testImplementation libs.junitVintageEngine | ||||
|  | @ -2739,6 +2762,7 @@ project(':jmh-benchmarks') { | |||
|       exclude group: 'net.sf.jopt-simple', module: 'jopt-simple' | ||||
|     } | ||||
|     implementation project(':server-common') | ||||
|     implementation project(':server') | ||||
|     implementation project(':clients') | ||||
|     implementation project(':group-coordinator') | ||||
|     implementation project(':metadata') | ||||
|  | @ -2966,6 +2990,7 @@ project(':connect:runtime') { | |||
|     testImplementation project(':metadata') | ||||
|     testImplementation project(':core').sourceSets.test.output | ||||
|     testImplementation project(':server-common') | ||||
|     testImplementation project(':server') | ||||
|     testImplementation project(':storage') | ||||
|     testImplementation project(':connect:test-plugins') | ||||
| 
 | ||||
|  |  | |||
|  | @ -47,6 +47,7 @@ | |||
|     <allow pkg="kafka.message"/> | ||||
|     <allow pkg="org.mockito"/> | ||||
|     <allow pkg="kafka.security.authorizer"/> | ||||
|     <allow pkg="org.apache.kafka.security.authorizer"/> | ||||
|     <allow pkg="org.apache.kafka.server"/> | ||||
|     <allow pkg="org.apache.kafka.storage"/> | ||||
|     <allow pkg="org.apache.kafka.clients"/> | ||||
|  |  | |||
|  | @ -105,10 +105,10 @@ | |||
|             <allow pkg="org.apache.kafka.server.util.json" /> | ||||
| 
 | ||||
|             <allow class="org.apache.kafka.server.util.TopicFilter.IncludeList" /> | ||||
|             <allow class="org.apache.kafka.test.TestUtils" /> | ||||
|             <subpackage name="timer"> | ||||
|                 <allow class="org.apache.kafka.server.util.MockTime" /> | ||||
|                 <allow class="org.apache.kafka.server.util.ShutdownableThread" /> | ||||
|                 <allow class="org.apache.kafka.test.TestUtils" /> | ||||
|             </subpackage> | ||||
|         </subpackage> | ||||
|     </subpackage> | ||||
|  |  | |||
|  | @ -605,6 +605,7 @@ | |||
|         <allow pkg="org.apache.kafka.metadata" /> | ||||
|         <allow pkg="org.eclipse.jetty.client"/> | ||||
|         <allow class="org.apache.kafka.storage.internals.log.CleanerConfig" /> | ||||
|         <allow class="org.apache.kafka.server.config.ZkConfigs" /> | ||||
|       </subpackage> | ||||
|     </subpackage> | ||||
| 
 | ||||
|  |  | |||
|  | @ -754,7 +754,7 @@ public interface Admin extends AutoCloseable { | |||
|      * </ul> | ||||
|      * | ||||
|      * @param options The options to use when creating delegation token. | ||||
|      * @return The DeleteRecordsResult. | ||||
|      * @return The CreateDelegationTokenResult. | ||||
|      */ | ||||
|     CreateDelegationTokenResult createDelegationToken(CreateDelegationTokenOptions options); | ||||
| 
 | ||||
|  |  | |||
|  | @ -30,6 +30,7 @@ import org.apache.kafka.common.errors.RetriableException; | |||
| import org.apache.kafka.common.errors.StaleMemberEpochException; | ||||
| import org.apache.kafka.common.errors.TimeoutException; | ||||
| import org.apache.kafka.common.errors.TopicAuthorizationException; | ||||
| import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; | ||||
| import org.apache.kafka.common.errors.UnstableOffsetCommitException; | ||||
| import org.apache.kafka.common.message.OffsetCommitRequestData; | ||||
| import org.apache.kafka.common.message.OffsetCommitResponseData; | ||||
|  | @ -272,14 +273,18 @@ public class CommitRequestManager implements RequestManager, MemberStateListener | |||
|      * Commit consumed offsets if auto-commit is enabled, regardless of the auto-commit interval. | ||||
|      * This is used for committing offsets before revoking partitions. This will retry committing | ||||
|      * the latest offsets until the request succeeds, fails with a fatal error, or the timeout | ||||
|      * expires. Note that this considers {@link Errors#STALE_MEMBER_EPOCH} as a retriable error, | ||||
|      * expires. Note that: | ||||
|      * - Considers {@link Errors#STALE_MEMBER_EPOCH} as a retriable error, | ||||
|      * and will retry it including the latest member ID and epoch received from the broker. | ||||
|      * - Considers {@link Errors#UNKNOWN_TOPIC_OR_PARTITION} as a fatal error, and will not retry | ||||
|      * it although the error extends RetriableException. The reason is that if a topic or partition | ||||
|      * is deleted, revocation would not finish in time since the auto commit would keep retrying. | ||||
|      * | ||||
|      * @return Future that will complete when the offsets are successfully committed. It will | ||||
|      * complete exceptionally if the commit fails with a non-retriable error, or if the retry | ||||
|      * timeout expires. | ||||
|      */ | ||||
|     public CompletableFuture<Void> maybeAutoCommitSyncNow(final long retryExpirationTimeMs) { | ||||
|     public CompletableFuture<Void> maybeAutoCommitSyncBeforeRevocation(final long retryExpirationTimeMs) { | ||||
|         if (!autoCommitEnabled()) { | ||||
|             return CompletableFuture.completedFuture(null); | ||||
|         } | ||||
|  | @ -287,11 +292,11 @@ public class CommitRequestManager implements RequestManager, MemberStateListener | |||
|         CompletableFuture<Void> result = new CompletableFuture<>(); | ||||
|         OffsetCommitRequestState requestState = | ||||
|             createOffsetCommitRequest(subscriptions.allConsumed(), Optional.of(retryExpirationTimeMs)); | ||||
|         autoCommitSyncNowWithRetries(requestState, result); | ||||
|         autoCommitSyncBeforeRevocationWithRetries(requestState, result); | ||||
|         return result; | ||||
|     } | ||||
| 
 | ||||
|     private void autoCommitSyncNowWithRetries(OffsetCommitRequestState requestAttempt, | ||||
|     private void autoCommitSyncBeforeRevocationWithRetries(OffsetCommitRequestState requestAttempt, | ||||
|                                                            CompletableFuture<Void> result) { | ||||
|         CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitAttempt = requestAutoCommit(requestAttempt); | ||||
|         commitAttempt.whenComplete((committedOffsets, error) -> { | ||||
|  | @ -300,16 +305,19 @@ public class CommitRequestManager implements RequestManager, MemberStateListener | |||
|             } else { | ||||
|                 if (error instanceof RetriableException || isStaleEpochErrorAndValidEpochAvailable(error)) { | ||||
|                     if (error instanceof TimeoutException && requestAttempt.isExpired) { | ||||
|                         log.debug("Auto-commit sync timed out and won't be retried anymore"); | ||||
|                         log.debug("Auto-commit sync before revocation timed out and won't be retried anymore"); | ||||
|                         result.completeExceptionally(error); | ||||
|                     } else if (error instanceof UnknownTopicOrPartitionException) { | ||||
|                         log.debug("Auto-commit sync before revocation failed because topic or partition were deleted"); | ||||
|                         result.completeExceptionally(error); | ||||
|                     } else { | ||||
|                         // Make sure the auto-commit is retries with the latest offsets | ||||
|                         requestAttempt.offsets = subscriptions.allConsumed(); | ||||
|                         requestAttempt.resetFuture(); | ||||
|                         autoCommitSyncNowWithRetries(requestAttempt, result); | ||||
|                         autoCommitSyncBeforeRevocationWithRetries(requestAttempt, result); | ||||
|                     } | ||||
|                 } else { | ||||
|                     log.debug("Auto-commit sync failed with non-retriable error", error); | ||||
|                     log.debug("Auto-commit sync before revocation failed with non-retriable error", error); | ||||
|                     result.completeExceptionally(error); | ||||
|                 } | ||||
|             } | ||||
|  |  | |||
|  | @ -228,11 +228,11 @@ public class MembershipManagerImpl implements MembershipManager { | |||
|     private boolean reconciliationInProgress; | ||||
| 
 | ||||
|     /** | ||||
|      * Epoch the member had when the reconciliation in progress started. This is used to identify if | ||||
|      * the member has rejoined while it was reconciling an assignment (in which case the result | ||||
|      * of the reconciliation is not applied.) | ||||
|      * True if a reconciliation is in progress and the member rejoined the group since the start | ||||
|      * of the reconciliation. Used to know that the reconciliation in progress should be | ||||
|      * interrupted and not be applied. | ||||
|      */ | ||||
|     private int memberEpochOnReconciliationStart; | ||||
|     private boolean rejoinedWhileReconciliationInProgress; | ||||
| 
 | ||||
|     /** | ||||
|      * If the member is currently leaving the group after a call to {@link #leaveGroup()}}, this | ||||
|  | @ -641,6 +641,9 @@ public class MembershipManagerImpl implements MembershipManager { | |||
|                     "the member is in FATAL state"); | ||||
|             return; | ||||
|         } | ||||
|         if (reconciliationInProgress) { | ||||
|             rejoinedWhileReconciliationInProgress = true; | ||||
|         } | ||||
|         resetEpoch(); | ||||
|         transitionTo(MemberState.JOINING); | ||||
|         clearPendingAssignmentsAndLocalNamesCache(); | ||||
|  | @ -958,7 +961,7 @@ public class MembershipManagerImpl implements MembershipManager { | |||
|         // best effort to commit the offsets in the case where the epoch might have changed while | ||||
|         // the current reconciliation is in process. Note this is using the rebalance timeout as | ||||
|         // it is the limit enforced by the broker to complete the reconciliation process. | ||||
|         commitResult = commitRequestManager.maybeAutoCommitSyncNow(getExpirationTimeForTimeout(rebalanceTimeoutMs)); | ||||
|         commitResult = commitRequestManager.maybeAutoCommitSyncBeforeRevocation(getExpirationTimeForTimeout(rebalanceTimeoutMs)); | ||||
| 
 | ||||
|         // Execute commit -> onPartitionsRevoked -> onPartitionsAssigned. | ||||
|         commitResult.whenComplete((__, commitReqError) -> { | ||||
|  | @ -972,7 +975,10 @@ public class MembershipManagerImpl implements MembershipManager { | |||
|                 log.debug("Auto-commit before reconciling new assignment completed successfully."); | ||||
|             } | ||||
| 
 | ||||
|             if (!maybeAbortReconciliation()) { | ||||
|                 revokeAndAssign(resolvedAssignment, assignedTopicIdPartitions, revokedPartitions, addedPartitions); | ||||
|             } | ||||
| 
 | ||||
|         }).exceptionally(error -> { | ||||
|             if (error != null) { | ||||
|                 log.error("Reconciliation failed.", error); | ||||
|  | @ -1010,33 +1016,23 @@ public class MembershipManagerImpl implements MembershipManager { | |||
|         // and assignment, executed sequentially). | ||||
|         CompletableFuture<Void> reconciliationResult = | ||||
|             revocationResult.thenCompose(__ -> { | ||||
|                 boolean memberHasRejoined = memberEpochOnReconciliationStart != memberEpoch; | ||||
|                 if (state == MemberState.RECONCILING && !memberHasRejoined) { | ||||
|                 if (!maybeAbortReconciliation()) { | ||||
|                     // Apply assignment | ||||
|                     return assignPartitions(assignedTopicIdPartitions, addedPartitions); | ||||
|                 } else { | ||||
|                     log.debug("Revocation callback completed but the member already " + | ||||
|                         "transitioned out of the reconciling state for epoch {} into " + | ||||
|                         "{} state with epoch {}. Interrupting reconciliation as it's " + | ||||
|                         "not relevant anymore,", memberEpochOnReconciliationStart, state, memberEpoch); | ||||
|                     String reason = interruptedReconciliationErrorMessage(); | ||||
|                     CompletableFuture<Void> res = new CompletableFuture<>(); | ||||
|                     res.completeExceptionally(new KafkaException("Interrupting reconciliation" + | ||||
|                         " after revocation. " + reason)); | ||||
|                     return res; | ||||
|                 } | ||||
|                 return CompletableFuture.completedFuture(null); | ||||
|             }); | ||||
| 
 | ||||
|         reconciliationResult.whenComplete((result, error) -> { | ||||
|             markReconciliationCompleted(); | ||||
|         reconciliationResult.whenComplete((__, error) -> { | ||||
|             if (error != null) { | ||||
|                 // Leaving member in RECONCILING state after callbacks fail. The member | ||||
|                 // won't send the ack, and the expectation is that the broker will kick the | ||||
|                 // member out of the group after the rebalance timeout expires, leading to a | ||||
|                 // RECONCILING -> FENCED transition. | ||||
|                 log.error("Reconciliation failed.", error); | ||||
|                 markReconciliationCompleted(); | ||||
|             } else { | ||||
|                 if (state == MemberState.RECONCILING) { | ||||
|                 if (reconciliationInProgress && !maybeAbortReconciliation()) { | ||||
|                     currentAssignment = resolvedAssignment; | ||||
| 
 | ||||
|                     // Reschedule the auto commit starting from now that the member has a new assignment. | ||||
|  | @ -1044,15 +1040,30 @@ public class MembershipManagerImpl implements MembershipManager { | |||
| 
 | ||||
|                     // Make assignment effective on the broker by transitioning to send acknowledge. | ||||
|                     transitionTo(MemberState.ACKNOWLEDGING); | ||||
|                 } else { | ||||
|                     String reason = interruptedReconciliationErrorMessage(); | ||||
|                     log.error("Interrupting reconciliation after partitions assigned callback " + | ||||
|                         "completed. " + reason); | ||||
|                     markReconciliationCompleted(); | ||||
|                 } | ||||
|             } | ||||
|         }); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * @return True if the reconciliation in progress should not continue. This could be because | ||||
|      * the member is not in RECONCILING state anymore (member failed or is leaving the group), or | ||||
|      * if it has rejoined the group (note that after rejoining the member could be RECONCILING | ||||
|      * again, so checking the state is not enough) | ||||
|      */ | ||||
|     boolean maybeAbortReconciliation() { | ||||
|         boolean shouldAbort = state != MemberState.RECONCILING || rejoinedWhileReconciliationInProgress; | ||||
|         if (shouldAbort) { | ||||
|             String reason = rejoinedWhileReconciliationInProgress ? | ||||
|                 "the member has re-joined the group" : | ||||
|                 "the member already transitioned out of the reconciling state into " + state; | ||||
|             log.info("Interrupting reconciliation that is not relevant anymore because " + reason); | ||||
|             markReconciliationCompleted(); | ||||
|         } | ||||
|         return shouldAbort; | ||||
|     } | ||||
| 
 | ||||
|     // Visible for testing. | ||||
|     void updateAssignment(Map<Uuid, SortedSet<Integer>> partitions) { | ||||
|         currentAssignment = new LocalAssignment(0, partitions); | ||||
|  | @ -1067,25 +1078,12 @@ public class MembershipManagerImpl implements MembershipManager { | |||
|         return result; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * @return Reason for interrupting a reconciliation progress when callbacks complete. | ||||
|      */ | ||||
|     private String interruptedReconciliationErrorMessage() { | ||||
|         String reason; | ||||
|         if (state != MemberState.RECONCILING) { | ||||
|             reason = "The member already transitioned out of the reconciling state into " + state; | ||||
|         } else { | ||||
|             reason = "The member has re-joined the group."; | ||||
|         } | ||||
|         return reason; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      *  Visible for testing. | ||||
|      */ | ||||
|     void markReconciliationInProgress() { | ||||
|         reconciliationInProgress = true; | ||||
|         memberEpochOnReconciliationStart = memberEpoch; | ||||
|         rejoinedWhileReconciliationInProgress = false; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|  | @ -1093,6 +1091,7 @@ public class MembershipManagerImpl implements MembershipManager { | |||
|      */ | ||||
|     void markReconciliationCompleted() { | ||||
|         reconciliationInProgress = false; | ||||
|         rejoinedWhileReconciliationInProgress = false; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|  |  | |||
|  | @ -270,13 +270,14 @@ public class Sender implements Runnable { | |||
|         while (!forceClose && transactionManager != null && transactionManager.hasOngoingTransaction()) { | ||||
|             if (!transactionManager.isCompleting()) { | ||||
|                 log.info("Aborting incomplete transaction due to shutdown"); | ||||
| 
 | ||||
|                 try { | ||||
|                     // It is possible for the transaction manager to throw errors when aborting. Catch these | ||||
|                     // so as not to interfere with the rest of the shutdown logic. | ||||
|                     transactionManager.beginAbort(); | ||||
|                 } catch (Exception e) { | ||||
|                     log.error("Error in kafka producer I/O thread while aborting transaction: ", e); | ||||
|                     log.error("Error in kafka producer I/O thread while aborting transaction when during closing: ", e); | ||||
|                     // Force close in case the transactionManager is in error states. | ||||
|                     forceClose = true; | ||||
|                 } | ||||
|             } | ||||
|             try { | ||||
|  |  | |||
|  | @ -1330,6 +1330,8 @@ public class TransactionManager { | |||
|                 // We could still receive INVALID_PRODUCER_EPOCH from old versioned transaction coordinator, | ||||
|                 // just treat it the same as PRODUCE_FENCED. | ||||
|                 fatalError(Errors.PRODUCER_FENCED.exception()); | ||||
|             } else if (error == Errors.TRANSACTION_ABORTABLE) { | ||||
|                 abortableError(error.exception()); | ||||
|             } else { | ||||
|                 fatalError(new KafkaException("Unexpected error in InitProducerIdResponse; " + error.message())); | ||||
|             } | ||||
|  | @ -1386,10 +1388,8 @@ public class TransactionManager { | |||
|                     // just treat it the same as PRODUCE_FENCED. | ||||
|                     fatalError(Errors.PRODUCER_FENCED.exception()); | ||||
|                     return; | ||||
|                 } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { | ||||
|                     fatalError(error.exception()); | ||||
|                     return; | ||||
|                 } else if (error == Errors.INVALID_TXN_STATE) { | ||||
|                 } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || | ||||
|                         error == Errors.INVALID_TXN_STATE) { | ||||
|                     fatalError(error.exception()); | ||||
|                     return; | ||||
|                 } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { | ||||
|  | @ -1401,6 +1401,9 @@ public class TransactionManager { | |||
|                 } else if (error == Errors.UNKNOWN_PRODUCER_ID || error == Errors.INVALID_PRODUCER_ID_MAPPING) { | ||||
|                     abortableErrorIfPossible(error.exception()); | ||||
|                     return; | ||||
|                 } else if (error == Errors.TRANSACTION_ABORTABLE) { | ||||
|                     abortableError(error.exception()); | ||||
|                     return; | ||||
|                 } else { | ||||
|                     log.error("Could not add partition {} due to unexpected error {}", topicPartition, error); | ||||
|                     hasPartitionErrors = true; | ||||
|  | @ -1504,6 +1507,8 @@ public class TransactionManager { | |||
|                 fatalError(error.exception()); | ||||
|             } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { | ||||
|                 abortableError(GroupAuthorizationException.forGroupId(key)); | ||||
|             } else if (error == Errors.TRANSACTION_ABORTABLE) { | ||||
|                 abortableError(error.exception()); | ||||
|             } else { | ||||
|                 fatalError(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to " + | ||||
|                         "unexpected error: %s", coordinatorType, key, | ||||
|  | @ -1552,12 +1557,13 @@ public class TransactionManager { | |||
|                 // We could still receive INVALID_PRODUCER_EPOCH from old versioned transaction coordinator, | ||||
|                 // just treat it the same as PRODUCE_FENCED. | ||||
|                 fatalError(Errors.PRODUCER_FENCED.exception()); | ||||
|             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { | ||||
|                 fatalError(error.exception()); | ||||
|             } else if (error == Errors.INVALID_TXN_STATE) { | ||||
|             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || | ||||
|                     error == Errors.INVALID_TXN_STATE) { | ||||
|                 fatalError(error.exception()); | ||||
|             } else if (error == Errors.UNKNOWN_PRODUCER_ID || error == Errors.INVALID_PRODUCER_ID_MAPPING) { | ||||
|                 abortableErrorIfPossible(error.exception()); | ||||
|             } else if (error == Errors.TRANSACTION_ABORTABLE) { | ||||
|                 abortableError(error.exception()); | ||||
|             } else { | ||||
|                 fatalError(new KafkaException("Unhandled error in EndTxnResponse: " + error.message())); | ||||
|             } | ||||
|  | @ -1611,12 +1617,13 @@ public class TransactionManager { | |||
|                 // We could still receive INVALID_PRODUCER_EPOCH from old versioned transaction coordinator, | ||||
|                 // just treat it the same as PRODUCE_FENCED. | ||||
|                 fatalError(Errors.PRODUCER_FENCED.exception()); | ||||
|             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { | ||||
|                 fatalError(error.exception()); | ||||
|             } else if (error == Errors.INVALID_TXN_STATE) { | ||||
|             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED || | ||||
|                     error == Errors.INVALID_TXN_STATE) { | ||||
|                 fatalError(error.exception()); | ||||
|             } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { | ||||
|                 abortableError(GroupAuthorizationException.forGroupId(builder.data.groupId())); | ||||
|             } else if (error == Errors.TRANSACTION_ABORTABLE) { | ||||
|                 abortableError(error.exception()); | ||||
|             } else { | ||||
|                 fatalError(new KafkaException("Unexpected error in AddOffsetsToTxnResponse: " + error.message())); | ||||
|             } | ||||
|  | @ -1679,7 +1686,8 @@ public class TransactionManager { | |||
|                 } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { | ||||
|                     abortableError(GroupAuthorizationException.forGroupId(builder.data.groupId())); | ||||
|                     break; | ||||
|                 } else if (error == Errors.FENCED_INSTANCE_ID) { | ||||
|                 } else if (error == Errors.FENCED_INSTANCE_ID || | ||||
|                         error == Errors.TRANSACTION_ABORTABLE) { | ||||
|                     abortableError(error.exception()); | ||||
|                     break; | ||||
|                 } else if (error == Errors.UNKNOWN_MEMBER_ID | ||||
|  |  | |||
|  | @ -25,6 +25,7 @@ import org.slf4j.Logger; | |||
| import org.slf4j.LoggerFactory; | ||||
| 
 | ||||
| import java.util.ArrayList; | ||||
| import java.util.Arrays; | ||||
| import java.util.Collections; | ||||
| import java.util.HashMap; | ||||
| import java.util.HashSet; | ||||
|  | @ -33,6 +34,8 @@ import java.util.Map; | |||
| import java.util.Set; | ||||
| import java.util.TreeMap; | ||||
| import java.util.concurrent.ConcurrentHashMap; | ||||
| import java.util.function.Predicate; | ||||
| import java.util.stream.Collectors; | ||||
| 
 | ||||
| /** | ||||
|  * A convenient base class for configurations to extend. | ||||
|  | @ -58,6 +61,8 @@ public class AbstractConfig { | |||
| 
 | ||||
|     private final ConfigDef definition; | ||||
| 
 | ||||
|     public static final String AUTOMATIC_CONFIG_PROVIDERS_PROPERTY = "org.apache.kafka.automatic.config.providers"; | ||||
| 
 | ||||
|     public static final String CONFIG_PROVIDERS_CONFIG = "config.providers"; | ||||
| 
 | ||||
|     private static final String CONFIG_PROVIDERS_PARAM = ".param."; | ||||
|  | @ -101,14 +106,11 @@ public class AbstractConfig { | |||
|      *                            the constructor to resolve any variables in {@code originals}; may be null or empty | ||||
|      * @param doLog               whether the configurations should be logged | ||||
|      */ | ||||
|     @SuppressWarnings({"unchecked", "this-escape"}) | ||||
|     @SuppressWarnings({"this-escape"}) | ||||
|     public AbstractConfig(ConfigDef definition, Map<?, ?> originals, Map<String, ?> configProviderProps, boolean doLog) { | ||||
|         /* check that all the keys are really strings */ | ||||
|         for (Map.Entry<?, ?> entry : originals.entrySet()) | ||||
|             if (!(entry.getKey() instanceof String)) | ||||
|                 throw new ConfigException(entry.getKey().toString(), entry.getValue(), "Key must be a string."); | ||||
|         Map<String, Object> originalMap = Utils.castToStringObjectMap(originals); | ||||
| 
 | ||||
|         this.originals = resolveConfigVariables(configProviderProps, (Map<String, Object>) originals); | ||||
|         this.originals = resolveConfigVariables(configProviderProps, originalMap); | ||||
|         this.values = definition.parse(this.originals); | ||||
|         Map<String, Object> configUpdates = postProcessParsedConfig(Collections.unmodifiableMap(this.values)); | ||||
|         for (Map.Entry<String, Object> update : configUpdates.entrySet()) { | ||||
|  | @ -521,6 +523,7 @@ public class AbstractConfig { | |||
|     private Map<String, ?> resolveConfigVariables(Map<String, ?> configProviderProps, Map<String, Object> originals) { | ||||
|         Map<String, String> providerConfigString; | ||||
|         Map<String, ?> configProperties; | ||||
|         Predicate<String> classNameFilter; | ||||
|         Map<String, Object> resolvedOriginals = new HashMap<>(); | ||||
|         // As variable configs are strings, parse the originals and obtain the potential variable configs. | ||||
|         Map<String, String> indirectVariables = extractPotentialVariables(originals); | ||||
|  | @ -529,11 +532,13 @@ public class AbstractConfig { | |||
|         if (configProviderProps == null || configProviderProps.isEmpty()) { | ||||
|             providerConfigString = indirectVariables; | ||||
|             configProperties = originals; | ||||
|             classNameFilter = automaticConfigProvidersFilter(); | ||||
|         } else { | ||||
|             providerConfigString = extractPotentialVariables(configProviderProps); | ||||
|             configProperties = configProviderProps; | ||||
|             classNameFilter = ignored -> true; | ||||
|         } | ||||
|         Map<String, ConfigProvider> providers = instantiateConfigProviders(providerConfigString, configProperties); | ||||
|         Map<String, ConfigProvider> providers = instantiateConfigProviders(providerConfigString, configProperties, classNameFilter); | ||||
| 
 | ||||
|         if (!providers.isEmpty()) { | ||||
|             ConfigTransformer configTransformer = new ConfigTransformer(providers); | ||||
|  | @ -547,6 +552,17 @@ public class AbstractConfig { | |||
|         return new ResolvingMap<>(resolvedOriginals, originals); | ||||
|     } | ||||
| 
 | ||||
|     private Predicate<String> automaticConfigProvidersFilter() { | ||||
|         String systemProperty = System.getProperty(AUTOMATIC_CONFIG_PROVIDERS_PROPERTY); | ||||
|         if (systemProperty == null) { | ||||
|             return ignored -> true; | ||||
|         } else { | ||||
|             return Arrays.stream(systemProperty.split(",")) | ||||
|                     .map(String::trim) | ||||
|                     .collect(Collectors.toSet())::contains; | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private Map<String, Object> configProviderProperties(String configProviderPrefix, Map<String, ?> providerConfigProperties) { | ||||
|         Map<String, Object> result = new HashMap<>(); | ||||
|         for (Map.Entry<String, ?> entry : providerConfigProperties.entrySet()) { | ||||
|  | @ -567,9 +583,14 @@ public class AbstractConfig { | |||
|      * | ||||
|      * @param indirectConfigs          The map of potential variable configs | ||||
|      * @param providerConfigProperties The map of config provider configs | ||||
|      * @return map map of config provider name and its instance. | ||||
|      * @param classNameFilter          Filter for config provider class names | ||||
|      * @return map of config provider name and its instance. | ||||
|      */ | ||||
|     private Map<String, ConfigProvider> instantiateConfigProviders(Map<String, String> indirectConfigs, Map<String, ?> providerConfigProperties) { | ||||
|     private Map<String, ConfigProvider> instantiateConfigProviders( | ||||
|             Map<String, String> indirectConfigs, | ||||
|             Map<String, ?> providerConfigProperties, | ||||
|             Predicate<String> classNameFilter | ||||
|     ) { | ||||
|         final String configProviders = indirectConfigs.get(CONFIG_PROVIDERS_CONFIG); | ||||
| 
 | ||||
|         if (configProviders == null || configProviders.isEmpty()) { | ||||
|  | @ -580,9 +601,15 @@ public class AbstractConfig { | |||
| 
 | ||||
|         for (String provider : configProviders.split(",")) { | ||||
|             String providerClass = providerClassProperty(provider); | ||||
|             if (indirectConfigs.containsKey(providerClass)) | ||||
|                 providerMap.put(provider, indirectConfigs.get(providerClass)); | ||||
| 
 | ||||
|             if (indirectConfigs.containsKey(providerClass)) { | ||||
|                 String providerClassName = indirectConfigs.get(providerClass); | ||||
|                 if (classNameFilter.test(providerClassName)) { | ||||
|                     providerMap.put(provider, providerClassName); | ||||
|                 } else { | ||||
|                     throw new ConfigException(providerClassName + " is not allowed. Update System property '" | ||||
|                             + AUTOMATIC_CONFIG_PROVIDERS_PROPERTY + "' to allow " + providerClassName); | ||||
|                 } | ||||
|             } | ||||
|         } | ||||
|         // Instantiate Config Providers | ||||
|         Map<String, ConfigProvider> configProviderInstances = new HashMap<>(); | ||||
|  |  | |||
|  | @ -16,7 +16,6 @@ | |||
|  */ | ||||
| package org.apache.kafka.common.config; | ||||
| 
 | ||||
| import org.apache.kafka.common.config.ConfigDef.Type; | ||||
| import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; | ||||
| import org.apache.kafka.common.utils.Java; | ||||
| import org.apache.kafka.common.utils.Utils; | ||||
|  | @ -144,7 +143,7 @@ public class SslConfigs { | |||
|                 .define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, ConfigDef.Type.STRING, null,  ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_LOCATION_DOC) | ||||
|                 .define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC) | ||||
|                 .define(SslConfigs.SSL_KEY_PASSWORD_CONFIG, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEY_PASSWORD_DOC) | ||||
|                 .define(SslConfigs.SSL_KEYSTORE_KEY_CONFIG, Type.PASSWORD, null,  ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_KEY_DOC) | ||||
|                 .define(SslConfigs.SSL_KEYSTORE_KEY_CONFIG, ConfigDef.Type.PASSWORD, null,  ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_KEY_DOC) | ||||
|                 .define(SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, ConfigDef.Type.PASSWORD, null,  ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_DOC) | ||||
|                 .define(SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG, ConfigDef.Type.PASSWORD, null,  ConfigDef.Importance.HIGH, SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_DOC) | ||||
|                 .define(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_TRUSTSTORE_TYPE_DOC) | ||||
|  |  | |||
|  | @ -67,13 +67,17 @@ public class TopicConfig { | |||
|         "(which consists of log segments) can grow to before we will discard old log segments to free up space if we " + | ||||
|         "are using the \"delete\" retention policy. By default there is no size limit only a time limit. " + | ||||
|         "Since this limit is enforced at the partition level, multiply it by the number of partitions to compute " + | ||||
|         "the topic retention in bytes."; | ||||
|         "the topic retention in bytes. Additionally, retention.bytes configuration " + | ||||
|         "operates independently of \"segment.ms\" and \"segment.bytes\" configurations. " + | ||||
|         "Moreover, it triggers the rolling of new segment if the retention.bytes is configured to zero."; | ||||
| 
 | ||||
|     public static final String RETENTION_MS_CONFIG = "retention.ms"; | ||||
|     public static final String RETENTION_MS_DOC = "This configuration controls the maximum time we will retain a " + | ||||
|         "log before we will discard old log segments to free up space if we are using the " + | ||||
|         "\"delete\" retention policy. This represents an SLA on how soon consumers must read " + | ||||
|         "their data. If set to -1, no time limit is applied."; | ||||
|         "their data. If set to -1, no time limit is applied. Additionally, retention.ms configuration " + | ||||
|         "operates independently of \"segment.ms\" and \"segment.bytes\" configurations. " + | ||||
|         "Moreover, it triggers the rolling of new segment if the retention.ms condition is satisfied."; | ||||
| 
 | ||||
|     public static final String REMOTE_LOG_STORAGE_ENABLE_CONFIG = "remote.storage.enable"; | ||||
|     public static final String REMOTE_LOG_STORAGE_ENABLE_DOC = "To enable tiered storage for a topic, set this configuration as true. " + | ||||
|  |  | |||
|  | @ -14,8 +14,10 @@ | |||
|  * See the License for the specific language governing permissions and | ||||
|  * limitations under the License. | ||||
|  */ | ||||
| package org.apache.kafka.server.config; | ||||
| package org.apache.kafka.common.errors; | ||||
| 
 | ||||
| public class ConfigEntityName { | ||||
|     public static final String DEFAULT = "<default>"; | ||||
| public class TransactionAbortableException extends ApiException { | ||||
|     public TransactionAbortableException(String message) { | ||||
|         super(message); | ||||
|     } | ||||
| } | ||||
|  | @ -26,9 +26,17 @@ public final class KafkaMetric implements Metric { | |||
|     private final Object lock; | ||||
|     private final Time time; | ||||
|     private final MetricValueProvider<?> metricValueProvider; | ||||
|     private MetricConfig config; | ||||
|     private volatile MetricConfig config; | ||||
| 
 | ||||
|     // public for testing | ||||
|     /** | ||||
|      * Create a metric to monitor an object that implements MetricValueProvider. | ||||
|      * @param lock The lock used to prevent race condition | ||||
|      * @param metricName The name of the metric | ||||
|      * @param valueProvider The metric value provider associated with this metric | ||||
|      * @param config The configuration of the metric | ||||
|      * @param time The time instance to use with the metrics | ||||
|      */ | ||||
|     public KafkaMetric(Object lock, MetricName metricName, MetricValueProvider<?> valueProvider, | ||||
|             MetricConfig config, Time time) { | ||||
|         this.metricName = metricName; | ||||
|  | @ -40,15 +48,29 @@ public final class KafkaMetric implements Metric { | |||
|         this.time = time; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Get the configuration of this metric. | ||||
|      * This is supposed to be used by server only. | ||||
|      * @return Return the config of this metric | ||||
|      */ | ||||
|     public MetricConfig config() { | ||||
|         return this.config; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Get the metric name | ||||
|      * @return Return the name of this metric | ||||
|      */ | ||||
|     @Override | ||||
|     public MetricName metricName() { | ||||
|         return this.metricName; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Take the metric and return the value, which could be a {@link Measurable} or a {@link Gauge} | ||||
|      * @return Return the metric value | ||||
|      * @throws IllegalStateException if the underlying metric is not a {@link Measurable} or a {@link Gauge}. | ||||
|      */ | ||||
|     @Override | ||||
|     public Object metricValue() { | ||||
|         long now = time.milliseconds(); | ||||
|  | @ -62,6 +84,11 @@ public final class KafkaMetric implements Metric { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Get the underlying metric provider, which should be a {@link Measurable} | ||||
|      * @return Return the metric provider | ||||
|      * @throws IllegalStateException if the underlying metric is not a {@link Measurable}. | ||||
|      */ | ||||
|     public Measurable measurable() { | ||||
|         if (this.metricValueProvider instanceof Measurable) | ||||
|             return (Measurable) metricValueProvider; | ||||
|  | @ -69,6 +96,11 @@ public final class KafkaMetric implements Metric { | |||
|             throw new IllegalStateException("Not a measurable: " + this.metricValueProvider.getClass()); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Take the metric and return the value, where the underlying metric provider should be a {@link Measurable} | ||||
|      * @param timeMs The time that this metric is taken | ||||
|      * @return Return the metric value if it's measurable, otherwise 0 | ||||
|      */ | ||||
|     double measurableValue(long timeMs) { | ||||
|         synchronized (this.lock) { | ||||
|             if (this.metricValueProvider instanceof Measurable) | ||||
|  | @ -78,6 +110,11 @@ public final class KafkaMetric implements Metric { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Set the metric config. | ||||
|      * This is supposed to be used by server only. | ||||
|      * @param config configuration for this metrics | ||||
|      */ | ||||
|     public void config(MetricConfig config) { | ||||
|         synchronized (lock) { | ||||
|             this.config = config; | ||||
|  |  | |||
|  | @ -138,6 +138,7 @@ import org.apache.kafka.common.errors.UnsupportedEndpointTypeException; | |||
| import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; | ||||
| import org.apache.kafka.common.errors.UnsupportedSaslMechanismException; | ||||
| import org.apache.kafka.common.errors.UnsupportedVersionException; | ||||
| import org.apache.kafka.common.errors.TransactionAbortableException; | ||||
| import org.slf4j.Logger; | ||||
| import org.slf4j.LoggerFactory; | ||||
| 
 | ||||
|  | @ -392,7 +393,8 @@ public enum Errors { | |||
|     UNKNOWN_CONTROLLER_ID(116, "This controller ID is not known.", UnknownControllerIdException::new), | ||||
|     UNKNOWN_SUBSCRIPTION_ID(117, "Client sent a push telemetry request with an invalid or outdated subscription ID.", UnknownSubscriptionIdException::new), | ||||
|     TELEMETRY_TOO_LARGE(118, "Client sent a push telemetry request larger than the maximum size the broker will accept.", TelemetryTooLargeException::new), | ||||
|     INVALID_REGISTRATION(119, "The controller has considered the broker registration to be invalid.", InvalidRegistrationException::new); | ||||
|     INVALID_REGISTRATION(119, "The controller has considered the broker registration to be invalid.", InvalidRegistrationException::new), | ||||
|     TRANSACTION_ABORTABLE(120, "The server encountered an error with the transaction. The client can abort the transaction to continue using this transactional ID.", TransactionAbortableException::new); | ||||
| 
 | ||||
|     private static final Logger log = LoggerFactory.getLogger(Errors.class); | ||||
| 
 | ||||
|  |  | |||
|  | @ -209,7 +209,7 @@ public class MemoryRecords extends AbstractRecords { | |||
|                                 partition, batch.lastOffset(), maxRecordBatchSize, filteredBatchSize); | ||||
| 
 | ||||
|                         MemoryRecordsBuilder.RecordsInfo info = builder.info(); | ||||
|                         filterResult.updateRetainedBatchMetadata(info.maxTimestamp, info.shallowOffsetOfMaxTimestamp, | ||||
|                         filterResult.updateRetainedBatchMetadata(info.maxTimestamp, info.offsetOfMaxTimestamp, | ||||
|                             maxOffset, retainedRecords.size(), filteredBatchSize); | ||||
|                     } | ||||
|                 } | ||||
|  | @ -399,7 +399,7 @@ public class MemoryRecords extends AbstractRecords { | |||
|         private int bytesRetained = 0; | ||||
|         private long maxOffset = -1L; | ||||
|         private long maxTimestamp = RecordBatch.NO_TIMESTAMP; | ||||
|         private long shallowOffsetOfMaxTimestamp = -1L; | ||||
|         private long offsetOfMaxTimestamp = -1L; | ||||
| 
 | ||||
|         private FilterResult(ByteBuffer outputBuffer) { | ||||
|             this.outputBuffer = outputBuffer; | ||||
|  | @ -411,21 +411,21 @@ public class MemoryRecords extends AbstractRecords { | |||
|                     retainedBatch.lastOffset(), numMessagesInBatch, bytesRetained); | ||||
|         } | ||||
| 
 | ||||
|         private void updateRetainedBatchMetadata(long maxTimestamp, long shallowOffsetOfMaxTimestamp, long maxOffset, | ||||
|         private void updateRetainedBatchMetadata(long maxTimestamp, long offsetOfMaxTimestamp, long maxOffset, | ||||
|                                                  int messagesRetained, int bytesRetained) { | ||||
|             validateBatchMetadata(maxTimestamp, shallowOffsetOfMaxTimestamp, maxOffset); | ||||
|             validateBatchMetadata(maxTimestamp, offsetOfMaxTimestamp, maxOffset); | ||||
|             if (maxTimestamp > this.maxTimestamp) { | ||||
|                 this.maxTimestamp = maxTimestamp; | ||||
|                 this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; | ||||
|                 this.offsetOfMaxTimestamp = offsetOfMaxTimestamp; | ||||
|             } | ||||
|             this.maxOffset = Math.max(maxOffset, this.maxOffset); | ||||
|             this.messagesRetained += messagesRetained; | ||||
|             this.bytesRetained += bytesRetained; | ||||
|         } | ||||
| 
 | ||||
|         private void validateBatchMetadata(long maxTimestamp, long shallowOffsetOfMaxTimestamp, long maxOffset) { | ||||
|             if (maxTimestamp != RecordBatch.NO_TIMESTAMP && shallowOffsetOfMaxTimestamp < 0) | ||||
|                 throw new IllegalArgumentException("shallowOffset undefined for maximum timestamp " + maxTimestamp); | ||||
|         private void validateBatchMetadata(long maxTimestamp, long offsetOfMaxTimestamp, long maxOffset) { | ||||
|             if (maxTimestamp != RecordBatch.NO_TIMESTAMP && offsetOfMaxTimestamp < 0) | ||||
|                 throw new IllegalArgumentException("offset undefined for maximum timestamp " + maxTimestamp); | ||||
|             if (maxOffset < 0) | ||||
|                 throw new IllegalArgumentException("maxOffset undefined"); | ||||
|         } | ||||
|  | @ -458,8 +458,8 @@ public class MemoryRecords extends AbstractRecords { | |||
|             return maxTimestamp; | ||||
|         } | ||||
| 
 | ||||
|         public long shallowOffsetOfMaxTimestamp() { | ||||
|             return shallowOffsetOfMaxTimestamp; | ||||
|         public long offsetOfMaxTimestamp() { | ||||
|             return offsetOfMaxTimestamp; | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -851,12 +851,12 @@ public class MemoryRecordsBuilder implements AutoCloseable { | |||
| 
 | ||||
|     public static class RecordsInfo { | ||||
|         public final long maxTimestamp; | ||||
|         public final long shallowOffsetOfMaxTimestamp; | ||||
|         public final long offsetOfMaxTimestamp; | ||||
| 
 | ||||
|         public RecordsInfo(long maxTimestamp, | ||||
|                            long shallowOffsetOfMaxTimestamp) { | ||||
|                            long offsetOfMaxTimestamp) { | ||||
|             this.maxTimestamp = maxTimestamp; | ||||
|             this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp; | ||||
|             this.offsetOfMaxTimestamp = offsetOfMaxTimestamp; | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -166,13 +166,6 @@ public class ApiVersionsResponse extends AbstractResponse { | |||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     public static ApiVersionCollection filterApis( | ||||
|         RecordVersion minRecordVersion, | ||||
|         ApiMessageType.ListenerType listenerType | ||||
|     ) { | ||||
|         return filterApis(minRecordVersion, listenerType, false, false); | ||||
|     } | ||||
| 
 | ||||
|     public static ApiVersionCollection filterApis( | ||||
|         RecordVersion minRecordVersion, | ||||
|         ApiMessageType.ListenerType listenerType, | ||||
|  |  | |||
|  | @ -29,8 +29,6 @@ import org.apache.kafka.common.utils.ByteBufferOutputStream; | |||
| import org.apache.kafka.common.utils.Utils; | ||||
| import org.slf4j.Logger; | ||||
| import org.slf4j.LoggerFactory; | ||||
| 
 | ||||
| import java.io.ByteArrayOutputStream; | ||||
| import java.io.IOException; | ||||
| import java.io.InputStream; | ||||
| import java.io.OutputStream; | ||||
|  | @ -204,16 +202,14 @@ public class ClientTelemetryUtils { | |||
|     public static ByteBuffer decompress(byte[] metrics, CompressionType compressionType) { | ||||
|         ByteBuffer data = ByteBuffer.wrap(metrics); | ||||
|         try (InputStream in = compressionType.wrapForInput(data, RecordBatch.CURRENT_MAGIC_VALUE, BufferSupplier.create()); | ||||
|             ByteArrayOutputStream out = new ByteArrayOutputStream()) { | ||||
| 
 | ||||
|             ByteBufferOutputStream out = new ByteBufferOutputStream(512)) { | ||||
|             byte[] bytes = new byte[data.capacity() * 2]; | ||||
|             int nRead; | ||||
|             while ((nRead = in.read(bytes, 0, bytes.length)) != -1) { | ||||
|                 out.write(bytes, 0, nRead); | ||||
|             } | ||||
| 
 | ||||
|             out.flush(); | ||||
|             return ByteBuffer.wrap(out.toByteArray()); | ||||
|             out.buffer().flip(); | ||||
|             return out.buffer(); | ||||
|         } catch (IOException e) { | ||||
|             throw new KafkaException("Failed to decompress metrics data", e); | ||||
|         } | ||||
|  |  | |||
|  | @ -43,6 +43,14 @@ public class Exit { | |||
|             Runtime.getRuntime().addShutdownHook(new Thread(runnable)); | ||||
|     }; | ||||
| 
 | ||||
|     private static final Procedure NOOP_HALT_PROCEDURE = (statusCode, message) -> { | ||||
|         throw new IllegalStateException("Halt called after resetting procedures; possible race condition present in test"); | ||||
|     }; | ||||
| 
 | ||||
|     private static final Procedure NOOP_EXIT_PROCEDURE = (statusCode, message) -> { | ||||
|         throw new IllegalStateException("Exit called after resetting procedures; possible race condition present in test"); | ||||
|     }; | ||||
| 
 | ||||
|     private volatile static Procedure exitProcedure = DEFAULT_EXIT_PROCEDURE; | ||||
|     private volatile static Procedure haltProcedure = DEFAULT_HALT_PROCEDURE; | ||||
|     private volatile static ShutdownHookAdder shutdownHookAdder = DEFAULT_SHUTDOWN_HOOK_ADDER; | ||||
|  | @ -67,26 +75,47 @@ public class Exit { | |||
|         shutdownHookAdder.addShutdownHook(name, runnable); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * For testing only, do not call in main code. | ||||
|      */ | ||||
|     public static void setExitProcedure(Procedure procedure) { | ||||
|         exitProcedure = procedure; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * For testing only, do not call in main code. | ||||
|      */ | ||||
|     public static void setHaltProcedure(Procedure procedure) { | ||||
|         haltProcedure = procedure; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * For testing only, do not call in main code. | ||||
|      */ | ||||
|     public static void setShutdownHookAdder(ShutdownHookAdder shutdownHookAdder) { | ||||
|         Exit.shutdownHookAdder = shutdownHookAdder; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * For testing only, do not call in main code. | ||||
|      * <p>Clears the procedure set in {@link #setExitProcedure(Procedure)}, but does not restore system default behavior of exiting the JVM. | ||||
|      */ | ||||
|     public static void resetExitProcedure() { | ||||
|         exitProcedure = DEFAULT_EXIT_PROCEDURE; | ||||
|         exitProcedure = NOOP_EXIT_PROCEDURE; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * For testing only, do not call in main code. | ||||
|      * <p>Clears the procedure set in {@link #setHaltProcedure(Procedure)}, but does not restore system default behavior of exiting the JVM. | ||||
|      */ | ||||
|     public static void resetHaltProcedure() { | ||||
|         haltProcedure = DEFAULT_HALT_PROCEDURE; | ||||
|         haltProcedure = NOOP_HALT_PROCEDURE; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * For testing only, do not call in main code. | ||||
|      * <p>Restores the system default shutdown hook behavior. | ||||
|      */ | ||||
|     public static void resetShutdownHookAdder() { | ||||
|         shutdownHookAdder = DEFAULT_SHUTDOWN_HOOK_ADDER; | ||||
|     } | ||||
|  |  | |||
|  | @ -1502,13 +1502,23 @@ public final class Utils { | |||
|      * @return a map including all elements in properties | ||||
|      */ | ||||
|     public static Map<String, Object> propsToMap(Properties properties) { | ||||
|         Map<String, Object> map = new HashMap<>(properties.size()); | ||||
|         for (Map.Entry<Object, Object> entry : properties.entrySet()) { | ||||
|         return castToStringObjectMap(properties); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Cast a map with arbitrary type keys to be keyed on String. | ||||
|      * @param inputMap A map with unknown type keys | ||||
|      * @return A map with the same contents as the input map, but with String keys | ||||
|      * @throws ConfigException if any key is not a String | ||||
|      */ | ||||
|     public static Map<String, Object> castToStringObjectMap(Map<?, ?> inputMap) { | ||||
|         Map<String, Object> map = new HashMap<>(inputMap.size()); | ||||
|         for (Map.Entry<?, ?> entry : inputMap.entrySet()) { | ||||
|             if (entry.getKey() instanceof String) { | ||||
|                 String k = (String) entry.getKey(); | ||||
|                 map.put(k, properties.get(k)); | ||||
|                 map.put(k, entry.getValue()); | ||||
|             } else { | ||||
|                 throw new ConfigException(entry.getKey().toString(), entry.getValue(), "Key must be a string."); | ||||
|                 throw new ConfigException(String.valueOf(entry.getKey()), entry.getValue(), "Key must be a string."); | ||||
|             } | ||||
|         } | ||||
|         return map; | ||||
|  | @ -1675,6 +1685,15 @@ public final class Utils { | |||
|         return result; | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Checks requirement. Throw {@link IllegalArgumentException} if {@code requirement} failed. | ||||
|      * @param requirement Requirement to check. | ||||
|      */ | ||||
|     public static void require(boolean requirement) { | ||||
|         if (!requirement) | ||||
|             throw new IllegalArgumentException("requirement failed"); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * A runnable that can throw checked exception. | ||||
|      */ | ||||
|  |  | |||
|  | @ -23,7 +23,9 @@ | |||
|   // Version 2 adds the support for new error code PRODUCER_FENCED. | ||||
|   // | ||||
|   // Version 3 enables flexible versions. | ||||
|   "validVersions": "0-3", | ||||
|   // | ||||
|   // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-4", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", | ||||
|  |  | |||
|  | @ -22,7 +22,9 @@ | |||
|   // Version 2 adds the support for new error code PRODUCER_FENCED. | ||||
|   // | ||||
|   // Version 3 enables flexible versions. | ||||
|   "validVersions": "0-3", | ||||
|   // | ||||
|   // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-4", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", | ||||
|  |  | |||
|  | @ -25,9 +25,11 @@ | |||
|   // Version 3 enables flexible versions. | ||||
|   // | ||||
|   // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions. | ||||
|   // | ||||
|   // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   // Versions 3 and below will be exclusively used by clients and versions 4 and above will be used by brokers. | ||||
|   "latestVersionUnstable": false, | ||||
|   "validVersions": "0-4", | ||||
|   "validVersions": "0-5", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "Transactions", "type": "[]AddPartitionsToTxnTransaction", "versions":  "4+", | ||||
|  |  | |||
|  | @ -24,7 +24,9 @@ | |||
|   // Version 3 enables flexible versions. | ||||
|   // | ||||
|   // Version 4 adds support to batch multiple transactions and a top level error code. | ||||
|   "validVersions": "0-4", | ||||
|   // | ||||
|   // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-5", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", | ||||
|  |  | |||
|  | @ -23,7 +23,9 @@ | |||
|   // Version 2 adds the support for new error code PRODUCER_FENCED. | ||||
|   // | ||||
|   // Version 3 enables flexible versions. | ||||
|   "validVersions": "0-3", | ||||
|   // | ||||
|   // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-4", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", | ||||
|  |  | |||
|  | @ -22,7 +22,9 @@ | |||
|   // Version 2 adds the support for new error code PRODUCER_FENCED. | ||||
|   // | ||||
|   // Version 3 enables flexible versions. | ||||
|   "validVersions": "0-3", | ||||
|   // | ||||
|   // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-4", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", | ||||
|  |  | |||
|  | @ -25,7 +25,9 @@ | |||
|   // Version 3 is the first flexible version. | ||||
|   // | ||||
|   // Version 4 adds support for batching via CoordinatorKeys (KIP-699) | ||||
|   "validVersions": "0-4", | ||||
|   // | ||||
|   // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-5", | ||||
|   "deprecatedVersions": "0", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|  |  | |||
|  | @ -24,7 +24,9 @@ | |||
|   // Version 3 is the first flexible version. | ||||
|   // | ||||
|   // Version 4 adds support for batching via Coordinators (KIP-699) | ||||
|   "validVersions": "0-4", | ||||
|   // | ||||
|   // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-5", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "ThrottleTimeMs", "type": "int32", "versions": "1+", "ignorable": true, | ||||
|  |  | |||
|  | @ -25,7 +25,9 @@ | |||
|   // Version 3 adds ProducerId and ProducerEpoch, allowing producers to try to resume after an INVALID_PRODUCER_EPOCH error | ||||
|   // | ||||
|   // Version 4 adds the support for new error code PRODUCER_FENCED. | ||||
|   "validVersions": "0-4", | ||||
|   // | ||||
|   // Verison 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-5", | ||||
|   "flexibleVersions": "2+", | ||||
|   "fields": [ | ||||
|     { "name": "TransactionalId", "type": "string", "versions": "0+", "nullableVersions": "0+", "entityType": "transactionalId", | ||||
|  |  | |||
|  | @ -24,7 +24,9 @@ | |||
|   // Version 3 is the same as version 2. | ||||
|   // | ||||
|   // Version 4 adds the support for new error code PRODUCER_FENCED. | ||||
|   "validVersions": "0-4", | ||||
|   // | ||||
|   // Version 5 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-5", | ||||
|   "flexibleVersions": "2+", | ||||
|   "fields": [ | ||||
|     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, | ||||
|  |  | |||
|  | @ -35,7 +35,9 @@ | |||
|   // Version 9 enables flexible versions. | ||||
|   // | ||||
|   // Version 10 is the same as version 9 (KIP-951). | ||||
|   "validVersions": "0-10", | ||||
|   // | ||||
|   // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-11", | ||||
|   "deprecatedVersions": "0-6", | ||||
|   "flexibleVersions": "9+", | ||||
|   "fields": [ | ||||
|  |  | |||
|  | @ -34,7 +34,9 @@ | |||
|   // Version 9 enables flexible versions. | ||||
|   // | ||||
|   // Version 10 adds 'CurrentLeader' and 'NodeEndpoints' as tagged fields (KIP-951) | ||||
|   "validVersions": "0-10", | ||||
|   // | ||||
|   // Version 11 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-11", | ||||
|   "flexibleVersions": "9+", | ||||
|   "fields": [ | ||||
|     { "name": "Responses", "type": "[]TopicProduceResponse", "versions": "0+", | ||||
|  |  | |||
|  | @ -23,7 +23,9 @@ | |||
|   // Version 2 adds the committed leader epoch. | ||||
|   // | ||||
|   // Version 3 adds the member.id, group.instance.id and generation.id. | ||||
|   "validVersions": "0-3", | ||||
|   // | ||||
|   // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-4", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId", | ||||
|  |  | |||
|  | @ -22,7 +22,9 @@ | |||
|   // Version 2 is the same as version 1. | ||||
|   // | ||||
|   // Version 3 adds illegal generation, fenced instance id, and unknown member id errors. | ||||
|   "validVersions": "0-3", | ||||
|   // | ||||
|   // Version 4 adds support for new error code TRANSACTION_ABORTABLE (KIP-890). | ||||
|   "validVersions": "0-4", | ||||
|   "flexibleVersions": "3+", | ||||
|   "fields": [ | ||||
|     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", | ||||
|  |  | |||
|  | @ -35,6 +35,7 @@ import java.util.Map; | |||
| import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; | ||||
| import static org.apache.kafka.common.config.ConfigDef.ValidString.in; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
|  | @ -136,7 +137,7 @@ public class CommonClientConfigsTest { | |||
|         TestConfig config = new TestConfig(Collections.emptyMap()); | ||||
|         List<MetricsReporter> reporters = CommonClientConfigs.metricsReporters("clientId", config); | ||||
|         assertEquals(1, reporters.size()); | ||||
|         assertTrue(reporters.get(0) instanceof JmxReporter); | ||||
|         assertInstanceOf(JmxReporter.class, reporters.get(0)); | ||||
| 
 | ||||
|         config = new TestConfig(Collections.singletonMap(CommonClientConfigs.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false")); | ||||
|         reporters = CommonClientConfigs.metricsReporters("clientId", config); | ||||
|  | @ -145,7 +146,7 @@ public class CommonClientConfigsTest { | |||
|         config = new TestConfig(Collections.singletonMap(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, JmxReporter.class.getName())); | ||||
|         reporters = CommonClientConfigs.metricsReporters("clientId", config); | ||||
|         assertEquals(1, reporters.size()); | ||||
|         assertTrue(reporters.get(0) instanceof JmxReporter); | ||||
|         assertInstanceOf(JmxReporter.class, reporters.get(0)); | ||||
| 
 | ||||
|         Map<String, String> props = new HashMap<>(); | ||||
|         props.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, JmxReporter.class.getName() + "," + MyJmxReporter.class.getName()); | ||||
|  |  | |||
|  | @ -271,6 +271,7 @@ import static org.apache.kafka.common.message.ListPartitionReassignmentsResponse | |||
| import static org.apache.kafka.common.message.ListPartitionReassignmentsResponseData.OngoingTopicReassignment; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
|  | @ -294,7 +295,7 @@ public class KafkaAdminClientTest { | |||
|         final AdminClientConfig config = newConfMap(AdminClientConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, "500"); | ||||
|         KafkaException exception = assertThrows(KafkaException.class, | ||||
|             () -> KafkaAdminClient.createInternal(config, null)); | ||||
|         assertTrue(exception.getCause() instanceof ConfigException); | ||||
|         assertInstanceOf(ConfigException.class, exception.getCause()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -499,7 +500,7 @@ public class KafkaAdminClientTest { | |||
|         ExecutionException e = assertThrows(ExecutionException.class, () -> env.adminClient().createTopics( | ||||
|                 singleton(new NewTopic("myTopic", Collections.singletonMap(0, asList(0, 1, 2)))), | ||||
|                 new CreateTopicsOptions().timeoutMs(10000)).all().get()); | ||||
|         assertTrue(e.getCause() instanceof IllegalStateException, | ||||
|         assertInstanceOf(IllegalStateException.class, e.getCause(), | ||||
|                 "Expected an IllegalStateException error, but got " + Utils.stackTrace(e)); | ||||
|     } | ||||
| 
 | ||||
|  | @ -711,7 +712,7 @@ public class KafkaAdminClientTest { | |||
|         if (error == Errors.NONE) { | ||||
|             return ApiVersionsResponse.createApiVersionsResponse( | ||||
|                 0, | ||||
|                 ApiVersionsResponse.filterApis(RecordVersion.current(), ApiMessageType.ListenerType.ZK_BROKER), | ||||
|                 ApiVersionsResponse.filterApis(RecordVersion.current(), ApiMessageType.ListenerType.ZK_BROKER, false, false), | ||||
|                 convertSupportedFeaturesMap(defaultFeatureMetadata().supportedFeatures()), | ||||
|                 Collections.singletonMap("test_feature_1", (short) 2), | ||||
|                 defaultFeatureMetadata().finalizedFeaturesEpoch().get(), | ||||
|  | @ -1456,38 +1457,38 @@ public class KafkaAdminClientTest { | |||
|         ExecutionException e = assertThrows(ExecutionException.class, () -> env.adminClient().createTopics( | ||||
|             singleton(new NewTopic("myTopic", Collections.singletonMap(0, asList(0, 1, 2)))), | ||||
|             new CreateTopicsOptions().timeoutMs(10000)).all().get()); | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
| 
 | ||||
|         Map<String, NewPartitions> counts = new HashMap<>(); | ||||
|         counts.put("my_topic", NewPartitions.increaseTo(3)); | ||||
|         counts.put("other_topic", NewPartitions.increaseTo(3, asList(asList(2), asList(3)))); | ||||
|         e = assertThrows(ExecutionException.class, () -> env.adminClient().createPartitions(counts).all().get()); | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
| 
 | ||||
|         e = assertThrows(ExecutionException.class, () -> env.adminClient().createAcls(asList(ACL1, ACL2)).all().get()); | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
| 
 | ||||
|         e = assertThrows(ExecutionException.class, () -> env.adminClient().describeAcls(FILTER1).values().get()); | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
| 
 | ||||
|         e = assertThrows(ExecutionException.class, () -> env.adminClient().deleteAcls(asList(FILTER1, FILTER2)).all().get()); | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
| 
 | ||||
|         e = assertThrows(ExecutionException.class, () -> env.adminClient().describeConfigs( | ||||
|             singleton(new ConfigResource(ConfigResource.Type.BROKER, "0"))).all().get()); | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
|     } | ||||
| 
 | ||||
|     private void callClientQuotasApisAndExpectAnAuthenticationError(AdminClientUnitTestEnv env) { | ||||
|         ExecutionException e = assertThrows(ExecutionException.class, | ||||
|             () -> env.adminClient().describeClientQuotas(ClientQuotaFilter.all()).entities().get()); | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
| 
 | ||||
|         ClientQuotaEntity entity = new ClientQuotaEntity(Collections.singletonMap(ClientQuotaEntity.USER, "user")); | ||||
|  | @ -1495,7 +1496,7 @@ public class KafkaAdminClientTest { | |||
|         e = assertThrows(ExecutionException.class, | ||||
|             () -> env.adminClient().alterClientQuotas(asList(alteration)).all().get()); | ||||
| 
 | ||||
|         assertTrue(e.getCause() instanceof AuthenticationException, | ||||
|         assertInstanceOf(AuthenticationException.class, e.getCause(), | ||||
|             "Expected an authentication error, but got " + Utils.stackTrace(e)); | ||||
|     } | ||||
| 
 | ||||
|  | @ -1877,7 +1878,7 @@ public class KafkaAdminClientTest { | |||
|                     env.cluster().nodeById(0)); | ||||
|             final DescribeLogDirsResult errorResult = env.adminClient().describeLogDirs(brokers); | ||||
|             ExecutionException exception = assertThrows(ExecutionException.class, () -> errorResult.allDescriptions().get()); | ||||
|             assertTrue(exception.getCause() instanceof ClusterAuthorizationException); | ||||
|             assertInstanceOf(ClusterAuthorizationException.class, exception.getCause()); | ||||
| 
 | ||||
|             // Empty results with an error with version >= 3 | ||||
|             env.kafkaClient().prepareResponseFrom( | ||||
|  | @ -1885,7 +1886,7 @@ public class KafkaAdminClientTest { | |||
|                     env.cluster().nodeById(0)); | ||||
|             final DescribeLogDirsResult errorResult2 = env.adminClient().describeLogDirs(brokers); | ||||
|             exception = assertThrows(ExecutionException.class, () -> errorResult2.allDescriptions().get()); | ||||
|             assertTrue(exception.getCause() instanceof UnknownServerException); | ||||
|             assertInstanceOf(UnknownServerException.class, exception.getCause()); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  | @ -1941,7 +1942,7 @@ public class KafkaAdminClientTest { | |||
|                     env.cluster().nodeById(0)); | ||||
|             final DescribeLogDirsResult errorResult = env.adminClient().describeLogDirs(brokers); | ||||
|             ExecutionException exception = assertThrows(ExecutionException.class, () -> errorResult.allDescriptions().get()); | ||||
|             assertTrue(exception.getCause() instanceof ClusterAuthorizationException); | ||||
|             assertInstanceOf(ClusterAuthorizationException.class, exception.getCause()); | ||||
| 
 | ||||
|             // Empty results with an error with version >= 3 | ||||
|             env.kafkaClient().prepareResponseFrom( | ||||
|  | @ -1949,7 +1950,7 @@ public class KafkaAdminClientTest { | |||
|                     env.cluster().nodeById(0)); | ||||
|             final DescribeLogDirsResult errorResult2 = env.adminClient().describeLogDirs(brokers); | ||||
|             exception = assertThrows(ExecutionException.class, () -> errorResult2.allDescriptions().get()); | ||||
|             assertTrue(exception.getCause() instanceof UnknownServerException); | ||||
|             assertInstanceOf(UnknownServerException.class, exception.getCause()); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  | @ -2168,14 +2169,9 @@ public class KafkaAdminClientTest { | |||
|             KafkaFuture<Void> myTopicResult = values.get("my_topic"); | ||||
|             myTopicResult.get(); | ||||
|             KafkaFuture<Void> otherTopicResult = values.get("other_topic"); | ||||
|             try { | ||||
|                 otherTopicResult.get(); | ||||
|                 fail("get() should throw ExecutionException"); | ||||
|             } catch (ExecutionException e0) { | ||||
|                 assertTrue(e0.getCause() instanceof InvalidTopicException); | ||||
|                 InvalidTopicException e = (InvalidTopicException) e0.getCause(); | ||||
|                 assertEquals("some detailed reason", e.getMessage()); | ||||
|             } | ||||
|             assertEquals("some detailed reason", | ||||
|                 assertInstanceOf(InvalidTopicException.class, | ||||
|                     assertThrows(ExecutionException.class, otherTopicResult::get).getCause()).getMessage()); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  | @ -2434,30 +2430,18 @@ public class KafkaAdminClientTest { | |||
| 
 | ||||
|             // "offset out of range" failure on records deletion for partition 1 | ||||
|             KafkaFuture<DeletedRecords> myTopicPartition1Result = values.get(myTopicPartition1); | ||||
|             try { | ||||
|                 myTopicPartition1Result.get(); | ||||
|                 fail("get() should throw ExecutionException"); | ||||
|             } catch (ExecutionException e0) { | ||||
|                 assertTrue(e0.getCause() instanceof OffsetOutOfRangeException); | ||||
|             } | ||||
|             assertInstanceOf(OffsetOutOfRangeException.class, | ||||
|                 assertThrows(ExecutionException.class, myTopicPartition1Result::get).getCause()); | ||||
| 
 | ||||
|             // not authorized to delete records for partition 2 | ||||
|             KafkaFuture<DeletedRecords> myTopicPartition2Result = values.get(myTopicPartition2); | ||||
|             try { | ||||
|                 myTopicPartition2Result.get(); | ||||
|                 fail("get() should throw ExecutionException"); | ||||
|             } catch (ExecutionException e1) { | ||||
|                 assertTrue(e1.getCause() instanceof TopicAuthorizationException); | ||||
|             } | ||||
|             assertInstanceOf(TopicAuthorizationException.class, | ||||
|                 assertThrows(ExecutionException.class, myTopicPartition2Result::get).getCause()); | ||||
| 
 | ||||
|             // the response does not contain a result for partition 3 | ||||
|             KafkaFuture<DeletedRecords> myTopicPartition3Result = values.get(myTopicPartition3); | ||||
|             try { | ||||
|                 myTopicPartition3Result.get(); | ||||
|                 fail("get() should throw ExecutionException"); | ||||
|             } catch (ExecutionException e1) { | ||||
|                 assertTrue(e1.getCause() instanceof ApiException); | ||||
|             } | ||||
|             assertInstanceOf(ApiException.class, | ||||
|                 assertThrows(ExecutionException.class, myTopicPartition3Result::get).getCause()); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  | @ -4736,8 +4720,8 @@ public class KafkaAdminClientTest { | |||
|                     new RemoveMembersFromConsumerGroupOptions() | ||||
|             ); | ||||
|             ExecutionException exception = assertThrows(ExecutionException.class, () -> partialFailureResults.all().get()); | ||||
|             assertTrue(exception.getCause() instanceof KafkaException); | ||||
|             assertTrue(exception.getCause().getCause() instanceof UnknownMemberIdException); | ||||
|             assertInstanceOf(KafkaException.class, exception.getCause()); | ||||
|             assertInstanceOf(UnknownMemberIdException.class, exception.getCause().getCause()); | ||||
| 
 | ||||
|             // Return with success for "removeAll" scenario | ||||
|             // 1 prepare response for AdminClient.describeConsumerGroups | ||||
|  |  | |||
|  | @ -38,9 +38,9 @@ import static java.util.Collections.singleton; | |||
| import static java.util.Collections.singletonList; | ||||
| import static org.apache.kafka.common.utils.Utils.mkSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| public class AbortTransactionHandlerTest { | ||||
|     private final LogContext logContext = new LogContext(); | ||||
|  | @ -214,7 +214,7 @@ public class AbortTransactionHandlerTest { | |||
|         assertEquals(emptySet(), result.completedKeys.keySet()); | ||||
|         assertEquals(emptyList(), result.unmappedKeys); | ||||
|         assertEquals(singleton(topicPartition), result.failedKeys.keySet()); | ||||
|         assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(topicPartition))); | ||||
|         assertInstanceOf(expectedExceptionType, result.failedKeys.get(topicPartition)); | ||||
|     } | ||||
| 
 | ||||
| } | ||||
|  |  | |||
|  | @ -37,8 +37,8 @@ import static java.util.Collections.singleton; | |||
| import static java.util.Collections.singletonMap; | ||||
| import static org.apache.kafka.common.utils.Utils.mkSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| public class CoordinatorStrategyTest { | ||||
| 
 | ||||
|  | @ -217,7 +217,7 @@ public class CoordinatorStrategyTest { | |||
|         assertFatalOldLookup(group, Errors.UNKNOWN_SERVER_ERROR); | ||||
| 
 | ||||
|         Throwable throwable = assertFatalOldLookup(group, Errors.GROUP_AUTHORIZATION_FAILED); | ||||
|         assertTrue(throwable instanceof GroupAuthorizationException); | ||||
|         assertInstanceOf(GroupAuthorizationException.class, throwable); | ||||
|         GroupAuthorizationException exception = (GroupAuthorizationException) throwable; | ||||
|         assertEquals("foo", exception.groupId()); | ||||
|     } | ||||
|  | @ -233,7 +233,7 @@ public class CoordinatorStrategyTest { | |||
|         assertEquals(singleton(key), result.failedKeys.keySet()); | ||||
| 
 | ||||
|         Throwable throwable = result.failedKeys.get(key); | ||||
|         assertTrue(error.exception().getClass().isInstance(throwable)); | ||||
|         assertInstanceOf(error.exception().getClass(), throwable); | ||||
|         return throwable; | ||||
|     } | ||||
| 
 | ||||
|  | @ -244,7 +244,7 @@ public class CoordinatorStrategyTest { | |||
|         assertFatalLookup(group, Errors.UNKNOWN_SERVER_ERROR); | ||||
| 
 | ||||
|         Throwable throwable = assertFatalLookup(group, Errors.GROUP_AUTHORIZATION_FAILED); | ||||
|         assertTrue(throwable instanceof GroupAuthorizationException); | ||||
|         assertInstanceOf(GroupAuthorizationException.class, throwable); | ||||
|         GroupAuthorizationException exception = (GroupAuthorizationException) throwable; | ||||
|         assertEquals("foo", exception.groupId()); | ||||
|     } | ||||
|  | @ -264,7 +264,7 @@ public class CoordinatorStrategyTest { | |||
|         assertEquals(singleton(key), result.failedKeys.keySet()); | ||||
| 
 | ||||
|         Throwable throwable = result.failedKeys.get(key); | ||||
|         assertTrue(error.exception().getClass().isInstance(throwable)); | ||||
|         assertInstanceOf(error.exception().getClass(), throwable); | ||||
|         return throwable; | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -21,7 +21,7 @@ import static java.util.Collections.emptySet; | |||
| import static java.util.Collections.singleton; | ||||
| import static java.util.Collections.singletonList; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| import java.util.Arrays; | ||||
| import java.util.Collection; | ||||
|  | @ -189,7 +189,7 @@ public class DeleteConsumerGroupOffsetsHandlerTest { | |||
|         assertEquals(emptySet(), result.completedKeys.keySet()); | ||||
|         assertEquals(emptyList(), result.unmappedKeys); | ||||
|         assertEquals(singleton(key), result.failedKeys.keySet()); | ||||
|         assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); | ||||
|         assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); | ||||
|     } | ||||
| 
 | ||||
|     private void assertPartitionFailed( | ||||
|  |  | |||
|  | @ -21,7 +21,7 @@ import static java.util.Collections.emptySet; | |||
| import static java.util.Collections.singleton; | ||||
| import static java.util.Collections.singletonList; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| import org.apache.kafka.common.Node; | ||||
| import org.apache.kafka.common.errors.GroupAuthorizationException; | ||||
|  | @ -125,6 +125,6 @@ public class DeleteConsumerGroupsHandlerTest { | |||
|         assertEquals(emptySet(), result.completedKeys.keySet()); | ||||
|         assertEquals(emptyList(), result.unmappedKeys); | ||||
|         assertEquals(singleton(key), result.failedKeys.keySet()); | ||||
|         assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); | ||||
|         assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -22,8 +22,8 @@ import static java.util.Collections.singleton; | |||
| import static java.util.Collections.singletonList; | ||||
| import static org.apache.kafka.common.utils.Utils.mkSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| import java.util.ArrayList; | ||||
| import java.util.Arrays; | ||||
|  | @ -369,7 +369,7 @@ public class DescribeConsumerGroupsHandlerTest { | |||
|         assertEquals(emptySet(), result.completedKeys.keySet()); | ||||
|         assertEquals(emptyList(), result.unmappedKeys); | ||||
|         assertEquals(singleton(key), result.failedKeys.keySet()); | ||||
|         assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); | ||||
|         assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); | ||||
|     } | ||||
| 
 | ||||
|     private void assertRequestAndKeys( | ||||
|  |  | |||
|  | @ -52,8 +52,8 @@ import static java.util.Collections.singletonList; | |||
| import static java.util.Collections.singletonMap; | ||||
| import static org.apache.kafka.common.utils.Utils.mkSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| public class DescribeProducersHandlerTest { | ||||
|     private DescribeProducersHandler newHandler( | ||||
|  | @ -137,7 +137,7 @@ public class DescribeProducersHandlerTest { | |||
|     public void testAuthorizationFailure() { | ||||
|         TopicPartition topicPartition = new TopicPartition("foo", 5); | ||||
|         Throwable exception = assertFatalError(topicPartition, Errors.TOPIC_AUTHORIZATION_FAILED); | ||||
|         assertTrue(exception instanceof TopicAuthorizationException); | ||||
|         assertInstanceOf(TopicAuthorizationException.class, exception); | ||||
|         TopicAuthorizationException authException = (TopicAuthorizationException) exception; | ||||
|         assertEquals(mkSet("foo"), authException.unauthorizedTopics()); | ||||
|     } | ||||
|  | @ -146,7 +146,7 @@ public class DescribeProducersHandlerTest { | |||
|     public void testInvalidTopic() { | ||||
|         TopicPartition topicPartition = new TopicPartition("foo", 5); | ||||
|         Throwable exception = assertFatalError(topicPartition, Errors.INVALID_TOPIC_EXCEPTION); | ||||
|         assertTrue(exception instanceof InvalidTopicException); | ||||
|         assertInstanceOf(InvalidTopicException.class, exception); | ||||
|         InvalidTopicException invalidTopicException = (InvalidTopicException) exception; | ||||
|         assertEquals(mkSet("foo"), invalidTopicException.invalidTopics()); | ||||
|     } | ||||
|  | @ -155,7 +155,7 @@ public class DescribeProducersHandlerTest { | |||
|     public void testUnexpectedError() { | ||||
|         TopicPartition topicPartition = new TopicPartition("foo", 5); | ||||
|         Throwable exception = assertFatalError(topicPartition, Errors.UNKNOWN_SERVER_ERROR); | ||||
|         assertTrue(exception instanceof UnknownServerException); | ||||
|         assertInstanceOf(UnknownServerException.class, exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -185,7 +185,7 @@ public class DescribeProducersHandlerTest { | |||
|         assertEquals(emptyList(), result.unmappedKeys); | ||||
|         assertEquals(mkSet(topicPartition), result.failedKeys.keySet()); | ||||
|         Throwable exception = result.failedKeys.get(topicPartition); | ||||
|         assertTrue(exception instanceof NotLeaderOrFollowerException); | ||||
|         assertInstanceOf(NotLeaderOrFollowerException.class, exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -37,7 +37,7 @@ import static java.util.Collections.emptyMap; | |||
| import static java.util.Collections.singletonList; | ||||
| import static org.apache.kafka.common.utils.Utils.mkSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| public class DescribeTransactionsHandlerTest { | ||||
|     private final LogContext logContext = new LogContext(); | ||||
|  | @ -107,7 +107,7 @@ public class DescribeTransactionsHandlerTest { | |||
|         assertEquals(mkSet(key), result.failedKeys.keySet()); | ||||
| 
 | ||||
|         Throwable throwable = result.failedKeys.get(key); | ||||
|         assertTrue(error.exception().getClass().isInstance(throwable)); | ||||
|         assertInstanceOf(error.exception().getClass(), throwable); | ||||
|     } | ||||
| 
 | ||||
|     private void assertRetriableError( | ||||
|  |  | |||
|  | @ -34,7 +34,7 @@ import static java.util.Collections.singleton; | |||
| import static java.util.Collections.singletonList; | ||||
| import static org.apache.kafka.common.utils.Utils.mkSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| public class FenceProducersHandlerTest { | ||||
|     private final LogContext logContext = new LogContext(); | ||||
|  | @ -96,7 +96,7 @@ public class FenceProducersHandlerTest { | |||
|         assertEquals(mkSet(key), result.failedKeys.keySet()); | ||||
| 
 | ||||
|         Throwable throwable = result.failedKeys.get(key); | ||||
|         assertTrue(error.exception().getClass().isInstance(throwable)); | ||||
|         assertInstanceOf(error.exception().getClass(), throwable); | ||||
|     } | ||||
| 
 | ||||
|     private void assertRetriableError( | ||||
|  |  | |||
|  | @ -21,6 +21,7 @@ import static java.util.Collections.emptySet; | |||
| import static java.util.Collections.singleton; | ||||
| import static java.util.Collections.singletonList; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| import java.util.Arrays; | ||||
|  | @ -403,7 +404,7 @@ public class ListConsumerGroupOffsetsHandlerTest { | |||
|         assertEquals(emptySet(), result.completedKeys.keySet()); | ||||
|         assertEquals(emptyList(), result.unmappedKeys); | ||||
|         assertEquals(singleton(key), result.failedKeys.keySet()); | ||||
|         assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); | ||||
|         assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); | ||||
|     } | ||||
| 
 | ||||
|     private void assertFailedForMultipleGroups( | ||||
|  | @ -415,7 +416,7 @@ public class ListConsumerGroupOffsetsHandlerTest { | |||
|         for (String g : groupToExceptionMap.keySet()) { | ||||
|             CoordinatorKey key = CoordinatorKey.byGroupId(g); | ||||
|             assertTrue(result.failedKeys.containsKey(key)); | ||||
|             assertTrue(groupToExceptionMap.get(g).isInstance(result.failedKeys.get(key))); | ||||
|             assertInstanceOf(groupToExceptionMap.get(g), result.failedKeys.get(key)); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -44,7 +44,7 @@ import static java.util.Collections.singletonMap; | |||
| import static org.apache.kafka.common.utils.Utils.mkSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| public class PartitionLeaderStrategyTest { | ||||
| 
 | ||||
|  | @ -78,7 +78,7 @@ public class PartitionLeaderStrategyTest { | |||
|     public void testTopicAuthorizationFailure() { | ||||
|         TopicPartition topicPartition = new TopicPartition("foo", 0); | ||||
|         Throwable exception = assertFatalTopicError(topicPartition, Errors.TOPIC_AUTHORIZATION_FAILED); | ||||
|         assertTrue(exception instanceof TopicAuthorizationException); | ||||
|         assertInstanceOf(TopicAuthorizationException.class, exception); | ||||
|         TopicAuthorizationException authException = (TopicAuthorizationException) exception; | ||||
|         assertEquals(mkSet("foo"), authException.unauthorizedTopics()); | ||||
|     } | ||||
|  | @ -87,7 +87,7 @@ public class PartitionLeaderStrategyTest { | |||
|     public void testInvalidTopicError() { | ||||
|         TopicPartition topicPartition = new TopicPartition("foo", 0); | ||||
|         Throwable exception = assertFatalTopicError(topicPartition, Errors.INVALID_TOPIC_EXCEPTION); | ||||
|         assertTrue(exception instanceof InvalidTopicException); | ||||
|         assertInstanceOf(InvalidTopicException.class, exception); | ||||
|         InvalidTopicException invalidTopicException = (InvalidTopicException) exception; | ||||
|         assertEquals(mkSet("foo"), invalidTopicException.invalidTopics()); | ||||
|     } | ||||
|  | @ -96,7 +96,7 @@ public class PartitionLeaderStrategyTest { | |||
|     public void testUnexpectedTopicError() { | ||||
|         TopicPartition topicPartition = new TopicPartition("foo", 0); | ||||
|         Throwable exception = assertFatalTopicError(topicPartition, Errors.UNKNOWN_SERVER_ERROR); | ||||
|         assertTrue(exception instanceof UnknownServerException); | ||||
|         assertInstanceOf(UnknownServerException.class, exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -121,7 +121,7 @@ public class PartitionLeaderStrategyTest { | |||
|     public void testUnexpectedPartitionError() { | ||||
|         TopicPartition topicPartition = new TopicPartition("foo", 0); | ||||
|         Throwable exception = assertFatalPartitionError(topicPartition, Errors.UNKNOWN_SERVER_ERROR); | ||||
|         assertTrue(exception instanceof UnknownServerException); | ||||
|         assertInstanceOf(UnknownServerException.class, exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -21,7 +21,7 @@ import static java.util.Collections.emptySet; | |||
| import static java.util.Collections.singleton; | ||||
| import static java.util.Collections.singletonList; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| import java.util.Arrays; | ||||
| import java.util.Collections; | ||||
|  | @ -164,7 +164,7 @@ public class RemoveMembersFromConsumerGroupHandlerTest { | |||
|         assertEquals(emptySet(), result.completedKeys.keySet()); | ||||
|         assertEquals(emptyList(), result.unmappedKeys); | ||||
|         assertEquals(singleton(key), result.failedKeys.keySet()); | ||||
|         assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(key))); | ||||
|         assertInstanceOf(expectedExceptionType, result.failedKeys.get(key)); | ||||
|     } | ||||
| 
 | ||||
|     private void assertMemberFailed( | ||||
|  |  | |||
|  | @ -35,7 +35,6 @@ import static org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.getAss | |||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| public class ConsumerPartitionAssignorTest { | ||||
| 
 | ||||
|  | @ -45,7 +44,7 @@ public class ConsumerPartitionAssignorTest { | |||
|                 Collections.singletonList(StickyAssignor.class.getName()), | ||||
|                 Collections.emptyMap() | ||||
|         ); | ||||
|         assertTrue(assignors.get(0) instanceof StickyAssignor); | ||||
|         assertInstanceOf(StickyAssignor.class, assignors.get(0)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -54,8 +53,8 @@ public class ConsumerPartitionAssignorTest { | |||
|                 Arrays.asList(StickyAssignor.class.getName(), CooperativeStickyAssignor.class.getName()), | ||||
|                 Collections.emptyMap() | ||||
|         ); | ||||
|         assertTrue(assignors.get(0) instanceof StickyAssignor); | ||||
|         assertTrue(assignors.get(1) instanceof CooperativeStickyAssignor); | ||||
|         assertInstanceOf(StickyAssignor.class, assignors.get(0)); | ||||
|         assertInstanceOf(CooperativeStickyAssignor.class, assignors.get(1)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -80,7 +79,7 @@ public class ConsumerPartitionAssignorTest { | |||
|                 initConsumerConfigWithClassTypes(Collections.singletonList(StickyAssignor.class)) | ||||
|                 .getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG); | ||||
|         List<ConsumerPartitionAssignor> assignors = getAssignorInstances(classTypes, Collections.emptyMap()); | ||||
|         assertTrue(assignors.get(0) instanceof StickyAssignor); | ||||
|         assertInstanceOf(StickyAssignor.class, assignors.get(0)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -91,8 +90,8 @@ public class ConsumerPartitionAssignorTest { | |||
| 
 | ||||
|         List<ConsumerPartitionAssignor> assignors = getAssignorInstances(classTypes, Collections.emptyMap()); | ||||
| 
 | ||||
|         assertTrue(assignors.get(0) instanceof StickyAssignor); | ||||
|         assertTrue(assignors.get(1) instanceof CooperativeStickyAssignor); | ||||
|         assertInstanceOf(StickyAssignor.class, assignors.get(0)); | ||||
|         assertInstanceOf(CooperativeStickyAssignor.class, assignors.get(1)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -145,6 +145,7 @@ import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID; | |||
| import static org.apache.kafka.common.utils.Utils.propsToMap; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotSame; | ||||
|  | @ -259,7 +260,7 @@ public class KafkaConsumerTest { | |||
|         props.setProperty(ConsumerConfig.ENABLE_METRICS_PUSH_CONFIG, "false"); | ||||
|         consumer = newConsumer(props, new StringDeserializer(), new StringDeserializer()); | ||||
|         assertEquals(1, consumer.metricsRegistry().reporters().size()); | ||||
|         assertTrue(consumer.metricsRegistry().reporters().get(0) instanceof JmxReporter); | ||||
|         assertInstanceOf(JmxReporter.class, consumer.metricsRegistry().reporters().get(0)); | ||||
|     } | ||||
| 
 | ||||
|     @ParameterizedTest | ||||
|  | @ -272,7 +273,7 @@ public class KafkaConsumerTest { | |||
|         props.setProperty(ConsumerConfig.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false"); | ||||
|         consumer = newConsumer(props, new StringDeserializer(), new StringDeserializer()); | ||||
|         assertEquals(1, consumer.metricsRegistry().reporters().size()); | ||||
|         assertTrue(consumer.metricsRegistry().reporters().get(0) instanceof ClientTelemetryReporter); | ||||
|         assertInstanceOf(ClientTelemetryReporter.class, consumer.metricsRegistry().reporters().get(0)); | ||||
|     } | ||||
| 
 | ||||
|     // TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol. | ||||
|  | @ -2071,7 +2072,7 @@ public class KafkaConsumerTest { | |||
|                 TestUtils.waitForCondition( | ||||
|                     () -> closeException.get() != null, "InterruptException did not occur within timeout."); | ||||
| 
 | ||||
|                 assertTrue(closeException.get() instanceof InterruptException, "Expected exception not thrown " + closeException); | ||||
|                 assertInstanceOf(InterruptException.class, closeException.get(), "Expected exception not thrown " + closeException); | ||||
|             } else { | ||||
|                 future.get(closeTimeoutMs, TimeUnit.MILLISECONDS); // Should succeed without TimeoutException or ExecutionException | ||||
|                 assertNull(closeException.get(), "Unexpected exception during close"); | ||||
|  |  | |||
|  | @ -78,6 +78,7 @@ import java.util.concurrent.atomic.AtomicBoolean; | |||
| import static java.util.Collections.emptyMap; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotSame; | ||||
| import static org.junit.jupiter.api.Assertions.assertSame; | ||||
|  | @ -341,7 +342,7 @@ public class AbstractCoordinatorTest { | |||
| 
 | ||||
|         RequestFuture<ByteBuffer> future = coordinator.sendJoinGroupRequest(); | ||||
|         assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.GROUP_MAX_SIZE_REACHED.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.GROUP_MAX_SIZE_REACHED.exception()); | ||||
|         assertFalse(future.isRetriable()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -359,7 +360,7 @@ public class AbstractCoordinatorTest { | |||
| 
 | ||||
|         mockTime.sleep(REBALANCE_TIMEOUT_MS - REQUEST_TIMEOUT_MS + AbstractCoordinator.JOIN_GROUP_TIMEOUT_LAPSE); | ||||
|         assertTrue(consumerClient.poll(future, mockTime.timer(0))); | ||||
|         assertTrue(future.exception() instanceof DisconnectException); | ||||
|         assertInstanceOf(DisconnectException.class, future.exception()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -377,7 +378,7 @@ public class AbstractCoordinatorTest { | |||
| 
 | ||||
|         mockTime.sleep(expectedRequestDeadline - mockTime.milliseconds() + 1); | ||||
|         assertTrue(consumerClient.poll(future, mockTime.timer(0))); | ||||
|         assertTrue(future.exception() instanceof DisconnectException); | ||||
|         assertInstanceOf(DisconnectException.class, future.exception()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -773,7 +774,7 @@ public class AbstractCoordinatorTest { | |||
|         mockClient.respond(joinGroupFollowerResponse(currGen.generationId + 1, memberId, JoinGroupRequest.UNKNOWN_MEMBER_ID, Errors.UNKNOWN_MEMBER_ID)); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.UNKNOWN_MEMBER_ID.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.UNKNOWN_MEMBER_ID.exception()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(newGen, coordinator.generation()); | ||||
|  | @ -813,7 +814,7 @@ public class AbstractCoordinatorTest { | |||
| 
 | ||||
|         mockClient.respond(syncGroupResponse(Errors.UNKNOWN_MEMBER_ID)); | ||||
|         assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.UNKNOWN_MEMBER_ID.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.UNKNOWN_MEMBER_ID.exception()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(newGen, coordinator.generation()); | ||||
|  | @ -853,7 +854,7 @@ public class AbstractCoordinatorTest { | |||
| 
 | ||||
|         mockClient.respond(syncGroupResponse(Errors.ILLEGAL_GENERATION)); | ||||
|         assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.ILLEGAL_GENERATION.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.ILLEGAL_GENERATION.exception()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(newGen, coordinator.generation()); | ||||
|  | @ -1027,7 +1028,8 @@ public class AbstractCoordinatorTest { | |||
|         mockClient.prepareResponse(syncGroupResponse(Errors.NONE)); | ||||
|         mockClient.prepareResponse(heartbeatResponse(Errors.FENCED_INSTANCE_ID)); | ||||
| 
 | ||||
|         try { | ||||
|         assertThrows(FencedInstanceIdException.class, | ||||
|             () -> { | ||||
|                 coordinator.ensureActiveGroup(); | ||||
|                 mockTime.sleep(HEARTBEAT_INTERVAL_MS); | ||||
|                 long startMs = System.currentTimeMillis(); | ||||
|  | @ -1035,10 +1037,8 @@ public class AbstractCoordinatorTest { | |||
|                     Thread.sleep(10); | ||||
|                     coordinator.pollHeartbeat(mockTime.milliseconds()); | ||||
|                 } | ||||
|             fail("Expected pollHeartbeat to raise fenced instance id exception in 1 second"); | ||||
|         } catch (RuntimeException exception) { | ||||
|             assertTrue(exception instanceof FencedInstanceIdException); | ||||
|         } | ||||
|             }, | ||||
|             "Expected pollHeartbeat to raise fenced instance id exception in 1 second"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -1069,7 +1069,7 @@ public class AbstractCoordinatorTest { | |||
|         RequestFuture<ByteBuffer> future = coordinator.sendJoinGroupRequest(); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, mockTime.timer(REQUEST_TIMEOUT_MS))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); | ||||
|         assertEquals(Errors.REBALANCE_IN_PROGRESS.message(), future.exception().getMessage()); | ||||
|         assertTrue(coordinator.rejoinNeededOrPending()); | ||||
| 
 | ||||
|  | @ -1157,7 +1157,7 @@ public class AbstractCoordinatorTest { | |||
|             leaveGroupResponse(Arrays.asList(memberResponse, memberResponse)); | ||||
|         RequestFuture<Void> leaveGroupFuture = setupLeaveGroup(response); | ||||
|         assertNotNull(leaveGroupFuture); | ||||
|         assertTrue(leaveGroupFuture.exception() instanceof IllegalStateException); | ||||
|         assertInstanceOf(IllegalStateException.class, leaveGroupFuture.exception()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -1178,7 +1178,7 @@ public class AbstractCoordinatorTest { | |||
|             leaveGroupResponse(Collections.singletonList(memberResponse)); | ||||
|         RequestFuture<Void> leaveGroupFuture = setupLeaveGroup(response); | ||||
|         assertNotNull(leaveGroupFuture); | ||||
|         assertTrue(leaveGroupFuture.exception() instanceof UnknownMemberIdException); | ||||
|         assertInstanceOf(UnknownMemberIdException.class, leaveGroupFuture.exception()); | ||||
|     } | ||||
| 
 | ||||
|     private RequestFuture<Void> setupLeaveGroup(LeaveGroupResponse leaveGroupResponse) { | ||||
|  |  | |||
|  | @ -729,7 +729,7 @@ public class AsyncKafkaConsumerTest { | |||
|         assertDoesNotThrow(() -> consumer.completeQuietly(() -> { | ||||
|             throw new KafkaException("Test exception"); | ||||
|         }, "test", exception)); | ||||
|         assertTrue(exception.get() instanceof KafkaException); | ||||
|         assertInstanceOf(KafkaException.class, exception.get()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -78,6 +78,7 @@ import static org.apache.kafka.clients.consumer.internals.ConsumerTestBuilder.DE | |||
| import static org.apache.kafka.test.TestUtils.assertFutureThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.fail; | ||||
|  | @ -911,7 +912,7 @@ public class CommitRequestManagerTest { | |||
|         long expirationTimeMs = time.milliseconds() + retryBackoffMs * 2; | ||||
| 
 | ||||
|         // Send commit request expected to be retried on STALE_MEMBER_EPOCH error while it does not expire | ||||
|         commitRequestManager.maybeAutoCommitSyncNow(expirationTimeMs); | ||||
|         commitRequestManager.maybeAutoCommitSyncBeforeRevocation(expirationTimeMs); | ||||
| 
 | ||||
|         int newEpoch = 8; | ||||
|         String memberId = "member1"; | ||||
|  | @ -922,7 +923,7 @@ public class CommitRequestManagerTest { | |||
| 
 | ||||
|         completeOffsetCommitRequestWithError(commitRequestManager, error); | ||||
| 
 | ||||
|         if (error.exception() instanceof RetriableException || error == Errors.STALE_MEMBER_EPOCH) { | ||||
|         if ((error.exception() instanceof RetriableException || error == Errors.STALE_MEMBER_EPOCH) && error != Errors.UNKNOWN_TOPIC_OR_PARTITION) { | ||||
|             assertEquals(1, commitRequestManager.pendingRequests.unsentOffsetCommits.size(), | ||||
|                 "Request to be retried should be added to the outbound queue"); | ||||
| 
 | ||||
|  | @ -1315,7 +1316,7 @@ public class CommitRequestManagerTest { | |||
|             final Errors error, | ||||
|             final boolean disconnected) { | ||||
|         AbstractRequest abstractRequest = request.requestBuilder().build(); | ||||
|         assertTrue(abstractRequest instanceof OffsetFetchRequest); | ||||
|         assertInstanceOf(OffsetFetchRequest.class, abstractRequest); | ||||
|         OffsetFetchRequest offsetFetchRequest = (OffsetFetchRequest) abstractRequest; | ||||
|         OffsetFetchResponse response = | ||||
|                 new OffsetFetchResponse(error, topicPartitionData); | ||||
|  |  | |||
|  | @ -127,6 +127,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; | |||
| import static org.apache.kafka.test.TestUtils.toSet; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
|  | @ -642,7 +643,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|             coordinator.commitOffsetsAsync(offsets, (offsets1, exception) -> { | ||||
|                 responses.incrementAndGet(); | ||||
|                 Throwable cause = exception.getCause(); | ||||
|                 assertTrue(cause instanceof DisconnectException, | ||||
|                 assertInstanceOf(DisconnectException.class, cause, | ||||
|                     "Unexpected exception cause type: " + (cause == null ? null : cause.getClass())); | ||||
|             }); | ||||
|         } | ||||
|  | @ -689,7 +690,7 @@ public abstract class ConsumerCoordinatorTest { | |||
| 
 | ||||
|                     @Override | ||||
|                     public void onFailure(RuntimeException e, RequestFuture<Object> future) { | ||||
|                         assertTrue(e instanceof DisconnectException, "Unexpected exception type: " + e.getClass()); | ||||
|                         assertInstanceOf(DisconnectException.class, e, "Unexpected exception type: " + e.getClass()); | ||||
|                         assertTrue(coordinator.coordinatorUnknown()); | ||||
|                         asyncCallbackInvoked.set(true); | ||||
|                     } | ||||
|  | @ -952,7 +953,7 @@ public abstract class ConsumerCoordinatorTest { | |||
| 
 | ||||
|         assertTrue(future.isDone()); | ||||
|         assertTrue(future.failed()); | ||||
|         assertTrue(future.exception() instanceof DisconnectException); | ||||
|         assertInstanceOf(DisconnectException.class, future.exception()); | ||||
|         assertTrue(coordinator.coordinatorUnknown()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -2362,8 +2363,8 @@ public abstract class ConsumerCoordinatorTest { | |||
|         coordinator.invokeCompletedOffsetCommitCallbacks(); | ||||
| 
 | ||||
|         assertTrue(coordinator.coordinatorUnknown()); | ||||
|         assertTrue(firstCommitCallback.exception instanceof RetriableCommitFailedException); | ||||
|         assertTrue(secondCommitCallback.exception instanceof RetriableCommitFailedException); | ||||
|         assertInstanceOf(RetriableCommitFailedException.class, firstCommitCallback.exception); | ||||
|         assertInstanceOf(RetriableCommitFailedException.class, secondCommitCallback.exception); | ||||
|         assertEquals(coordinator.inFlightAsyncCommits.get(), 0); | ||||
|     } | ||||
| 
 | ||||
|  | @ -2597,7 +2598,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|         assertEquals(coordinator.inFlightAsyncCommits.get(), 0); | ||||
|         coordinator.invokeCompletedOffsetCommitCallbacks(); | ||||
|         assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked); | ||||
|         assertTrue(mockOffsetCommitCallback.exception instanceof RetriableCommitFailedException); | ||||
|         assertInstanceOf(RetriableCommitFailedException.class, mockOffsetCommitCallback.exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -2614,7 +2615,7 @@ public abstract class ConsumerCoordinatorTest { | |||
| 
 | ||||
|         assertTrue(coordinator.coordinatorUnknown()); | ||||
|         assertEquals(1, cb.invoked); | ||||
|         assertTrue(cb.exception instanceof RetriableCommitFailedException); | ||||
|         assertInstanceOf(RetriableCommitFailedException.class, cb.exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -2631,7 +2632,7 @@ public abstract class ConsumerCoordinatorTest { | |||
| 
 | ||||
|         assertTrue(coordinator.coordinatorUnknown()); | ||||
|         assertEquals(1, cb.invoked); | ||||
|         assertTrue(cb.exception instanceof RetriableCommitFailedException); | ||||
|         assertInstanceOf(RetriableCommitFailedException.class, cb.exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -2648,7 +2649,7 @@ public abstract class ConsumerCoordinatorTest { | |||
| 
 | ||||
|         assertTrue(coordinator.coordinatorUnknown()); | ||||
|         assertEquals(1, cb.invoked); | ||||
|         assertTrue(cb.exception instanceof RetriableCommitFailedException); | ||||
|         assertInstanceOf(RetriableCommitFailedException.class, cb.exception); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -2794,7 +2795,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|         coordinator.setNewState(AbstractCoordinator.MemberState.PREPARING_REBALANCE); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, time.timer(30000))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(newGen, coordinator.generation()); | ||||
|  | @ -2842,7 +2843,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|         coordinator.setNewGeneration(AbstractCoordinator.Generation.NO_GENERATION); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, time.timer(30000))); | ||||
|         assertTrue(future.exception().getClass().isInstance(new CommitFailedException())); | ||||
|         assertInstanceOf(future.exception().getClass(), new CommitFailedException()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(AbstractCoordinator.Generation.NO_GENERATION, coordinator.generation()); | ||||
|  | @ -2872,7 +2873,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|         coordinator.setNewState(AbstractCoordinator.MemberState.PREPARING_REBALANCE); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, time.timer(30000))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(newGen, coordinator.generation()); | ||||
|  | @ -2897,7 +2898,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|         coordinator.setNewGeneration(AbstractCoordinator.Generation.NO_GENERATION); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, time.timer(30000))); | ||||
|         assertTrue(future.exception().getClass().isInstance(new CommitFailedException())); | ||||
|         assertInstanceOf(future.exception().getClass(), new CommitFailedException()); | ||||
| 
 | ||||
|         // the generation should be reset | ||||
|         assertEquals(AbstractCoordinator.Generation.NO_GENERATION, coordinator.generation()); | ||||
|  | @ -2947,7 +2948,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|         coordinator.setNewGeneration(newGen); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, time.timer(30000))); | ||||
|         assertTrue(future.exception().getClass().isInstance(Errors.REBALANCE_IN_PROGRESS.exception())); | ||||
|         assertInstanceOf(future.exception().getClass(), Errors.REBALANCE_IN_PROGRESS.exception()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(newGen, coordinator.generation()); | ||||
|  | @ -2976,7 +2977,7 @@ public abstract class ConsumerCoordinatorTest { | |||
|         coordinator.setNewGeneration(newGen); | ||||
| 
 | ||||
|         assertTrue(consumerClient.poll(future, time.timer(30000))); | ||||
|         assertTrue(future.exception().getClass().isInstance(new CommitFailedException())); | ||||
|         assertInstanceOf(future.exception().getClass(), new CommitFailedException()); | ||||
| 
 | ||||
|         // the generation should not be reset | ||||
|         assertEquals(newGen, coordinator.generation()); | ||||
|  |  | |||
|  | @ -49,6 +49,7 @@ import java.util.concurrent.atomic.AtomicInteger; | |||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.fail; | ||||
|  | @ -94,7 +95,7 @@ public class ConsumerNetworkClientTest { | |||
|         final RequestFuture<ClientResponse> future = consumerClient.send(node, heartbeat()); | ||||
|         consumerClient.poll(future); | ||||
|         assertTrue(future.failed()); | ||||
|         assertTrue(future.exception() instanceof AuthenticationException, "Expected only an authentication error."); | ||||
|         assertInstanceOf(AuthenticationException.class, future.exception(), "Expected only an authentication error."); | ||||
| 
 | ||||
|         time.sleep(30); // wait less than the backoff period | ||||
|         assertTrue(client.connectionFailed(node)); | ||||
|  | @ -102,7 +103,7 @@ public class ConsumerNetworkClientTest { | |||
|         final RequestFuture<ClientResponse> future2 = consumerClient.send(node, heartbeat()); | ||||
|         consumerClient.poll(future2); | ||||
|         assertTrue(future2.failed()); | ||||
|         assertTrue(future2.exception() instanceof AuthenticationException, "Expected only an authentication error."); | ||||
|         assertInstanceOf(AuthenticationException.class, future2.exception(), "Expected only an authentication error."); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -127,7 +128,7 @@ public class ConsumerNetworkClientTest { | |||
|         consumerClient.disconnectAsync(node); | ||||
|         consumerClient.pollNoWakeup(); | ||||
|         assertTrue(future.failed()); | ||||
|         assertTrue(future.exception() instanceof DisconnectException); | ||||
|         assertInstanceOf(DisconnectException.class, future.exception()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -139,7 +140,7 @@ public class ConsumerNetworkClientTest { | |||
|         consumerClient.disconnectAsync(node); | ||||
|         consumerClient.pollNoWakeup(); | ||||
|         assertTrue(future.failed()); | ||||
|         assertTrue(future.exception() instanceof DisconnectException); | ||||
|         assertInstanceOf(DisconnectException.class, future.exception()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -159,7 +160,7 @@ public class ConsumerNetworkClientTest { | |||
| 
 | ||||
|         assertFalse(consumerClient.hasPendingRequests()); | ||||
|         assertTrue(future.failed()); | ||||
|         assertTrue(future.exception() instanceof TimeoutException); | ||||
|         assertInstanceOf(TimeoutException.class, future.exception()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -227,7 +228,7 @@ public class ConsumerNetworkClientTest { | |||
|         consumerClient.disconnectAsync(node); | ||||
|         t.join(); | ||||
|         assertTrue(future.failed()); | ||||
|         assertTrue(future.exception() instanceof DisconnectException); | ||||
|         assertInstanceOf(DisconnectException.class, future.exception()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -37,6 +37,7 @@ import java.util.Collections; | |||
| import java.util.Optional; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.mockito.ArgumentMatchers.argThat; | ||||
|  | @ -202,7 +203,7 @@ public class CoordinatorRequestManagerTest { | |||
|         Errors error | ||||
|     ) { | ||||
|         AbstractRequest abstractRequest = request.requestBuilder().build(); | ||||
|         assertTrue(abstractRequest instanceof FindCoordinatorRequest); | ||||
|         assertInstanceOf(FindCoordinatorRequest.class, abstractRequest); | ||||
|         FindCoordinatorRequest findCoordinatorRequest = (FindCoordinatorRequest) abstractRequest; | ||||
| 
 | ||||
|         FindCoordinatorResponse findCoordinatorResponse = | ||||
|  |  | |||
|  | @ -470,13 +470,13 @@ public class MembershipManagerImplTest { | |||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * This is the case where a member is stuck reconciling an assignment A (waiting on | ||||
|      * metadata, commit or callbacks), and it rejoins (due to fence or unsubscribe/subscribe). If | ||||
|      * the reconciliation of A completes it should not be applied (it should not update the | ||||
|      * assignment on the member or send ack). | ||||
|      * This is the case where a member is stuck reconciling an assignment A (waiting for commit | ||||
|      * to complete), and it rejoins (due to fence or unsubscribe/subscribe). If the | ||||
|      * reconciliation of A completes it should be interrupted, and it should not update the | ||||
|      * assignment on the member or send ack. | ||||
|      */ | ||||
|     @Test | ||||
|     public void testDelayedReconciliationResultDiscardedIfMemberRejoins() { | ||||
|     public void testDelayedReconciliationResultDiscardedAfterCommitIfMemberRejoins() { | ||||
|         MembershipManagerImpl membershipManager = createMemberInStableState(); | ||||
|         Uuid topicId1 = Uuid.randomUuid(); | ||||
|         String topic1 = "topic1"; | ||||
|  | @ -496,25 +496,83 @@ public class MembershipManagerImplTest { | |||
|         testFencedMemberReleasesAssignmentAndTransitionsToJoining(membershipManager); | ||||
|         clearInvocations(subscriptionState); | ||||
| 
 | ||||
|         // Get new assignment A2 after rejoining. This should not trigger a reconciliation just | ||||
|         // yet because there is another on in progress, but should keep the new assignment ready | ||||
|         // to be reconciled next. | ||||
|         Uuid topicId3 = Uuid.randomUuid(); | ||||
|         mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId3, "topic3", owned); | ||||
|         receiveAssignmentAfterRejoin(topicId3, Collections.singletonList(5), membershipManager); | ||||
|         verifyReconciliationNotTriggered(membershipManager); | ||||
|         Map<Uuid, SortedSet<Integer>> assignmentAfterRejoin = topicIdPartitionsMap(topicId3, 5); | ||||
|         assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); | ||||
|         Map<Uuid, SortedSet<Integer>> assignmentAfterRejoin = receiveAssignmentAfterRejoin( | ||||
|             Collections.singletonList(5), membershipManager, owned); | ||||
| 
 | ||||
|         // Reconciliation completes when the member has already re-joined the group. Should not | ||||
|         // update the subscription state or send ack. | ||||
|         // proceed with the revocation, update the subscription state or send ack. | ||||
|         commitResult.complete(null); | ||||
|         verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); | ||||
|         assertNotEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); | ||||
|         assertInitialReconciliationDiscardedAfterRejoin(membershipManager, assignmentAfterRejoin); | ||||
|     } | ||||
| 
 | ||||
|         // Assignment received after rejoining should be ready to reconcile on the next | ||||
|         // reconciliation loop. | ||||
|         assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); | ||||
|     /** | ||||
|      * This is the case where a member is stuck reconciling an assignment A (waiting for | ||||
|      * onPartitionsRevoked callback to complete), and it rejoins (due to fence or | ||||
|      * unsubscribe/subscribe). When the reconciliation of A completes it should be interrupted, | ||||
|      * and it should not update the assignment on the member or send ack. | ||||
|      */ | ||||
|     @Test | ||||
|     public void testDelayedReconciliationResultDiscardedAfterPartitionsRevokedCallbackIfMemberRejoins() { | ||||
|         MembershipManagerImpl membershipManager = createMemberInStableState(); | ||||
|         Uuid topicId1 = Uuid.randomUuid(); | ||||
|         String topic1 = "topic1"; | ||||
|         List<TopicIdPartition> owned = Collections.singletonList(new TopicIdPartition(topicId1, | ||||
|             new TopicPartition(topic1, 0))); | ||||
|         mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId1, topic1, owned); | ||||
| 
 | ||||
|         // Reconciliation that does not complete stuck on onPartitionsRevoked callback | ||||
|         ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); | ||||
|         ConsumerRebalanceListenerCallbackCompletedEvent callbackCompletedEvent = | ||||
|             mockNewAssignmentStuckOnPartitionsRevokedCallback(membershipManager, topicId1, topic1, | ||||
|             Arrays.asList(1, 2), owned.get(0).topicPartition(), invoker); | ||||
|         Map<Uuid, SortedSet<Integer>> assignment1 = topicIdPartitionsMap(topicId1,  1, 2); | ||||
|         assertEquals(assignment1, membershipManager.topicPartitionsAwaitingReconciliation()); | ||||
| 
 | ||||
|         // Get fenced and rejoin while still reconciling. Get new assignment to reconcile after rejoining. | ||||
|         testFencedMemberReleasesAssignmentAndTransitionsToJoining(membershipManager); | ||||
|         clearInvocations(subscriptionState); | ||||
| 
 | ||||
|         Map<Uuid, SortedSet<Integer>> assignmentAfterRejoin = receiveAssignmentAfterRejoin( | ||||
|             Collections.singletonList(5), membershipManager, owned); | ||||
| 
 | ||||
|         // onPartitionsRevoked callback completes when the member has already re-joined the group. | ||||
|         // Should not proceed with the assignment, update the subscription state or send ack. | ||||
|         completeCallback(callbackCompletedEvent, membershipManager); | ||||
|         assertInitialReconciliationDiscardedAfterRejoin(membershipManager, assignmentAfterRejoin); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * This is the case where a member is stuck reconciling an assignment A (waiting for | ||||
|      * onPartitionsAssigned callback to complete), and it rejoins (due to fence or | ||||
|      * unsubscribe/subscribe). If the reconciliation of A completes it should be interrupted, and it | ||||
|      * should not update the assignment on the member or send ack. | ||||
|      */ | ||||
|     @Test | ||||
|     public void testDelayedReconciliationResultDiscardedAfterPartitionsAssignedCallbackIfMemberRejoins() { | ||||
|         MembershipManagerImpl membershipManager = createMemberInStableState(); | ||||
|         Uuid topicId1 = Uuid.randomUuid(); | ||||
|         String topic1 = "topic1"; | ||||
| 
 | ||||
|         // Reconciliation that does not complete stuck on onPartitionsAssigned callback | ||||
|         ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); | ||||
|         int newPartition = 1; | ||||
|         ConsumerRebalanceListenerCallbackCompletedEvent callbackCompletedEvent = | ||||
|             mockNewAssignmentStuckOnPartitionsAssignedCallback(membershipManager, topicId1, | ||||
|                 topic1, newPartition, invoker); | ||||
|         Map<Uuid, SortedSet<Integer>> assignment1 = topicIdPartitionsMap(topicId1,  newPartition); | ||||
|         assertEquals(assignment1, membershipManager.topicPartitionsAwaitingReconciliation()); | ||||
| 
 | ||||
|         // Get fenced and rejoin while still reconciling. Get new assignment to reconcile after rejoining. | ||||
|         testFencedMemberReleasesAssignmentAndTransitionsToJoining(membershipManager); | ||||
|         clearInvocations(subscriptionState); | ||||
| 
 | ||||
|         Map<Uuid, SortedSet<Integer>> assignmentAfterRejoin = receiveAssignmentAfterRejoin( | ||||
|             Collections.singletonList(5), membershipManager, Collections.emptyList()); | ||||
| 
 | ||||
|         // onPartitionsAssigned callback completes when the member has already re-joined the group. | ||||
|         // Should not update the subscription state or send ack. | ||||
|         completeCallback(callbackCompletedEvent, membershipManager); | ||||
|         assertInitialReconciliationDiscardedAfterRejoin(membershipManager, assignmentAfterRejoin); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|  | @ -539,7 +597,7 @@ public class MembershipManagerImplTest { | |||
|         membershipManager.onHeartbeatSuccess(createConsumerGroupHeartbeatResponse(assignment2).data()); | ||||
|         assertEquals(MemberState.RECONCILING, membershipManager.state()); | ||||
|         CompletableFuture<Void> commitResult = new CompletableFuture<>(); | ||||
|         when(commitRequestManager.maybeAutoCommitSyncNow(anyLong())).thenReturn(commitResult); | ||||
|         when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(commitResult); | ||||
|         membershipManager.poll(time.milliseconds()); | ||||
| 
 | ||||
|         // Get fenced, commit completes | ||||
|  | @ -2147,7 +2205,7 @@ public class MembershipManagerImplTest { | |||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testRebalanceMetricsForMultipleReconcilations() { | ||||
|     public void testRebalanceMetricsForMultipleReconciliations() { | ||||
|         MembershipManagerImpl membershipManager = createMemberInStableState(); | ||||
|         ConsumerRebalanceListenerInvoker invoker = consumerRebalanceListenerInvoker(); | ||||
| 
 | ||||
|  | @ -2299,6 +2357,57 @@ public class MembershipManagerImplTest { | |||
|         return commitResult; | ||||
|     } | ||||
| 
 | ||||
|     private ConsumerRebalanceListenerCallbackCompletedEvent mockNewAssignmentStuckOnPartitionsRevokedCallback( | ||||
|         MembershipManagerImpl membershipManager, Uuid topicId, String topicName, | ||||
|         List<Integer> partitions, TopicPartition ownedPartition, ConsumerRebalanceListenerInvoker invoker) { | ||||
|         doNothing().when(subscriptionState).markPendingRevocation(anySet()); | ||||
|         CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); | ||||
|         when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(ownedPartition)); | ||||
|         when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); | ||||
|         when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); | ||||
|         when(commitRequestManager.autoCommitEnabled()).thenReturn(false); | ||||
| 
 | ||||
|         when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); | ||||
|         receiveAssignment(topicId, partitions, membershipManager); | ||||
|         membershipManager.poll(time.milliseconds()); | ||||
|         verifyReconciliationTriggered(membershipManager); | ||||
|         clearInvocations(membershipManager); | ||||
|         assertEquals(MemberState.RECONCILING, membershipManager.state()); | ||||
| 
 | ||||
|         return performCallback( | ||||
|             membershipManager, | ||||
|             invoker, | ||||
|             ConsumerRebalanceListenerMethodName.ON_PARTITIONS_REVOKED, | ||||
|             topicPartitions(ownedPartition.topic(), ownedPartition.partition()), | ||||
|             false | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     private ConsumerRebalanceListenerCallbackCompletedEvent mockNewAssignmentStuckOnPartitionsAssignedCallback( | ||||
|         MembershipManagerImpl membershipManager, Uuid topicId, String topicName, int newPartition, | ||||
|         ConsumerRebalanceListenerInvoker invoker) { | ||||
|         CounterConsumerRebalanceListener listener = new CounterConsumerRebalanceListener(); | ||||
|         when(subscriptionState.assignedPartitions()).thenReturn(Collections.emptySet()); | ||||
|         when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); | ||||
|         when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); | ||||
|         when(commitRequestManager.autoCommitEnabled()).thenReturn(false); | ||||
| 
 | ||||
|         when(metadata.topicNames()).thenReturn(Collections.singletonMap(topicId, topicName)); | ||||
|         receiveAssignment(topicId, Collections.singletonList(newPartition), membershipManager); | ||||
|         membershipManager.poll(time.milliseconds()); | ||||
|         verifyReconciliationTriggered(membershipManager); | ||||
|         clearInvocations(membershipManager); | ||||
|         assertEquals(MemberState.RECONCILING, membershipManager.state()); | ||||
| 
 | ||||
|         return performCallback( | ||||
|             membershipManager, | ||||
|             invoker, | ||||
|             ConsumerRebalanceListenerMethodName.ON_PARTITIONS_ASSIGNED, | ||||
|             topicPartitions(topicName, newPartition), | ||||
|             false | ||||
|         ); | ||||
|     } | ||||
| 
 | ||||
|     private void verifyReconciliationTriggered(MembershipManagerImpl membershipManager) { | ||||
|         verify(membershipManager).markReconciliationInProgress(); | ||||
|         assertEquals(MemberState.RECONCILING, membershipManager.state()); | ||||
|  | @ -2348,7 +2457,7 @@ public class MembershipManagerImplTest { | |||
|         if (withAutoCommit) { | ||||
|             when(commitRequestManager.autoCommitEnabled()).thenReturn(true); | ||||
|             CompletableFuture<Void> commitResult = new CompletableFuture<>(); | ||||
|             when(commitRequestManager.maybeAutoCommitSyncNow(anyLong())).thenReturn(commitResult); | ||||
|             when(commitRequestManager.maybeAutoCommitSyncBeforeRevocation(anyLong())).thenReturn(commitResult); | ||||
|             return commitResult; | ||||
|         } else { | ||||
|             return CompletableFuture.completedFuture(null); | ||||
|  | @ -2480,7 +2589,15 @@ public class MembershipManagerImplTest { | |||
|         membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); | ||||
|     } | ||||
| 
 | ||||
|     private void receiveAssignmentAfterRejoin(Uuid topicId, List<Integer> partitions, MembershipManager membershipManager) { | ||||
|     private Map<Uuid, SortedSet<Integer>> receiveAssignmentAfterRejoin(List<Integer> partitions, | ||||
|                                                                        MembershipManagerImpl membershipManager, | ||||
|                                                                        Collection<TopicIdPartition> owned) { | ||||
|         // Get new assignment after rejoining. This should not trigger a reconciliation just | ||||
|         // yet because there is another one in progress, but should keep the new assignment ready | ||||
|         // to be reconciled next. | ||||
|         Uuid topicId = Uuid.randomUuid(); | ||||
|         mockOwnedPartitionAndAssignmentReceived(membershipManager, topicId, "topic3", owned); | ||||
| 
 | ||||
|         ConsumerGroupHeartbeatResponseData.Assignment targetAssignment = new ConsumerGroupHeartbeatResponseData.Assignment() | ||||
|                 .setTopicPartitions(Collections.singletonList( | ||||
|                         new ConsumerGroupHeartbeatResponseData.TopicPartitions() | ||||
|  | @ -2489,6 +2606,29 @@ public class MembershipManagerImplTest { | |||
|         ConsumerGroupHeartbeatResponse heartbeatResponse = | ||||
|                 createConsumerGroupHeartbeatResponseWithBumpedEpoch(targetAssignment); | ||||
|         membershipManager.onHeartbeatSuccess(heartbeatResponse.data()); | ||||
| 
 | ||||
|         verifyReconciliationNotTriggered(membershipManager); | ||||
|         Map<Uuid, SortedSet<Integer>> assignmentAfterRejoin = topicIdPartitionsMap(topicId, 5); | ||||
|         assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); | ||||
|         return assignmentAfterRejoin; | ||||
|     } | ||||
| 
 | ||||
|     private void assertInitialReconciliationDiscardedAfterRejoin( | ||||
|         MembershipManagerImpl membershipManager, | ||||
|         Map<Uuid, SortedSet<Integer>> assignmentAfterRejoin) { | ||||
|         verify(subscriptionState, never()).markPendingRevocation(any()); | ||||
|         verify(subscriptionState, never()).assignFromSubscribed(anyCollection()); | ||||
|         assertNotEquals(MemberState.ACKNOWLEDGING, membershipManager.state()); | ||||
| 
 | ||||
|         // Assignment received after rejoining should be ready to reconcile on the next | ||||
|         // reconciliation loop. | ||||
|         assertEquals(assignmentAfterRejoin, membershipManager.topicPartitionsAwaitingReconciliation()); | ||||
| 
 | ||||
|         // Stale reconciliation should have been aborted and a new one should be triggered on the next poll. | ||||
|         assertFalse(membershipManager.reconciliationInProgress()); | ||||
|         clearInvocations(membershipManager); | ||||
|         membershipManager.poll(time.milliseconds()); | ||||
|         verify(membershipManager).markReconciliationInProgress(); | ||||
|     } | ||||
| 
 | ||||
|     private void receiveEmptyAssignment(MembershipManager membershipManager) { | ||||
|  | @ -2578,7 +2718,6 @@ public class MembershipManagerImplTest { | |||
|         when(subscriptionState.assignedPartitions()).thenReturn(Collections.singleton(ownedPartition)); | ||||
|         when(subscriptionState.hasAutoAssignedPartitions()).thenReturn(true); | ||||
|         when(subscriptionState.rebalanceListener()).thenReturn(Optional.of(listener)); | ||||
|         // doNothing().when(subscriptionState).markPendingRevocation(anySet()); | ||||
|         when(commitRequestManager.autoCommitEnabled()).thenReturn(false); | ||||
|         membershipManager.transitionToFenced(); | ||||
|         return performCallback( | ||||
|  |  | |||
|  | @ -87,6 +87,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; | |||
| import static org.apache.kafka.test.TestUtils.assertOptional; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
|  | @ -462,7 +463,7 @@ public class OffsetFetcherTest { | |||
|         OffsetResetStrategy strategy, | ||||
|         AbstractRequest request | ||||
|     ) { | ||||
|         assertTrue(request instanceof ListOffsetsRequest); | ||||
|         assertInstanceOf(ListOffsetsRequest.class, request); | ||||
| 
 | ||||
|         ListOffsetsRequest req = (ListOffsetsRequest) request; | ||||
|         assertEquals(singleton(tp.topic()), req.data().topics().stream() | ||||
|  |  | |||
|  | @ -835,7 +835,7 @@ public class OffsetsRequestManagerTest { | |||
|             final int endOffset) { | ||||
| 
 | ||||
|         AbstractRequest abstractRequest = request.requestBuilder().build(); | ||||
|         assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); | ||||
|         assertInstanceOf(OffsetsForLeaderEpochRequest.class, abstractRequest); | ||||
|         OffsetsForLeaderEpochRequest offsetsForLeaderEpochRequest = (OffsetsForLeaderEpochRequest) abstractRequest; | ||||
|         OffsetForLeaderEpochResponseData data = new OffsetForLeaderEpochResponseData(); | ||||
|         partitions.forEach(tp -> { | ||||
|  | @ -870,7 +870,7 @@ public class OffsetsRequestManagerTest { | |||
|             final Map<TopicPartition, Errors> partitionErrors) { | ||||
| 
 | ||||
|         AbstractRequest abstractRequest = request.requestBuilder().build(); | ||||
|         assertTrue(abstractRequest instanceof OffsetsForLeaderEpochRequest); | ||||
|         assertInstanceOf(OffsetsForLeaderEpochRequest.class, abstractRequest); | ||||
|         OffsetsForLeaderEpochRequest offsetsForLeaderEpochRequest = (OffsetsForLeaderEpochRequest) abstractRequest; | ||||
|         OffsetForLeaderEpochResponseData data = new OffsetForLeaderEpochResponseData(); | ||||
|         partitionErrors.keySet().forEach(tp -> { | ||||
|  | @ -931,7 +931,7 @@ public class OffsetsRequestManagerTest { | |||
|             final boolean disconnected, | ||||
|             final AuthenticationException authenticationException) { | ||||
|         AbstractRequest abstractRequest = request.requestBuilder().build(); | ||||
|         assertTrue(abstractRequest instanceof ListOffsetsRequest); | ||||
|         assertInstanceOf(ListOffsetsRequest.class, abstractRequest); | ||||
|         ListOffsetsRequest offsetFetchRequest = (ListOffsetsRequest) abstractRequest; | ||||
|         ListOffsetsResponse response = buildListOffsetsResponse(topicResponses); | ||||
|         return new ClientResponse( | ||||
|  |  | |||
|  | @ -56,6 +56,7 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_ | |||
| import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.mockito.Mockito.spy; | ||||
| 
 | ||||
|  | @ -222,7 +223,7 @@ public class TopicMetadataRequestManagerTest { | |||
|             final String topic, | ||||
|             final Errors error) { | ||||
|         AbstractRequest abstractRequest = request.requestBuilder().build(); | ||||
|         assertTrue(abstractRequest instanceof MetadataRequest); | ||||
|         assertInstanceOf(MetadataRequest.class, abstractRequest); | ||||
|         MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; | ||||
|         Cluster cluster = mockCluster(3, 0); | ||||
|         List<MetadataResponse.TopicMetadata> topics = new ArrayList<>(); | ||||
|  | @ -248,7 +249,7 @@ public class TopicMetadataRequestManagerTest { | |||
|         final NetworkClientDelegate.UnsentRequest request, | ||||
|         final Errors error) { | ||||
|         AbstractRequest abstractRequest = request.requestBuilder().build(); | ||||
|         assertTrue(abstractRequest instanceof MetadataRequest); | ||||
|         assertInstanceOf(MetadataRequest.class, abstractRequest); | ||||
|         MetadataRequest metadataRequest = (MetadataRequest) abstractRequest; | ||||
|         Cluster cluster = mockCluster(3, 0); | ||||
|         List<MetadataResponse.TopicMetadata> topics = new ArrayList<>(); | ||||
|  |  | |||
|  | @ -33,7 +33,6 @@ import static org.junit.jupiter.api.Assertions.assertInstanceOf; | |||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.fail; | ||||
| import static org.mockito.Mockito.mock; | ||||
| import static org.mockito.Mockito.never; | ||||
| import static org.mockito.Mockito.verify; | ||||
|  | @ -174,14 +173,7 @@ public class WakeupTriggerTest { | |||
| 
 | ||||
|     private void assertWakeupExceptionIsThrown(final CompletableFuture<?> future) { | ||||
|         assertTrue(future.isCompletedExceptionally()); | ||||
|         try { | ||||
|             future.get(DEFAULT_TIMEOUT_MS, TimeUnit.MILLISECONDS); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof WakeupException); | ||||
|             return; | ||||
|         } catch (Exception e) { | ||||
|             fail("The task should throw an ExecutionException but got:" + e); | ||||
|         } | ||||
|         fail("The task should throw an ExecutionException"); | ||||
|         assertInstanceOf(WakeupException.class, | ||||
|             assertThrows(ExecutionException.class, () -> future.get(DEFAULT_TIMEOUT_MS, TimeUnit.MILLISECONDS)).getCause()); | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -127,6 +127,7 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals; | |||
| import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
|  | @ -491,7 +492,7 @@ public class KafkaProducerTest { | |||
|         props.setProperty(ProducerConfig.ENABLE_METRICS_PUSH_CONFIG, "false"); | ||||
|         KafkaProducer<String, String> producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); | ||||
|         assertEquals(1, producer.metrics.reporters().size()); | ||||
|         assertTrue(producer.metrics.reporters().get(0) instanceof JmxReporter); | ||||
|         assertInstanceOf(JmxReporter.class, producer.metrics.reporters().get(0)); | ||||
|         producer.close(); | ||||
|     } | ||||
| 
 | ||||
|  | @ -503,7 +504,7 @@ public class KafkaProducerTest { | |||
|         props.setProperty(ProducerConfig.AUTO_INCLUDE_JMX_REPORTER_CONFIG, "false"); | ||||
|         KafkaProducer<String, String> producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); | ||||
|         assertEquals(1, producer.metrics.reporters().size()); | ||||
|         assertTrue(producer.metrics.reporters().get(0) instanceof ClientTelemetryReporter); | ||||
|         assertInstanceOf(ClientTelemetryReporter.class, producer.metrics.reporters().get(0)); | ||||
|         producer.close(); | ||||
|     } | ||||
| 
 | ||||
|  | @ -692,7 +693,7 @@ public class KafkaProducerTest { | |||
|             TestUtils.waitForCondition(() -> closeException.get() != null, | ||||
|                     "InterruptException did not occur within timeout."); | ||||
| 
 | ||||
|             assertTrue(closeException.get() instanceof InterruptException, "Expected exception not thrown " + closeException); | ||||
|             assertInstanceOf(InterruptException.class, closeException.get(), "Expected exception not thrown " + closeException); | ||||
|         } finally { | ||||
|             executor.shutdownNow(); | ||||
|         } | ||||
|  | @ -848,9 +849,7 @@ public class KafkaProducerTest { | |||
|         verify(metadata, times(4)).awaitUpdate(anyInt(), anyLong()); | ||||
|         verify(metadata, times(5)).fetch(); | ||||
|         try { | ||||
|             future.get(); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof TimeoutException); | ||||
|             assertInstanceOf(TimeoutException.class, assertThrows(ExecutionException.class, future::get).getCause()); | ||||
|         } finally { | ||||
|             producer.close(Duration.ofMillis(0)); | ||||
|         } | ||||
|  | @ -917,9 +916,7 @@ public class KafkaProducerTest { | |||
|         verify(metadata, times(4)).awaitUpdate(anyInt(), anyLong()); | ||||
|         verify(metadata, times(5)).fetch(); | ||||
|         try { | ||||
|             future.get(); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof TimeoutException); | ||||
|             assertInstanceOf(TimeoutException.class, assertThrows(ExecutionException.class, future::get).getCause()); | ||||
|         } finally { | ||||
|             producer.close(Duration.ofMillis(0)); | ||||
|         } | ||||
|  |  | |||
|  | @ -43,6 +43,7 @@ import static java.util.Collections.singletonList; | |||
| import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
|  | @ -262,7 +263,7 @@ public class MockProducerTest { | |||
|         producer.initTransactions(); | ||||
|         producer.fenceProducer(); | ||||
|         Throwable e = assertThrows(KafkaException.class, () -> producer.send(null)); | ||||
|         assertTrue(e.getCause() instanceof ProducerFencedException, "The root cause of the exception should be ProducerFenced"); | ||||
|         assertInstanceOf(ProducerFencedException.class, e.getCause(), "The root cause of the exception should be ProducerFenced"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -42,6 +42,7 @@ import org.apache.kafka.common.errors.TopicAuthorizationException; | |||
| import org.apache.kafka.common.errors.TransactionAbortedException; | ||||
| import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; | ||||
| import org.apache.kafka.common.errors.UnsupportedVersionException; | ||||
| import org.apache.kafka.common.errors.TransactionAbortableException; | ||||
| import org.apache.kafka.common.internals.ClusterResourceListeners; | ||||
| import org.apache.kafka.common.message.AddPartitionsToTxnResponseData; | ||||
| import org.apache.kafka.common.message.ApiMessageType; | ||||
|  | @ -114,6 +115,7 @@ import java.util.concurrent.atomic.AtomicReference; | |||
| import static org.apache.kafka.clients.producer.internals.ProducerTestUtils.runUntil; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
|  | @ -130,9 +132,11 @@ import static org.mockito.ArgumentMatchers.anyString; | |||
| import static org.mockito.ArgumentMatchers.eq; | ||||
| import static org.mockito.Mockito.atLeastOnce; | ||||
| import static org.mockito.Mockito.inOrder; | ||||
| import static org.mockito.Mockito.mock; | ||||
| import static org.mockito.Mockito.spy; | ||||
| import static org.mockito.Mockito.times; | ||||
| import static org.mockito.Mockito.verify; | ||||
| import static org.mockito.Mockito.when; | ||||
| 
 | ||||
| public class SenderTest { | ||||
|     private static final int MAX_REQUEST_SIZE = 1024 * 1024; | ||||
|  | @ -733,7 +737,7 @@ public class SenderTest { | |||
|         prepareAndReceiveInitProducerId(producerId, Errors.CLUSTER_AUTHORIZATION_FAILED); | ||||
|         assertFalse(transactionManager.hasProducerId()); | ||||
|         assertTrue(transactionManager.hasError()); | ||||
|         assertTrue(transactionManager.lastError() instanceof ClusterAuthorizationException); | ||||
|         assertInstanceOf(ClusterAuthorizationException.class, transactionManager.lastError()); | ||||
|         assertEquals(-1, transactionManager.producerIdAndEpoch().epoch); | ||||
| 
 | ||||
|         assertSendFailure(ClusterAuthorizationException.class); | ||||
|  | @ -768,11 +772,7 @@ public class SenderTest { | |||
|         }, produceResponse(tp0, -1L, Errors.TOPIC_AUTHORIZATION_FAILED, 0)); | ||||
|         sender.runOnce(); | ||||
|         assertTrue(future.isDone()); | ||||
|         try { | ||||
|             future.get(); | ||||
|         } catch (Exception e) { | ||||
|             assertTrue(e.getCause() instanceof TopicAuthorizationException); | ||||
|         } | ||||
|         assertInstanceOf(TopicAuthorizationException.class, assertThrows(Exception.class, future::get).getCause()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -2539,12 +2539,10 @@ public class SenderTest { | |||
|         time.sleep(deliveryTimeoutMs); | ||||
|         sender.runOnce();  // receive first response | ||||
|         assertEquals(0, sender.inFlightBatches(tp0).size(), "Expect zero in-flight batch in accumulator"); | ||||
|         try { | ||||
|             request.get(); | ||||
|             fail("The expired batch should throw a TimeoutException"); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof TimeoutException); | ||||
|         } | ||||
|         assertInstanceOf( | ||||
|             TimeoutException.class, | ||||
|             assertThrows(ExecutionException.class, request::get).getCause(), | ||||
|             "The expired batch should throw a TimeoutException"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -2578,10 +2576,10 @@ public class SenderTest { | |||
|             KafkaException exception = TestUtils.assertFutureThrows(future, KafkaException.class); | ||||
|             Integer index = futureEntry.getKey(); | ||||
|             if (index == 0 || index == 2) { | ||||
|                 assertTrue(exception instanceof InvalidRecordException); | ||||
|                 assertInstanceOf(InvalidRecordException.class, exception); | ||||
|                 assertEquals(index.toString(), exception.getMessage()); | ||||
|             } else if (index == 3) { | ||||
|                 assertTrue(exception instanceof InvalidRecordException); | ||||
|                 assertInstanceOf(InvalidRecordException.class, exception); | ||||
|                 assertEquals(Errors.INVALID_RECORD.message(), exception.getMessage()); | ||||
|             } else { | ||||
|                 assertEquals(KafkaException.class, exception.getClass()); | ||||
|  | @ -2722,10 +2720,10 @@ public class SenderTest { | |||
|         assertEquals(0, sender.inFlightBatches(tp0).size(), "Expect zero in-flight batch in accumulator"); | ||||
| 
 | ||||
|         ExecutionException e = assertThrows(ExecutionException.class, request1::get); | ||||
|         assertTrue(e.getCause() instanceof TimeoutException); | ||||
|         assertInstanceOf(TimeoutException.class, e.getCause()); | ||||
| 
 | ||||
|         e = assertThrows(ExecutionException.class, request2::get); | ||||
|         assertTrue(e.getCause() instanceof TimeoutException); | ||||
|         assertInstanceOf(TimeoutException.class, e.getCause()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -3151,6 +3149,45 @@ public class SenderTest { | |||
| 
 | ||||
|         txnManager.beginTransaction(); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testTransactionAbortablenExceptionIsAnAbortableError() throws Exception { | ||||
|         ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); | ||||
|         apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); | ||||
|         TransactionManager txnManager = new TransactionManager(logContext, "textTransactionAbortableException", 60000, 100, apiVersions); | ||||
| 
 | ||||
|         setupWithTransactionState(txnManager); | ||||
|         doInitTransactions(txnManager, producerIdAndEpoch); | ||||
| 
 | ||||
|         txnManager.beginTransaction(); | ||||
|         txnManager.maybeAddPartition(tp0); | ||||
|         client.prepareResponse(buildAddPartitionsToTxnResponseData(0, Collections.singletonMap(tp0, Errors.NONE))); | ||||
|         sender.runOnce(); | ||||
| 
 | ||||
|         Future<RecordMetadata> request = appendToAccumulator(tp0); | ||||
|         sender.runOnce();  // send request | ||||
|         sendIdempotentProducerResponse(0, tp0, Errors.TRANSACTION_ABORTABLE, -1); | ||||
| 
 | ||||
|         // Return TransactionAbortableException error. It should be abortable. | ||||
|         sender.runOnce(); | ||||
|         assertFutureFailure(request, TransactionAbortableException.class); | ||||
|         assertTrue(txnManager.hasAbortableError()); | ||||
|         TransactionalRequestResult result = txnManager.beginAbort(); | ||||
|         sender.runOnce(); | ||||
| 
 | ||||
|         // Once the transaction is aborted, we should be able to begin a new one. | ||||
|         respondToEndTxn(Errors.NONE); | ||||
|         sender.runOnce(); | ||||
|         assertTrue(txnManager::isInitializing); | ||||
|         prepareInitProducerResponse(Errors.NONE, producerIdAndEpoch.producerId, producerIdAndEpoch.epoch); | ||||
|         sender.runOnce(); | ||||
|         assertTrue(txnManager::isReady); | ||||
| 
 | ||||
|         assertTrue(result.isSuccessful()); | ||||
|         result.await(); | ||||
| 
 | ||||
|         txnManager.beginTransaction(); | ||||
|     } | ||||
|     @Test | ||||
|     public void testProducerBatchRetriesWhenPartitionLeaderChanges() throws Exception { | ||||
|         Metrics m = new Metrics(); | ||||
|  | @ -3239,6 +3276,26 @@ public class SenderTest { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     // This test is expected to run fast. If timeout, the sender is not able to close properly. | ||||
|     @Timeout(5) | ||||
|     @Test | ||||
|     public void testSenderShouldCloseWhenTransactionManagerInErrorState() throws Exception { | ||||
|         metrics.close(); | ||||
|         Map<String, String> clientTags = Collections.singletonMap("client-id", "clientA"); | ||||
|         metrics = new Metrics(new MetricConfig().tags(clientTags)); | ||||
|         TransactionManager transactionManager = mock(TransactionManager.class); | ||||
|         SenderMetricsRegistry metricsRegistry = new SenderMetricsRegistry(metrics); | ||||
|         Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, | ||||
|                 1, metricsRegistry, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, transactionManager, apiVersions); | ||||
|         when(transactionManager.hasOngoingTransaction()).thenReturn(true); | ||||
|         when(transactionManager.beginAbort()).thenThrow(new IllegalStateException()); | ||||
|         sender.initiateClose(); | ||||
| 
 | ||||
|         // The sender should directly get closed. | ||||
|         sender.run(); | ||||
|         verify(transactionManager, times(1)).close(); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|      * Test the scenario that FetchResponse returns NOT_LEADER_OR_FOLLOWER, indicating change in leadership, but it | ||||
|      * does not contain new leader info(defined in KIP-951). | ||||
|  |  | |||
|  | @ -40,6 +40,7 @@ import org.apache.kafka.common.errors.TopicAuthorizationException; | |||
| import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; | ||||
| import org.apache.kafka.common.errors.UnsupportedForMessageFormatException; | ||||
| import org.apache.kafka.common.errors.UnsupportedVersionException; | ||||
| import org.apache.kafka.common.errors.TransactionAbortableException; | ||||
| import org.apache.kafka.common.header.Header; | ||||
| import org.apache.kafka.common.internals.ClusterResourceListeners; | ||||
| import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; | ||||
|  | @ -109,6 +110,7 @@ import static java.util.Collections.singletonList; | |||
| import static java.util.Collections.singletonMap; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
|  | @ -883,7 +885,7 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         runUntil(transactionManager::hasFatalError); | ||||
|         assertTrue(transactionManager.hasFatalError()); | ||||
|         assertTrue(transactionManager.lastError() instanceof UnsupportedVersionException); | ||||
|         assertInstanceOf(UnsupportedVersionException.class, transactionManager.lastError()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -902,7 +904,7 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         runUntil(transactionManager::hasFatalError); | ||||
|         assertTrue(transactionManager.hasFatalError()); | ||||
|         assertTrue(transactionManager.lastError() instanceof UnsupportedVersionException); | ||||
|         assertInstanceOf(UnsupportedVersionException.class, transactionManager.lastError()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -920,10 +922,10 @@ public class TransactionManagerTest { | |||
|         prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp, Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT)); | ||||
|         runUntil(transactionManager::hasError); | ||||
| 
 | ||||
|         assertTrue(transactionManager.lastError() instanceof UnsupportedForMessageFormatException); | ||||
|         assertInstanceOf(UnsupportedForMessageFormatException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof UnsupportedForMessageFormatException); | ||||
|         assertInstanceOf(UnsupportedForMessageFormatException.class, sendOffsetsResult.error()); | ||||
|         assertFatalError(UnsupportedForMessageFormatException.class); | ||||
|     } | ||||
| 
 | ||||
|  | @ -954,10 +956,10 @@ public class TransactionManagerTest { | |||
|         }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.FENCED_INSTANCE_ID))); | ||||
| 
 | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof FencedInstanceIdException); | ||||
|         assertInstanceOf(FencedInstanceIdException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof FencedInstanceIdException); | ||||
|         assertInstanceOf(FencedInstanceIdException.class, sendOffsetsResult.error()); | ||||
|         assertAbortableError(FencedInstanceIdException.class); | ||||
|     } | ||||
| 
 | ||||
|  | @ -987,10 +989,10 @@ public class TransactionManagerTest { | |||
|         }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.UNKNOWN_MEMBER_ID))); | ||||
| 
 | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof CommitFailedException); | ||||
|         assertInstanceOf(CommitFailedException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof CommitFailedException); | ||||
|         assertInstanceOf(CommitFailedException.class, sendOffsetsResult.error()); | ||||
|         assertAbortableError(CommitFailedException.class); | ||||
|     } | ||||
| 
 | ||||
|  | @ -1022,10 +1024,10 @@ public class TransactionManagerTest { | |||
|         }, new TxnOffsetCommitResponse(0, singletonMap(tp, Errors.ILLEGAL_GENERATION))); | ||||
| 
 | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof CommitFailedException); | ||||
|         assertInstanceOf(CommitFailedException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof CommitFailedException); | ||||
|         assertInstanceOf(CommitFailedException.class, sendOffsetsResult.error()); | ||||
|         assertAbortableError(CommitFailedException.class); | ||||
|     } | ||||
| 
 | ||||
|  | @ -1128,7 +1130,7 @@ public class TransactionManagerTest { | |||
|         runUntil(transactionManager::hasError); | ||||
| 
 | ||||
|         assertTrue(transactionManager.hasFatalError()); | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); | ||||
|         assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); | ||||
|         assertFalse(initPidResult.isSuccessful()); | ||||
|         assertThrows(TransactionalIdAuthorizationException.class, initPidResult::await); | ||||
|         assertFatalError(TransactionalIdAuthorizationException.class); | ||||
|  | @ -1162,11 +1164,11 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, false, CoordinatorType.GROUP, consumerGroupId); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); | ||||
|         assertInstanceOf(GroupAuthorizationException.class, transactionManager.lastError()); | ||||
| 
 | ||||
|         runUntil(sendOffsetsResult::isCompleted); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); | ||||
|         assertInstanceOf(GroupAuthorizationException.class, sendOffsetsResult.error()); | ||||
| 
 | ||||
|         GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); | ||||
|         assertEquals(consumerGroupId, exception.groupId()); | ||||
|  | @ -1191,10 +1193,10 @@ public class TransactionManagerTest { | |||
|         prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp1, Errors.GROUP_AUTHORIZATION_FAILED)); | ||||
| 
 | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); | ||||
|         assertInstanceOf(GroupAuthorizationException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); | ||||
|         assertInstanceOf(GroupAuthorizationException.class, sendOffsetsResult.error()); | ||||
|         assertFalse(transactionManager.hasPendingOffsetCommits()); | ||||
| 
 | ||||
|         GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); | ||||
|  | @ -1215,10 +1217,10 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         prepareAddOffsetsToTxnResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, consumerGroupId, producerId, epoch); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); | ||||
|         assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); | ||||
|         assertInstanceOf(TransactionalIdAuthorizationException.class, sendOffsetsResult.error()); | ||||
| 
 | ||||
|         assertFatalError(TransactionalIdAuthorizationException.class); | ||||
|     } | ||||
|  | @ -1235,10 +1237,10 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         prepareAddOffsetsToTxnResponse(Errors.INVALID_TXN_STATE, consumerGroupId, producerId, epoch); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof InvalidTxnStateException); | ||||
|         assertInstanceOf(InvalidTxnStateException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof InvalidTxnStateException); | ||||
|         assertInstanceOf(InvalidTxnStateException.class, sendOffsetsResult.error()); | ||||
| 
 | ||||
|         assertFatalError(InvalidTxnStateException.class); | ||||
|     } | ||||
|  | @ -1260,10 +1262,10 @@ public class TransactionManagerTest { | |||
|         prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)); | ||||
|         runUntil(transactionManager::hasError); | ||||
| 
 | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); | ||||
|         assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); | ||||
|         assertInstanceOf(TransactionalIdAuthorizationException.class, sendOffsetsResult.error()); | ||||
| 
 | ||||
|         assertFatalError(TransactionalIdAuthorizationException.class); | ||||
|     } | ||||
|  | @ -1289,7 +1291,7 @@ public class TransactionManagerTest { | |||
|         prepareAddPartitionsToTxn(errors); | ||||
|         runUntil(transactionManager::hasError); | ||||
| 
 | ||||
|         assertTrue(transactionManager.lastError() instanceof TopicAuthorizationException); | ||||
|         assertInstanceOf(TopicAuthorizationException.class, transactionManager.lastError()); | ||||
|         assertFalse(transactionManager.isPartitionPendingAdd(tp0)); | ||||
|         assertFalse(transactionManager.isPartitionPendingAdd(tp1)); | ||||
|         assertFalse(transactionManager.isPartitionAdded(tp0)); | ||||
|  | @ -1351,7 +1353,7 @@ public class TransactionManagerTest { | |||
|         assertTrue(commitResult.isCompleted()); | ||||
|         TestUtils.assertFutureThrows(firstPartitionAppend, KafkaException.class); | ||||
|         TestUtils.assertFutureThrows(secondPartitionAppend, KafkaException.class); | ||||
|         assertTrue(commitResult.error() instanceof TopicAuthorizationException); | ||||
|         assertInstanceOf(TopicAuthorizationException.class, commitResult.error()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -1626,7 +1628,7 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         prepareAddPartitionsToTxn(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); | ||||
|         assertInstanceOf(TransactionalIdAuthorizationException.class, transactionManager.lastError()); | ||||
| 
 | ||||
|         assertFatalError(TransactionalIdAuthorizationException.class); | ||||
|     } | ||||
|  | @ -1642,7 +1644,7 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         prepareAddPartitionsToTxn(tp, Errors.INVALID_TXN_STATE); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof InvalidTxnStateException); | ||||
|         assertInstanceOf(InvalidTxnStateException.class, transactionManager.lastError()); | ||||
| 
 | ||||
|         assertFatalError(InvalidTxnStateException.class); | ||||
|     } | ||||
|  | @ -1866,7 +1868,7 @@ public class TransactionManagerTest { | |||
|             responseFuture.get(); | ||||
|             fail("Expected to get a ExecutionException from the response"); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof ProducerFencedException); | ||||
|             assertInstanceOf(ProducerFencedException.class, e.getCause()); | ||||
|         } | ||||
| 
 | ||||
|         // make sure the exception was thrown directly from the follow-up calls. | ||||
|  | @ -1932,13 +1934,13 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         // First we will get an EndTxn for abort. | ||||
|         assertNotNull(handler); | ||||
|         assertTrue(handler.requestBuilder() instanceof EndTxnRequest.Builder); | ||||
|         assertInstanceOf(EndTxnRequest.Builder.class, handler.requestBuilder()); | ||||
| 
 | ||||
|         handler = transactionManager.nextRequest(false); | ||||
| 
 | ||||
|         // Second we will see an InitPid for handling InvalidProducerEpoch. | ||||
|         assertNotNull(handler); | ||||
|         assertTrue(handler.requestBuilder() instanceof InitProducerIdRequest.Builder); | ||||
|         assertInstanceOf(InitProducerIdRequest.Builder.class, handler.requestBuilder()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -2433,7 +2435,7 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         assertTrue(addOffsetsResult.isCompleted()); | ||||
|         assertFalse(addOffsetsResult.isSuccessful()); | ||||
|         assertTrue(addOffsetsResult.error() instanceof UnsupportedVersionException); | ||||
|         assertInstanceOf(UnsupportedVersionException.class, addOffsetsResult.error()); | ||||
|         assertFatalError(UnsupportedVersionException.class); | ||||
|     } | ||||
| 
 | ||||
|  | @ -2614,13 +2616,11 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         runUntil(responseFuture::isDone); | ||||
| 
 | ||||
|         try { | ||||
|         // make sure the produce was expired. | ||||
|             responseFuture.get(); | ||||
|             fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof  TimeoutException); | ||||
|         } | ||||
|         assertInstanceOf( | ||||
|             TimeoutException.class, | ||||
|             assertThrows(ExecutionException.class, responseFuture::get).getCause(), | ||||
|             "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         assertTrue(transactionManager.hasAbortableError()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -2664,21 +2664,17 @@ public class TransactionManagerTest { | |||
|         runUntil(firstBatchResponse::isDone); | ||||
|         runUntil(secondBatchResponse::isDone); | ||||
| 
 | ||||
|         try { | ||||
|         // make sure the produce was expired. | ||||
|             firstBatchResponse.get(); | ||||
|             fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof  TimeoutException); | ||||
|         } | ||||
|         assertInstanceOf( | ||||
|             TimeoutException.class, | ||||
|             assertThrows(ExecutionException.class, firstBatchResponse::get).getCause(), | ||||
|             "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         // make sure the produce was expired. | ||||
|         assertInstanceOf( | ||||
|             TimeoutException.class, | ||||
|             assertThrows(ExecutionException.class, secondBatchResponse::get).getCause(), | ||||
|             "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
| 
 | ||||
|         try { | ||||
|             // make sure the produce was expired. | ||||
|             secondBatchResponse.get(); | ||||
|             fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof  TimeoutException); | ||||
|         } | ||||
|         assertTrue(transactionManager.hasAbortableError()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -2713,13 +2709,11 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         runUntil(responseFuture::isDone);  // We should try to flush the produce, but expire it instead without sending anything. | ||||
| 
 | ||||
|         try { | ||||
|         // make sure the produce was expired. | ||||
|             responseFuture.get(); | ||||
|             fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof  TimeoutException); | ||||
|         } | ||||
|         assertInstanceOf( | ||||
|             TimeoutException.class, | ||||
|             assertThrows(ExecutionException.class, responseFuture::get).getCause(), | ||||
|             "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         runUntil(commitResult::isCompleted);  // the commit shouldn't be completed without being sent since the produce request failed. | ||||
|         assertFalse(commitResult.isSuccessful());  // the commit shouldn't succeed since the produce request failed. | ||||
|         assertThrows(TimeoutException.class, commitResult::await); | ||||
|  | @ -2784,13 +2778,11 @@ public class TransactionManagerTest { | |||
| 
 | ||||
|         runUntil(responseFuture::isDone);  // We should try to flush the produce, but expire it instead without sending anything. | ||||
| 
 | ||||
|         try { | ||||
|         // make sure the produce was expired. | ||||
|             responseFuture.get(); | ||||
|             fail("Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         } catch (ExecutionException e) { | ||||
|             assertTrue(e.getCause() instanceof  TimeoutException); | ||||
|         } | ||||
|         assertInstanceOf( | ||||
|             TimeoutException.class, | ||||
|             assertThrows(ExecutionException.class, responseFuture::get).getCause(), | ||||
|             "Expected to get a TimeoutException since the queued ProducerBatch should have been expired"); | ||||
|         runUntil(commitResult::isCompleted); | ||||
|         assertFalse(commitResult.isSuccessful());  // the commit should have been dropped. | ||||
| 
 | ||||
|  | @ -3523,6 +3515,126 @@ public class TransactionManagerTest { | |||
|         assertFalse(transactionManager.hasOngoingTransaction()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testTransactionAbortableExceptionInInitProducerId() { | ||||
|         TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); | ||||
|         prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); | ||||
|         runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); | ||||
|         assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); | ||||
| 
 | ||||
|         prepareInitPidResponse(Errors.TRANSACTION_ABORTABLE, false, producerId, RecordBatch.NO_PRODUCER_EPOCH); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(initPidResult.isCompleted()); | ||||
|         assertFalse(initPidResult.isSuccessful()); | ||||
|         assertThrows(TransactionAbortableException.class, initPidResult::await); | ||||
|         assertAbortableError(TransactionAbortableException.class); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testTransactionAbortableExceptionInAddPartitions() { | ||||
|         final TopicPartition tp = new TopicPartition("foo", 0); | ||||
| 
 | ||||
|         doInitTransactions(); | ||||
| 
 | ||||
|         transactionManager.beginTransaction(); | ||||
|         transactionManager.maybeAddPartition(tp); | ||||
| 
 | ||||
|         prepareAddPartitionsToTxn(tp, Errors.TRANSACTION_ABORTABLE); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionAbortableException); | ||||
| 
 | ||||
|         assertAbortableError(TransactionAbortableException.class); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testTransactionAbortableExceptionInFindCoordinator() { | ||||
|         doInitTransactions(); | ||||
| 
 | ||||
|         transactionManager.beginTransaction(); | ||||
|         TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( | ||||
|                 singletonMap(new TopicPartition("foo", 0), new OffsetAndMetadata(39L)), new ConsumerGroupMetadata(consumerGroupId)); | ||||
| 
 | ||||
|         prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, producerId, epoch); | ||||
|         runUntil(() -> !transactionManager.hasPartitionsToAdd()); | ||||
| 
 | ||||
|         prepareFindCoordinatorResponse(Errors.TRANSACTION_ABORTABLE, false, CoordinatorType.GROUP, consumerGroupId); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionAbortableException); | ||||
| 
 | ||||
|         runUntil(sendOffsetsResult::isCompleted); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof TransactionAbortableException); | ||||
| 
 | ||||
|         assertAbortableError(TransactionAbortableException.class); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testTransactionAbortableExceptionInEndTxn() throws InterruptedException { | ||||
|         doInitTransactions(); | ||||
| 
 | ||||
|         transactionManager.beginTransaction(); | ||||
|         transactionManager.maybeAddPartition(tp0); | ||||
|         TransactionalRequestResult commitResult = transactionManager.beginCommit(); | ||||
| 
 | ||||
|         Future<RecordMetadata> responseFuture = appendToAccumulator(tp0); | ||||
| 
 | ||||
|         assertFalse(responseFuture.isDone()); | ||||
|         prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, producerId); | ||||
|         prepareProduceResponse(Errors.NONE, producerId, epoch); | ||||
|         prepareEndTxnResponse(Errors.TRANSACTION_ABORTABLE, TransactionResult.COMMIT, producerId, epoch); | ||||
| 
 | ||||
|         runUntil(commitResult::isCompleted); | ||||
|         runUntil(responseFuture::isDone); | ||||
| 
 | ||||
|         assertThrows(KafkaException.class, commitResult::await); | ||||
|         assertFalse(commitResult.isSuccessful()); | ||||
|         assertTrue(commitResult.isAcked()); | ||||
| 
 | ||||
|         assertAbortableError(TransactionAbortableException.class); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testTransactionAbortableExceptionInAddOffsetsToTxn() { | ||||
|         final TopicPartition tp = new TopicPartition("foo", 0); | ||||
| 
 | ||||
|         doInitTransactions(); | ||||
| 
 | ||||
|         transactionManager.beginTransaction(); | ||||
|         TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( | ||||
|                 singletonMap(tp, new OffsetAndMetadata(39L)), new ConsumerGroupMetadata(consumerGroupId)); | ||||
| 
 | ||||
|         prepareAddOffsetsToTxnResponse(Errors.TRANSACTION_ABORTABLE, consumerGroupId, producerId, epoch); | ||||
|         runUntil(transactionManager::hasError); | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionAbortableException); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof TransactionAbortableException); | ||||
| 
 | ||||
|         assertAbortableError(TransactionAbortableException.class); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testTransactionAbortableExceptionInTxnOffsetCommit() { | ||||
|         final TopicPartition tp = new TopicPartition("foo", 0); | ||||
| 
 | ||||
|         doInitTransactions(); | ||||
| 
 | ||||
|         transactionManager.beginTransaction(); | ||||
|         TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( | ||||
|                 singletonMap(tp, new OffsetAndMetadata(39L)), new ConsumerGroupMetadata(consumerGroupId)); | ||||
| 
 | ||||
|         prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, producerId, epoch); | ||||
|         prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); | ||||
|         prepareTxnOffsetCommitResponse(consumerGroupId, producerId, epoch, singletonMap(tp, Errors.TRANSACTION_ABORTABLE)); | ||||
|         runUntil(transactionManager::hasError); | ||||
| 
 | ||||
|         assertTrue(transactionManager.lastError() instanceof TransactionAbortableException); | ||||
|         assertTrue(sendOffsetsResult.isCompleted()); | ||||
|         assertFalse(sendOffsetsResult.isSuccessful()); | ||||
|         assertTrue(sendOffsetsResult.error() instanceof TransactionAbortableException); | ||||
|         assertAbortableError(TransactionAbortableException.class); | ||||
|     } | ||||
| 
 | ||||
|     private FutureRecordMetadata appendToAccumulator(TopicPartition tp) throws InterruptedException { | ||||
|         final long nowMs = time.milliseconds(); | ||||
|         return accumulator.append(tp.topic(), tp.partition(), nowMs, "key".getBytes(), "value".getBytes(), Record.EMPTY_HEADERS, | ||||
|  |  | |||
|  | @ -34,6 +34,7 @@ import java.util.concurrent.TimeUnit; | |||
| import java.util.concurrent.TimeoutException; | ||||
| import java.util.function.Supplier; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
|  | @ -316,7 +317,7 @@ public class KafkaFutureTest { | |||
|         assertIsFailed(dependantFuture); | ||||
|         awaitAndAssertResult(future, 21, null); | ||||
|         Throwable cause = awaitAndAssertFailure(dependantFuture, CompletionException.class, "java.lang.RuntimeException: We require more vespene gas"); | ||||
|         assertTrue(cause.getCause() instanceof RuntimeException); | ||||
|         assertInstanceOf(RuntimeException.class, cause.getCause()); | ||||
|         assertEquals(cause.getCause().getMessage(), "We require more vespene gas"); | ||||
|     } | ||||
| 
 | ||||
|  | @ -438,7 +439,7 @@ public class KafkaFutureTest { | |||
|         assertFalse(dependantFuture.isDone()); | ||||
|         assertTrue(future.cancel(true)); | ||||
|         assertTrue(ran[0]); | ||||
|         assertTrue(err[0] instanceof CancellationException); | ||||
|         assertInstanceOf(CancellationException.class, err[0]); | ||||
|     } | ||||
| 
 | ||||
|     private static class CompleterThread<T> extends Thread { | ||||
|  |  | |||
|  | @ -21,7 +21,7 @@ import org.junit.jupiter.api.Test; | |||
| 
 | ||||
| import java.io.IOException; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| 
 | ||||
| /** | ||||
|  | @ -48,7 +48,7 @@ public class TopicPartitionTest { | |||
| 
 | ||||
|         //deserialize the byteArray and check if the values are same as original | ||||
|         Object deserializedObject = Serializer.deserialize(byteArray); | ||||
|         assertTrue(deserializedObject instanceof TopicPartition); | ||||
|         assertInstanceOf(TopicPartition.class, deserializedObject); | ||||
|         checkValues((TopicPartition) deserializedObject); | ||||
|     } | ||||
| 
 | ||||
|  | @ -57,7 +57,7 @@ public class TopicPartitionTest { | |||
|         // assert serialized TopicPartition object in file (serializedData/topicPartitionSerializedfile) is | ||||
|         // deserializable into TopicPartition and is compatible | ||||
|         Object deserializedObject = Serializer.deserialize(fileName); | ||||
|         assertTrue(deserializedObject instanceof TopicPartition); | ||||
|         assertInstanceOf(TopicPartition.class, deserializedObject); | ||||
|         checkValues((TopicPartition) deserializedObject); | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -19,6 +19,7 @@ package org.apache.kafka.common.config; | |||
| import org.apache.kafka.common.KafkaException; | ||||
| import org.apache.kafka.common.config.ConfigDef.Importance; | ||||
| import org.apache.kafka.common.config.ConfigDef.Type; | ||||
| import org.apache.kafka.common.config.provider.FileConfigProvider; | ||||
| import org.apache.kafka.common.config.types.Password; | ||||
| import org.apache.kafka.common.metrics.FakeMetricsReporter; | ||||
| import org.apache.kafka.common.metrics.JmxReporter; | ||||
|  | @ -26,7 +27,10 @@ import org.apache.kafka.common.metrics.MetricsReporter; | |||
| import org.apache.kafka.common.security.TestSecurityConfig; | ||||
| import org.apache.kafka.common.config.provider.MockVaultConfigProvider; | ||||
| import org.apache.kafka.common.config.provider.MockFileConfigProvider; | ||||
| import org.apache.kafka.common.utils.Utils; | ||||
| import org.apache.kafka.test.MockConsumerInterceptor; | ||||
| import org.junit.jupiter.api.AfterEach; | ||||
| import org.junit.jupiter.api.BeforeEach; | ||||
| import org.junit.jupiter.api.Test; | ||||
| 
 | ||||
| import java.util.Arrays; | ||||
|  | @ -46,6 +50,23 @@ import static org.junit.jupiter.api.Assertions.assertEquals; | |||
| 
 | ||||
| public class AbstractConfigTest { | ||||
| 
 | ||||
|     private String propertyValue; | ||||
| 
 | ||||
|     @BeforeEach | ||||
|     public void setup() { | ||||
|         propertyValue = System.getProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY); | ||||
|         System.clearProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY); | ||||
|     } | ||||
| 
 | ||||
|     @AfterEach | ||||
|     public void teardown() { | ||||
|         if (propertyValue != null) { | ||||
|             System.setProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY, propertyValue); | ||||
|         } else { | ||||
|             System.clearProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testConfiguredInstances() { | ||||
|         testValidInputs("    "); | ||||
|  | @ -254,12 +275,7 @@ public class AbstractConfigTest { | |||
|         Properties props = new Properties(); | ||||
|         props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, configValue); | ||||
|         TestConfig config = new TestConfig(props); | ||||
|         try { | ||||
|             config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); | ||||
|             fail("Expected a config exception due to invalid props :" + props); | ||||
|         } catch (KafkaException e) { | ||||
|             // this is good | ||||
|         } | ||||
|         assertThrows(KafkaException.class, () -> config.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -349,16 +365,6 @@ public class AbstractConfigTest { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     @SuppressWarnings("unchecked") | ||||
|     public Map<String, ?> convertPropertiesToMap(Map<?, ?> props) { | ||||
|         for (Map.Entry<?, ?> entry : props.entrySet()) { | ||||
|             if (!(entry.getKey() instanceof String)) | ||||
|                 throw new ConfigException(entry.getKey().toString(), entry.getValue(), | ||||
|                     "Key must be a string."); | ||||
|         } | ||||
|         return (Map<String, ?>) props; | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testOriginalWithOverrides() { | ||||
|         Properties props = new Properties(); | ||||
|  | @ -389,6 +395,43 @@ public class AbstractConfigTest { | |||
|         MockFileConfigProvider.assertClosed(id); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testOriginalsWithConfigProvidersPropsExcluded() { | ||||
|         System.setProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY, MockVaultConfigProvider.class.getName() + " , " + FileConfigProvider.class.getName()); | ||||
|         Properties props = new Properties(); | ||||
| 
 | ||||
|         // Test Case: Config provider that is not an allowed class | ||||
|         props.put("config.providers", "file"); | ||||
|         props.put("config.providers.file.class", MockFileConfigProvider.class.getName()); | ||||
|         String id = UUID.randomUUID().toString(); | ||||
|         props.put("config.providers.file.param.testId", id); | ||||
|         props.put("prefix.ssl.truststore.location.number", 5); | ||||
|         props.put("sasl.kerberos.service.name", "service name"); | ||||
|         props.put("sasl.kerberos.key", "${file:/usr/kerberos:key}"); | ||||
|         props.put("sasl.kerberos.password", "${file:/usr/kerberos:password}"); | ||||
|         assertThrows(ConfigException.class, () -> new TestIndirectConfigResolution(props, Collections.emptyMap())); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testOriginalsWithConfigProvidersPropsIncluded() { | ||||
|         System.setProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY, MockFileConfigProvider.class.getName() + " , " + FileConfigProvider.class.getName()); | ||||
|         Properties props = new Properties(); | ||||
| 
 | ||||
|         // Test Case: Config provider that is an allowed class | ||||
|         props.put("config.providers", "file"); | ||||
|         props.put("config.providers.file.class", MockFileConfigProvider.class.getName()); | ||||
|         String id = UUID.randomUUID().toString(); | ||||
|         props.put("config.providers.file.param.testId", id); | ||||
|         props.put("prefix.ssl.truststore.location.number", 5); | ||||
|         props.put("sasl.kerberos.service.name", "service name"); | ||||
|         props.put("sasl.kerberos.key", "${file:/usr/kerberos:key}"); | ||||
|         props.put("sasl.kerberos.password", "${file:/usr/kerberos:password}"); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, Collections.emptyMap()); | ||||
|         assertEquals("testKey", config.originals().get("sasl.kerberos.key")); | ||||
|         assertEquals("randomPassword", config.originals().get("sasl.kerberos.password")); | ||||
|         MockFileConfigProvider.assertClosed(id); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testConfigProvidersPropsAsParam() { | ||||
|         // Test Case: Valid Test Case for ConfigProviders as a separate variable | ||||
|  | @ -400,7 +443,7 @@ public class AbstractConfigTest { | |||
|         Properties props = new Properties(); | ||||
|         props.put("sasl.kerberos.key", "${file:/usr/kerberos:key}"); | ||||
|         props.put("sasl.kerberos.password", "${file:/usr/kerberos:password}"); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, convertPropertiesToMap(providers)); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, Utils.castToStringObjectMap(providers)); | ||||
|         assertEquals("testKey", config.originals().get("sasl.kerberos.key")); | ||||
|         assertEquals("randomPassword", config.originals().get("sasl.kerberos.password")); | ||||
|         MockFileConfigProvider.assertClosed(id); | ||||
|  | @ -417,7 +460,7 @@ public class AbstractConfigTest { | |||
|         Properties props = new Properties(); | ||||
|         props.put("sasl.kerberos.key", "${file:/usr/kerberos:key}"); | ||||
|         Map<?, ?> immutableMap = Collections.unmodifiableMap(props); | ||||
|         Map<String, ?> provMap = convertPropertiesToMap(providers); | ||||
|         Map<String, ?> provMap = Utils.castToStringObjectMap(providers); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(immutableMap, provMap); | ||||
|         assertEquals("testKey", config.originals().get("sasl.kerberos.key")); | ||||
|         MockFileConfigProvider.assertClosed(id); | ||||
|  | @ -437,7 +480,7 @@ public class AbstractConfigTest { | |||
|         props.put("sasl.kerberos.password", "${file:/usr/kerberos:password}"); | ||||
|         props.put("sasl.truststore.key", "${vault:/usr/truststore:truststoreKey}"); | ||||
|         props.put("sasl.truststore.password", "${vault:/usr/truststore:truststorePassword}"); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, convertPropertiesToMap(providers)); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, Utils.castToStringObjectMap(providers)); | ||||
|         assertEquals("testKey", config.originals().get("sasl.kerberos.key")); | ||||
|         assertEquals("randomPassword", config.originals().get("sasl.kerberos.password")); | ||||
|         assertEquals("testTruststoreKey", config.originals().get("sasl.truststore.key")); | ||||
|  | @ -453,12 +496,33 @@ public class AbstractConfigTest { | |||
|         props.put("config.providers.file.class", | ||||
|             "org.apache.kafka.common.config.provider.InvalidConfigProvider"); | ||||
|         props.put("testKey", "${test:/foo/bar/testpath:testKey}"); | ||||
|         try { | ||||
|             new TestIndirectConfigResolution(props); | ||||
|             fail("Expected a config exception due to invalid props :" + props); | ||||
|         } catch (KafkaException e) { | ||||
|             // this is good | ||||
|         assertThrows(KafkaException.class, () -> new TestIndirectConfigResolution(props)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testAutoConfigResolutionWithInvalidConfigProviderClassExcluded() { | ||||
|         String invalidConfigProvider = "org.apache.kafka.common.config.provider.InvalidConfigProvider"; | ||||
|         System.setProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY, ""); | ||||
|         // Test Case: Any config provider specified while the system property is empty | ||||
|         Properties props = new Properties(); | ||||
|         props.put("config.providers", "file"); | ||||
|         props.put("config.providers.file.class", invalidConfigProvider); | ||||
|         props.put("testKey", "${test:/foo/bar/testpath:testKey}"); | ||||
|         KafkaException e = assertThrows(KafkaException.class, () -> new TestIndirectConfigResolution(props, Collections.emptyMap())); | ||||
|         assertTrue(e.getMessage().contains(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testAutoConfigResolutionWithInvalidConfigProviderClassIncluded() { | ||||
|         String invalidConfigProvider = "org.apache.kafka.common.config.provider.InvalidConfigProvider"; | ||||
|         System.setProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY, invalidConfigProvider); | ||||
|         // Test Case: Invalid config provider specified, but is also included in the system property | ||||
|         Properties props = new Properties(); | ||||
|         props.put("config.providers", "file"); | ||||
|         props.put("config.providers.file.class", invalidConfigProvider); | ||||
|         props.put("testKey", "${test:/foo/bar/testpath:testKey}"); | ||||
|         KafkaException e = assertThrows(KafkaException.class, () -> new TestIndirectConfigResolution(props, Collections.emptyMap())); | ||||
|         assertFalse(e.getMessage().contains(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY)); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -496,13 +560,15 @@ public class AbstractConfigTest { | |||
|         props.put("config.providers", "file"); | ||||
|         props.put("config.providers.file.class", MockVaultConfigProvider.class.getName()); | ||||
| 
 | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, convertPropertiesToMap(providers)); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, Utils.castToStringObjectMap(providers)); | ||||
|         assertEquals("${file:/usr/kerberos:key}", config.originals().get("sasl.kerberos.key")); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|     public void testConfigProviderConfigurationWithConfigParams() { | ||||
|         // Test Case: Valid Test Case With Multiple ConfigProviders as a separate variable | ||||
|         // should have no effect | ||||
|         System.setProperty(AbstractConfig.AUTOMATIC_CONFIG_PROVIDERS_PROPERTY, MockFileConfigProvider.class.getName()); | ||||
|         // Test Case: Specify a config provider not allowed, but passed via the trusted providers argument | ||||
|         Properties providers = new Properties(); | ||||
|         providers.put("config.providers", "vault"); | ||||
|         providers.put("config.providers.vault.class", MockVaultConfigProvider.class.getName()); | ||||
|  | @ -512,7 +578,7 @@ public class AbstractConfigTest { | |||
|         props.put("sasl.truststore.key", "${vault:/usr/truststore:truststoreKey}"); | ||||
|         props.put("sasl.truststore.password", "${vault:/usr/truststore:truststorePassword}"); | ||||
|         props.put("sasl.truststore.location", "${vault:/usr/truststore:truststoreLocation}"); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, convertPropertiesToMap(providers)); | ||||
|         TestIndirectConfigResolution config = new TestIndirectConfigResolution(props, Utils.castToStringObjectMap(providers)); | ||||
|         assertEquals("/usr/vault", config.originals().get("sasl.truststore.location")); | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -30,6 +30,7 @@ import java.util.Properties; | |||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
|  | @ -40,7 +41,7 @@ public class ChannelBuildersTest { | |||
|         Map<String, Object> configs = new HashMap<>(); | ||||
|         configs.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, ConfigurableKafkaPrincipalBuilder.class); | ||||
|         KafkaPrincipalBuilder builder = ChannelBuilders.createPrincipalBuilder(configs, null, null); | ||||
|         assertTrue(builder instanceof ConfigurableKafkaPrincipalBuilder); | ||||
|         assertInstanceOf(ConfigurableKafkaPrincipalBuilder.class, builder); | ||||
|         assertTrue(((ConfigurableKafkaPrincipalBuilder) builder).configured); | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -73,6 +73,7 @@ import javax.net.ssl.SSLSession; | |||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assumptions.assumeTrue; | ||||
|  | @ -1064,7 +1065,7 @@ public class SslTransportLayerTest { | |||
| 
 | ||||
|         CertStores newServerCertStores = certBuilder(true, "server", args.useInlinePem).addHostName("localhost").build(); | ||||
|         Map<String, Object> newKeystoreConfigs = newServerCertStores.keyStoreProps(); | ||||
|         assertTrue(serverChannelBuilder instanceof ListenerReconfigurable, "SslChannelBuilder not reconfigurable"); | ||||
|         assertInstanceOf(ListenerReconfigurable.class, serverChannelBuilder, "SslChannelBuilder not reconfigurable"); | ||||
|         ListenerReconfigurable reconfigurableBuilder = (ListenerReconfigurable) serverChannelBuilder; | ||||
|         assertEquals(listenerName, reconfigurableBuilder.listenerName()); | ||||
|         reconfigurableBuilder.validateReconfiguration(newKeystoreConfigs); | ||||
|  | @ -1192,7 +1193,7 @@ public class SslTransportLayerTest { | |||
|         CertStores newClientCertStores = certBuilder(true, "client", args.useInlinePem).addHostName("localhost").build(); | ||||
|         args.sslClientConfigs = args.getTrustingConfig(newClientCertStores, args.serverCertStores); | ||||
|         Map<String, Object> newTruststoreConfigs = newClientCertStores.trustStoreProps(); | ||||
|         assertTrue(serverChannelBuilder instanceof ListenerReconfigurable, "SslChannelBuilder not reconfigurable"); | ||||
|         assertInstanceOf(ListenerReconfigurable.class, serverChannelBuilder, "SslChannelBuilder not reconfigurable"); | ||||
|         ListenerReconfigurable reconfigurableBuilder = (ListenerReconfigurable) serverChannelBuilder; | ||||
|         assertEquals(listenerName, reconfigurableBuilder.listenerName()); | ||||
|         reconfigurableBuilder.validateReconfiguration(newTruststoreConfigs); | ||||
|  |  | |||
|  | @ -44,6 +44,7 @@ import static org.apache.kafka.common.record.DefaultRecordBatch.RECORDS_COUNT_OF | |||
| import static org.apache.kafka.common.record.DefaultRecordBatch.RECORDS_OFFSET; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertIterableEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
|  | @ -425,12 +426,12 @@ public class DefaultRecordBatchTest { | |||
| 
 | ||||
|             if (CompressionType.NONE == compressionType) { | ||||
|                 // assert that for uncompressed data stream record iterator is not used | ||||
|                 assertTrue(skipKeyValueIterator instanceof DefaultRecordBatch.RecordIterator); | ||||
|                 assertInstanceOf(DefaultRecordBatch.RecordIterator.class, skipKeyValueIterator); | ||||
|                 // superficial validation for correctness. Deep validation is already performed in other tests | ||||
|                 assertEquals(Utils.toList(records.records()).size(), Utils.toList(skipKeyValueIterator).size()); | ||||
|             } else { | ||||
|                 // assert that a streaming iterator is used for compressed records | ||||
|                 assertTrue(skipKeyValueIterator instanceof DefaultRecordBatch.StreamRecordIterator); | ||||
|                 assertInstanceOf(DefaultRecordBatch.StreamRecordIterator.class, skipKeyValueIterator); | ||||
|                 // assert correctness for compressed records | ||||
|                 assertIterableEquals(Arrays.asList( | ||||
|                         new PartialDefaultRecord(9, (byte) 0, 0L, 1L, -1, 1, 1), | ||||
|  |  | |||
|  | @ -378,7 +378,7 @@ public class MemoryRecordsBuilderTest { | |||
|         MemoryRecordsBuilder.RecordsInfo info = builder.info(); | ||||
|         assertEquals(logAppendTime, info.maxTimestamp); | ||||
|         // When logAppendTime is used, the first offset of the batch will be the offset of maxTimestamp | ||||
|         assertEquals(0L, info.shallowOffsetOfMaxTimestamp); | ||||
|         assertEquals(0L, info.offsetOfMaxTimestamp); | ||||
| 
 | ||||
|         for (RecordBatch batch : records.batches()) { | ||||
|             if (magic == MAGIC_VALUE_V0) { | ||||
|  | @ -414,9 +414,9 @@ public class MemoryRecordsBuilderTest { | |||
| 
 | ||||
|         if (magic == MAGIC_VALUE_V0) | ||||
|             // in MAGIC_VALUE_V0's case, we don't have timestamp info in records, so always return -1. | ||||
|             assertEquals(-1L, info.shallowOffsetOfMaxTimestamp); | ||||
|             assertEquals(-1L, info.offsetOfMaxTimestamp); | ||||
|         else | ||||
|             assertEquals(1L, info.shallowOffsetOfMaxTimestamp); | ||||
|             assertEquals(1L, info.offsetOfMaxTimestamp); | ||||
| 
 | ||||
|         int i = 0; | ||||
|         long[] expectedTimestamps = new long[] {0L, 2L, 1L}; | ||||
|  | @ -495,10 +495,10 @@ public class MemoryRecordsBuilderTest { | |||
|         MemoryRecordsBuilder.RecordsInfo info = builder.info(); | ||||
|         if (magic == MAGIC_VALUE_V0) { | ||||
|             assertEquals(-1, info.maxTimestamp); | ||||
|             assertEquals(-1L, info.shallowOffsetOfMaxTimestamp); | ||||
|             assertEquals(-1L, info.offsetOfMaxTimestamp); | ||||
|         } else { | ||||
|             assertEquals(2L, info.maxTimestamp); | ||||
|             assertEquals(2L, info.shallowOffsetOfMaxTimestamp); | ||||
|             assertEquals(2L, info.offsetOfMaxTimestamp); | ||||
|         } | ||||
| 
 | ||||
|         long i = 0L; | ||||
|  |  | |||
|  | @ -352,7 +352,7 @@ public class MemoryRecordsTest { | |||
|                     assertEquals(0, filterResult.messagesRetained()); | ||||
|                     assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained()); | ||||
|                     assertEquals(12, filterResult.maxTimestamp()); | ||||
|                     assertEquals(baseOffset + 1, filterResult.shallowOffsetOfMaxTimestamp()); | ||||
|                     assertEquals(baseOffset + 1, filterResult.offsetOfMaxTimestamp()); | ||||
| 
 | ||||
|                     // Verify filtered records | ||||
|                     filtered.flip(); | ||||
|  | @ -413,7 +413,7 @@ public class MemoryRecordsTest { | |||
|         assertEquals(0, filterResult.messagesRetained()); | ||||
|         assertEquals(DefaultRecordBatch.RECORD_BATCH_OVERHEAD, filterResult.bytesRetained()); | ||||
|         assertEquals(timestamp, filterResult.maxTimestamp()); | ||||
|         assertEquals(baseOffset, filterResult.shallowOffsetOfMaxTimestamp()); | ||||
|         assertEquals(baseOffset, filterResult.offsetOfMaxTimestamp()); | ||||
|         assertTrue(filterResult.outputBuffer().position() > 0); | ||||
| 
 | ||||
|         // Verify filtered records | ||||
|  | @ -893,7 +893,7 @@ public class MemoryRecordsTest { | |||
|         assertEquals(filtered.limit(), result.bytesRetained()); | ||||
|         if (magic > RecordBatch.MAGIC_VALUE_V0) { | ||||
|             assertEquals(20L, result.maxTimestamp()); | ||||
|             assertEquals(4L, result.shallowOffsetOfMaxTimestamp()); | ||||
|             assertEquals(4L, result.offsetOfMaxTimestamp()); | ||||
|         } | ||||
| 
 | ||||
|         MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); | ||||
|  |  | |||
|  | @ -40,6 +40,7 @@ import java.nio.ByteBuffer; | |||
| import java.util.Collections; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
|  | @ -62,7 +63,7 @@ public class RequestContextTest { | |||
|         requestBuffer.flip(); | ||||
| 
 | ||||
|         RequestAndSize requestAndSize = context.parseRequest(requestBuffer); | ||||
|         assertTrue(requestAndSize.request instanceof ApiVersionsRequest); | ||||
|         assertInstanceOf(ApiVersionsRequest.class, requestAndSize.request); | ||||
|         ApiVersionsRequest request = (ApiVersionsRequest) requestAndSize.request; | ||||
|         assertTrue(request.hasUnsupportedRequestVersion()); | ||||
| 
 | ||||
|  |  | |||
|  | @ -45,6 +45,7 @@ import java.util.List; | |||
| import java.util.Map; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| public abstract class SaslAuthenticatorFailureDelayTest { | ||||
|  | @ -227,7 +228,7 @@ public abstract class SaslAuthenticatorFailureDelayTest { | |||
|                                                            String mechanism, String expectedErrorMessage) throws Exception { | ||||
|         ChannelState finalState = createAndCheckClientConnectionFailure(securityProtocol, node); | ||||
|         Exception exception = finalState.exception(); | ||||
|         assertTrue(exception instanceof SaslAuthenticationException, "Invalid exception class " + exception.getClass()); | ||||
|         assertInstanceOf(SaslAuthenticationException.class, exception, "Invalid exception class " + exception.getClass()); | ||||
|         if (expectedErrorMessage == null) | ||||
|             expectedErrorMessage = "Authentication failed during authentication due to invalid credentials with SASL mechanism " + mechanism; | ||||
|         assertEquals(expectedErrorMessage, exception.getMessage()); | ||||
|  |  | |||
|  | @ -140,6 +140,7 @@ import java.util.stream.IntStream; | |||
| import static org.apache.kafka.common.protocol.ApiKeys.LIST_OFFSETS; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
|  | @ -1204,7 +1205,7 @@ public class SaslAuthenticatorTest { | |||
|         try { | ||||
|             createClientConnection(securityProtocol, "invalid"); | ||||
|         } catch (Exception e) { | ||||
|             assertTrue(e.getCause() instanceof LoginException, "Unexpected exception " + e.getCause()); | ||||
|             assertInstanceOf(LoginException.class, e.getCause(), "Unexpected exception " + e.getCause()); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  | @ -1805,12 +1806,12 @@ public class SaslAuthenticatorTest { | |||
|         // Server with extensions, but without a token should fail to start up since it could indicate a configuration error | ||||
|         saslServerConfigs.put("listener.name.sasl_ssl.oauthbearer." + SaslConfigs.SASL_JAAS_CONFIG, | ||||
|                 TestJaasConfig.jaasConfigProperty("OAUTHBEARER", Collections.singletonMap("unsecuredLoginExtension_test", "something"))); | ||||
|         try { | ||||
|             createEchoServer(securityProtocol); | ||||
|             fail("Server created with invalid login config containing extensions without a token"); | ||||
|         } catch (Throwable e) { | ||||
|             assertTrue(e.getCause() instanceof LoginException, "Unexpected exception " + Utils.stackTrace(e)); | ||||
|         } | ||||
| 
 | ||||
|         Throwable throwable = assertThrows( | ||||
|             Throwable.class, | ||||
|             () -> createEchoServer(securityProtocol), | ||||
|             "Server created with invalid login config containing extensions without a token"); | ||||
|         assertInstanceOf(LoginException.class, throwable.getCause(), "Unexpected exception " + Utils.stackTrace(throwable)); | ||||
|     } | ||||
| 
 | ||||
|     /** | ||||
|  | @ -2221,7 +2222,7 @@ public class SaslAuthenticatorTest { | |||
|             String mechanism, String expectedErrorMessage) throws Exception { | ||||
|         ChannelState finalState = createAndCheckClientConnectionFailure(securityProtocol, node); | ||||
|         Exception exception = finalState.exception(); | ||||
|         assertTrue(exception instanceof SaslAuthenticationException, "Invalid exception class " + exception.getClass()); | ||||
|         assertInstanceOf(SaslAuthenticationException.class, exception, "Invalid exception class " + exception.getClass()); | ||||
|         String expectedExceptionMessage = expectedErrorMessage != null ? expectedErrorMessage : | ||||
|                 "Authentication failed during authentication due to invalid credentials with SASL mechanism " + mechanism; | ||||
|         assertEquals(expectedExceptionMessage, exception.getMessage()); | ||||
|  |  | |||
|  | @ -21,6 +21,7 @@ import static org.apache.kafka.common.config.SaslConfigs.SASL_OAUTHBEARER_TOKEN_ | |||
| import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_ID_CONFIG; | ||||
| import static org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginCallbackHandler.CLIENT_SECRET_CONFIG; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
|  | @ -213,7 +214,7 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { | |||
|         Map<String, ?> configs = getSaslConfigs(SASL_OAUTHBEARER_TOKEN_ENDPOINT_URL, accessTokenFile.toURI().toString()); | ||||
|         Map<String, Object> jaasConfigs = Collections.emptyMap(); | ||||
|         configureHandler(handler, configs, jaasConfigs); | ||||
|         assertTrue(handler.getAccessTokenRetriever() instanceof FileTokenRetriever); | ||||
|         assertInstanceOf(FileTokenRetriever.class, handler.getAccessTokenRetriever()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -224,7 +225,7 @@ public class OAuthBearerLoginCallbackHandlerTest extends OAuthBearerTest { | |||
|         jaasConfigs.put(CLIENT_ID_CONFIG, "an ID"); | ||||
|         jaasConfigs.put(CLIENT_SECRET_CONFIG, "a secret"); | ||||
|         configureHandler(handler, configs, jaasConfigs); | ||||
|         assertTrue(handler.getAccessTokenRetriever() instanceof HttpAccessTokenRetriever); | ||||
|         assertInstanceOf(HttpAccessTokenRetriever.class, handler.getAccessTokenRetriever()); | ||||
|     } | ||||
| 
 | ||||
|     private void testInvalidAccessToken(String accessToken, String expectedMessageSubstring) throws Exception { | ||||
|  |  | |||
|  | @ -17,6 +17,7 @@ | |||
| 
 | ||||
| package org.apache.kafka.common.security.oauthbearer.internals.secured; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.fail; | ||||
| import static org.mockito.Mockito.mock; | ||||
|  | @ -39,6 +40,7 @@ import java.util.Map; | |||
| import java.util.concurrent.ExecutionException; | ||||
| import java.util.function.Consumer; | ||||
| import javax.security.auth.login.AppConfigurationEntry; | ||||
| 
 | ||||
| import org.apache.kafka.common.config.AbstractConfig; | ||||
| import org.apache.kafka.common.config.ConfigDef; | ||||
| import org.apache.kafka.common.security.auth.AuthenticateCallbackHandler; | ||||
|  | @ -65,19 +67,7 @@ public abstract class OAuthBearerTest { | |||
|     protected void assertThrowsWithMessage(Class<? extends Exception> clazz, | ||||
|         Executable executable, | ||||
|         String substring) { | ||||
|         boolean failed = false; | ||||
| 
 | ||||
|         try { | ||||
|             executable.execute(); | ||||
|         } catch (Throwable t) { | ||||
|             failed = true; | ||||
|             assertTrue(clazz.isInstance(t), String.format("Test failed by exception %s, but expected %s", t.getClass(), clazz)); | ||||
| 
 | ||||
|             assertErrorMessageContains(t.getMessage(), substring); | ||||
|         } | ||||
| 
 | ||||
|         if (!failed) | ||||
|             fail("Expected test to fail with " + clazz + " that contains the string " + substring); | ||||
|         assertErrorMessageContains(assertThrows(clazz, executable).getMessage(), substring); | ||||
|     } | ||||
| 
 | ||||
|     protected void assertErrorMessageContains(String actual, String expectedSubstring) { | ||||
|  |  | |||
|  | @ -17,8 +17,8 @@ | |||
| package org.apache.kafka.common.security.oauthbearer.internals.unsecured; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| import java.nio.charset.StandardCharsets; | ||||
| import java.util.Arrays; | ||||
|  | @ -77,8 +77,8 @@ public class OAuthBearerUnsecuredValidatorCallbackHandlerTest { | |||
|                     + (includeOptionalIssuedAtClaim ? comma(ISSUED_AT_CLAIM_TEXT) : "") + "}"; | ||||
|             Object validationResult = validationResult(UNSECURED_JWT_HEADER_JSON, claimsJson, | ||||
|                     MODULE_OPTIONS_MAP_NO_SCOPE_REQUIRED); | ||||
|             assertTrue(validationResult instanceof OAuthBearerValidatorCallback); | ||||
|             assertTrue(((OAuthBearerValidatorCallback) validationResult).token() instanceof OAuthBearerUnsecuredJws); | ||||
|             assertInstanceOf(OAuthBearerValidatorCallback.class, validationResult); | ||||
|             assertInstanceOf(OAuthBearerUnsecuredJws.class, ((OAuthBearerValidatorCallback) validationResult).token()); | ||||
|         } | ||||
|     } | ||||
| 
 | ||||
|  | @ -103,8 +103,8 @@ public class OAuthBearerUnsecuredValidatorCallbackHandlerTest { | |||
|         String claimsJson = "{" + SUB_CLAIM_TEXT + comma(EXPIRATION_TIME_CLAIM_TEXT) + comma(SCOPE_CLAIM_TEXT) + "}"; | ||||
|         Object validationResult = validationResult(UNSECURED_JWT_HEADER_JSON, claimsJson, | ||||
|                 MODULE_OPTIONS_MAP_REQUIRE_EXISTING_SCOPE); | ||||
|         assertTrue(validationResult instanceof OAuthBearerValidatorCallback); | ||||
|         assertTrue(((OAuthBearerValidatorCallback) validationResult).token() instanceof OAuthBearerUnsecuredJws); | ||||
|         assertInstanceOf(OAuthBearerValidatorCallback.class, validationResult); | ||||
|         assertInstanceOf(OAuthBearerUnsecuredJws.class, ((OAuthBearerValidatorCallback) validationResult).token()); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -123,7 +123,7 @@ public class OAuthBearerUnsecuredValidatorCallbackHandlerTest { | |||
|             Map<String, String> moduleOptionsMap, String optionalFailureScope) throws OAuthBearerConfigException, | ||||
|             OAuthBearerIllegalTokenException { | ||||
|         Object validationResultObj = validationResult(headerJson, claimsJson, moduleOptionsMap); | ||||
|         assertTrue(validationResultObj instanceof OAuthBearerValidatorCallback); | ||||
|         assertInstanceOf(OAuthBearerValidatorCallback.class, validationResultObj); | ||||
|         OAuthBearerValidatorCallback callback = (OAuthBearerValidatorCallback) validationResultObj; | ||||
|         assertNull(callback.token()); | ||||
|         assertNull(callback.errorOpenIDConfiguration()); | ||||
|  |  | |||
|  | @ -51,6 +51,7 @@ import org.junit.jupiter.api.Test; | |||
| import static org.apache.kafka.common.security.ssl.SslFactory.CertificateEntries.ensureCompatible; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotSame; | ||||
|  | @ -474,7 +475,7 @@ public abstract class SslFactoryTest { | |||
|         clientSslConfig.put(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, TestSslUtils.TestSslEngineFactory.class); | ||||
|         SslFactory sslFactory = new SslFactory(Mode.CLIENT); | ||||
|         sslFactory.configure(clientSslConfig); | ||||
|         assertTrue(sslFactory.sslEngineFactory() instanceof TestSslUtils.TestSslEngineFactory, | ||||
|         assertInstanceOf(TestSslUtils.TestSslEngineFactory.class, sslFactory.sslEngineFactory(), | ||||
|             "SslEngineFactory must be of expected type"); | ||||
|     } | ||||
| 
 | ||||
|  | @ -507,7 +508,7 @@ public abstract class SslFactoryTest { | |||
|         serverSslConfig.put(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, TestSslUtils.TestSslEngineFactory.class); | ||||
|         SslFactory sslFactory = new SslFactory(Mode.SERVER); | ||||
|         sslFactory.configure(serverSslConfig); | ||||
|         assertTrue(sslFactory.sslEngineFactory() instanceof TestSslUtils.TestSslEngineFactory, | ||||
|         assertInstanceOf(TestSslUtils.TestSslEngineFactory.class, sslFactory.sslEngineFactory(), | ||||
|             "SslEngineFactory must be of expected type"); | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -17,9 +17,9 @@ | |||
| package org.apache.kafka.common.serialization; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| import org.apache.kafka.clients.CommonClientConfigs; | ||||
| import org.apache.kafka.common.KafkaException; | ||||
|  | @ -45,7 +45,7 @@ public class ListDeserializerTest { | |||
|         listDeserializer.configure(props, true); | ||||
|         final Deserializer<?> inner = listDeserializer.innerDeserializer(); | ||||
|         assertNotNull(inner, "Inner deserializer should be not null"); | ||||
|         assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); | ||||
|         assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -55,7 +55,7 @@ public class ListDeserializerTest { | |||
|         listDeserializer.configure(props, false); | ||||
|         final Deserializer<?> inner = listDeserializer.innerDeserializer(); | ||||
|         assertNotNull(inner, "Inner deserializer should be not null"); | ||||
|         assertTrue(inner instanceof IntegerDeserializer, "Inner deserializer type should be IntegerDeserializer"); | ||||
|         assertInstanceOf(IntegerDeserializer.class, inner, "Inner deserializer type should be IntegerDeserializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -65,7 +65,7 @@ public class ListDeserializerTest { | |||
|         listDeserializer.configure(props, true); | ||||
|         final Deserializer<?> inner = listDeserializer.innerDeserializer(); | ||||
|         assertNotNull(inner, "Inner deserializer should be not null"); | ||||
|         assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); | ||||
|         assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -75,7 +75,7 @@ public class ListDeserializerTest { | |||
|         listDeserializer.configure(props, false); | ||||
|         final Deserializer<?> inner = listDeserializer.innerDeserializer(); | ||||
|         assertNotNull(inner, "Inner deserializer should be not null"); | ||||
|         assertTrue(inner instanceof StringDeserializer, "Inner deserializer type should be StringDeserializer"); | ||||
|         assertInstanceOf(StringDeserializer.class, inner, "Inner deserializer type should be StringDeserializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -17,9 +17,9 @@ | |||
| package org.apache.kafka.common.serialization; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| 
 | ||||
| import org.apache.kafka.clients.CommonClientConfigs; | ||||
| import org.apache.kafka.common.KafkaException; | ||||
|  | @ -43,7 +43,7 @@ public class ListSerializerTest { | |||
|         listSerializer.configure(props, true); | ||||
|         final Serializer<?> inner = listSerializer.getInnerSerializer(); | ||||
|         assertNotNull(inner, "Inner serializer should be not null"); | ||||
|         assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); | ||||
|         assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -52,7 +52,7 @@ public class ListSerializerTest { | |||
|         listSerializer.configure(props, false); | ||||
|         final Serializer<?> inner = listSerializer.getInnerSerializer(); | ||||
|         assertNotNull(inner, "Inner serializer should be not null"); | ||||
|         assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); | ||||
|         assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -61,7 +61,7 @@ public class ListSerializerTest { | |||
|         listSerializer.configure(props, true); | ||||
|         final Serializer<?> inner = listSerializer.getInnerSerializer(); | ||||
|         assertNotNull(inner, "Inner serializer should be not null"); | ||||
|         assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); | ||||
|         assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  | @ -70,7 +70,7 @@ public class ListSerializerTest { | |||
|         listSerializer.configure(props, false); | ||||
|         final Serializer<?> inner = listSerializer.getInnerSerializer(); | ||||
|         assertNotNull(inner, "Inner serializer should be not null"); | ||||
|         assertTrue(inner instanceof StringSerializer, "Inner serializer type should be StringSerializer"); | ||||
|         assertInstanceOf(StringSerializer.class, inner, "Inner serializer type should be StringSerializer"); | ||||
|     } | ||||
| 
 | ||||
|     @Test | ||||
|  |  | |||
|  | @ -59,6 +59,7 @@ import java.util.concurrent.atomic.AtomicReference; | |||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
|  | @ -247,7 +248,7 @@ public class ClientTelemetryReporterTest { | |||
|         Optional<AbstractRequest.Builder<?>> requestOptional = telemetrySender.createRequest(); | ||||
|         assertNotNull(requestOptional); | ||||
|         assertTrue(requestOptional.isPresent()); | ||||
|         assertTrue(requestOptional.get().build() instanceof GetTelemetrySubscriptionsRequest); | ||||
|         assertInstanceOf(GetTelemetrySubscriptionsRequest.class, requestOptional.get().build()); | ||||
|         GetTelemetrySubscriptionsRequest request = (GetTelemetrySubscriptionsRequest) requestOptional.get().build(); | ||||
| 
 | ||||
|         GetTelemetrySubscriptionsRequest expectedResult = new GetTelemetrySubscriptionsRequest.Builder( | ||||
|  | @ -266,7 +267,7 @@ public class ClientTelemetryReporterTest { | |||
|         Optional<AbstractRequest.Builder<?>> requestOptional = telemetrySender.createRequest(); | ||||
|         assertNotNull(requestOptional); | ||||
|         assertTrue(requestOptional.isPresent()); | ||||
|         assertTrue(requestOptional.get().build() instanceof GetTelemetrySubscriptionsRequest); | ||||
|         assertInstanceOf(GetTelemetrySubscriptionsRequest.class, requestOptional.get().build()); | ||||
|         GetTelemetrySubscriptionsRequest request = (GetTelemetrySubscriptionsRequest) requestOptional.get().build(); | ||||
| 
 | ||||
|         GetTelemetrySubscriptionsRequest expectedResult = new GetTelemetrySubscriptionsRequest.Builder( | ||||
|  | @ -290,7 +291,7 @@ public class ClientTelemetryReporterTest { | |||
|         Optional<AbstractRequest.Builder<?>> requestOptional = telemetrySender.createRequest(); | ||||
|         assertNotNull(requestOptional); | ||||
|         assertTrue(requestOptional.isPresent()); | ||||
|         assertTrue(requestOptional.get().build() instanceof PushTelemetryRequest); | ||||
|         assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); | ||||
|         PushTelemetryRequest request = (PushTelemetryRequest) requestOptional.get().build(); | ||||
| 
 | ||||
|         PushTelemetryRequest expectedResult = new PushTelemetryRequest.Builder( | ||||
|  | @ -373,7 +374,7 @@ public class ClientTelemetryReporterTest { | |||
|         Optional<AbstractRequest.Builder<?>> requestOptional = telemetrySender.createRequest(); | ||||
|         assertNotNull(requestOptional); | ||||
|         assertTrue(requestOptional.isPresent()); | ||||
|         assertTrue(requestOptional.get().build() instanceof PushTelemetryRequest); | ||||
|         assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); | ||||
|         PushTelemetryRequest request = (PushTelemetryRequest) requestOptional.get().build(); | ||||
| 
 | ||||
|         assertEquals(subscription.clientInstanceId(), request.data().clientInstanceId()); | ||||
|  | @ -401,7 +402,7 @@ public class ClientTelemetryReporterTest { | |||
|             Optional<AbstractRequest.Builder<?>> requestOptional = telemetrySender.createRequest(); | ||||
|             assertNotNull(requestOptional); | ||||
|             assertTrue(requestOptional.isPresent()); | ||||
|             assertTrue(requestOptional.get().build() instanceof PushTelemetryRequest); | ||||
|             assertInstanceOf(PushTelemetryRequest.class, requestOptional.get().build()); | ||||
|             PushTelemetryRequest request = (PushTelemetryRequest) requestOptional.get().build(); | ||||
| 
 | ||||
|             assertEquals(subscription.clientInstanceId(), request.data().clientInstanceId()); | ||||
|  |  | |||
|  | @ -19,6 +19,7 @@ package org.apache.kafka.common.telemetry.internals; | |||
| import org.apache.kafka.common.Uuid; | ||||
| import org.apache.kafka.common.protocol.Errors; | ||||
| import org.apache.kafka.common.record.CompressionType; | ||||
| import org.apache.kafka.common.utils.Utils; | ||||
| import org.junit.jupiter.api.Test; | ||||
| import org.junit.jupiter.params.ParameterizedTest; | ||||
| import org.junit.jupiter.params.provider.EnumSource; | ||||
|  | @ -132,9 +133,9 @@ public class ClientTelemetryUtilsTest { | |||
|         } else { | ||||
|             assertArrayEquals(testString, compressed); | ||||
|         } | ||||
| 
 | ||||
|         ByteBuffer decompressed = ClientTelemetryUtils.decompress(compressed, compressionType); | ||||
|         assertNotNull(decompressed); | ||||
|         assertArrayEquals(testString, decompressed.array()); | ||||
|         byte[] actualResult = Utils.toArray(decompressed); | ||||
|         assertArrayEquals(testString, actualResult); | ||||
|     } | ||||
| } | ||||
|  | @ -74,6 +74,7 @@ import java.util.regex.Pattern; | |||
| 
 | ||||
| import static java.util.Arrays.asList; | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
|  | @ -556,7 +557,7 @@ public class TestUtils { | |||
|      */ | ||||
|     public static <T extends Throwable> T assertFutureThrows(Future<?> future, Class<T> exceptionCauseClass) { | ||||
|         ExecutionException exception = assertThrows(ExecutionException.class, future::get); | ||||
|         assertTrue(exceptionCauseClass.isInstance(exception.getCause()), | ||||
|         assertInstanceOf(exceptionCauseClass, exception.getCause(), | ||||
|             "Unexpected exception cause " + exception.getCause()); | ||||
|         return exceptionCauseClass.cast(exception.getCause()); | ||||
|     } | ||||
|  |  | |||
|  | @ -38,6 +38,7 @@ import java.util.Map; | |||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertNull; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
|  | @ -829,14 +830,14 @@ public class ValuesTest { | |||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Decimal.schema(0), schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof BigDecimal); | ||||
|         assertInstanceOf(BigDecimal.class, schemaAndValue.value()); | ||||
|         assertEquals(value, ((BigDecimal) schemaAndValue.value()).unscaledValue()); | ||||
|         value = BigInteger.valueOf(Long.MIN_VALUE).subtract(new BigInteger("1")); | ||||
|         schemaAndValue = Values.parseString( | ||||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Decimal.schema(0), schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof BigDecimal); | ||||
|         assertInstanceOf(BigDecimal.class, schemaAndValue.value()); | ||||
|         assertEquals(value, ((BigDecimal) schemaAndValue.value()).unscaledValue()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -847,14 +848,14 @@ public class ValuesTest { | |||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT8_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Byte); | ||||
|         assertInstanceOf(Byte.class, schemaAndValue.value()); | ||||
|         assertEquals(value.byteValue(), ((Byte) schemaAndValue.value()).byteValue()); | ||||
|         value = Byte.MIN_VALUE; | ||||
|         schemaAndValue = Values.parseString( | ||||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT8_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Byte); | ||||
|         assertInstanceOf(Byte.class, schemaAndValue.value()); | ||||
|         assertEquals(value.byteValue(), ((Byte) schemaAndValue.value()).byteValue()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -865,14 +866,14 @@ public class ValuesTest { | |||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT16_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Short); | ||||
|         assertInstanceOf(Short.class, schemaAndValue.value()); | ||||
|         assertEquals(value.shortValue(), ((Short) schemaAndValue.value()).shortValue()); | ||||
|         value = Short.MIN_VALUE; | ||||
|         schemaAndValue = Values.parseString( | ||||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT16_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Short); | ||||
|         assertInstanceOf(Short.class, schemaAndValue.value()); | ||||
|         assertEquals(value.shortValue(), ((Short) schemaAndValue.value()).shortValue()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -883,14 +884,14 @@ public class ValuesTest { | |||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT32_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Integer); | ||||
|         assertInstanceOf(Integer.class, schemaAndValue.value()); | ||||
|         assertEquals(value.intValue(), ((Integer) schemaAndValue.value()).intValue()); | ||||
|         value = Integer.MIN_VALUE; | ||||
|         schemaAndValue = Values.parseString( | ||||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT32_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Integer); | ||||
|         assertInstanceOf(Integer.class, schemaAndValue.value()); | ||||
|         assertEquals(value.intValue(), ((Integer) schemaAndValue.value()).intValue()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -901,14 +902,14 @@ public class ValuesTest { | |||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT64_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Long); | ||||
|         assertInstanceOf(Long.class, schemaAndValue.value()); | ||||
|         assertEquals(value.longValue(), ((Long) schemaAndValue.value()).longValue()); | ||||
|         value = Long.MIN_VALUE; | ||||
|         schemaAndValue = Values.parseString( | ||||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.INT64_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Long); | ||||
|         assertInstanceOf(Long.class, schemaAndValue.value()); | ||||
|         assertEquals(value.longValue(), ((Long) schemaAndValue.value()).longValue()); | ||||
|     } | ||||
| 
 | ||||
|  | @ -919,14 +920,14 @@ public class ValuesTest { | |||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.FLOAT32_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Float); | ||||
|         assertInstanceOf(Float.class, schemaAndValue.value()); | ||||
|         assertEquals(value, (Float) schemaAndValue.value(), 0); | ||||
|         value = -Float.MAX_VALUE; | ||||
|         schemaAndValue = Values.parseString( | ||||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.FLOAT32_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Float); | ||||
|         assertInstanceOf(Float.class, schemaAndValue.value()); | ||||
|         assertEquals(value, (Float) schemaAndValue.value(), 0); | ||||
|     } | ||||
| 
 | ||||
|  | @ -937,14 +938,14 @@ public class ValuesTest { | |||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.FLOAT64_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Double); | ||||
|         assertInstanceOf(Double.class, schemaAndValue.value()); | ||||
|         assertEquals(value, (Double) schemaAndValue.value(), 0); | ||||
|         value = -Double.MAX_VALUE; | ||||
|         schemaAndValue = Values.parseString( | ||||
|             String.valueOf(value) | ||||
|         ); | ||||
|         assertEquals(Schema.FLOAT64_SCHEMA, schemaAndValue.schema()); | ||||
|         assertTrue(schemaAndValue.value() instanceof Double); | ||||
|         assertInstanceOf(Double.class, schemaAndValue.value()); | ||||
|         assertEquals(value, (Double) schemaAndValue.value(), 0); | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -76,7 +76,7 @@ public class MirrorClientConfig extends AbstractConfig { | |||
|     public static final String PRODUCER_CLIENT_PREFIX = "producer."; | ||||
| 
 | ||||
|     MirrorClientConfig(Map<?, ?> props) { | ||||
|         super(CONFIG_DEF, props, true); | ||||
|         super(CONFIG_DEF, props, Utils.castToStringObjectMap(props), true); | ||||
|     } | ||||
| 
 | ||||
|     public ReplicationPolicy replicationPolicy() { | ||||
|  |  | |||
|  | @ -32,8 +32,8 @@ import java.util.Collections; | |||
| import java.util.Map; | ||||
| import java.util.concurrent.ExecutionException; | ||||
| 
 | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.junit.jupiter.api.Assertions.assertThrows; | ||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||
| import static org.mockito.ArgumentMatchers.any; | ||||
| import static org.mockito.Mockito.mock; | ||||
| import static org.mockito.Mockito.verify; | ||||
|  | @ -121,7 +121,7 @@ public class MirrorUtilsTest { | |||
|         when(admin.createTopics(any(), any())).thenReturn(ctr); | ||||
|         Throwable ce = assertThrows(ConnectException.class, () -> MirrorUtils.createCompactedTopic(TOPIC, (short) 1, (short) 1, admin), "Should have exception thrown"); | ||||
| 
 | ||||
|         assertTrue(ce.getCause() instanceof InvalidConfigurationException); | ||||
|         assertInstanceOf(InvalidConfigurationException.class, ce.getCause()); | ||||
|         verify(future).get(); | ||||
|         verify(ctr).values(); | ||||
|         verify(admin).createTopics(any(), any()); | ||||
|  | @ -135,7 +135,7 @@ public class MirrorUtilsTest { | |||
|         when(admin.createTopics(any(), any())).thenReturn(ctr); | ||||
|         Throwable ce = assertThrows(ConnectException.class, () -> MirrorUtils.createCompactedTopic(TOPIC, (short) 1, (short) 1, admin), "Should have exception thrown"); | ||||
| 
 | ||||
|         assertTrue(ce.getCause() instanceof TimeoutException); | ||||
|         assertInstanceOf(TimeoutException.class, ce.getCause()); | ||||
|         verify(future).get(); | ||||
|         verify(ctr).values(); | ||||
|         verify(admin).createTopics(any(), any()); | ||||
|  |  | |||
|  | @ -125,7 +125,7 @@ public abstract class AbstractConnectCli<T extends WorkerConfig> { | |||
| 
 | ||||
|         RestClient restClient = new RestClient(config); | ||||
| 
 | ||||
|         ConnectRestServer restServer = new ConnectRestServer(config.rebalanceTimeout(), restClient, workerProps); | ||||
|         ConnectRestServer restServer = new ConnectRestServer(config.rebalanceTimeout(), restClient, config.originals()); | ||||
|         restServer.initializeServer(); | ||||
| 
 | ||||
|         URI advertisedUrl = restServer.advertisedUrl(); | ||||
|  |  | |||
|  | @ -438,9 +438,15 @@ public class WorkerConfig extends AbstractConfig { | |||
| 
 | ||||
|     @SuppressWarnings("this-escape") | ||||
|     public WorkerConfig(ConfigDef definition, Map<String, String> props) { | ||||
|         super(definition, props); | ||||
|         super(definition, props, Utils.castToStringObjectMap(props), true); | ||||
|         logInternalConverterRemovalWarnings(props); | ||||
|         logPluginPathConfigProviderWarning(props); | ||||
|     } | ||||
| 
 | ||||
|     @Override | ||||
|     public Map<String, Object> originals() { | ||||
|         Map<String, Object> map = super.originals(); | ||||
|         map.remove(AbstractConfig.CONFIG_PROVIDERS_CONFIG); | ||||
|         return map; | ||||
|     } | ||||
| } | ||||
|  |  | |||
|  | @ -258,7 +258,7 @@ public abstract class RestServerConfig extends AbstractConfig { | |||
|     } | ||||
| 
 | ||||
|     protected RestServerConfig(ConfigDef configDef, Map<?, ?> props) { | ||||
|         super(configDef, props); | ||||
|         super(configDef, props, Utils.castToStringObjectMap(props), true); | ||||
|     } | ||||
| 
 | ||||
|     // Visible for testing | ||||
|  |  | |||
|  | @ -293,7 +293,7 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme | |||
|     private final String topic; | ||||
|     // Data is passed to the log already serialized. We use a converter to handle translating to/from generic Connect | ||||
|     // format to serialized form | ||||
|     private final KafkaBasedLog<String, byte[]> configLog; | ||||
|     private KafkaBasedLog<String, byte[]> configLog; | ||||
|     // Connector -> # of tasks | ||||
|     final Map<String, Integer> connectorTaskCounts = new HashMap<>(); | ||||
|     // Connector and task configs: name or id -> config map | ||||
|  | @ -329,6 +329,11 @@ public class KafkaConfigBackingStore extends KafkaTopicBasedBackingStore impleme | |||
|     private final Map<String, Object> fencableProducerProps; | ||||
|     private final Time time; | ||||
| 
 | ||||
|     //VisibleForTesting | ||||
|     void setConfigLog(KafkaBasedLog<String, byte[]> configLog) { | ||||
|         this.configLog = configLog; | ||||
|     } | ||||
| 
 | ||||
|     @Deprecated | ||||
|     public KafkaConfigBackingStore(Converter converter, DistributedConfig config, WorkerConfigTransformer configTransformer) { | ||||
|         this(converter, config, configTransformer, null, "connect-distributed-"); | ||||
|  |  | |||
|  | @ -115,8 +115,10 @@ public class OffsetUtils { | |||
|         } | ||||
| 
 | ||||
|         if (!(keyList.get(1) instanceof Map)) { | ||||
|             if (keyList.get(1) != null) { | ||||
|                 log.warn("Ignoring offset partition key with an unexpected format for the second element in the partition key list. " + | ||||
|                         "Expected type: {}, actual type: {}", Map.class.getName(), className(keyList.get(1))); | ||||
|             } | ||||
|             return; | ||||
|         } | ||||
| 
 | ||||
|  |  | |||
|  | @ -78,6 +78,7 @@ import static org.junit.Assert.assertEquals; | |||
| import static org.junit.Assert.assertFalse; | ||||
| import static org.junit.Assert.assertNotNull; | ||||
| import static org.junit.Assert.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| /** | ||||
|  * Test simple operations on the workers of a Connect cluster. | ||||
|  | @ -860,7 +861,7 @@ public class ConnectWorkerIntegrationTest { | |||
|                         return false; | ||||
|                     } catch (Throwable t) { | ||||
|                         latestError.set(t); | ||||
|                         assertTrue(t instanceof ConnectRestException); | ||||
|                         assertInstanceOf(ConnectRestException.class, t); | ||||
|                         ConnectRestException restException = (ConnectRestException) t; | ||||
| 
 | ||||
|                         assertEquals(INTERNAL_SERVER_ERROR.getStatusCode(), restException.statusCode()); | ||||
|  |  | |||
|  | @ -104,6 +104,7 @@ import static org.junit.Assert.assertFalse; | |||
| import static org.junit.Assert.assertNotNull; | ||||
| import static org.junit.Assert.assertThrows; | ||||
| import static org.junit.Assert.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| 
 | ||||
| @Category(IntegrationTest.class) | ||||
| public class ExactlyOnceSourceIntegrationTest { | ||||
|  | @ -1078,7 +1079,7 @@ public class ExactlyOnceSourceIntegrationTest { | |||
|     @SuppressWarnings("unchecked") | ||||
|     private static <T> T assertAndCast(Object o, Class<T> klass, String objectDescription) { | ||||
|         String className = o == null ? "null" : o.getClass().getName(); | ||||
|         assertTrue(objectDescription + " should be " + klass.getName() + "; was " + className + " instead", klass.isInstance(o)); | ||||
|         assertInstanceOf(klass, o, objectDescription + " should be " + klass.getName() + "; was " + className + " instead"); | ||||
|         return (T) o; | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
|  | @ -51,6 +51,7 @@ import static org.junit.Assert.assertEquals; | |||
| import static org.junit.Assert.assertFalse; | ||||
| import static org.junit.Assert.assertNotNull; | ||||
| import static org.junit.Assert.assertTrue; | ||||
| import static org.junit.jupiter.api.Assertions.assertInstanceOf; | ||||
| import static org.mockito.ArgumentMatchers.any; | ||||
| import static org.mockito.ArgumentMatchers.eq; | ||||
| import static org.mockito.ArgumentMatchers.isNull; | ||||
|  | @ -529,7 +530,7 @@ public class WorkerConnectorTest { | |||
|         ArgumentCaptor<Throwable> exceptionCapture = ArgumentCaptor.forClass(Throwable.class); | ||||
|         verify(listener).onFailure(eq(CONNECTOR), exceptionCapture.capture()); | ||||
|         Throwable e = exceptionCapture.getValue(); | ||||
|         assertTrue(e instanceof ConnectException); | ||||
|         assertInstanceOf(ConnectException.class, e); | ||||
|         assertTrue(e.getMessage().contains("must be a subclass of")); | ||||
|     } | ||||
| 
 | ||||
|  |  | |||
Some files were not shown because too many files have changed in this diff Show More
		Loading…
	
		Reference in New Issue