hachikuji commented on a change in pull request #9476: URL: https://github.com/apache/kafka/pull/9476#discussion_r510387310
########## File path: raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java ########## @@ -0,0 +1,648 @@ +/* + * 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. + */ +package org.apache.kafka.raft; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.BeginQuorumEpochRequestData; +import org.apache.kafka.common.message.BeginQuorumEpochResponseData; +import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState; +import org.apache.kafka.common.message.DescribeQuorumResponseData; +import org.apache.kafka.common.message.EndQuorumEpochRequestData; +import org.apache.kafka.common.message.EndQuorumEpochResponseData; +import org.apache.kafka.common.message.FetchRequestData; +import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.LeaderChangeMessage.Voter; +import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.message.VoteRequestData; +import org.apache.kafka.common.message.VoteResponseData; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.ControlRecordUtils; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.requests.BeginQuorumEpochResponse; +import org.apache.kafka.common.requests.DescribeQuorumResponse; +import org.apache.kafka.common.requests.VoteResponse; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.TestUtils; +import org.mockito.Mockito; +import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +final class RaftClientTestContext { + private static final int FETCH_MAX_WAIT_MS = 0; + + static final TopicPartition METADATA_PARTITION = new TopicPartition("metadata", 0); + static final int LOCAL_ID = 0; Review comment: I'm somewhat inclined to add the local id to the builder rather than making it constant. It makes the builder a bit more self-contained. On a similar note, it would be nice to push these state config values into the builder as well. ########## File path: raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java ########## @@ -0,0 +1,648 @@ +/* + * 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. + */ +package org.apache.kafka.raft; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.BeginQuorumEpochRequestData; +import org.apache.kafka.common.message.BeginQuorumEpochResponseData; +import org.apache.kafka.common.message.DescribeQuorumResponseData.ReplicaState; +import org.apache.kafka.common.message.DescribeQuorumResponseData; +import org.apache.kafka.common.message.EndQuorumEpochRequestData; +import org.apache.kafka.common.message.EndQuorumEpochResponseData; +import org.apache.kafka.common.message.FetchRequestData; +import org.apache.kafka.common.message.FetchResponseData; +import org.apache.kafka.common.message.LeaderChangeMessage.Voter; +import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.message.VoteRequestData; +import org.apache.kafka.common.message.VoteResponseData; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.ControlRecordType; +import org.apache.kafka.common.record.ControlRecordUtils; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.Record; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.requests.BeginQuorumEpochResponse; +import org.apache.kafka.common.requests.DescribeQuorumResponse; +import org.apache.kafka.common.requests.VoteResponse; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.TestUtils; +import org.mockito.Mockito; +import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +final class RaftClientTestContext { + private static final int FETCH_MAX_WAIT_MS = 0; + + static final TopicPartition METADATA_PARTITION = new TopicPartition("metadata", 0); + static final int LOCAL_ID = 0; + + static final int ELECTION_BACKOFF_MAX_MS = 100; + static final int ELECTION_TIMEOUT_MS = 10000; + // fetch timeout is usually larger than election timeout + static final int FETCH_TIMEOUT_MS = 50000; + static final int REQUEST_TIMEOUT_MS = 5000; + static final int RETRY_BACKOFF_MS = 50; + + private final QuorumStateStore quorumStateStore; + private final Random random; + + final KafkaRaftClient client; + final Metrics metrics; + final MockLog log; + final MockNetworkChannel channel; + final MockTime time; + final Set<Integer> voters; + + public static final class Builder { + private final QuorumStateStore quorumStateStore = new MockQuorumStateStore(); + private final Random random = Mockito.spy(new Random(1)); + private final MockLog log = new MockLog(METADATA_PARTITION); + private final Set<Integer> voters; + + Builder(Set<Integer> voters) { + this.voters = voters; + } + + Builder withElectedLeader(int epoch, int leaderId) throws IOException { + quorumStateStore.writeElectionState(ElectionState.withElectedLeader(epoch, leaderId, voters)); + return this; + } + + Builder withUnknownLeader(int epoch) throws IOException { + quorumStateStore.writeElectionState(ElectionState.withUnknownLeader(epoch, voters)); + return this; + } + + Builder withVotedCandidate(int epoch, int votedId) throws IOException { + quorumStateStore.writeElectionState(ElectionState.withVotedCandidate(epoch, votedId, voters)); + return this; + } + + Builder updateRandom(Consumer<Random> consumer) { + consumer.accept(random); + return this; + } + + Builder updateLog(Consumer<MockLog> consumer) { + consumer.accept(log); + return this; + } + + RaftClientTestContext build() throws IOException { + MockTime time = new MockTime(); + Metrics metrics = new Metrics(time); + MockNetworkChannel channel = new MockNetworkChannel(); + LogContext logContext = new LogContext(); + QuorumState quorum = new QuorumState(LOCAL_ID, voters, ELECTION_TIMEOUT_MS, FETCH_TIMEOUT_MS, + quorumStateStore, time, logContext, random); + + Map<Integer, InetSocketAddress> voterAddresses = voters.stream().collect(Collectors.toMap( + Function.identity(), + RaftClientTestContext::mockAddress + )); Review comment: nit: this indentation looks kind of funky ########## File path: raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java ########## @@ -90,470 +76,480 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class KafkaRaftClientTest { - private static final TopicPartition METADATA_PARTITION = new TopicPartition("metadata", 0); - - private final int localId = 0; - private final int electionTimeoutMs = 10000; - private final int electionBackoffMaxMs = 100; - private final int fetchTimeoutMs = 50000; // fetch timeout is usually larger than election timeout - private final int retryBackoffMs = 50; - private final int requestTimeoutMs = 5000; - private final int fetchMaxWaitMs = 0; - - private final MockTime time = new MockTime(); - private final MockLog log = new MockLog(METADATA_PARTITION); - private final MockNetworkChannel channel = new MockNetworkChannel(); - private final Random random = Mockito.spy(new Random(1)); - private final QuorumStateStore quorumStateStore = new MockQuorumStateStore(); - - @AfterEach - public void cleanUp() throws IOException { - quorumStateStore.clear(); - } - - private InetSocketAddress mockAddress(int id) { - return new InetSocketAddress("localhost", 9990 + id); - } - - private KafkaRaftClient buildClient(Set<Integer> voters) throws IOException { - return buildClient(voters, new Metrics(time)); - } - - private KafkaRaftClient buildClient(Set<Integer> voters, Metrics metrics) throws IOException { - LogContext logContext = new LogContext(); - QuorumState quorum = new QuorumState(localId, voters, electionTimeoutMs, fetchTimeoutMs, - quorumStateStore, time, logContext, random); - - Map<Integer, InetSocketAddress> voterAddresses = voters.stream().collect(Collectors.toMap( - Function.identity(), - this::mockAddress - )); - - KafkaRaftClient client = new KafkaRaftClient(channel, log, quorum, time, metrics, - new MockFuturePurgatory<>(time), new MockFuturePurgatory<>(time), voterAddresses, - electionBackoffMaxMs, retryBackoffMs, requestTimeoutMs, fetchMaxWaitMs, logContext, random); - - client.initialize(); - - return client; - } - @Test public void testInitializeSingleMemberQuorum() throws IOException { - buildClient(Collections.singleton(localId)); - assertEquals(ElectionState.withElectedLeader(1, localId, Collections.singleton(localId)), - quorumStateStore.readElectionState()); + RaftClientTestContext context = RaftClientTestContext.build(Collections.singleton(LOCAL_ID)); + assertEquals( + ElectionState.withElectedLeader(1, LOCAL_ID, Collections.singleton(LOCAL_ID)), + context.quorumStateStore.readElectionState() + ); } @Test public void testInitializeAsLeaderFromStateStoreSingleMemberQuorum() throws Exception { // Start off as leader. We should still bump the epoch after initialization int initialEpoch = 2; - Set<Integer> voters = Collections.singleton(localId); - quorumStateStore.writeElectionState(ElectionState.withElectedLeader(initialEpoch, localId, voters)); - - KafkaRaftClient client = buildClient(voters); - assertEquals(1L, log.endOffset().offset); - assertEquals(initialEpoch + 1, log.lastFetchedEpoch()); - assertEquals(new LeaderAndEpoch(OptionalInt.of(localId), initialEpoch + 1), - client.currentLeaderAndEpoch()); - assertEquals(ElectionState.withElectedLeader(initialEpoch + 1, localId, voters), - quorumStateStore.readElectionState()); + Set<Integer> voters = Collections.singleton(LOCAL_ID); + RaftClientTestContext context = new RaftClientTestContext.Builder() + .updateQuorumStateStore(quorumStateStore -> { + assertDoesNotThrow(() -> { + quorumStateStore.writeElectionState( + ElectionState.withElectedLeader(initialEpoch, LOCAL_ID, voters) + ); + }); + }) + .build(voters); + + assertEquals(1L, context.log.endOffset().offset); + assertEquals(initialEpoch + 1, context.log.lastFetchedEpoch()); + assertEquals(new LeaderAndEpoch(OptionalInt.of(LOCAL_ID), initialEpoch + 1), + context.client.currentLeaderAndEpoch()); + assertEquals(ElectionState.withElectedLeader(initialEpoch + 1, LOCAL_ID, voters), + context.quorumStateStore.readElectionState()); } @Test public void testInitializeAsLeaderFromStateStore() throws Exception { - Set<Integer> voters = Utils.mkSet(localId, 1); + Set<Integer> voters = Utils.mkSet(LOCAL_ID, 1); int epoch = 2; - Mockito.doReturn(0).when(random).nextInt(electionTimeoutMs); - quorumStateStore.writeElectionState(ElectionState.withElectedLeader(epoch, localId, voters)); - KafkaRaftClient client = buildClient(voters); - assertEquals(0L, log.endOffset().offset); - assertEquals(ElectionState.withUnknownLeader(epoch, voters), quorumStateStore.readElectionState()); + RaftClientTestContext context = new RaftClientTestContext.Builder() + .updateRandom(random -> { + Mockito.doReturn(0).when(random).nextInt(RaftClientTestContext.ELECTION_TIMEOUT_MS); + }) + .updateQuorumStateStore(quorumStateStore -> { + assertDoesNotThrow(() -> { + quorumStateStore.writeElectionState(ElectionState.withElectedLeader(epoch, LOCAL_ID, voters)); + }); + }) + .build(voters); + - time.sleep(electionTimeoutMs); - pollUntilSend(client); - assertSentVoteRequest(epoch + 1, 0, 0L); + assertEquals(0L, context.log.endOffset().offset); + assertEquals(ElectionState.withUnknownLeader(epoch, voters), context.quorumStateStore.readElectionState()); + + context.time.sleep(RaftClientTestContext.ELECTION_TIMEOUT_MS); + context.pollUntilSend(); + context.assertSentVoteRequest(epoch + 1, 0, 0L); } @Test public void testInitializeAsCandidateFromStateStore() throws Exception { // Need 3 node to require a 2-node majority - Set<Integer> voters = Utils.mkSet(localId, 1, 2); - quorumStateStore.writeElectionState(ElectionState.withVotedCandidate(2, localId, voters)); + Set<Integer> voters = Utils.mkSet(LOCAL_ID, 1, 2); + + RaftClientTestContext context = new RaftClientTestContext.Builder() + .updateQuorumStateStore(quorumStateStore -> { + assertDoesNotThrow(() -> { + quorumStateStore.writeElectionState(ElectionState.withVotedCandidate(2, LOCAL_ID, voters)); + }); + }) + .build(voters); - KafkaRaftClient client = buildClient(voters); - assertEquals(0L, log.endOffset().offset); + assertEquals(0L, context.log.endOffset().offset); // Send out vote requests. - client.poll(); + context.client.poll(); - List<RaftRequest.Outbound> voteRequests = collectVoteRequests(2, 0, 0); + List<RaftRequest.Outbound> voteRequests = context.collectVoteRequests(2, 0, 0); assertEquals(2, voteRequests.size()); } @Test public void testInitializeAsCandidateAndBecomeLeader() throws Exception { final int otherNodeId = 1; - Set<Integer> voters = Utils.mkSet(localId, otherNodeId); - KafkaRaftClient client = buildClient(voters); + Set<Integer> voters = Utils.mkSet(LOCAL_ID, otherNodeId); + RaftClientTestContext context = RaftClientTestContext.build(voters); - assertEquals(ElectionState.withUnknownLeader(0, voters), quorumStateStore.readElectionState()); - time.sleep(2 * electionTimeoutMs); + assertEquals(ElectionState.withUnknownLeader(0, voters), context.quorumStateStore.readElectionState()); + context.time.sleep(2 * RaftClientTestContext.ELECTION_TIMEOUT_MS); - pollUntilSend(client); - assertEquals(ElectionState.withVotedCandidate(1, localId, voters), quorumStateStore.readElectionState()); + context.pollUntilSend(); + assertEquals(ElectionState.withVotedCandidate(1, LOCAL_ID, voters), context.quorumStateStore.readElectionState()); - int correlationId = assertSentVoteRequest(1, 0, 0L); - deliverResponse(correlationId, otherNodeId, voteResponse(true, Optional.empty(), 1)); + int correlationId = context.assertSentVoteRequest(1, 0, 0L); + context.deliverResponse(correlationId, otherNodeId, RaftClientTestContext.voteResponse(true, Optional.empty(), 1)); // Become leader after receiving the vote - client.poll(); - assertEquals(ElectionState.withElectedLeader(1, localId, voters), quorumStateStore.readElectionState()); - long electionTimestamp = time.milliseconds(); + context.client.poll(); + assertEquals(ElectionState.withElectedLeader(1, LOCAL_ID, voters), context.quorumStateStore.readElectionState()); + long electionTimestamp = context.time.milliseconds(); // Leader change record appended - assertEquals(1L, log.endOffset().offset); - assertEquals(1L, log.lastFlushedOffset()); + assertEquals(1L, context.log.endOffset().offset); + assertEquals(1L, context.log.lastFlushedOffset()); // Send BeginQuorumEpoch to voters - client.poll(); - assertSentBeginQuorumEpochRequest(1); + context.client.poll(); + context.assertSentBeginQuorumEpochRequest(1); - Records records = log.read(0, Isolation.UNCOMMITTED).records; + Records records = context.log.read(0, Isolation.UNCOMMITTED).records; RecordBatch batch = records.batches().iterator().next(); assertTrue(batch.isControlBatch()); Record record = batch.iterator().next(); assertEquals(electionTimestamp, record.timestamp()); - verifyLeaderChangeMessage(localId, Collections.singletonList(otherNodeId), - record.key(), record.value()); + RaftClientTestContext.verifyLeaderChangeMessage(LOCAL_ID, Collections.singletonList(otherNodeId), record.key(), record.value()); Review comment: Yeah, that's fair. It looks like the code current just includes all followers. I guess we need to carry over the voters into the `LeaderState` if we want to implement the description above. Let's open a separate sub-task for https://issues.apache.org/jira/browse/KAFKA-9876 and decide what we want to do there. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org