Merge branch 'trunk' into jbodTiered2

This commit is contained in:
Luke Chen 2024-04-05 16:05:09 +08:00
commit 51d8967bcf
400 changed files with 10999 additions and 7795 deletions

44
.github/workflows/docker_scan.yml vendored Normal file
View File

@ -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

View File

@ -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

View File

@ -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 "$@"

View File

@ -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 %*

View File

@ -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')
@ -2896,7 +2920,7 @@ project(':connect:json') {
api libs.jacksonAfterburner
implementation libs.slf4jApi
testImplementation libs.junitJupiter
testRuntimeOnly libs.slf4jlog4j
@ -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')

View File

@ -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"/>

View File

@ -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>

View File

@ -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>

View File

@ -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);

View File

@ -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,12 +292,12 @@ 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,
CompletableFuture<Void> result) {
private void autoCommitSyncBeforeRevocationWithRetries(OffsetCommitRequestState requestAttempt,
CompletableFuture<Void> result) {
CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commitAttempt = requestAutoCommit(requestAttempt);
commitAttempt.whenComplete((committedOffsets, error) -> {
if (error == null) {
@ -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);
}
}

View File

@ -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.");
}
revokeAndAssign(resolvedAssignment, assignedTopicIdPartitions, revokedPartitions, addedPartitions);
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;
}
/**

View File

@ -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 {

View File

@ -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

View File

@ -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<>();

View File

@ -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)

View File

@ -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. " +

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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);

View File

@ -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,
int messagesRetained, int bytesRetained) {
validateBatchMetadata(maxTimestamp, shallowOffsetOfMaxTimestamp, maxOffset);
private void updateRetainedBatchMetadata(long maxTimestamp, long offsetOfMaxTimestamp, long maxOffset,
int messagesRetained, int bytesRetained) {
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;
}
}

View File

@ -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;
}
}

View File

@ -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,

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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.
*/

View File

@ -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",

View File

@ -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+",

View File

@ -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+",

View File

@ -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+",

View File

@ -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",

View File

@ -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+",

View File

@ -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": [

View File

@ -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,

View File

@ -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",

View File

@ -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,

View File

@ -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": [

View File

@ -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+",

View File

@ -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",

View File

@ -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+",

View File

@ -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());

View File

@ -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

View File

@ -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));
}
}

View File

@ -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;
}

View File

@ -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(

View File

@ -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));
}
}

View File

@ -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(

View File

@ -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

View File

@ -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(

View File

@ -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(

View File

@ -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));
}
}

View File

@ -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

View File

@ -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(

View File

@ -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

View File

@ -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");

View File

@ -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,18 +1028,17 @@ public class AbstractCoordinatorTest {
mockClient.prepareResponse(syncGroupResponse(Errors.NONE));
mockClient.prepareResponse(heartbeatResponse(Errors.FENCED_INSTANCE_ID));
try {
coordinator.ensureActiveGroup();
mockTime.sleep(HEARTBEAT_INTERVAL_MS);
long startMs = System.currentTimeMillis();
while (System.currentTimeMillis() - startMs < 1000) {
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);
}
assertThrows(FencedInstanceIdException.class,
() -> {
coordinator.ensureActiveGroup();
mockTime.sleep(HEARTBEAT_INTERVAL_MS);
long startMs = System.currentTimeMillis();
while (System.currentTimeMillis() - startMs < 1000) {
Thread.sleep(10);
coordinator.pollHeartbeat(mockTime.milliseconds());
}
},
"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) {

View File

@ -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

View File

@ -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);

View File

@ -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());

View File

@ -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

View File

@ -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 =

View File

@ -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(

View File

@ -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()

View File

@ -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(

View File

@ -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<>();

View File

@ -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());
}
}

View File

@ -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));
}

View File

@ -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

View File

@ -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).

View File

@ -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);
}
// make sure the produce was expired.
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);
}
// make sure the produce was expired.
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);
}
// make sure the produce was expired.
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);
}
// make sure the produce was expired.
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,

View File

@ -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 {

View File

@ -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);
}
}

View File

@ -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"));
}

View File

@ -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);
}

View File

@ -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);

View File

@ -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),

View File

@ -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;

View File

@ -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);

View File

@ -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());

View File

@ -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());

View File

@ -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());

View File

@ -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 {

View File

@ -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) {

View File

@ -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());

View File

@ -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");
}

View File

@ -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

View File

@ -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

View File

@ -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());

View File

@ -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);
}
}

View File

@ -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());
}

View File

@ -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);
}

View File

@ -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() {

View File

@ -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());

View File

@ -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();

View File

@ -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;
}
}

View File

@ -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

View File

@ -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-");

View File

@ -115,8 +115,10 @@ public class OffsetUtils {
}
if (!(keyList.get(1) instanceof Map)) {
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)));
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;
}

View File

@ -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());

View File

@ -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;
}

View File

@ -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