MINOR: Various cleanups in metadata (#14734)

- Remove unused code, suppression
- Simplify/fix test assertions
- Javadoc cleanups

Reviewers: Josep Prat <josep.prat@aiven.io>
This commit is contained in:
Mickael Maison 2023-11-14 09:25:09 +01:00 committed by GitHub
parent 49d3122d42
commit 832627fc78
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
36 changed files with 128 additions and 170 deletions

View File

@ -35,7 +35,7 @@ public enum BrokerRegistrationFencingChange {
private final static Map<Byte, BrokerRegistrationFencingChange> VALUE_TO_ENUM = private final static Map<Byte, BrokerRegistrationFencingChange> VALUE_TO_ENUM =
Arrays.stream(BrokerRegistrationFencingChange.values()). Arrays.stream(BrokerRegistrationFencingChange.values()).
collect(Collectors.toMap(v -> Byte.valueOf(v.value()), Function.identity())); collect(Collectors.toMap(v -> v.value(), Function.identity()));
public static Optional<BrokerRegistrationFencingChange> fromValue(byte value) { public static Optional<BrokerRegistrationFencingChange> fromValue(byte value) {
return Optional.ofNullable(VALUE_TO_ENUM.get(value)); return Optional.ofNullable(VALUE_TO_ENUM.get(value));

View File

@ -36,7 +36,7 @@ public enum BrokerRegistrationInControlledShutdownChange {
private final static Map<Byte, BrokerRegistrationInControlledShutdownChange> VALUE_TO_ENUM = private final static Map<Byte, BrokerRegistrationInControlledShutdownChange> VALUE_TO_ENUM =
Arrays.stream(BrokerRegistrationInControlledShutdownChange.values()). Arrays.stream(BrokerRegistrationInControlledShutdownChange.values()).
collect(Collectors.toMap(v -> Byte.valueOf(v.value()), Function.identity())); collect(Collectors.toMap(v -> v.value(), Function.identity()));
public static Optional<BrokerRegistrationInControlledShutdownChange> fromValue(byte value) { public static Optional<BrokerRegistrationInControlledShutdownChange> fromValue(byte value) {
return Optional.ofNullable(VALUE_TO_ENUM.get(value)); return Optional.ofNullable(VALUE_TO_ENUM.get(value));

View File

@ -33,7 +33,7 @@ import java.util.Objects;
*/ */
public final class DelegationTokenData { public final class DelegationTokenData {
private TokenInformation tokenInformation; private final TokenInformation tokenInformation;
public static DelegationTokenData fromRecord(DelegationTokenRecord record) { public static DelegationTokenData fromRecord(DelegationTokenRecord record) {
List<KafkaPrincipal> renewers = new ArrayList<>(); List<KafkaPrincipal> renewers = new ArrayList<>();
@ -62,7 +62,7 @@ public final class DelegationTokenData {
return new DelegationTokenRecord() return new DelegationTokenRecord()
.setOwner(tokenInformation.ownerAsString()) .setOwner(tokenInformation.ownerAsString())
.setRequester(tokenInformation.tokenRequesterAsString()) .setRequester(tokenInformation.tokenRequesterAsString())
.setRenewers(new ArrayList<String>(tokenInformation.renewersAsString())) .setRenewers(new ArrayList<>(tokenInformation.renewersAsString()))
.setIssueTimestamp(tokenInformation.issueTimestamp()) .setIssueTimestamp(tokenInformation.issueTimestamp())
.setMaxTimestamp(tokenInformation.maxTimestamp()) .setMaxTimestamp(tokenInformation.maxTimestamp())
.setExpirationTimestamp(tokenInformation.expiryTimestamp()) .setExpirationTimestamp(tokenInformation.expiryTimestamp())

View File

@ -56,7 +56,7 @@ public enum LeaderRecoveryState {
private final byte value; private final byte value;
private LeaderRecoveryState(byte value) { LeaderRecoveryState(byte value) {
this.value = value; this.value = value;
} }

View File

@ -358,8 +358,8 @@ public class PartitionRegistration {
if (options.metadataVersion().isDirectoryAssignmentSupported()) { if (options.metadataVersion().isDirectoryAssignmentSupported()) {
record.setDirectories(Uuid.toList(directories)); record.setDirectories(Uuid.toList(directories));
} else { } else {
for (int i = 0; i < directories.length; i++) { for (Uuid directory : directories) {
if (!DirectoryId.UNASSIGNED.equals(directories[i])) { if (!DirectoryId.UNASSIGNED.equals(directory)) {
options.handleLoss("the directory assignment state of one or more replicas"); options.handleLoss("the directory assignment state of one or more replicas");
break; break;
} }

View File

@ -122,7 +122,7 @@ public class StandardAuthorizer implements ClusterMetadataAuthorizer {
Map<Endpoint, CompletableFuture<Void>> result = new HashMap<>(); Map<Endpoint, CompletableFuture<Void>> result = new HashMap<>();
for (Endpoint endpoint : serverInfo.endpoints()) { for (Endpoint endpoint : serverInfo.endpoints()) {
if (serverInfo.earlyStartListeners().contains( if (serverInfo.earlyStartListeners().contains(
endpoint.listenerName().orElseGet(() -> ""))) { endpoint.listenerName().orElse(""))) {
result.put(endpoint, CompletableFuture.completedFuture(null)); result.put(endpoint, CompletableFuture.completedFuture(null));
} else { } else {
result.put(endpoint, initialLoadFuture); result.put(endpoint, initialLoadFuture);

View File

@ -33,89 +33,86 @@ import org.apache.kafka.metadata.OptionalStringComparator;
/** /**
* The striped replica placer. * The striped replica placer.
* * <p>
* * <h3>Goals</h3>
* GOALS * <p>The design of this placer attempts to satisfy a few competing goals. Firstly, we want
* The design of this placer attempts to satisfy a few competing goals. Firstly, we want * to spread the replicas as evenly as we can across racks. In the simple case where
* to spread the replicas as evenly as we can across racks. In the simple case where * broker racks have not been configured, this goal is a no-op, of course. But it is the
* broker racks have not been configured, this goal is a no-op, of course. But it is the
* highest priority goal in multi-rack clusters. * highest priority goal in multi-rack clusters.
* *
* Our second goal is to spread the replicas evenly across brokers. Since we are placing * <p>Our second goal is to spread the replicas evenly across brokers. Since we are placing
* multiple partitions, we try to avoid putting each partition on the same set of * multiple partitions, we try to avoid putting each partition on the same set of
* replicas, even if it does satisfy the rack placement goal. If any specific broker is * replicas, even if it does satisfy the rack placement goal. If any specific broker is
* fenced, we would like the new leaders to distributed evenly across the remaining * fenced, we would like the new leaders to distributed evenly across the remaining
* brokers. * brokers.
* *
* However, we treat the rack placement goal as higher priority than this goal-- if you * <p>However, we treat the rack placement goal as higher priority than this goal-- if you
* configure 10 brokers in rack A and B, and 1 broker in rack C, you will end up with a * configure 10 brokers in rack A and B, and 1 broker in rack C, you will end up with a
* lot of partitions on that one broker in rack C. If you were to place a lot of * lot of partitions on that one broker in rack C. If you were to place a lot of
* partitions with replication factor 3, each partition would try to get a replica there. * partitions with replication factor 3, each partition would try to get a replica there.
* In general racks are supposed to be about the same size -- if they aren't, this is a * In general racks are supposed to be about the same size -- if they aren't, this is a
* user error. * user error.
* *
* Finally, we would prefer to place replicas on unfenced brokers, rather than on fenced * <p>Finally, we would prefer to place replicas on unfenced brokers, rather than on fenced
* brokers. * brokers.
* * <p>
* * <h3>Constraints</h3>
* CONSTRAINTS * In addition to these goals, we have two constraints. Unlike the goals, these are not
* In addition to these goals, we have two constraints. Unlike the goals, these are not * optional -- they are mandatory. Placement will fail if a constraint cannot be
* optional -- they are mandatory. Placement will fail if a constraint cannot be * satisfied. The first constraint is that we can't place more than one replica on the
* satisfied. The first constraint is that we can't place more than one replica on the * same broker. This imposes an upper limit on replication factor-- for example, a 3-node
* same broker. This imposes an upper limit on replication factor-- for example, a 3-node * cluster can't have any topics with replication factor 4. This constraint comes from
* cluster can't have any topics with replication factor 4. This constraint comes from
* Kafka's internal design. * Kafka's internal design.
* *
* The second constraint is that the leader of each partition must be an unfenced broker. * <p>The second constraint is that the leader of each partition must be an unfenced broker.
* This constraint is a bit arbitrary. In theory, we could allow people to create * This constraint is a bit arbitrary. In theory, we could allow people to create
* new topics even if every broker were fenced. However, this would be confusing for * new topics even if every broker were fenced. However, this would be confusing for
* users. * users.
* * <p>
* * <h3>Algorithm</h3>
* ALGORITHM * <p>The StripedReplicaPlacer constructor loads the broker data into rack objects. Each
* The StripedReplicaPlacer constructor loads the broker data into rack objects. Each
* rack object contains a sorted list of fenced brokers, and a separate sorted list of * rack object contains a sorted list of fenced brokers, and a separate sorted list of
* unfenced brokers. The racks themselves are organized into a sorted list, stored inside * unfenced brokers. The racks themselves are organized into a sorted list, stored inside
* the top-level RackList object. * the top-level RackList object.
* *
* The general idea is that we place replicas on to racks in a round-robin fashion. So if * <p>The general idea is that we place replicas on to racks in a round-robin fashion. So if
* we had racks A, B, C, and D, and we were creating a new partition with replication * we had racks A, B, C, and D, and we were creating a new partition with replication
* factor 3, our first replica might come from A, our second from B, and our third from C. * factor 3, our first replica might come from A, our second from B, and our third from C.
* Of course our placement would not be very fair if we always started with rack A. * Of course our placement would not be very fair if we always started with rack A.
* Therefore, we generate a random starting offset when the RackList is created. So one * Therefore, we generate a random starting offset when the RackList is created. So one
* time we might go B, C, D. Another time we might go C, D, A. And so forth. * time we might go B, C, D. Another time we might go C, D, A. And so forth.
* *
* Note that each partition we generate advances the starting offset by one. * <p>Note that each partition we generate advances the starting offset by one.
* So in our 4-rack cluster, with 3 partitions, we might choose these racks: * So in our 4-rack cluster, with 3 partitions, we might choose these racks:
* * <pre>
* partition 1: A, B, C * partition 1: A, B, C
* partition 2: B, C, A * partition 2: B, C, A
* partition 3: C, A, B * partition 3: C, A, B
* * </pre>
* This is what generates the characteristic "striped" pattern of this placer. * This is what generates the characteristic "striped" pattern of this placer.
* *
* So far I haven't said anything about how we choose a replica from within a rack. In * <p>So far I haven't said anything about how we choose a replica from within a rack. In
* fact, this is also done in a round-robin fashion. So if rack A had replica A0, A1, A2, * fact, this is also done in a round-robin fashion. So if rack A had replica A0, A1, A2,
* and A3, we might return A0 the first time, A1, the second, A2 the third, and so on. * and A3, we might return A0 the first time, A1, the second, A2 the third, and so on.
* Just like with the racks, we add a random starting offset to mix things up a bit. * Just like with the racks, we add a random starting offset to mix things up a bit.
* *
* So let's say you had a cluster with racks A, B, and C, and each rack had 3 replicas, * <p>So let's say you had a cluster with racks A, B, and C, and each rack had 3 replicas,
* for 9 nodes in total. * for 9 nodes in total.
* If all the offsets were 0, you'd get placements like this: * If all the offsets were 0, you'd get placements like this:
* * <pre>
* partition 1: A0, B0, C0 * partition 1: A0, B0, C0
* partition 2: B1, C1, A1 * partition 2: B1, C1, A1
* partition 3: C2, A2, B2 * partition 3: C2, A2, B2
* * </pre>
* One additional complication with choosing a replica within a rack is that we want to * <p>One additional complication with choosing a replica within a rack is that we want to
* choose the unfenced replicas first. In a big cluster with lots of nodes available, * choose the unfenced replicas first. In a big cluster with lots of nodes available,
* we'd prefer not to place a new partition on a node that is fenced. Therefore, we * we'd prefer not to place a new partition on a node that is fenced. Therefore, we
* actually maintain two lists, rather than the single list I described above. * actually maintain two lists, rather than the single list I described above.
* We only start using the fenced node list when the unfenced node list is totally * We only start using the fenced node list when the unfenced node list is totally
* exhausted. * exhausted.
* *
* Furthermore, we cannot place the first replica (the leader) of a new partition on a * <p>Furthermore, we cannot place the first replica (the leader) of a new partition on a
* fenced replica. Therefore, we have some special logic to ensure that this doesn't * fenced replica. Therefore, we have some special logic to ensure that this doesn't
* happen. * happen.
*/ */
public class StripedReplicaPlacer implements ReplicaPlacer { public class StripedReplicaPlacer implements ReplicaPlacer {
@ -272,7 +269,6 @@ public class StripedReplicaPlacer implements ReplicaPlacer {
/** /**
* The names of all the racks in the cluster. * The names of all the racks in the cluster.
*
* Racks which have at least one unfenced broker come first (in sorted order), * Racks which have at least one unfenced broker come first (in sorted order),
* followed by racks which have only fenced brokers (also in sorted order). * followed by racks which have only fenced brokers (also in sorted order).
*/ */

View File

@ -113,8 +113,7 @@ public final class BatchFileReader implements Iterator<BatchFileReader.BatchAndT
private BatchAndType nextControlBatch(FileChannelRecordBatch input) { private BatchAndType nextControlBatch(FileChannelRecordBatch input) {
List<ApiMessageAndVersion> messages = new ArrayList<>(); List<ApiMessageAndVersion> messages = new ArrayList<>();
for (Iterator<Record> iter = input.iterator(); iter.hasNext(); ) { for (Record record : input) {
Record record = iter.next();
try { try {
short typeId = ControlRecordType.parseTypeId(record.key()); short typeId = ControlRecordType.parseTypeId(record.key());
ControlRecordType type = ControlRecordType.fromTypeId(typeId); ControlRecordType type = ControlRecordType.fromTypeId(typeId);
@ -179,6 +178,6 @@ public final class BatchFileReader implements Iterator<BatchFileReader.BatchAndT
} catch (Exception e) { } catch (Exception e) {
log.error("Error closing fileRecords", e); log.error("Error closing fileRecords", e);
} }
this.batchIterator = Collections.<FileChannelRecordBatch>emptyList().iterator(); this.batchIterator = Collections.emptyIterator();
} }
} }

View File

@ -49,7 +49,7 @@ public final class RecordRedactor {
case USER_SCRAM_CREDENTIAL_RECORD: { case USER_SCRAM_CREDENTIAL_RECORD: {
UserScramCredentialRecord record = (UserScramCredentialRecord) message; UserScramCredentialRecord record = (UserScramCredentialRecord) message;
return "UserScramCredentialRecord(" return "UserScramCredentialRecord("
+ "name=" + ((record.name() == null) ? "null" : "'" + record.name().toString() + "'") + "name=" + ((record.name() == null) ? "null" : "'" + record.name() + "'")
+ ", mechanism=" + record.mechanism() + ", mechanism=" + record.mechanism()
+ ", salt=(redacted)" + ", salt=(redacted)"
+ ", storedKey=(redacted)" + ", storedKey=(redacted)"

View File

@ -240,7 +240,7 @@ public class ClusterControlManagerTest {
} }
@Test @Test
public void testRegistrationWithIncorrectClusterId() throws Exception { public void testRegistrationWithIncorrectClusterId() {
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder(). FeatureControlManager featureControl = new FeatureControlManager.Builder().
setSnapshotRegistry(snapshotRegistry). setSnapshotRegistry(snapshotRegistry).
@ -317,7 +317,7 @@ public class ClusterControlManagerTest {
} }
@Test @Test
public void testUnregister() throws Exception { public void testUnregister() {
RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord(). RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().
setBrokerId(1). setBrokerId(1).
setBrokerEpoch(100). setBrokerEpoch(100).
@ -365,7 +365,7 @@ public class ClusterControlManagerTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(ints = {3, 10}) @ValueSource(ints = {3, 10})
public void testPlaceReplicas(int numUsableBrokers) throws Exception { public void testPlaceReplicas(int numUsableBrokers) {
MockTime time = new MockTime(0, 0, 0); MockTime time = new MockTime(0, 0, 0);
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder(). FeatureControlManager featureControl = new FeatureControlManager.Builder().
@ -418,7 +418,7 @@ public class ClusterControlManagerTest {
@ParameterizedTest @ParameterizedTest
@EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"}) @EnumSource(value = MetadataVersion.class, names = {"IBP_3_3_IV2", "IBP_3_3_IV3"})
public void testRegistrationsToRecords(MetadataVersion metadataVersion) throws Exception { public void testRegistrationsToRecords(MetadataVersion metadataVersion) {
MockTime time = new MockTime(0, 0, 0); MockTime time = new MockTime(0, 0, 0);
SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext());
FeatureControlManager featureControl = new FeatureControlManager.Builder(). FeatureControlManager featureControl = new FeatureControlManager.Builder().

View File

@ -122,8 +122,6 @@ public class ProducerIdControlManagerTest {
@Test @Test
public void testUnknownBrokerOrEpoch() { public void testUnknownBrokerOrEpoch() {
ControllerResult<ProducerIdsBlock> result;
assertThrows(StaleBrokerEpochException.class, () -> assertThrows(StaleBrokerEpochException.class, () ->
producerIdControlManager.generateNextProducerId(99, 0)); producerIdControlManager.generateNextProducerId(99, 0));

View File

@ -30,8 +30,6 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource; import org.junit.jupiter.params.provider.ValueSource;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
@ -46,13 +44,13 @@ import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.f
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.pause; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.pause;
import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.registerBrokersAndUnfence; import static org.apache.kafka.controller.QuorumControllerIntegrationTestUtils.registerBrokersAndUnfence;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(value = 40) @Timeout(value = 40)
public class QuorumControllerMetricsIntegrationTest { public class QuorumControllerMetricsIntegrationTest {
private final static Logger log = LoggerFactory.getLogger(QuorumControllerMetricsIntegrationTest.class);
static class MockControllerMetrics extends QuorumControllerMetrics { static class MockControllerMetrics extends QuorumControllerMetrics {
final AtomicBoolean closed = new AtomicBoolean(false); final AtomicBoolean closed = new AtomicBoolean(false);
@ -179,7 +177,7 @@ public class QuorumControllerMetricsIntegrationTest {
for (QuorumController controller : controlEnv.controllers()) { for (QuorumController controller : controlEnv.controllers()) {
// Inactive controllers don't set these metrics. // Inactive controllers don't set these metrics.
if (!controller.isActive()) { if (!controller.isActive()) {
assertEquals(false, controller.controllerMetrics().active()); assertFalse(controller.controllerMetrics().active());
assertEquals(0L, controller.controllerMetrics().timedOutHeartbeats()); assertEquals(0L, controller.controllerMetrics().timedOutHeartbeats());
assertEquals(0L, controller.controllerMetrics().operationsTimedOut()); assertEquals(0L, controller.controllerMetrics().operationsTimedOut());
} }

View File

@ -1102,8 +1102,8 @@ public class QuorumControllerTest {
} }
static class InitialSnapshot implements AutoCloseable { static class InitialSnapshot implements AutoCloseable {
File tempDir = null; File tempDir;
BatchFileWriter writer = null; BatchFileWriter writer;
public InitialSnapshot(List<ApiMessageAndVersion> records) throws Exception { public InitialSnapshot(List<ApiMessageAndVersion> records) throws Exception {
tempDir = TestUtils.tempDirectory(); tempDir = TestUtils.tempDirectory();
@ -1292,7 +1292,7 @@ public class QuorumControllerTest {
controllerBuilder.setZkMigrationEnabled(migrationEnabled); controllerBuilder.setZkMigrationEnabled(migrationEnabled);
}). }).
setBootstrapMetadata(BootstrapMetadata.fromVersion(metadataVersion, "test")). setBootstrapMetadata(BootstrapMetadata.fromVersion(metadataVersion, "test")).
build(); build()
) { ) {
QuorumController active = controlEnv.activeController(); QuorumController active = controlEnv.activeController();
ZkMigrationState zkMigrationState = active.appendReadEvent("read migration state", OptionalLong.empty(), ZkMigrationState zkMigrationState = active.appendReadEvent("read migration state", OptionalLong.empty(),
@ -1317,7 +1317,7 @@ public class QuorumControllerTest {
controllerBuilder.setZkMigrationEnabled(true); controllerBuilder.setZkMigrationEnabled(true);
}). }).
setBootstrapMetadata(bootstrapMetadata). setBootstrapMetadata(bootstrapMetadata).
build(); build()
) { ) {
QuorumController active = controlEnv.activeController(); QuorumController active = controlEnv.activeController();
assertEquals(active.featureControl().zkMigrationState(), ZkMigrationState.MIGRATION); assertEquals(active.featureControl().zkMigrationState(), ZkMigrationState.MIGRATION);
@ -1453,7 +1453,7 @@ public class QuorumControllerTest {
@Test @Test
public void testMigrationsEnabledForOldBootstrapMetadataVersion() throws Exception { public void testMigrationsEnabledForOldBootstrapMetadataVersion() throws Exception {
try ( try (
LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build(); LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build()
) { ) {
QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv). QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder -> { setControllerBuilderInitializer(controllerBuilder -> {
@ -1571,7 +1571,7 @@ public class QuorumControllerTest {
@Test @Test
public void testFailoverDuringMigrationTransaction() throws Exception { public void testFailoverDuringMigrationTransaction() throws Exception {
try ( try (
LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).build(); LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).build()
) { ) {
QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv). QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder -> controllerBuilder.setZkMigrationEnabled(true)). setControllerBuilderInitializer(controllerBuilder -> controllerBuilder.setZkMigrationEnabled(true)).
@ -1616,7 +1616,7 @@ public class QuorumControllerTest {
@EnumSource(value = MetadataVersion.class, names = {"IBP_3_4_IV0", "IBP_3_5_IV0", "IBP_3_6_IV0", "IBP_3_6_IV1"}) @EnumSource(value = MetadataVersion.class, names = {"IBP_3_4_IV0", "IBP_3_5_IV0", "IBP_3_6_IV0", "IBP_3_6_IV1"})
public void testBrokerHeartbeatDuringMigration(MetadataVersion metadataVersion) throws Exception { public void testBrokerHeartbeatDuringMigration(MetadataVersion metadataVersion) throws Exception {
try ( try (
LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build(); LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(1).build()
) { ) {
QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv). QuorumControllerTestEnv.Builder controlEnvBuilder = new QuorumControllerTestEnv.Builder(logEnv).
setControllerBuilderInitializer(controllerBuilder -> setControllerBuilderInitializer(controllerBuilder ->

View File

@ -17,7 +17,6 @@
package org.apache.kafka.controller; package org.apache.kafka.controller;
import org.apache.kafka.clients.ApiVersions;
import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata;
import org.apache.kafka.metalog.LocalLogManagerTestEnv; import org.apache.kafka.metalog.LocalLogManagerTestEnv;
import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.LeaderAndEpoch;
@ -98,7 +97,6 @@ public class QuorumControllerTestEnv implements AutoCloseable {
int numControllers = logEnv.logManagers().size(); int numControllers = logEnv.logManagers().size();
this.controllers = new ArrayList<>(numControllers); this.controllers = new ArrayList<>(numControllers);
try { try {
ApiVersions apiVersions = new ApiVersions();
List<Integer> nodeIds = IntStream.range(0, numControllers).boxed().collect(Collectors.toList()); List<Integer> nodeIds = IntStream.range(0, numControllers).boxed().collect(Collectors.toList());
for (int nodeId = 0; nodeId < numControllers; nodeId++) { for (int nodeId = 0; nodeId < numControllers; nodeId++) {
QuorumController.Builder builder = new QuorumController.Builder(nodeId, logEnv.clusterId()); QuorumController.Builder builder = new QuorumController.Builder(nodeId, logEnv.clusterId());

View File

@ -921,8 +921,8 @@ public class ReplicationControlManagerTest {
shrinkIsrResult, topicIdPartition, NONE); shrinkIsrResult, topicIdPartition, NONE);
assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, shrinkIsrResponse); assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, shrinkIsrResponse);
PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
assertTrue(Arrays.equals(new int[]{1, 2}, partition.elr), partition.toString()); assertArrayEquals(new int[]{1, 2}, partition.elr, partition.toString());
assertTrue(Arrays.equals(new int[]{}, partition.lastKnownElr), partition.toString()); assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
PartitionData expandIsrRequest = newAlterPartition( PartitionData expandIsrRequest = newAlterPartition(
replicationControl, topicIdPartition, isrWithDefaultEpoch(0, 1), LeaderRecoveryState.RECOVERED); replicationControl, topicIdPartition, isrWithDefaultEpoch(0, 1), LeaderRecoveryState.RECOVERED);
@ -932,8 +932,8 @@ public class ReplicationControlManagerTest {
expandIsrResult, topicIdPartition, NONE); expandIsrResult, topicIdPartition, NONE);
assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, expandIsrResponse); assertConsistentAlterPartitionResponse(replicationControl, topicIdPartition, expandIsrResponse);
partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
assertTrue(Arrays.equals(new int[]{}, partition.elr), partition.toString()); assertArrayEquals(new int[]{}, partition.elr, partition.toString());
assertTrue(Arrays.equals(new int[]{}, partition.lastKnownElr), partition.toString()); assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
} }
@Test @Test
@ -952,19 +952,19 @@ public class ReplicationControlManagerTest {
ctx.fenceBrokers(Utils.mkSet(2, 3)); ctx.fenceBrokers(Utils.mkSet(2, 3));
PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
assertTrue(Arrays.equals(new int[]{3}, partition.elr), partition.toString()); assertArrayEquals(new int[]{3}, partition.elr, partition.toString());
assertTrue(Arrays.equals(new int[]{}, partition.lastKnownElr), partition.toString()); assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
ctx.fenceBrokers(Utils.mkSet(1, 2, 3)); ctx.fenceBrokers(Utils.mkSet(1, 2, 3));
partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
assertTrue(Arrays.equals(new int[]{1, 3}, partition.elr), partition.toString()); assertArrayEquals(new int[]{1, 3}, partition.elr, partition.toString());
assertTrue(Arrays.equals(new int[]{}, partition.lastKnownElr), partition.toString()); assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
ctx.unfenceBrokers(0, 1, 2, 3); ctx.unfenceBrokers(0, 1, 2, 3);
partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
assertTrue(Arrays.equals(new int[]{1, 3}, partition.elr), partition.toString()); assertArrayEquals(new int[]{1, 3}, partition.elr, partition.toString());
assertTrue(Arrays.equals(new int[]{}, partition.lastKnownElr), partition.toString()); assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
} }
@Test @Test
@ -1000,16 +1000,16 @@ public class ReplicationControlManagerTest {
ctx.fenceBrokers(Utils.mkSet(1, 2, 3)); ctx.fenceBrokers(Utils.mkSet(1, 2, 3));
PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
assertTrue(Arrays.equals(new int[]{2, 3}, partition.elr), partition.toString()); assertArrayEquals(new int[]{2, 3}, partition.elr, partition.toString());
assertTrue(Arrays.equals(new int[]{}, partition.lastKnownElr), partition.toString()); assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
ctx.unfenceBrokers(2); ctx.unfenceBrokers(2);
ctx.fenceBrokers(Utils.mkSet(0, 1)); ctx.fenceBrokers(Utils.mkSet(0, 1));
partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId()); partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
assertTrue(Arrays.equals(new int[]{0, 3}, partition.elr), partition.toString()); assertArrayEquals(new int[]{0, 3}, partition.elr, partition.toString());
assertTrue(Arrays.equals(new int[]{2}, partition.isr), partition.toString()); assertArrayEquals(new int[]{2}, partition.isr, partition.toString());
assertEquals(2, partition.leader, partition.toString()); assertEquals(2, partition.leader, partition.toString());
assertTrue(Arrays.equals(new int[]{}, partition.lastKnownElr), partition.toString()); assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
} }
@ParameterizedTest @ParameterizedTest
@ -1152,7 +1152,7 @@ public class ReplicationControlManagerTest {
long brokerEpoch, long brokerEpoch,
Uuid topicId, Uuid topicId,
AlterPartitionRequestData.PartitionData partitionData AlterPartitionRequestData.PartitionData partitionData
) throws Exception { ) {
AlterPartitionRequestData request = new AlterPartitionRequestData() AlterPartitionRequestData request = new AlterPartitionRequestData()
.setBrokerId(brokerId) .setBrokerId(brokerId)
.setBrokerEpoch(brokerEpoch); .setBrokerEpoch(brokerEpoch);
@ -1424,7 +1424,6 @@ public class ReplicationControlManagerTest {
anonymousContextFor(ApiKeys.CREATE_TOPICS); anonymousContextFor(ApiKeys.CREATE_TOPICS);
ControllerResult<CreateTopicsResponseData> createResult = ControllerResult<CreateTopicsResponseData> createResult =
replicationControl.createTopics(createTopicsRequestContext, request, Collections.singleton("foo")); replicationControl.createTopics(createTopicsRequestContext, request, Collections.singleton("foo"));
CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData();
CreatableTopicResult createdTopic = createResult.response().topics().find("foo"); CreatableTopicResult createdTopic = createResult.response().topics().find("foo");
assertEquals(NONE.code(), createdTopic.errorCode()); assertEquals(NONE.code(), createdTopic.errorCode());
ctx.replay(createResult.records()); ctx.replay(createResult.records());

View File

@ -37,7 +37,7 @@ import java.util.Optional;
@Timeout(40) @Timeout(40)
public class AclsDeltaTest { public class AclsDeltaTest {
private Uuid aclId = Uuid.fromString("iOZpss6VQUmD6blnqzl50g"); private final Uuid aclId = Uuid.fromString("iOZpss6VQUmD6blnqzl50g");
@Test @Test
public void testRemovesDeleteIfNotInImage() { public void testRemovesDeleteIfNotInImage() {

View File

@ -37,8 +37,6 @@ import org.apache.kafka.metadata.VersionRange;
import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
@ -55,7 +53,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
@Timeout(value = 40) @Timeout(value = 40)
public class ClusterImageTest { public class ClusterImageTest {
private static final Logger log = LoggerFactory.getLogger(ClusterImageTest.class);
public final static ClusterImage IMAGE1; public final static ClusterImage IMAGE1;

View File

@ -56,7 +56,7 @@ public class DelegationTokenImageTest {
tokenId, tokenId,
SecurityUtils.parseKafkaPrincipal(KafkaPrincipal.USER_TYPE + ":" + "fred"), SecurityUtils.parseKafkaPrincipal(KafkaPrincipal.USER_TYPE + ":" + "fred"),
SecurityUtils.parseKafkaPrincipal(KafkaPrincipal.USER_TYPE + ":" + "fred"), SecurityUtils.parseKafkaPrincipal(KafkaPrincipal.USER_TYPE + ":" + "fred"),
new ArrayList<KafkaPrincipal>(), new ArrayList<>(),
0, 0,
1000, 1000,
expireTimestamp); expireTimestamp);

View File

@ -28,7 +28,7 @@ import java.util.List;
public class FakeSnapshotWriter implements SnapshotWriter<ApiMessageAndVersion> { public class FakeSnapshotWriter implements SnapshotWriter<ApiMessageAndVersion> {
private final OffsetAndEpoch snapshotId; private final OffsetAndEpoch snapshotId;
private List<List<ApiMessageAndVersion>> batches = new ArrayList<>(); private final List<List<ApiMessageAndVersion>> batches = new ArrayList<>();
private boolean frozen = false; private boolean frozen = false;
private boolean closed = false; private boolean closed = false;
@ -79,7 +79,7 @@ public class FakeSnapshotWriter implements SnapshotWriter<ApiMessageAndVersion>
@Override @Override
public long freeze() { public long freeze() {
frozen = true; frozen = true;
return batches.size() * 100; return batches.size() * 100L;
} }
@Override @Override

View File

@ -103,7 +103,7 @@ public class ImageDowngradeTest {
* Test downgrading to a MetadataVersion that doesn't support inControlledShutdown. * Test downgrading to a MetadataVersion that doesn't support inControlledShutdown.
*/ */
@Test @Test
public void testPreControlledShutdownStateVersion() throws Throwable { public void testPreControlledShutdownStateVersion() {
writeWithExpectedLosses(MetadataVersion.IBP_3_3_IV2, writeWithExpectedLosses(MetadataVersion.IBP_3_3_IV2,
Arrays.asList( Arrays.asList(
"the inControlledShutdown state of one or more brokers"), "the inControlledShutdown state of one or more brokers"),

View File

@ -118,10 +118,6 @@ public class MetadataImageTest {
.build(), Optional.empty()); .build(), Optional.empty());
} }
private static void testToImage(MetadataImage image, ImageWriterOptions options) {
testToImage(image, options, Optional.empty());
}
static void testToImage(MetadataImage image, ImageWriterOptions options, Optional<List<ApiMessageAndVersion>> fromRecords) { static void testToImage(MetadataImage image, ImageWriterOptions options, Optional<List<ApiMessageAndVersion>> fromRecords) {
testToImage(image, fromRecords.orElseGet(() -> getImageRecords(image, options))); testToImage(image, fromRecords.orElseGet(() -> getImageRecords(image, options)));
} }

View File

@ -19,8 +19,6 @@ package org.apache.kafka.image;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1; import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1;
import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV0; import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV0;
@ -31,7 +29,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(value = 40) @Timeout(value = 40)
public class MetadataVersionChangeTest { public class MetadataVersionChangeTest {
private static final Logger log = LoggerFactory.getLogger(MetadataVersionChangeTest.class);
private final static MetadataVersionChange CHANGE_3_0_IV1_TO_3_3_IV0 = private final static MetadataVersionChange CHANGE_3_0_IV1_TO_3_3_IV0 =
new MetadataVersionChange(IBP_3_0_IV1, IBP_3_3_IV0); new MetadataVersionChange(IBP_3_0_IV1, IBP_3_3_IV0);
@ -40,19 +37,19 @@ public class MetadataVersionChangeTest {
new MetadataVersionChange(IBP_3_3_IV0, IBP_3_0_IV1); new MetadataVersionChange(IBP_3_3_IV0, IBP_3_0_IV1);
@Test @Test
public void testIsUpgrade() throws Throwable { public void testIsUpgrade() {
assertTrue(CHANGE_3_0_IV1_TO_3_3_IV0.isUpgrade()); assertTrue(CHANGE_3_0_IV1_TO_3_3_IV0.isUpgrade());
assertFalse(CHANGE_3_3_IV0_TO_3_0_IV1.isUpgrade()); assertFalse(CHANGE_3_3_IV0_TO_3_0_IV1.isUpgrade());
} }
@Test @Test
public void testIsDowngrade() throws Throwable { public void testIsDowngrade() {
assertFalse(CHANGE_3_0_IV1_TO_3_3_IV0.isDowngrade()); assertFalse(CHANGE_3_0_IV1_TO_3_3_IV0.isDowngrade());
assertTrue(CHANGE_3_3_IV0_TO_3_0_IV1.isDowngrade()); assertTrue(CHANGE_3_3_IV0_TO_3_0_IV1.isDowngrade());
} }
@Test @Test
public void testMetadataVersionChangeExceptionToString() throws Throwable { public void testMetadataVersionChangeExceptionToString() {
assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata " + assertEquals("org.apache.kafka.image.MetadataVersionChangeException: The metadata " +
"version is changing from 3.0-IV1 to 3.3-IV0", "version is changing from 3.0-IV1 to 3.3-IV0",
new MetadataVersionChangeException(CHANGE_3_0_IV1_TO_3_3_IV0).toString()); new MetadataVersionChangeException(CHANGE_3_0_IV1_TO_3_3_IV0).toString());

View File

@ -45,10 +45,6 @@ public class MetadataLoaderMetricsTest {
new AtomicReference<>(MetadataProvenance.EMPTY); new AtomicReference<>(MetadataProvenance.EMPTY);
final MetadataLoaderMetrics metrics; final MetadataLoaderMetrics metrics;
FakeMetadataLoaderMetrics() {
this(Optional.empty());
}
FakeMetadataLoaderMetrics(MetricsRegistry registry) { FakeMetadataLoaderMetrics(MetricsRegistry registry) {
this(Optional.of(registry)); this(Optional.of(registry));
} }

View File

@ -135,7 +135,7 @@ public class SnapshotEmitterTest {
} }
@Test @Test
public void testEmit() throws Exception { public void testEmit() {
MockRaftClient mockRaftClient = new MockRaftClient(); MockRaftClient mockRaftClient = new MockRaftClient();
MockTime time = new MockTime(0, 10000L, 20000L); MockTime time = new MockTime(0, 10000L, 20000L);
SnapshotEmitter emitter = new SnapshotEmitter.Builder(). SnapshotEmitter emitter = new SnapshotEmitter.Builder().

View File

@ -21,12 +21,9 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@Timeout(value = 40) @Timeout(value = 40)
public class BrokerStateTest { public class BrokerStateTest {
private static final Logger log = LoggerFactory.getLogger(BrokerStateTest.class);
@Test @Test
public void testFromValue() { public void testFromValue() {

View File

@ -116,14 +116,14 @@ public class ListenerInfoTest {
} }
@Test @Test
public void testToControllerRegistrationRequestFailsOnNullHost() throws Exception { public void testToControllerRegistrationRequestFailsOnNullHost() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
toControllerRegistrationRequest()); toControllerRegistrationRequest());
} }
@Test @Test
public void testToControllerRegistrationRequestFailsOnZeroPort() throws Exception { public void testToControllerRegistrationRequestFailsOnZeroPort() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
withWildcardHostnamesResolved(). withWildcardHostnamesResolved().
@ -141,14 +141,14 @@ public class ListenerInfoTest {
} }
@Test @Test
public void testToControllerRegistrationRecordFailsOnNullHost() throws Exception { public void testToControllerRegistrationRecordFailsOnNullHost() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
toControllerRegistrationRecord()); toControllerRegistrationRecord());
} }
@Test @Test
public void testToControllerRegistrationRecordFailsOnZeroPort() throws Exception { public void testToControllerRegistrationRecordFailsOnZeroPort() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
withWildcardHostnamesResolved(). withWildcardHostnamesResolved().
@ -166,14 +166,14 @@ public class ListenerInfoTest {
} }
@Test @Test
public void testToBrokerRegistrationRequestFailsOnNullHost() throws Exception { public void testToBrokerRegistrationRequestFailsOnNullHost() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
toBrokerRegistrationRequest()); toBrokerRegistrationRequest());
} }
@Test @Test
public void testToBrokerRegistrationRequestFailsOnZeroPort() throws Exception { public void testToBrokerRegistrationRequestFailsOnZeroPort() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
withWildcardHostnamesResolved(). withWildcardHostnamesResolved().
@ -191,14 +191,14 @@ public class ListenerInfoTest {
} }
@Test @Test
public void testToBrokerRegistrationRecordFailsOnNullHost() throws Exception { public void testToBrokerRegistrationRecordFailsOnNullHost() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
toBrokerRegistrationRecord()); toBrokerRegistrationRecord());
} }
@Test @Test
public void testToBrokerRegistrationRecordFailsOnZeroPort() throws Exception { public void testToBrokerRegistrationRecordFailsOnZeroPort() {
assertThrows(RuntimeException.class, assertThrows(RuntimeException.class,
() -> ListenerInfo.create(Arrays.asList(INTERNAL)). () -> ListenerInfo.create(Arrays.asList(INTERNAL)).
withWildcardHostnamesResolved(). withWildcardHostnamesResolved().

View File

@ -280,8 +280,8 @@ public class PartitionRegistrationTest {
PartitionRecord expectRecord = new PartitionRecord(). PartitionRecord expectRecord = new PartitionRecord().
setTopicId(topicID). setTopicId(topicID).
setPartitionId(0). setPartitionId(0).
setReplicas(Arrays.asList(new Integer[]{0, 1, 2, 3, 4})). setReplicas(Arrays.asList(0, 1, 2, 3, 4)).
setIsr(Arrays.asList(new Integer[]{0, 1})). setIsr(Arrays.asList(0, 1)).
setLeader(0). setLeader(0).
setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()). setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()).
setLeaderEpoch(0). setLeaderEpoch(0).
@ -290,8 +290,8 @@ public class PartitionRegistrationTest {
when(metadataVersion.partitionRecordVersion()).thenReturn(version); when(metadataVersion.partitionRecordVersion()).thenReturn(version);
if (version > 0) { if (version > 0) {
expectRecord. expectRecord.
setEligibleLeaderReplicas(Arrays.asList(new Integer[]{2, 3})). setEligibleLeaderReplicas(Arrays.asList(2, 3)).
setLastKnownELR(Arrays.asList(new Integer[]{4})); setLastKnownELR(Arrays.asList(4));
} else { } else {
when(metadataVersion.isElrSupported()).thenReturn(false); when(metadataVersion.isElrSupported()).thenReturn(false);
} }
@ -318,6 +318,7 @@ public class PartitionRegistrationTest {
assertEquals(Replicas.toList(Replicas.NONE), Replicas.toList(partitionRegistration.addingReplicas)); assertEquals(Replicas.toList(Replicas.NONE), Replicas.toList(partitionRegistration.addingReplicas));
} }
@Test
public void testPartitionRegistrationToRecord_ElrShouldBeNullIfEmpty() { public void testPartitionRegistrationToRecord_ElrShouldBeNullIfEmpty() {
PartitionRegistration.Builder builder = new PartitionRegistration.Builder(). PartitionRegistration.Builder builder = new PartitionRegistration.Builder().
setReplicas(new int[]{0, 1, 2, 3, 4}). setReplicas(new int[]{0, 1, 2, 3, 4}).
@ -331,8 +332,8 @@ public class PartitionRegistrationTest {
PartitionRecord expectRecord = new PartitionRecord(). PartitionRecord expectRecord = new PartitionRecord().
setTopicId(topicID). setTopicId(topicID).
setPartitionId(0). setPartitionId(0).
setReplicas(Arrays.asList(new Integer[]{0, 1, 2, 3, 4})). setReplicas(Arrays.asList(0, 1, 2, 3, 4)).
setIsr(Arrays.asList(new Integer[]{0, 1})). setIsr(Arrays.asList(0, 1)).
setLeader(0). setLeader(0).
setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()). setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()).
setLeaderEpoch(0). setLeaderEpoch(0).
@ -342,8 +343,9 @@ public class PartitionRegistrationTest {
setMetadataVersion(MetadataVersion.latest()). setMetadataVersion(MetadataVersion.latest()).
setLossHandler(exceptions::add). setLossHandler(exceptions::add).
build(); build();
assertEquals(new ApiMessageAndVersion(expectRecord, (short) 2), partitionRegistration.toRecord(topicID, 0, options)); assertEquals(new ApiMessageAndVersion(expectRecord, (short) 1), partitionRegistration.toRecord(topicID, 0, options));
assertEquals(Replicas.toList(Replicas.NONE), Replicas.toList(partitionRegistration.addingReplicas)); assertEquals(Replicas.toList(Replicas.NONE), Replicas.toList(partitionRegistration.addingReplicas));
assertTrue(exceptions.isEmpty());
} }
@Property @Property

View File

@ -273,7 +273,6 @@ public class RecordTestUtils {
* *
* @param o The input object. It will be modified in-place. * @param o The input object. It will be modified in-place.
*/ */
@SuppressWarnings("unchecked")
public static void deepSortRecords(Object o) throws Exception { public static void deepSortRecords(Object o) throws Exception {
if (o == null) { if (o == null) {
return; return;

View File

@ -26,7 +26,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(value = 40) @Timeout(value = 40)
public class VersionRangeTest { public class VersionRangeTest {
@SuppressWarnings("unchecked")
private static VersionRange v(int a, int b) { private static VersionRange v(int a, int b) {
assertTrue(a <= Short.MAX_VALUE); assertTrue(a <= Short.MAX_VALUE);
assertTrue(a >= Short.MIN_VALUE); assertTrue(a >= Short.MIN_VALUE);

View File

@ -48,7 +48,7 @@ public class BootstrapDirectoryTest {
static class BootstrapTestDirectory implements AutoCloseable { static class BootstrapTestDirectory implements AutoCloseable {
File directory = null; File directory = null;
synchronized BootstrapTestDirectory createDirectory() throws Exception { synchronized BootstrapTestDirectory createDirectory() {
directory = TestUtils.tempDirectory("BootstrapTestDirectory"); directory = TestUtils.tempDirectory("BootstrapTestDirectory");
return this; return this;
} }
@ -98,7 +98,7 @@ public class BootstrapDirectoryTest {
} }
@Test @Test
public void testMissingDirectory() throws Exception { public void testMissingDirectory() {
assertEquals("No such directory as ./non/existent/directory", assertEquals("No such directory as ./non/existent/directory",
assertThrows(RuntimeException.class, () -> assertThrows(RuntimeException.class, () ->
new BootstrapDirectory("./non/existent/directory", Optional.empty()).read()).getMessage()); new BootstrapDirectory("./non/existent/directory", Optional.empty()).read()).getMessage());

View File

@ -48,7 +48,7 @@ public class BootstrapMetadataTest {
setFeatureLevel((short) 6), (short) 0))); setFeatureLevel((short) 6), (short) 0)));
@Test @Test
public void testFromVersion() throws Exception { public void testFromVersion() {
assertEquals(new BootstrapMetadata(Collections.singletonList( assertEquals(new BootstrapMetadata(Collections.singletonList(
new ApiMessageAndVersion(new FeatureLevelRecord(). new ApiMessageAndVersion(new FeatureLevelRecord().
setName(FEATURE_NAME). setName(FEATURE_NAME).
@ -58,20 +58,20 @@ public class BootstrapMetadataTest {
} }
@Test @Test
public void testFromRecordsList() throws Exception { public void testFromRecordsList() {
assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV2, "bar"), assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV2, "bar"),
BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "bar")); BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "bar"));
} }
@Test @Test
public void testFromRecordsListWithoutMetadataVersion() throws Exception { public void testFromRecordsListWithoutMetadataVersion() {
assertEquals("No FeatureLevelRecord for metadata.version was found in the bootstrap " + assertEquals("No FeatureLevelRecord for metadata.version was found in the bootstrap " +
"metadata from quux", assertThrows(RuntimeException.class, "metadata from quux", assertThrows(RuntimeException.class,
() -> BootstrapMetadata.fromRecords(emptyList(), "quux")).getMessage()); () -> BootstrapMetadata.fromRecords(emptyList(), "quux")).getMessage());
} }
@Test @Test
public void testCopyWithOnlyVersion() throws Exception { public void testCopyWithOnlyVersion() {
assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1.subList(2, 3), IBP_3_3_IV2, "baz"), assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1.subList(2, 3), IBP_3_3_IV2, "baz"),
BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "baz").copyWithOnlyVersion()); BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "baz").copyWithOnlyVersion());
} }
@ -82,7 +82,7 @@ public class BootstrapMetadataTest {
setFeatureLevel(IBP_3_0_IV1.featureLevel()), (short) 0))); setFeatureLevel(IBP_3_0_IV1.featureLevel()), (short) 0)));
@Test @Test
public void testFromRecordsListWithOldMetadataVersion() throws Exception { public void testFromRecordsListWithOldMetadataVersion() {
RuntimeException exception = assertThrows(RuntimeException.class, RuntimeException exception = assertThrows(RuntimeException.class,
() -> BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux")); () -> BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux"));
assertEquals("Bootstrap metadata versions before 3.3-IV0 are not supported. Can't load " + assertEquals("Bootstrap metadata versions before 3.3-IV0 are not supported. Can't load " +

View File

@ -36,7 +36,7 @@ public class CapturingDelegationTokenMigrationClient implements DelegationTokenM
@Override @Override
public List<String> getDelegationTokens() { public List<String> getDelegationTokens() {
return new ArrayList<String>(); return new ArrayList<>();
} }
@Override @Override

View File

@ -102,7 +102,7 @@ final public class BatchFileWriterReaderTest {
assertEquals(0, apiMessageAndVersion.version()); assertEquals(0, apiMessageAndVersion.version());
SnapshotFooterRecord footerRecord = (SnapshotFooterRecord) apiMessageAndVersion.message(); SnapshotFooterRecord footerRecord = (SnapshotFooterRecord) apiMessageAndVersion.message();
assertEquals(0, headerRecord.version()); assertEquals(0, footerRecord.version());
} }
} }
} }

View File

@ -176,6 +176,11 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
*/ */
private final TreeMap<Long, LocalBatch> batches = new TreeMap<>(); private final TreeMap<Long, LocalBatch> batches = new TreeMap<>();
/**
* Maps committed offset to snapshot reader.
*/
private final NavigableMap<Long, RawSnapshotReader> snapshots = new TreeMap<>();
/** /**
* The current leader. * The current leader.
*/ */
@ -192,11 +197,6 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
*/ */
private long initialMaxReadOffset = Long.MAX_VALUE; private long initialMaxReadOffset = Long.MAX_VALUE;
/**
* Maps committed offset to snapshot reader.
*/
private NavigableMap<Long, RawSnapshotReader> snapshots = new TreeMap<>();
public SharedLogData(Optional<RawSnapshotReader> snapshot) { public SharedLogData(Optional<RawSnapshotReader> snapshot) {
if (snapshot.isPresent()) { if (snapshot.isPresent()) {
RawSnapshotReader initialSnapshot = snapshot.get(); RawSnapshotReader initialSnapshot = snapshot.get();
@ -515,7 +515,7 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
* result is half the records getting appended with leader election following that. * result is half the records getting appended with leader election following that.
* This is done to emulate having some of the records not getting committed. * This is done to emulate having some of the records not getting committed.
*/ */
private AtomicBoolean resignAfterNonAtomicCommit = new AtomicBoolean(false); private final AtomicBoolean resignAfterNonAtomicCommit = new AtomicBoolean(false);
public LocalLogManager(LogContext logContext, public LocalLogManager(LogContext logContext,
int nodeId, int nodeId,
@ -827,7 +827,6 @@ public final class LocalLogManager implements RaftClient<ApiMessageAndVersion>,
// the leader epoch has already advanced. resign is a no op. // the leader epoch has already advanced. resign is a no op.
log.debug("Ignoring call to resign from epoch {}. Either we are not the leader or the provided epoch is " + log.debug("Ignoring call to resign from epoch {}. Either we are not the leader or the provided epoch is " +
"smaller than the current epoch {}", epoch, currentEpoch); "smaller than the current epoch {}", epoch, currentEpoch);
return;
} }
} }

View File

@ -108,7 +108,7 @@ public class LocalLogManagerTest {
long highestOffset = -1; long highestOffset = -1;
for (String event : listener.serializedEvents()) { for (String event : listener.serializedEvents()) {
if (event.startsWith(LAST_COMMITTED_OFFSET)) { if (event.startsWith(LAST_COMMITTED_OFFSET)) {
long offset = Long.valueOf( long offset = Long.parseLong(
event.substring(LAST_COMMITTED_OFFSET.length() + 1)); event.substring(LAST_COMMITTED_OFFSET.length() + 1));
if (offset < highestOffset) { if (offset < highestOffset) {
throw new RuntimeException("Invalid offset: " + offset + throw new RuntimeException("Invalid offset: " + offset +

View File

@ -54,13 +54,9 @@ public class MockMetaLogManagerListener implements RaftClient.Listener<ApiMessag
for (ApiMessageAndVersion messageAndVersion : batch.records()) { for (ApiMessageAndVersion messageAndVersion : batch.records()) {
ApiMessage message = messageAndVersion.message(); ApiMessage message = messageAndVersion.message();
StringBuilder bld = new StringBuilder(); serializedEvents.add(COMMIT + " " + message.toString());
bld.append(COMMIT).append(" ").append(message.toString());
serializedEvents.add(bld.toString());
} }
StringBuilder bld = new StringBuilder(); serializedEvents.add(LAST_COMMITTED_OFFSET + " " + lastCommittedOffset);
bld.append(LAST_COMMITTED_OFFSET).append(" ").append(lastCommittedOffset);
serializedEvents.add(bld.toString());
} }
} finally { } finally {
reader.close(); reader.close();
@ -76,13 +72,9 @@ public class MockMetaLogManagerListener implements RaftClient.Listener<ApiMessag
for (ApiMessageAndVersion messageAndVersion : batch.records()) { for (ApiMessageAndVersion messageAndVersion : batch.records()) {
ApiMessage message = messageAndVersion.message(); ApiMessage message = messageAndVersion.message();
StringBuilder bld = new StringBuilder(); serializedEvents.add(SNAPSHOT + " " + message.toString());
bld.append(SNAPSHOT).append(" ").append(message.toString());
serializedEvents.add(bld.toString());
} }
StringBuilder bld = new StringBuilder(); serializedEvents.add(LAST_COMMITTED_OFFSET + " " + lastCommittedOffset);
bld.append(LAST_COMMITTED_OFFSET).append(" ").append(lastCommittedOffset);
serializedEvents.add(bld.toString());
} }
} finally { } finally {
reader.close(); reader.close();
@ -95,14 +87,10 @@ public class MockMetaLogManagerListener implements RaftClient.Listener<ApiMessag
this.leaderAndEpoch = newLeaderAndEpoch; this.leaderAndEpoch = newLeaderAndEpoch;
if (newLeaderAndEpoch.isLeader(nodeId)) { if (newLeaderAndEpoch.isLeader(nodeId)) {
StringBuilder bld = new StringBuilder(); String bld = NEW_LEADER + " " + nodeId + " " + newLeaderAndEpoch.epoch();
bld.append(NEW_LEADER).append(" "). serializedEvents.add(bld);
append(nodeId).append(" ").append(newLeaderAndEpoch.epoch());
serializedEvents.add(bld.toString());
} else if (oldLeaderAndEpoch.isLeader(nodeId)) { } else if (oldLeaderAndEpoch.isLeader(nodeId)) {
StringBuilder bld = new StringBuilder(); serializedEvents.add(RENOUNCE + " " + newLeaderAndEpoch.epoch());
bld.append(RENOUNCE).append(" ").append(newLeaderAndEpoch.epoch());
serializedEvents.add(bld.toString());
} }
} }