From 136f584c05c1d381544a95cfcde7adc43f6a7ae9 Mon Sep 17 00:00:00 2001 From: zhaohaidao Date: Mon, 5 Apr 2021 23:11:58 +0800 Subject: [PATCH 001/155] KAFKA-12615: Fix `Selector.clear()` javadoc typo (#10477) The second `clearCompletedSends()` reference should be `clearCompletedReceives()`. Reviewers: Ismael Juma Co-authored-by: Zhao Haiyuan --- .../src/main/java/org/apache/kafka/common/network/Selector.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 33f587f4809e9..38880f2038e24 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -825,7 +825,7 @@ public void clearCompletedSends() { * Clears all the results from the previous poll. This is invoked by Selector at the start of * a poll() when all the results from the previous poll are expected to have been handled. *

- * SocketServer uses {@link #clearCompletedSends()} and {@link #clearCompletedSends()} to + * SocketServer uses {@link #clearCompletedSends()} and {@link #clearCompletedReceives()} to * clear `completedSends` and `completedReceives` as soon as they are processed to avoid * holding onto large request/response buffers from multiple connections longer than necessary. * Clients rely on Selector invoking {@link #clear()} at the start of each poll() since memory usage From 161c5638b8b1b12fdac04fa56bbaeb270fa36f8d Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 5 Apr 2021 08:42:04 -0700 Subject: [PATCH 002/155] KAFKA-12614: Use Jenkinsfile for trunk and release branch builds (#10473) * Run all JDK/Scala version combinations for trunk/release branch builds. * Only retry failures in PR builds for now (we can remove this distinction if/when we report flaky failures as described in KAFKA-12216). * Disable concurrent builds * Send email to dev list on build failure * Use triple double quotes in `doValidation` since we use string interpolation for `SCALA_VERSION`. * Update release.py to output new `Unit/integration tests` Jenkins link Reviewers: Gwen Shapira , David Arthur --- Jenkinsfile | 136 +++++++++++++++++++++++++++++++++++++++++++++------- release.py | 2 +- 2 files changed, 121 insertions(+), 17 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index bc63364fc6a79..07db6e4b648ee 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -26,19 +26,26 @@ def setupGradle() { } def doValidation() { - sh ''' + sh """ ./gradlew -PscalaVersion=$SCALA_VERSION clean compileJava compileScala compileTestJava compileTestScala \ spotlessScalaCheck checkstyleMain checkstyleTest spotbugsMain rat \ --profile --no-daemon --continue -PxmlSpotBugsReport=true - ''' + """ } -def doTest(target = "unitTest integrationTest") { - sh """ - ./gradlew -PscalaVersion=$SCALA_VERSION ${target} \ - --profile --no-daemon --continue -PtestLoggingEvents=started,passed,skipped,failed \ - -PignoreFailures=true -PmaxParallelForks=2 -PmaxTestRetries=1 -PmaxTestRetryFailures=5 - """ +def isChangeRequest(env) { + env.CHANGE_ID != null && !env.CHANGE_ID.isEmpty() +} + +def retryFlagsString(env) { + if (isChangeRequest(env)) " -PmaxTestRetries=1 -PmaxTestRetryFailures=5" + else "" +} + +def doTest(env, target = "unitTest integrationTest") { + sh """./gradlew -PscalaVersion=$SCALA_VERSION ${target} \ + --profile --no-daemon --continue -PtestLoggingEvents=started,passed,skipped,failed \ + -PignoreFailures=true -PmaxParallelForks=2""" + retryFlagsString(env) junit '**/build/test-results/**/TEST-*.xml' } @@ -95,10 +102,16 @@ def tryStreamsArchetype() { pipeline { agent none + + options { + disableConcurrentBuilds() + } + stages { stage('Build') { parallel { - stage('JDK 8') { + + stage('JDK 8 and Scala 2.12') { agent { label 'ubuntu' } tools { jdk 'jdk_1.8_latest' @@ -114,12 +127,12 @@ pipeline { steps { setupGradle() doValidation() - doTest() + doTest(env) tryStreamsArchetype() } } - stage('JDK 11') { + stage('JDK 11 and Scala 2.13') { agent { label 'ubuntu' } tools { jdk 'jdk_11_latest' @@ -134,12 +147,12 @@ pipeline { steps { setupGradle() doValidation() - doTest() + doTest(env) echo 'Skipping Kafka Streams archetype test for Java 11' } } - - stage('JDK 15') { + + stage('JDK 15 and Scala 2.13') { agent { label 'ubuntu' } tools { jdk 'jdk_15_latest' @@ -154,7 +167,7 @@ pipeline { steps { setupGradle() doValidation() - doTest() + doTest(env) echo 'Skipping Kafka Streams archetype test for Java 15' } } @@ -172,11 +185,102 @@ pipeline { setupGradle() doValidation() catchError(buildResult: 'SUCCESS', stageResult: 'FAILURE') { - doTest('unitTest') + doTest(env, 'unitTest') } echo 'Skipping Kafka Streams archetype test for ARM build' } } + + // To avoid excessive Jenkins resource usage, we only run the stages + // above at the PR stage. The ones below are executed after changes + // are pushed to trunk and/or release branches. We achieve this via + // the `when` clause. + + stage('JDK 8 and Scala 2.13') { + when { + not { changeRequest() } + beforeAgent true + } + agent { label 'ubuntu' } + tools { + jdk 'jdk_1.8_latest' + maven 'maven_3_latest' + } + options { + timeout(time: 8, unit: 'HOURS') + timestamps() + } + environment { + SCALA_VERSION=2.13 + } + steps { + setupGradle() + doValidation() + doTest(env) + tryStreamsArchetype() + } + } + + stage('JDK 11 and Scala 2.12') { + when { + not { changeRequest() } + beforeAgent true + } + agent { label 'ubuntu' } + tools { + jdk 'jdk_11_latest' + } + options { + timeout(time: 8, unit: 'HOURS') + timestamps() + } + environment { + SCALA_VERSION=2.12 + } + steps { + setupGradle() + doValidation() + doTest(env) + echo 'Skipping Kafka Streams archetype test for Java 11' + } + } + + stage('JDK 15 and Scala 2.12') { + when { + not { changeRequest() } + beforeAgent true + } + agent { label 'ubuntu' } + tools { + jdk 'jdk_15_latest' + } + options { + timeout(time: 8, unit: 'HOURS') + timestamps() + } + environment { + SCALA_VERSION=2.12 + } + steps { + setupGradle() + doValidation() + doTest(env) + echo 'Skipping Kafka Streams archetype test for Java 15' + } + } + } + } + } + + post { + always { + script { + if (!isChangeRequest(env)) { + step([$class: 'Mailer', + notifyEveryUnstableBuild: true, + recipients: "dev@kafka.apache.org", + sendToIndividuals: false]) + } } } } diff --git a/release.py b/release.py index 5f2cede49763a..751e375f7a984 100755 --- a/release.py +++ b/release.py @@ -731,7 +731,7 @@ def select_gpg_key(): https://kafka.apache.org/%(docs_version)s/protocol.html * Successful Jenkins builds for the %(dev_branch)s branch: -Unit/integration tests: https://builds.apache.org/job/kafka-%(dev_branch)s-jdk8// +Unit/integration tests: https://ci-builds.apache.org/job/Kafka/job/kafka/job/%(dev_branch)s// System tests: https://jenkins.confluent.io/job/system-test-kafka/job/%(dev_branch)s// /************************************** From 4f47a565e29539d1c252c36c59f5f24d105cec4b Mon Sep 17 00:00:00 2001 From: dengziming Date: Tue, 6 Apr 2021 00:27:50 +0800 Subject: [PATCH 003/155] KAFKA-12539; Refactor KafkaRaftCllient handleVoteRequest to reduce cyclomatic complexity (#10393) 1. Add `canGrantVote` to `EpochState` 2. Move the if-else in `KafkaRaftCllient.handleVoteRequest` to `EpochState` 3. Add unit tests for `canGrantVote` Reviewers: Jason Gustafson --- checkstyle/suppressions.xml | 2 +- .../org/apache/kafka/raft/CandidateState.java | 16 +++- .../org/apache/kafka/raft/EpochState.java | 11 +++ .../org/apache/kafka/raft/FollowerState.java | 15 +++- .../apache/kafka/raft/KafkaRaftClient.java | 40 ++-------- .../org/apache/kafka/raft/LeaderState.java | 7 ++ .../org/apache/kafka/raft/QuorumState.java | 37 ++++++--- .../org/apache/kafka/raft/ResignedState.java | 14 +++- .../apache/kafka/raft/UnattachedState.java | 16 +++- .../org/apache/kafka/raft/VotedState.java | 18 ++++- .../apache/kafka/raft/CandidateStateTest.java | 76 +++++++++++++------ .../apache/kafka/raft/FollowerStateTest.java | 58 ++++++++------ .../apache/kafka/raft/LeaderStateTest.java | 13 ++++ .../apache/kafka/raft/ResignedStateTest.java | 45 ++++++++--- .../kafka/raft/UnattachedStateTest.java | 43 ++++++++--- .../org/apache/kafka/raft/VotedStateTest.java | 44 ++++++++--- 16 files changed, 330 insertions(+), 125 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 0496d2fca46cf..5473b035ea919 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -68,7 +68,7 @@ files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/> + files="(ConsumerCoordinator|Fetcher|Sender|KafkaProducer|BufferPool|ConfigDef|RecordAccumulator|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer).java"/> diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 08f69067995b2..001fc19c11358 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; import java.util.HashMap; import java.util.Map; @@ -34,6 +36,7 @@ public class CandidateState implements EpochState { private final int electionTimeoutMs; private final Timer electionTimer; private final Timer backoffTimer; + private final Logger log; /** * The life time of a candidate state is the following: @@ -52,7 +55,8 @@ protected CandidateState( Set voters, Optional highWatermark, int retries, - int electionTimeoutMs + int electionTimeoutMs, + LogContext logContext ) { this.localId = localId; this.epoch = epoch; @@ -62,6 +66,7 @@ protected CandidateState( this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); this.backoffTimer = time.timer(0); + this.log = logContext.logger(CandidateState.class); for (Integer voterId : voters) { voteStates.put(voterId, State.UNRECORDED); @@ -235,6 +240,15 @@ public Optional highWatermark() { return highWatermark; } + @Override + public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { + // Still reject vote request even candidateId = localId, Although the candidate votes for + // itself, this vote is implicit and not "granted". + log.debug("Rejecting vote request from candidate {} since we are already candidate in epoch {}", + candidateId, epoch); + return false; + } + @Override public String toString() { return "Candidate(" + diff --git a/raft/src/main/java/org/apache/kafka/raft/EpochState.java b/raft/src/main/java/org/apache/kafka/raft/EpochState.java index b32a200c5f77d..89e8f0ac235b8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/EpochState.java +++ b/raft/src/main/java/org/apache/kafka/raft/EpochState.java @@ -25,6 +25,17 @@ default Optional highWatermark() { return Optional.empty(); } + /** + * Decide whether to grant a vote to a candidate, it is the responsibility of the caller to invoke + * {@link QuorumState##transitionToVoted(int, int)} if vote is granted. + * + * @param candidateId The ID of the voter who attempt to become leader + * @param isLogUpToDate Whether the candidate’s log is at least as up-to-date as receiver’s log, it + * is the responsibility of the caller to compare the log in advance + * @return true If grant vote. + */ + boolean canGrantVote(int candidateId, boolean isLogUpToDate); + /** * Get the current election state, which is guaranteed to be immutable. */ diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index e1ef7aa6d56be..8bfad3abc71af 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -16,9 +16,11 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; import org.apache.kafka.snapshot.RawSnapshotWriter; +import org.slf4j.Logger; import java.io.IOException; import java.util.Optional; @@ -39,13 +41,16 @@ public class FollowerState implements EpochState { */ private Optional fetchingSnapshot; + private final Logger log; + public FollowerState( Time time, int epoch, int leaderId, Set voters, Optional highWatermark, - int fetchTimeoutMs + int fetchTimeoutMs, + LogContext logContext ) { this.fetchTimeoutMs = fetchTimeoutMs; this.epoch = epoch; @@ -54,6 +59,7 @@ public FollowerState( this.fetchTimer = time.timer(fetchTimeoutMs); this.highWatermark = highWatermark; this.fetchingSnapshot = Optional.empty(); + this.log = logContext.logger(FollowerState.class); } @Override @@ -139,6 +145,13 @@ public void setFetchingSnapshot(Optional fetchingSnapshot) th this.fetchingSnapshot = fetchingSnapshot; } + @Override + public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { + log.debug("Rejecting vote request from candidate {} since we already have a leader {} in epoch {}", + candidateId, leaderId(), epoch); + return false; + } + @Override public String toString() { return "FollowerState(" + diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 2cde196b0315e..a3dbbdd3a66fd 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -592,44 +592,14 @@ private VoteResponseData handleVoteRequest( transitionToUnattached(candidateEpoch); } - final boolean voteGranted; - if (quorum.isLeader()) { - logger.debug("Rejecting vote request {} with epoch {} since we are already leader on that epoch", - request, candidateEpoch); - voteGranted = false; - } else if (quorum.isCandidate()) { - logger.debug("Rejecting vote request {} with epoch {} since we are already candidate on that epoch", - request, candidateEpoch); - voteGranted = false; - } else if (quorum.isResigned()) { - logger.debug("Rejecting vote request {} with epoch {} since we have resigned as candidate/leader in this epoch", - request, candidateEpoch); - voteGranted = false; - } else if (quorum.isFollower()) { - FollowerState state = quorum.followerStateOrThrow(); - logger.debug("Rejecting vote request {} with epoch {} since we already have a leader {} on that epoch", - request, candidateEpoch, state.leaderId()); - voteGranted = false; - } else if (quorum.isVoted()) { - VotedState state = quorum.votedStateOrThrow(); - voteGranted = state.votedId() == candidateId; + OffsetAndEpoch lastEpochEndOffsetAndEpoch = new OffsetAndEpoch(lastEpochEndOffset, lastEpoch); + boolean voteGranted = quorum.canGrantVote(candidateId, lastEpochEndOffsetAndEpoch.compareTo(endOffset()) >= 0); - if (!voteGranted) { - logger.debug("Rejecting vote request {} with epoch {} since we already have voted for " + - "another candidate {} on that epoch", request, candidateEpoch, state.votedId()); - } - } else if (quorum.isUnattached()) { - OffsetAndEpoch lastEpochEndOffsetAndEpoch = new OffsetAndEpoch(lastEpochEndOffset, lastEpoch); - voteGranted = lastEpochEndOffsetAndEpoch.compareTo(endOffset()) >= 0; - - if (voteGranted) { - transitionToVoted(candidateId, candidateEpoch); - } - } else { - throw new IllegalStateException("Unexpected quorum state " + quorum); + if (voteGranted && quorum.isUnattached()) { + transitionToVoted(candidateId, candidateEpoch); } - logger.info("Vote request {} is {}", request, voteGranted ? "granted" : "rejected"); + logger.info("Vote request {} with epoch {} is {}", request, candidateEpoch, voteGranted ? "granted" : "rejected"); return buildVoteResponse(Errors.NONE, voteGranted); } diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index d939aacbb1436..95c5628bf09f4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -304,6 +304,13 @@ public String toString() { } } + @Override + public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { + log.debug("Rejecting vote request from candidate {} since we are already leader in epoch {}", + candidateId, epoch); + return false; + } + @Override public String toString() { return "Leader(" + diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 6216d0f5847e2..86f8c187d8f35 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -146,7 +146,8 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IOException, logEndOffsetAndEpoch.epoch, voters, Optional.empty(), - randomElectionTimeoutMs() + randomElectionTimeoutMs(), + logContext ); } else if (localId.isPresent() && election.isLeader(localId.getAsInt())) { // If we were previously a leader, then we will start out as resigned @@ -161,7 +162,8 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IOException, election.epoch, voters, randomElectionTimeoutMs(), - Collections.emptyList() + Collections.emptyList(), + logContext ); } else if (localId.isPresent() && election.isVotedCandidate(localId.getAsInt())) { initialState = new CandidateState( @@ -171,7 +173,8 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IOException, voters, Optional.empty(), 1, - randomElectionTimeoutMs() + randomElectionTimeoutMs(), + logContext ); } else if (election.hasVoted()) { initialState = new VotedState( @@ -180,7 +183,8 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IOException, election.votedId(), voters, Optional.empty(), - randomElectionTimeoutMs() + randomElectionTimeoutMs(), + logContext ); } else if (election.hasLeader()) { initialState = new FollowerState( @@ -189,7 +193,8 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IOException, election.leaderId(), voters, Optional.empty(), - fetchTimeoutMs + fetchTimeoutMs, + logContext ); } else { initialState = new UnattachedState( @@ -197,7 +202,8 @@ public void initialize(OffsetAndEpoch logEndOffsetAndEpoch) throws IOException, election.epoch, voters, Optional.empty(), - randomElectionTimeoutMs() + randomElectionTimeoutMs(), + logContext ); } @@ -271,7 +277,8 @@ public void transitionToResigned(List preferredSuccessors) { epoch, voters, randomElectionTimeoutMs(), - preferredSuccessors + preferredSuccessors, + logContext ); log.info("Completed transition to {}", state); } @@ -305,7 +312,8 @@ public void transitionToUnattached(int epoch) throws IOException { epoch, voters, state.highWatermark(), - electionTimeoutMs + electionTimeoutMs, + logContext )); } @@ -348,7 +356,8 @@ public void transitionToVoted( candidateId, voters, state.highWatermark(), - randomElectionTimeoutMs() + randomElectionTimeoutMs(), + logContext )); } @@ -383,7 +392,8 @@ public void transitionToFollower( leaderId, voters, state.highWatermark(), - fetchTimeoutMs + fetchTimeoutMs, + logContext )); } @@ -407,7 +417,8 @@ public void transitionToCandidate() throws IOException { voters, state.highWatermark(), retries, - electionTimeoutMs + electionTimeoutMs, + logContext )); } @@ -460,6 +471,10 @@ private int randomElectionTimeoutMs() { return electionTimeoutMs + random.nextInt(electionTimeoutMs); } + public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { + return state.canGrantVote(candidateId, isLogUpToDate); + } + public FollowerState followerStateOrThrow() { if (isFollower()) return (FollowerState) state; diff --git a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java index c1608aa9fc99e..899823ac631f3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ResignedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/ResignedState.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; import java.util.HashSet; import java.util.List; @@ -45,6 +47,7 @@ public class ResignedState implements EpochState { private final Set unackedVoters; private final Timer electionTimer; private final List preferredSuccessors; + private final Logger log; public ResignedState( Time time, @@ -52,7 +55,8 @@ public ResignedState( int epoch, Set voters, long electionTimeoutMs, - List preferredSuccessors + List preferredSuccessors, + LogContext logContext ) { this.localId = localId; this.epoch = epoch; @@ -62,6 +66,7 @@ public ResignedState( this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); this.preferredSuccessors = preferredSuccessors; + this.log = logContext.logger(ResignedState.class); } @Override @@ -125,6 +130,13 @@ public List preferredSuccessors() { return preferredSuccessors; } + @Override + public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { + log.debug("Rejecting vote request from candidate {} since we have resigned as candidate/leader in epoch {}", + candidateId, epoch); + return false; + } + @Override public String name() { return "Resigned"; diff --git a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java index 62b82e232f17a..4dc5fc796de65 100644 --- a/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/UnattachedState.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; import java.util.Optional; import java.util.OptionalInt; @@ -34,19 +36,22 @@ public class UnattachedState implements EpochState { private final long electionTimeoutMs; private final Timer electionTimer; private final Optional highWatermark; + private final Logger log; public UnattachedState( Time time, int epoch, Set voters, Optional highWatermark, - long electionTimeoutMs + long electionTimeoutMs, + LogContext logContext ) { this.epoch = epoch; this.voters = voters; this.highWatermark = highWatermark; this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); + this.log = logContext.logger(UnattachedState.class); } @Override @@ -88,6 +93,15 @@ public Optional highWatermark() { return highWatermark; } + @Override + public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { + if (!isLogUpToDate) { + log.debug("Rejecting vote request from candidate {} since candidate epoch/offset is not up to date with us", + candidateId); + } + return isLogUpToDate; + } + @Override public String toString() { return "Unattached(" + diff --git a/raft/src/main/java/org/apache/kafka/raft/VotedState.java b/raft/src/main/java/org/apache/kafka/raft/VotedState.java index 4138176b7c305..2ae5026f52b27 100644 --- a/raft/src/main/java/org/apache/kafka/raft/VotedState.java +++ b/raft/src/main/java/org/apache/kafka/raft/VotedState.java @@ -16,8 +16,10 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.slf4j.Logger; import java.util.Optional; import java.util.OptionalInt; @@ -36,6 +38,7 @@ public class VotedState implements EpochState { private final int electionTimeoutMs; private final Timer electionTimer; private final Optional highWatermark; + private final Logger log; public VotedState( Time time, @@ -43,7 +46,8 @@ public VotedState( int votedId, Set voters, Optional highWatermark, - int electionTimeoutMs + int electionTimeoutMs, + LogContext logContext ) { this.epoch = epoch; this.votedId = votedId; @@ -51,6 +55,7 @@ public VotedState( this.highWatermark = highWatermark; this.electionTimeoutMs = electionTimeoutMs; this.electionTimer = time.timer(electionTimeoutMs); + this.log = logContext.logger(VotedState.class); } @Override @@ -92,6 +97,17 @@ public void overrideElectionTimeout(long currentTimeMs, long timeoutMs) { electionTimer.reset(timeoutMs); } + @Override + public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { + if (votedId() == candidateId) { + return true; + } + + log.debug("Rejecting vote request from candidate {} since we already have voted for " + + "another candidate {} in epoch {}", candidateId, votedId(), epoch); + return false; + } + @Override public Optional highWatermark() { return highWatermark; diff --git a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java index 8c4633baa8271..71a2375408e1b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/CandidateStateTest.java @@ -16,12 +16,16 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; 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.ValueSource; import java.util.Collections; import java.util.Optional; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -33,11 +37,27 @@ public class CandidateStateTest { private final int epoch = 5; private final MockTime time = new MockTime(); private final int electionTimeoutMs = 5000; + private final LogContext logContext = new LogContext(); + + private CandidateState newCandidateState( + Set voters, + Optional highWatermark + ) { + return new CandidateState( + time, + localId, + epoch, + voters, + highWatermark, + 0, + electionTimeoutMs, + logContext + ); + } @Test public void testSingleNodeQuorum() { - CandidateState state = new CandidateState(time, localId, epoch, - Collections.singleton(localId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState(Collections.singleton(localId), Optional.empty()); assertTrue(state.isVoteGranted()); assertFalse(state.isVoteRejected()); assertEquals(Collections.emptySet(), state.unrecordedVoters()); @@ -46,8 +66,7 @@ public void testSingleNodeQuorum() { @Test public void testTwoNodeQuorumVoteRejected() { int otherNodeId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, otherNodeId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState(Utils.mkSet(localId, otherNodeId), Optional.empty()); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); assertEquals(Collections.singleton(otherNodeId), state.unrecordedVoters()); @@ -59,8 +78,8 @@ public void testTwoNodeQuorumVoteRejected() { @Test public void testTwoNodeQuorumVoteGranted() { int otherNodeId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, otherNodeId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, otherNodeId), Optional.empty()); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); assertEquals(Collections.singleton(otherNodeId), state.unrecordedVoters()); @@ -74,8 +93,8 @@ public void testTwoNodeQuorumVoteGranted() { public void testThreeNodeQuorumVoteGranted() { int node1 = 1; int node2 = 2; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, node1, node2), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, node1, node2), Optional.empty()); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); assertEquals(Utils.mkSet(node1, node2), state.unrecordedVoters()); @@ -93,8 +112,8 @@ public void testThreeNodeQuorumVoteGranted() { public void testThreeNodeQuorumVoteRejected() { int node1 = 1; int node2 = 2; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, node1, node2), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, node1, node2), Optional.empty()); assertFalse(state.isVoteGranted()); assertFalse(state.isVoteRejected()); assertEquals(Utils.mkSet(node1, node2), state.unrecordedVoters()); @@ -111,16 +130,16 @@ public void testThreeNodeQuorumVoteRejected() { @Test public void testCannotRejectVoteFromLocalId() { int otherNodeId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, otherNodeId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, otherNodeId), Optional.empty()); assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(localId)); } @Test public void testCannotChangeVoteGrantedToRejected() { int otherNodeId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, otherNodeId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, otherNodeId), Optional.empty()); assertTrue(state.recordGrantedVote(otherNodeId)); assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(otherNodeId)); assertTrue(state.isVoteGranted()); @@ -129,8 +148,8 @@ public void testCannotChangeVoteGrantedToRejected() { @Test public void testCannotChangeVoteRejectedToGranted() { int otherNodeId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, otherNodeId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, otherNodeId), Optional.empty()); assertTrue(state.recordRejectedVote(otherNodeId)); assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(otherNodeId)); assertTrue(state.isVoteRejected()); @@ -139,8 +158,8 @@ public void testCannotChangeVoteRejectedToGranted() { @Test public void testCannotGrantOrRejectNonVoters() { int nonVoterId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Collections.singleton(localId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Collections.singleton(localId), Optional.empty()); assertThrows(IllegalArgumentException.class, () -> state.recordGrantedVote(nonVoterId)); assertThrows(IllegalArgumentException.class, () -> state.recordRejectedVote(nonVoterId)); } @@ -148,8 +167,8 @@ public void testCannotGrantOrRejectNonVoters() { @Test public void testIdempotentGrant() { int otherNodeId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, otherNodeId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, otherNodeId), Optional.empty()); assertTrue(state.recordGrantedVote(otherNodeId)); assertFalse(state.recordGrantedVote(otherNodeId)); } @@ -157,10 +176,23 @@ public void testIdempotentGrant() { @Test public void testIdempotentReject() { int otherNodeId = 1; - CandidateState state = new CandidateState(time, localId, epoch, - Utils.mkSet(localId, otherNodeId), Optional.empty(), 0, electionTimeoutMs); + CandidateState state = newCandidateState( + Utils.mkSet(localId, otherNodeId), Optional.empty()); assertTrue(state.recordRejectedVote(otherNodeId)); assertFalse(state.recordRejectedVote(otherNodeId)); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantVote(boolean isLogUpToDate) { + CandidateState state = newCandidateState( + Utils.mkSet(1, 2, 3), + Optional.empty() + ); + + assertFalse(state.canGrantVote(1, isLogUpToDate)); + assertFalse(state.canGrantVote(2, isLogUpToDate)); + assertFalse(state.canGrantVote(3, isLogUpToDate)); + } + } diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index 2965f059e4561..42c6bc9ba7c74 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -16,12 +16,16 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; 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.ValueSource; import java.util.Optional; import java.util.OptionalLong; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -30,21 +34,29 @@ public class FollowerStateTest { private final MockTime time = new MockTime(); + private final LogContext logContext = new LogContext(); + private final int epoch = 5; + private final int fetchTimeoutMs = 15000; + int leaderId = 3; - @Test - public void testFetchTimeoutExpiration() { - int epoch = 5; - int leaderId = 3; - int fetchTimeoutMs = 15000; - - FollowerState state = new FollowerState( + private FollowerState newFollowerState( + Set voters, + Optional highWatermark + ) { + return new FollowerState( time, epoch, leaderId, - Utils.mkSet(1, 2, 3), - Optional.empty(), - fetchTimeoutMs + voters, + highWatermark, + fetchTimeoutMs, + logContext ); + } + + @Test + public void testFetchTimeoutExpiration() { + FollowerState state = newFollowerState(Utils.mkSet(1, 2, 3), Optional.empty()); assertFalse(state.hasFetchTimeoutExpired(time.milliseconds())); assertEquals(fetchTimeoutMs, state.remainingFetchTimeMs(time.milliseconds())); @@ -60,18 +72,7 @@ public void testFetchTimeoutExpiration() { @Test public void testMonotonicHighWatermark() { - int epoch = 5; - int leaderId = 3; - int fetchTimeoutMs = 15000; - - FollowerState state = new FollowerState( - time, - epoch, - leaderId, - Utils.mkSet(1, 2, 3), - Optional.empty(), - fetchTimeoutMs - ); + FollowerState state = newFollowerState(Utils.mkSet(1, 2, 3), Optional.empty()); OptionalLong highWatermark = OptionalLong.of(15L); state.updateHighWatermark(highWatermark); @@ -81,4 +82,17 @@ public void testMonotonicHighWatermark() { assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantVote(boolean isLogUpToDate) { + FollowerState state = newFollowerState( + Utils.mkSet(1, 2, 3), + Optional.empty() + ); + + assertFalse(state.canGrantVote(1, isLogUpToDate)); + assertFalse(state.canGrantVote(2, isLogUpToDate)); + assertFalse(state.canGrantVote(3, isLogUpToDate)); + } + } diff --git a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java index cdadf6e00123a..c7490b5a913e9 100644 --- a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java @@ -18,7 +18,10 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +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.ValueSource; import java.util.Arrays; import java.util.Collections; @@ -254,6 +257,16 @@ public void testObserverStateExpiration() { assertEquals(emptySet(), state.getObserverStates(time.milliseconds()).keySet()); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantVote(boolean isLogUpToDate) { + LeaderState state = newLeaderState(Utils.mkSet(1, 2, 3), 1); + + assertFalse(state.canGrantVote(1, isLogUpToDate)); + assertFalse(state.canGrantVote(2, isLogUpToDate)); + assertFalse(state.canGrantVote(3, isLogUpToDate)); + } + private static class MockOffsetMetadata implements OffsetMetadata { private final String value; diff --git a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java index 20e2e8493d3d7..770297bf04672 100644 --- a/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/ResignedStateTest.java @@ -16,11 +16,15 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; 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.ValueSource; import java.util.Collections; +import java.util.List; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -30,23 +34,32 @@ class ResignedStateTest { private final MockTime time = new MockTime(); + private final LogContext logContext = new LogContext(); + int electionTimeoutMs = 5000; + int localId = 0; + int epoch = 5; - @Test - public void testResignedState() { - int electionTimeoutMs = 5000; - int localId = 0; - int remoteId = 1; - int epoch = 5; - Set voters = Utils.mkSet(localId, remoteId); - - ResignedState state = new ResignedState( + private ResignedState newResignedState( + Set voters, + List preferredSuccessors + ) { + return new ResignedState( time, localId, epoch, voters, electionTimeoutMs, - Collections.emptyList() + preferredSuccessors, + logContext ); + } + + @Test + public void testResignedState() { + int remoteId = 1; + Set voters = Utils.mkSet(localId, remoteId); + + ResignedState state = newResignedState(voters, Collections.emptyList()); assertEquals(ElectionState.withElectedLeader(epoch, localId, voters), state.election()); assertEquals(epoch, state.epoch()); @@ -65,4 +78,16 @@ public void testResignedState() { assertTrue(state.hasElectionTimeoutExpired(time.milliseconds())); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantVote(boolean isLogUpToDate) { + ResignedState state = newResignedState( + Utils.mkSet(1, 2, 3), + Collections.emptyList() + ); + + assertFalse(state.canGrantVote(1, isLogUpToDate)); + assertFalse(state.canGrantVote(2, isLogUpToDate)); + assertFalse(state.canGrantVote(3, isLogUpToDate)); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java index bd00baaf22b3e..96f2a520dac71 100644 --- a/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/UnattachedStateTest.java @@ -16,9 +16,12 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; 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.ValueSource; import java.util.Optional; import java.util.Set; @@ -30,19 +33,28 @@ public class UnattachedStateTest { private final MockTime time = new MockTime(); + private final LogContext logContext = new LogContext(); + private final int epoch = 5; + private final int electionTimeoutMs = 10000; + + private UnattachedState newUnattachedState(Set voters, Optional highWatermark) { + return new UnattachedState( + time, + epoch, + voters, + highWatermark, + electionTimeoutMs, + logContext + ); + } @Test public void testElectionTimeout() { Set voters = Utils.mkSet(1, 2, 3); - int epoch = 1; - int electionTimeoutMs = 10000; - - UnattachedState state = new UnattachedState( - time, - epoch, - voters, - Optional.empty(), - electionTimeoutMs + + UnattachedState state = newUnattachedState( + voters, + Optional.empty() ); assertEquals(epoch, state.epoch()); @@ -59,4 +71,17 @@ public void testElectionTimeout() { assertEquals(0, state.remainingElectionTimeMs(time.milliseconds())); assertTrue(state.hasElectionTimeoutExpired(time.milliseconds())); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantVote(boolean isLogUpToDate) { + UnattachedState state = newUnattachedState( + Utils.mkSet(1, 2, 3), + Optional.empty() + ); + + assertEquals(isLogUpToDate, state.canGrantVote(1, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(2, isLogUpToDate)); + assertEquals(isLogUpToDate, state.canGrantVote(3, isLogUpToDate)); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java b/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java index 843855bbcb73f..317b80f8e0ddb 100644 --- a/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/VotedStateTest.java @@ -16,9 +16,12 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; 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.ValueSource; import java.util.Optional; import java.util.Set; @@ -30,22 +33,31 @@ class VotedStateTest { private final MockTime time = new MockTime(); + private final LogContext logContext = new LogContext(); + private final int epoch = 5; + private final int votedId = 1; + private final int electionTimeoutMs = 10000; - @Test - public void testElectionTimeout() { - Set voters = Utils.mkSet(1, 2, 3); - int epoch = 5; - int votedId = 1; - int electionTimeoutMs = 10000; - - VotedState state = new VotedState( + private VotedState newVotedState( + Set voters, + Optional highWatermark + ) { + return new VotedState( time, epoch, votedId, voters, - Optional.empty(), - electionTimeoutMs + highWatermark, + electionTimeoutMs, + logContext ); + } + + @Test + public void testElectionTimeout() { + Set voters = Utils.mkSet(1, 2, 3); + + VotedState state = newVotedState(voters, Optional.empty()); assertEquals(epoch, state.epoch()); assertEquals(votedId, state.votedId()); @@ -62,4 +74,16 @@ public void testElectionTimeout() { assertTrue(state.hasElectionTimeoutExpired(time.milliseconds())); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testGrantVote(boolean isLogUpToDate) { + VotedState state = newVotedState( + Utils.mkSet(1, 2, 3), + Optional.empty() + ); + + assertTrue(state.canGrantVote(1, isLogUpToDate)); + assertFalse(state.canGrantVote(2, isLogUpToDate)); + assertFalse(state.canGrantVote(3, isLogUpToDate)); + } } From 66ba91733c58ddc3e9465a5b1b234be0b2b06be4 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 5 Apr 2021 11:50:54 -0700 Subject: [PATCH 004/155] KAFKA-12548; Propagate record error messages to application (#10445) KIP-467 added a field in the produce response to allow the broker to indicate which specific records failed validation. This patch adds the logic to propagate this message up to the application. Reviewers: Guozhang Wang --- .../kafka/clients/producer/MockProducer.java | 7 +- .../internals/FutureRecordMetadata.java | 13 +-- .../internals/ProduceRequestResult.java | 18 ++-- .../producer/internals/ProducerBatch.java | 83 +++++++++++++---- .../clients/producer/internals/Sender.java | 93 ++++++++++++++----- .../common/message/ProduceRequest.json | 2 +- .../common/message/ProduceResponse.json | 6 +- .../clients/producer/RecordSendTest.java | 9 +- .../producer/internals/ProducerBatchTest.java | 75 ++++++++++++--- .../internals/RecordAccumulatorTest.java | 6 +- .../producer/internals/SenderTest.java | 91 ++++++++++++++++-- .../internals/TransactionManagerTest.java | 18 ++-- .../kafka/api/PlaintextProducerSendTest.scala | 2 - 13 files changed, 326 insertions(+), 97 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 6eefb897b8d20..650b69795d882 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -527,7 +527,12 @@ public Completion(long offset, } public void complete(RuntimeException e) { - result.set(e == null ? offset : -1L, RecordBatch.NO_TIMESTAMP, e); + if (e == null) { + result.set(offset, RecordBatch.NO_TIMESTAMP, null); + } else { + result.set(-1, RecordBatch.NO_TIMESTAMP, index -> e); + } + if (callback != null) { if (e == null) callback.onCompletion(metadata, null); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java index d1a643b319679..a9665a991bd3a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java @@ -30,7 +30,7 @@ public final class FutureRecordMetadata implements Future { private final ProduceRequestResult result; - private final long relativeOffset; + private final int batchIndex; private final long createTimestamp; private final Long checksum; private final int serializedKeySize; @@ -38,10 +38,10 @@ public final class FutureRecordMetadata implements Future { private final Time time; private volatile FutureRecordMetadata nextRecordMetadata = null; - public FutureRecordMetadata(ProduceRequestResult result, long relativeOffset, long createTimestamp, + public FutureRecordMetadata(ProduceRequestResult result, int batchIndex, long createTimestamp, Long checksum, int serializedKeySize, int serializedValueSize, Time time) { this.result = result; - this.relativeOffset = relativeOffset; + this.batchIndex = batchIndex; this.createTimestamp = createTimestamp; this.checksum = checksum; this.serializedKeySize = serializedKeySize; @@ -94,8 +94,9 @@ void chain(FutureRecordMetadata futureRecordMetadata) { } RecordMetadata valueOrError() throws ExecutionException { - if (this.result.error() != null) - throw new ExecutionException(this.result.error()); + RuntimeException exception = this.result.error(batchIndex); + if (exception != null) + throw new ExecutionException(exception); else return value(); } @@ -107,7 +108,7 @@ Long checksumOrNull() { RecordMetadata value() { if (nextRecordMetadata != null) return nextRecordMetadata.value(); - return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.relativeOffset, + return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.batchIndex, timestamp(), this.checksum, this.serializedKeySize, this.serializedValueSize); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java index 1e8c787963284..9077b107ab03e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java @@ -22,7 +22,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; - +import java.util.function.Function; /** * A class that models the future completion of a produce request for a single partition. There is one of these per @@ -36,7 +36,7 @@ public class ProduceRequestResult { private volatile Long baseOffset = null; private volatile long logAppendTime = RecordBatch.NO_TIMESTAMP; - private volatile RuntimeException error; + private volatile Function errorsByIndex; /** * Create an instance of this class. @@ -52,12 +52,12 @@ public ProduceRequestResult(TopicPartition topicPartition) { * * @param baseOffset The base offset assigned to the record * @param logAppendTime The log append time or -1 if CreateTime is being used - * @param error The error that occurred if there was one, or null + * @param errorsByIndex Function mapping the batch index to the exception, or null if the response was successful */ - public void set(long baseOffset, long logAppendTime, RuntimeException error) { + public void set(long baseOffset, long logAppendTime, Function errorsByIndex) { this.baseOffset = baseOffset; this.logAppendTime = logAppendTime; - this.error = error; + this.errorsByIndex = errorsByIndex; } /** @@ -110,8 +110,12 @@ public long logAppendTime() { /** * The error thrown (generally on the server) while processing this request */ - public RuntimeException error() { - return error; + public RuntimeException error(int batchIndex) { + if (errorsByIndex == null) { + return null; + } else { + return errorsByIndex.apply(batchIndex); + } } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java index cfd3a6794cc04..6f76c799b281c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.producer.internals; import org.apache.kafka.clients.producer.Callback; -import org.apache.kafka.common.utils.ProducerIdAndEpoch; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.RecordBatchTooLargeException; @@ -32,6 +31,7 @@ import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.ProduceResponse; +import org.apache.kafka.common.utils.ProducerIdAndEpoch; import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,8 +42,10 @@ import java.util.Deque; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V2; import static org.apache.kafka.common.record.RecordBatch.NO_TIMESTAMP; @@ -156,16 +158,46 @@ public void abort(RuntimeException exception) { throw new IllegalStateException("Batch has already been completed in final state " + finalState.get()); log.trace("Aborting batch for partition {}", topicPartition, exception); - completeFutureAndFireCallbacks(ProduceResponse.INVALID_OFFSET, RecordBatch.NO_TIMESTAMP, exception); + completeFutureAndFireCallbacks(ProduceResponse.INVALID_OFFSET, RecordBatch.NO_TIMESTAMP, index -> exception); } /** - * Return `true` if {@link #done(long, long, RuntimeException)} has been invoked at least once, `false` otherwise. + * Check if the batch has been completed (either successfully or exceptionally). + * @return `true` if the batch has been completed, `false` otherwise. */ public boolean isDone() { return finalState() != null; } + /** + * Complete the batch successfully. + * @param baseOffset The base offset of the messages assigned by the server + * @param logAppendTime The log append time or -1 if CreateTime is being used + * @return true if the batch was completed as a result of this call, and false + * if it had been completed previously + */ + public boolean complete(long baseOffset, long logAppendTime) { + return done(baseOffset, logAppendTime, null, null); + } + + /** + * Complete the batch exceptionally. The provided top-level exception will be used + * for each record future contained in the batch. + * + * @param topLevelException top-level partition error + * @param recordExceptions Record exception function mapping batchIndex to the respective record exception + * @return true if the batch was completed as a result of this call, and false + * if it had been completed previously + */ + public boolean completeExceptionally( + RuntimeException topLevelException, + Function recordExceptions + ) { + Objects.requireNonNull(topLevelException); + Objects.requireNonNull(recordExceptions); + return done(ProduceResponse.INVALID_OFFSET, RecordBatch.NO_TIMESTAMP, topLevelException, recordExceptions); + } + /** * Finalize the state of a batch. Final state, once set, is immutable. This function may be called * once or twice on a batch. It may be called twice if @@ -181,20 +213,25 @@ public boolean isDone() { * * @param baseOffset The base offset of the messages assigned by the server * @param logAppendTime The log append time or -1 if CreateTime is being used - * @param exception The exception that occurred (or null if the request was successful) + * @param topLevelException The exception that occurred (or null if the request was successful) + * @param recordExceptions Record exception function mapping batchIndex to the respective record exception * @return true if the batch was completed successfully and false if the batch was previously aborted */ - public boolean done(long baseOffset, long logAppendTime, RuntimeException exception) { - final FinalState tryFinalState = (exception == null) ? FinalState.SUCCEEDED : FinalState.FAILED; - + private boolean done( + long baseOffset, + long logAppendTime, + RuntimeException topLevelException, + Function recordExceptions + ) { + final FinalState tryFinalState = (topLevelException == null) ? FinalState.SUCCEEDED : FinalState.FAILED; if (tryFinalState == FinalState.SUCCEEDED) { log.trace("Successfully produced messages to {} with base offset {}.", topicPartition, baseOffset); } else { - log.trace("Failed to produce messages to {} with base offset {}.", topicPartition, baseOffset, exception); + log.trace("Failed to produce messages to {} with base offset {}.", topicPartition, baseOffset, topLevelException); } if (this.finalState.compareAndSet(null, tryFinalState)) { - completeFutureAndFireCallbacks(baseOffset, logAppendTime, exception); + completeFutureAndFireCallbacks(baseOffset, logAppendTime, recordExceptions); return true; } @@ -215,20 +252,26 @@ public boolean done(long baseOffset, long logAppendTime, RuntimeException except return false; } - private void completeFutureAndFireCallbacks(long baseOffset, long logAppendTime, RuntimeException exception) { + private void completeFutureAndFireCallbacks( + long baseOffset, + long logAppendTime, + Function recordExceptions + ) { // Set the future before invoking the callbacks as we rely on its state for the `onCompletion` call - produceFuture.set(baseOffset, logAppendTime, exception); + produceFuture.set(baseOffset, logAppendTime, recordExceptions); // execute callbacks - for (Thunk thunk : thunks) { + for (int i = 0; i < thunks.size(); i++) { try { - if (exception == null) { - RecordMetadata metadata = thunk.future.value(); - if (thunk.callback != null) + Thunk thunk = thunks.get(i); + if (thunk.callback != null) { + if (recordExceptions == null) { + RecordMetadata metadata = thunk.future.value(); thunk.callback.onCompletion(metadata, null); - } else { - if (thunk.callback != null) + } else { + RuntimeException exception = recordExceptions.apply(i); thunk.callback.onCompletion(null, exception); + } } } catch (Exception e) { log.error("Error executing user-provided callback on message for topic-partition '{}'", topicPartition, e); @@ -280,7 +323,7 @@ public Deque split(int splitBatchSize) { batch.closeForRecordAppends(); } - produceFuture.set(ProduceResponse.INVALID_OFFSET, NO_TIMESTAMP, new RecordBatchTooLargeException()); + produceFuture.set(ProduceResponse.INVALID_OFFSET, NO_TIMESTAMP, index -> new RecordBatchTooLargeException()); produceFuture.done(); if (hasSequence()) { @@ -420,8 +463,8 @@ public void close() { * Abort the record builder and reset the state of the underlying buffer. This is used prior to aborting * the batch with {@link #abort(RuntimeException)} and ensures that no record previously appended can be * read. This is used in scenarios where we want to ensure a batch ultimately gets aborted, but in which - * it is not safe to invoke the completion callbacks (e.g. because we are holding a lock, - * {@link RecordAccumulator#abortBatches()}). + * it is not safe to invoke the completion callbacks (e.g. because we are holding a lock, such as + * when aborting batches in {@link RecordAccumulator}). */ public void abortRecordAppends() { recordsBuilder.abort(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 8f2549343cf00..8b31094fda981 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -24,6 +24,8 @@ import org.apache.kafka.clients.NetworkClientUtils; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; @@ -60,10 +62,9 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.Function; import java.util.stream.Collectors; -import static org.apache.kafka.common.record.RecordBatch.NO_TIMESTAMP; - /** * The background thread that handles the sending of produce requests to the Kafka cluster. This thread makes metadata * requests to renew its view of the cluster and then sends produce requests to the appropriate nodes. @@ -380,7 +381,7 @@ private long sendProducerData(long now) { for (ProducerBatch expiredBatch : expiredBatches) { String errorMessage = "Expiring " + expiredBatch.recordCount + " record(s) for " + expiredBatch.topicPartition + ":" + (now - expiredBatch.createdMs) + " ms has passed since batch creation"; - failBatch(expiredBatch, -1, NO_TIMESTAMP, new TimeoutException(errorMessage), false); + failBatch(expiredBatch, new TimeoutException(errorMessage), false); if (transactionManager != null && expiredBatch.inRetry()) { // This ensures that no new batches are drained until the current in flight batches are fully resolved. transactionManager.markSequenceUnresolved(expiredBatch); @@ -635,17 +636,10 @@ private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionRespons // The only thing we can do is to return success to the user and not return a valid offset and timestamp. completeBatch(batch, response); } else { - final RuntimeException exception; - if (error == Errors.TOPIC_AUTHORIZATION_FAILED) - exception = new TopicAuthorizationException(Collections.singleton(batch.topicPartition.topic())); - else if (error == Errors.CLUSTER_AUTHORIZATION_FAILED) - exception = new ClusterAuthorizationException("The producer is not authorized to do idempotent sends"); - else - exception = error.exception(response.errorMessage); // tell the user the result of their request. We only adjust sequence numbers if the batch didn't exhaust // its retries -- if it did, we don't know whether the sequence number was accepted or not, and // thus it is not safe to reassign the sequence. - failBatch(batch, response, exception, batch.attempts() < this.retries); + failBatch(batch, response, batch.attempts() < this.retries); } if (error.exception() instanceof InvalidMetadataException) { if (error.exception() instanceof UnknownTopicOrPartitionException) { @@ -689,30 +683,87 @@ private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionRespons transactionManager.handleCompletedBatch(batch, response); } - if (batch.done(response.baseOffset, response.logAppendTime, null)) { + if (batch.complete(response.baseOffset, response.logAppendTime)) { maybeRemoveAndDeallocateBatch(batch); } } private void failBatch(ProducerBatch batch, ProduceResponse.PartitionResponse response, - RuntimeException exception, boolean adjustSequenceNumbers) { - failBatch(batch, response.baseOffset, response.logAppendTime, exception, adjustSequenceNumbers); + final RuntimeException topLevelException; + if (response.error == Errors.TOPIC_AUTHORIZATION_FAILED) + topLevelException = new TopicAuthorizationException(Collections.singleton(batch.topicPartition.topic())); + else if (response.error == Errors.CLUSTER_AUTHORIZATION_FAILED) + topLevelException = new ClusterAuthorizationException("The producer is not authorized to do idempotent sends"); + else + topLevelException = response.error.exception(response.errorMessage); + + if (response.recordErrors == null || response.recordErrors.isEmpty()) { + failBatch(batch, topLevelException, adjustSequenceNumbers); + } else { + Map recordErrorMap = new HashMap<>(response.recordErrors.size()); + for (ProduceResponse.RecordError recordError : response.recordErrors) { + // The API leaves us with some awkwardness interpreting the errors in the response. + // We cannot differentiate between different error cases (such as INVALID_TIMESTAMP) + // from the single error code at the partition level, so instead we use INVALID_RECORD + // for all failed records and rely on the message to distinguish the cases. + final String errorMessage; + if (recordError.message != null) { + errorMessage = recordError.message; + } else if (response.errorMessage != null) { + errorMessage = response.errorMessage; + } else { + errorMessage = response.error.message(); + } + + // If the batch contained only a single record error, then we can unambiguously + // use the exception type corresponding to the partition-level error code. + if (response.recordErrors.size() == 1) { + recordErrorMap.put(recordError.batchIndex, response.error.exception(errorMessage)); + } else { + recordErrorMap.put(recordError.batchIndex, new InvalidRecordException(errorMessage)); + } + } + + Function recordExceptions = batchIndex -> { + RuntimeException exception = recordErrorMap.get(batchIndex); + if (exception != null) { + return exception; + } else { + // If the response contains record errors, then the records which failed validation + // will be present in the response. To avoid confusion for the remaining records, we + // return a generic exception. + return new KafkaException("Failed to append record because it was part of a batch " + + "which had one more more invalid records"); + } + }; + + failBatch(batch, topLevelException, recordExceptions, adjustSequenceNumbers); + } } - private void failBatch(ProducerBatch batch, - long baseOffset, - long logAppendTime, - RuntimeException exception, - boolean adjustSequenceNumbers) { + private void failBatch( + ProducerBatch batch, + RuntimeException topLevelException, + boolean adjustSequenceNumbers + ) { + failBatch(batch, topLevelException, batchIndex -> topLevelException, adjustSequenceNumbers); + } + + private void failBatch( + ProducerBatch batch, + RuntimeException topLevelException, + Function recordExceptions, + boolean adjustSequenceNumbers + ) { if (transactionManager != null) { - transactionManager.handleFailedBatch(batch, exception, adjustSequenceNumbers); + transactionManager.handleFailedBatch(batch, topLevelException, adjustSequenceNumbers); } this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount); - if (batch.done(baseOffset, logAppendTime, exception)) { + if (batch.completeExceptionally(topLevelException, recordExceptions)) { maybeRemoveAndDeallocateBatch(batch); } } diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index 121cd42f44df8..90900af580018 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -41,7 +41,7 @@ { "name": "Acks", "type": "int16", "versions": "0+", "about": "The number of acknowledgments the producer requires the leader to have received before considering a request complete. Allowed values: 0 for no acknowledgments, 1 for only the leader and -1 for the full ISR." }, { "name": "TimeoutMs", "type": "int32", "versions": "0+", - "about": "The timeout to await a response in miliseconds." }, + "about": "The timeout to await a response in milliseconds." }, { "name": "TopicData", "type": "[]TopicProduceData", "versions": "0+", "about": "Each topic to produce to.", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", "mapKey": true, diff --git a/clients/src/main/resources/common/message/ProduceResponse.json b/clients/src/main/resources/common/message/ProduceResponse.json index d6e5566eab3c2..0c47f6d938e04 100644 --- a/clients/src/main/resources/common/message/ProduceResponse.json +++ b/clients/src/main/resources/common/message/ProduceResponse.json @@ -53,10 +53,10 @@ "about": "The log start offset." }, { "name": "RecordErrors", "type": "[]BatchIndexAndErrorMessage", "versions": "8+", "ignorable": true, "about": "The batch indices of records that caused the batch to be dropped", "fields": [ - { "name": "BatchIndex", "type": "int32", "versions": "8+", - "about": "The batch index of the record that cause the batch to be dropped" }, + { "name": "BatchIndex", "type": "int32", "versions": "8+", + "about": "The batch index of the record that cause the batch to be dropped" }, { "name": "BatchIndexErrorMessage", "type": "string", "default": "null", "versions": "8+", "nullableVersions": "8+", - "about": "The error message of the record that caused the batch to be dropped"} + "about": "The error message of the record that caused the batch to be dropped"} ]}, { "name": "ErrorMessage", "type": "string", "default": "null", "versions": "8+", "nullableVersions": "8+", "ignorable": true, "about": "The global error message summarizing the common root cause of the records that caused the batch to be dropped"} diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java index 3af658bbace62..b7cfd4e583eda 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java @@ -39,7 +39,7 @@ public class RecordSendTest { private final TopicPartition topicPartition = new TopicPartition("test", 0); private final long baseOffset = 45; - private final long relOffset = 5; + private final int relOffset = 5; /** * Test that waiting on a request that never completes times out @@ -89,7 +89,12 @@ public ProduceRequestResult asyncRequest(final long baseOffset, final RuntimeExc public void run() { try { sleep(timeout); - request.set(baseOffset, RecordBatch.NO_TIMESTAMP, error); + if (error == null) { + request.set(baseOffset, RecordBatch.NO_TIMESTAMP, null); + } else { + request.set(-1L, RecordBatch.NO_TIMESTAMP, index -> error); + } + request.done(); } catch (InterruptedException e) { } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java index af867ba17df97..b6ee58245aedc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java @@ -29,12 +29,18 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.Deque; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.ExecutionException; +import java.util.function.Function; import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V0; import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V1; @@ -43,6 +49,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; 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; @@ -50,7 +57,7 @@ public class ProducerBatchTest { private final long now = 1488748346917L; - private final MemoryRecordsBuilder memoryRecordsBuilder = MemoryRecords.builder(ByteBuffer.allocate(128), + private final MemoryRecordsBuilder memoryRecordsBuilder = MemoryRecords.builder(ByteBuffer.allocate(512), CompressionType.NONE, TimestampType.CREATE_TIME, 128); @Test @@ -75,8 +82,8 @@ public void testBatchAbort() throws Exception { assertNull(callback.metadata); // subsequent completion should be ignored - assertFalse(batch.done(500L, 2342342341L, null)); - assertFalse(batch.done(-1, -1, new KafkaException())); + assertFalse(batch.complete(500L, 2342342341L)); + assertFalse(batch.completeExceptionally(new KafkaException(), index -> new KafkaException())); assertEquals(1, callback.invocations); assertTrue(future.isDone()); @@ -121,18 +128,11 @@ public void testBatchCannotCompleteTwice() throws Exception { ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); MockCallback callback = new MockCallback(); FutureRecordMetadata future = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, callback, now); - batch.done(500L, 10L, null); + batch.complete(500L, 10L); assertEquals(1, callback.invocations); assertNull(callback.exception); assertNotNull(callback.metadata); - - try { - batch.done(1000L, 20L, null); - fail("Expected exception from done"); - } catch (IllegalStateException e) { - // expected - } - + assertThrows(IllegalStateException.class, () -> batch.complete(1000L, 20L)); RecordMetadata recordMetadata = future.get(); assertEquals(500L, recordMetadata.offset()); assertEquals(10L, recordMetadata.timestamp()); @@ -266,6 +266,57 @@ public void testShouldNotAttemptAppendOnceRecordsBuilderIsClosedForAppends() { assertNull(batch.tryAppend(now + 1, null, new byte[10], Record.EMPTY_HEADERS, null, now + 1)); } + @Test + public void testCompleteExceptionallyWithRecordErrors() { + int recordCount = 5; + RuntimeException topLevelException = new RuntimeException(); + + Map recordExceptionMap = new HashMap<>(); + recordExceptionMap.put(0, new RuntimeException()); + recordExceptionMap.put(3, new RuntimeException()); + + Function recordExceptions = batchIndex -> + recordExceptionMap.getOrDefault(batchIndex, topLevelException); + + testCompleteExceptionally(recordCount, topLevelException, recordExceptions); + } + + @Test + public void testCompleteExceptionallyWithNullRecordErrors() { + int recordCount = 5; + RuntimeException topLevelException = new RuntimeException(); + assertThrows(NullPointerException.class, () -> + testCompleteExceptionally(recordCount, topLevelException, null)); + } + + private void testCompleteExceptionally( + int recordCount, + RuntimeException topLevelException, + Function recordExceptions + ) { + ProducerBatch batch = new ProducerBatch( + new TopicPartition("topic", 1), + memoryRecordsBuilder, + now + ); + + List futures = new ArrayList<>(recordCount); + for (int i = 0; i < recordCount; i++) { + futures.add(batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, null, now)); + } + assertEquals(recordCount, batch.recordCount); + + batch.completeExceptionally(topLevelException, recordExceptions); + assertTrue(batch.isDone()); + + for (int i = 0; i < futures.size(); i++) { + FutureRecordMetadata future = futures.get(i); + RuntimeException caughtException = TestUtils.assertFutureThrows(future, RuntimeException.class); + RuntimeException expectedException = recordExceptions.apply(i); + assertEquals(expectedException, caughtException); + } + } + private static class MockCallback implements Callback { private int invocations = 0; private RecordMetadata metadata; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 3608aabdebf6c..9cc350abc17fb 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -754,7 +754,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { drained = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); assertFalse(drained.isEmpty()); assertFalse(drained.get(node1.id()).isEmpty()); - drained.get(node1.id()).get(0).done(acked.get(), 100L, null); + drained.get(node1.id()).get(0).complete(acked.get(), 100L); assertEquals(1, acked.get(), "The first message should have been acked."); assertTrue(future1.isDone()); assertEquals(0, future1.get().offset()); @@ -762,7 +762,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { drained = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); assertFalse(drained.isEmpty()); assertFalse(drained.get(node1.id()).isEmpty()); - drained.get(node1.id()).get(0).done(acked.get(), 100L, null); + drained.get(node1.id()).get(0).complete(acked.get(), 100L); assertEquals(2, acked.get(), "Both message should have been acked."); assertTrue(future2.isDone()); assertEquals(1, future2.get().offset()); @@ -1010,7 +1010,7 @@ private BatchDrainedResult completeOrSplitBatches(RecordAccumulator accum, int b // release the resource of the original big batch. numSplit++; } else { - batch.done(0L, 0L, null); + batch.complete(0L, 0L); } accum.deallocate(batch); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index c98ac632a0f7d..52e801f87b3c6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.MetricNameTemplate; @@ -44,6 +45,8 @@ import org.apache.kafka.common.message.EndTxnResponseData; import org.apache.kafka.common.message.InitProducerIdResponseData; import org.apache.kafka.common.message.ProduceRequestData; +import org.apache.kafka.common.message.ProduceResponseData; +import org.apache.kafka.common.message.ProduceResponseData.BatchIndexAndErrorMessage; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -86,6 +89,7 @@ import org.mockito.InOrder; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collections; import java.util.Deque; import java.util.HashMap; @@ -2450,6 +2454,48 @@ public void testInflightBatchesExpireOnDeliveryTimeout() throws InterruptedExcep } } + @Test + public void testRecordErrorPropagatedToApplication() throws InterruptedException { + int recordCount = 5; + + setup(); + + Map futures = new HashMap<>(recordCount); + for (int i = 0; i < recordCount; i++) { + futures.put(i, appendToAccumulator(tp0)); + } + + sender.runOnce(); // send request + assertEquals(1, client.inFlightRequestCount()); + assertEquals(1, sender.inFlightBatches(tp0).size()); + + OffsetAndError offsetAndError = new OffsetAndError(-1L, Errors.INVALID_RECORD, Arrays.asList( + new BatchIndexAndErrorMessage().setBatchIndex(0).setBatchIndexErrorMessage("0"), + new BatchIndexAndErrorMessage().setBatchIndex(2).setBatchIndexErrorMessage("2"), + new BatchIndexAndErrorMessage().setBatchIndex(3) + )); + + client.respond(produceResponse(Collections.singletonMap(tp0, offsetAndError))); + sender.runOnce(); + + for (Map.Entry futureEntry : futures.entrySet()) { + FutureRecordMetadata future = futureEntry.getValue(); + assertTrue(future.isDone()); + + KafkaException exception = TestUtils.assertFutureThrows(future, KafkaException.class); + Integer index = futureEntry.getKey(); + if (index == 0 || index == 2) { + assertTrue(exception instanceof InvalidRecordException); + assertEquals(index.toString(), exception.getMessage()); + } else if (index == 3) { + assertTrue(exception instanceof InvalidRecordException); + assertEquals(Errors.INVALID_RECORD.message(), exception.getMessage()); + } else { + assertEquals(KafkaException.class, exception.getClass()); + } + } + } + @Test public void testWhenFirstBatchExpireNoSendSecondBatchIfGuaranteeOrder() throws InterruptedException { long deliveryTimeoutMs = 1500L; @@ -2867,13 +2913,25 @@ private MockClient.RequestMatcher produceRequestMatcher(final TopicPartition tp, }; } - class OffsetAndError { - long offset; - Errors error; - OffsetAndError(long offset, Errors error) { + private static class OffsetAndError { + final long offset; + final Errors error; + final List recordErrors; + + OffsetAndError( + long offset, + Errors error, + List recordErrors + ) { this.offset = offset; this.error = error; + this.recordErrors = recordErrors; + } + + OffsetAndError(long offset, Errors error) { + this(offset, error, Collections.emptyList()); } + } private FutureRecordMetadata appendToAccumulator(TopicPartition tp) throws InterruptedException { @@ -2893,16 +2951,29 @@ private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors e return new ProduceResponse(partResp, throttleTimeMs); } - @SuppressWarnings("deprecation") private ProduceResponse produceResponse(Map responses) { - Map partResponses = new LinkedHashMap<>(); + ProduceResponseData data = new ProduceResponseData(); + for (Map.Entry entry : responses.entrySet()) { - ProduceResponse.PartitionResponse response = new ProduceResponse.PartitionResponse(entry.getValue().error, - entry.getValue().offset, RecordBatch.NO_TIMESTAMP, -1); - partResponses.put(entry.getKey(), response); + TopicPartition topicPartition = entry.getKey(); + ProduceResponseData.TopicProduceResponse topicData = data.responses().find(topicPartition.topic()); + if (topicData == null) { + topicData = new ProduceResponseData.TopicProduceResponse().setName(topicPartition.topic()); + data.responses().add(topicData); + } + + OffsetAndError offsetAndError = entry.getValue(); + ProduceResponseData.PartitionProduceResponse partitionData = + new ProduceResponseData.PartitionProduceResponse() + .setIndex(topicPartition.partition()) + .setBaseOffset(offsetAndError.offset) + .setErrorCode(offsetAndError.error.code()) + .setRecordErrors(offsetAndError.recordErrors); + + topicData.partitionResponses().add(partitionData); } - return new ProduceResponse(partResponses); + return new ProduceResponse(data); } private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs) { return produceResponse(tp, offset, error, throttleTimeMs, -1L, null); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index 9f097fc423b56..a14d29522cb18 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -595,7 +595,7 @@ public void testBumpEpochAndResetSequenceNumbersAfterUnknownProducerId() { long b1AppendTime = time.milliseconds(); ProduceResponse.PartitionResponse b1Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - b1.done(500L, b1AppendTime, null); + b1.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(b1, b1Response); // We get an UNKNOWN_PRODUCER_ID, so bump the epoch and set sequence numbers back to 0 @@ -3084,12 +3084,12 @@ MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(t long b1AppendTime = time.milliseconds(); ProduceResponse.PartitionResponse t0b1Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp0b1.done(500L, b1AppendTime, null); + tp0b1.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp0b1, t0b1Response); ProduceResponse.PartitionResponse t1b1Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp1b1.done(500L, b1AppendTime, null); + tp1b1.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp1b1, t1b1Response); // We bump the epoch and set sequence numbers back to 0 @@ -3134,7 +3134,7 @@ MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(t // After successfully retrying, there should be no in-flight batches for tp1 and the sequence should be 0 t1b2Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp1b2.done(500L, b1AppendTime, null); + tp1b2.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp1b2, t1b2Response); transactionManager.maybeUpdateProducerIdAndEpoch(tp1); @@ -3149,7 +3149,7 @@ MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(t ProduceResponse.PartitionResponse t1b3Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp1b3.done(500L, b1AppendTime, null); + tp1b3.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp1b3, t1b3Response); transactionManager.maybeUpdateProducerIdAndEpoch(tp1); @@ -3208,12 +3208,12 @@ MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(t long b1AppendTime = time.milliseconds(); ProduceResponse.PartitionResponse t0b1Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp0b1.done(500L, b1AppendTime, null); + tp0b1.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp0b1, t0b1Response); ProduceResponse.PartitionResponse t1b1Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp1b1.done(500L, b1AppendTime, null); + tp1b1.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp1b1, t1b1Response); // We bump the epoch and set sequence numbers back to 0 @@ -3258,7 +3258,7 @@ MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(t // After successfully retrying, there should be no in-flight batches for tp1 and the sequence should be 0 t1b2Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp1b2.done(500L, b1AppendTime, null); + tp1b2.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp1b2, t1b2Response); transactionManager.maybeUpdateProducerIdAndEpoch(tp1); @@ -3273,7 +3273,7 @@ MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(new Metrics(t ProduceResponse.PartitionResponse t1b3Response = new ProduceResponse.PartitionResponse( Errors.NONE, 500L, b1AppendTime, 0L); - tp1b3.done(500L, b1AppendTime, null); + tp1b3.complete(500L, b1AppendTime); transactionManager.handleCompletedBatch(tp1b3, t1b3Response); assertFalse(transactionManager.hasInflightBatches(tp1)); diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index 526f7266a1b29..38febbcd7a3d3 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -100,7 +100,6 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { val e = assertThrows(classOf[ExecutionException], () => producer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get()).getCause assertTrue(e.isInstanceOf[InvalidTimestampException]) - assertEquals("One or more records have been rejected due to invalid timestamp", e.getMessage) } finally { producer.close() } @@ -111,7 +110,6 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { val e = assertThrows(classOf[ExecutionException], () => compressedProducer.send(new ProducerRecord(topic, 0, System.currentTimeMillis() - 1001, "key".getBytes, "value".getBytes)).get()).getCause assertTrue(e.isInstanceOf[InvalidTimestampException]) - assertEquals("One or more records have been rejected due to invalid timestamp", e.getMessage) } finally { compressedProducer.close() } From cad514bff9140b75a3da18c56efaf26df68310c7 Mon Sep 17 00:00:00 2001 From: Boyang Chen Date: Mon, 5 Apr 2021 15:54:57 -0700 Subject: [PATCH 005/155] KAFKA-12294; forward auto topic request within envelope on behalf of clients (#10142) When auto-creating topics in KIP-500, the broker will send a `CreateTopics` request to the controller. It is useful in this case to preserve the original principal from the corresponding `Metadata` request by wrapping the `CreateTopics` request in an envelope so that the controller may repeat the authorization and to improve auditability. This follows a similar pattern to how standard `CreateTopics` requests are forwarded to the controller. Reviewers: Jason Gustafson --- .../common/requests/AbstractRequest.java | 13 ++ .../common/requests/RequestResponseTest.java | 47 ++++++ .../server/AutoTopicCreationManager.scala | 60 ++++++-- .../kafka/server/ForwardingManager.scala | 31 ++-- .../main/scala/kafka/server/KafkaApis.scala | 4 +- .../server/AutoTopicCreationManagerTest.scala | 134 +++++++++++++++++- .../unit/kafka/server/KafkaApisTest.scala | 18 ++- 7 files changed, 271 insertions(+), 36 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 7802293c9bbfd..3137e3e500aef 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -101,6 +101,19 @@ public final Send toSend(RequestHeader header) { return SendBuilder.buildRequestSend(header, data()); } + /** + * Serializes header and body without prefixing with size (unlike `toSend`, which does include a size prefix). + */ + public final ByteBuffer serializeWithHeader(RequestHeader header) { + if (header.apiKey() != apiKey) { + throw new IllegalArgumentException("Could not build request " + apiKey + " with header api key " + header.apiKey()); + } + if (header.apiVersion() != version) { + throw new IllegalArgumentException("Could not build request version " + version + " with header version " + header.apiVersion()); + } + return RequestUtils.serialize(header.data(), header.headerVersion(), data(), version); + } + // Visible for testing public final ByteBuffer serialize() { return MessageUtil.toByteBuffer(data(), version); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 725ae054d9b05..8f567aa915433 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -71,6 +71,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection; import org.apache.kafka.common.message.CreateTopicsRequestData.CreateableTopicConfig; import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicConfigs; @@ -983,6 +984,52 @@ public void testFetchRequestCompat() { data.write(writer, cache, (short) 2); } + @Test + public void testSerializeWithHeader() { + CreatableTopicCollection topicsToCreate = new CreatableTopicCollection(1); + topicsToCreate.add(new CreatableTopic() + .setName("topic") + .setNumPartitions(3) + .setReplicationFactor((short) 2)); + + CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest.Builder( + new CreateTopicsRequestData() + .setTimeoutMs(10) + .setTopics(topicsToCreate) + ).build(); + + short requestVersion = ApiKeys.CREATE_TOPICS.latestVersion(); + RequestHeader requestHeader = new RequestHeader(ApiKeys.CREATE_TOPICS, requestVersion, "client", 2); + ByteBuffer serializedRequest = createTopicsRequest.serializeWithHeader(requestHeader); + + RequestHeader parsedHeader = RequestHeader.parse(serializedRequest); + assertEquals(requestHeader, parsedHeader); + + RequestAndSize parsedRequest = AbstractRequest.parseRequest( + ApiKeys.CREATE_TOPICS, requestVersion, serializedRequest); + + assertEquals(createTopicsRequest.data(), parsedRequest.request.data()); + } + + @Test + public void testSerializeWithInconsistentHeaderApiKey() { + CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest.Builder( + new CreateTopicsRequestData() + ).build(); + short requestVersion = ApiKeys.CREATE_TOPICS.latestVersion(); + RequestHeader requestHeader = new RequestHeader(DELETE_TOPICS, requestVersion, "client", 2); + assertThrows(IllegalArgumentException.class, () -> createTopicsRequest.serializeWithHeader(requestHeader)); + } + + @Test + public void testSerializeWithInconsistentHeaderVersion() { + CreateTopicsRequest createTopicsRequest = new CreateTopicsRequest.Builder( + new CreateTopicsRequestData() + ).build((short) 2); + RequestHeader requestHeader = new RequestHeader(CREATE_TOPICS, (short) 1, "client", 2); + assertThrows(IllegalArgumentException.class, () -> createTopicsRequest.serializeWithHeader(requestHeader)); + } + @Test public void testJoinGroupRequestVersion0RebalanceTimeout() { final short version = 0; diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala index bac9fb7bc2755..616b7a22de6e5 100644 --- a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -17,9 +17,9 @@ package kafka.server -import java.util.{Collections, Properties} import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicReference +import java.util.{Collections, Properties} import kafka.controller.KafkaController import kafka.coordinator.group.GroupCoordinator @@ -32,8 +32,8 @@ import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANS import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreateableTopicConfig, CreateableTopicConfigCollection} import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic -import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{ApiError, CreateTopicsRequest} +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.requests.{ApiError, CreateTopicsRequest, RequestContext, RequestHeader} import scala.collection.{Map, Seq, Set, mutable} import scala.jdk.CollectionConverters._ @@ -42,7 +42,8 @@ trait AutoTopicCreationManager { def createTopics( topicNames: Set[String], - controllerMutationQuota: ControllerMutationQuota + controllerMutationQuota: ControllerMutationQuota, + metadataRequestContext: Option[RequestContext] ): Seq[MetadataResponseTopic] } @@ -77,16 +78,27 @@ class DefaultAutoTopicCreationManager( private val inflightTopics = Collections.newSetFromMap(new ConcurrentHashMap[String, java.lang.Boolean]()) + /** + * Initiate auto topic creation for the given topics. + * + * @param topics the topics to create + * @param controllerMutationQuota the controller mutation quota for topic creation + * @param metadataRequestContext defined when creating topics on behalf of the client. The goal here is to preserve + * original client principal for auditing, thus needing to wrap a plain CreateTopicsRequest + * inside Envelope to send to the controller when forwarding is enabled. + * @return auto created topic metadata responses + */ override def createTopics( topics: Set[String], - controllerMutationQuota: ControllerMutationQuota + controllerMutationQuota: ControllerMutationQuota, + metadataRequestContext: Option[RequestContext] ): Seq[MetadataResponseTopic] = { val (creatableTopics, uncreatableTopicResponses) = filterCreatableTopics(topics) val creatableTopicResponses = if (creatableTopics.isEmpty) { Seq.empty } else if (controller.isEmpty || !controller.get.isActive && channelManager.isDefined) { - sendCreateTopicRequest(creatableTopics) + sendCreateTopicRequest(creatableTopics, metadataRequestContext) } else { createTopicsInZk(creatableTopics, controllerMutationQuota) } @@ -145,7 +157,8 @@ class DefaultAutoTopicCreationManager( } private def sendCreateTopicRequest( - creatableTopics: Map[String, CreatableTopic] + creatableTopics: Map[String, CreatableTopic], + metadataRequestContext: Option[RequestContext] ): Seq[MetadataResponseTopic] = { val topicsToCreate = new CreateTopicsRequestData.CreatableTopicCollection(creatableTopics.size) topicsToCreate.addAll(creatableTopics.values.asJavaCollection) @@ -156,17 +169,44 @@ class DefaultAutoTopicCreationManager( .setTopics(topicsToCreate) ) - channelManager.get.sendRequest(createTopicsRequest, new ControllerRequestCompletionHandler { + val requestCompletionHandler = new ControllerRequestCompletionHandler { override def onTimeout(): Unit = { debug(s"Auto topic creation timed out for ${creatableTopics.keys}.") clearInflightRequests(creatableTopics) } override def onComplete(response: ClientResponse): Unit = { - debug(s"Auto topic creation completed for ${creatableTopics.keys}.") + debug(s"Auto topic creation completed for ${creatableTopics.keys} with response ${response.responseBody.toString}.") clearInflightRequests(creatableTopics) } - }) + } + + val channelManager = this.channelManager.getOrElse { + throw new IllegalStateException("Channel manager must be defined in order to send CreateTopic requests.") + } + + val request = metadataRequestContext.map { context => + val requestVersion = + channelManager.controllerApiVersions() match { + case None => + // We will rely on the Metadata request to be retried in the case + // that the latest version is not usable by the controller. + ApiKeys.CREATE_TOPICS.latestVersion() + case Some(nodeApiVersions) => + nodeApiVersions.latestUsableVersion(ApiKeys.CREATE_TOPICS) + } + + // Borrow client information such as client id and correlation id from the original request, + // in order to correlate the create request with the original metadata request. + val requestHeader = new RequestHeader(ApiKeys.CREATE_TOPICS, + requestVersion, + context.clientId, + context.correlationId) + ForwardingManager.buildEnvelopeRequest(context, + createTopicsRequest.build(requestVersion).serializeWithHeader(requestHeader)) + }.getOrElse(createTopicsRequest) + + channelManager.sendRequest(request, requestCompletionHandler) val creatableTopicResponses = creatableTopics.keySet.toSeq.map { topic => new MetadataResponseTopic() diff --git a/core/src/main/scala/kafka/server/ForwardingManager.scala b/core/src/main/scala/kafka/server/ForwardingManager.scala index 788dd4c06c8c6..e84592b292dae 100644 --- a/core/src/main/scala/kafka/server/ForwardingManager.scala +++ b/core/src/main/scala/kafka/server/ForwardingManager.scala @@ -24,7 +24,7 @@ import kafka.utils.Logging import org.apache.kafka.clients.{ClientResponse, NodeApiVersions} import org.apache.kafka.common.errors.TimeoutException import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, EnvelopeRequest, EnvelopeResponse, RequestHeader} +import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, EnvelopeRequest, EnvelopeResponse, RequestContext, RequestHeader} import scala.compat.java8.OptionConverters._ @@ -43,6 +43,20 @@ object ForwardingManager { ): ForwardingManager = { new ForwardingManagerImpl(channelManager) } + + private[server] def buildEnvelopeRequest(context: RequestContext, + forwardRequestBuffer: ByteBuffer): EnvelopeRequest.Builder = { + val principalSerde = context.principalSerde.asScala.getOrElse( + throw new IllegalArgumentException(s"Cannot deserialize principal from request context $context " + + "since there is no serde defined") + ) + val serializedPrincipal = principalSerde.serialize(context.principal) + new EnvelopeRequest.Builder( + forwardRequestBuffer, + serializedPrincipal, + context.clientAddress.getAddress + ) + } } class ForwardingManagerImpl( @@ -61,18 +75,9 @@ class ForwardingManagerImpl( request: RequestChannel.Request, responseCallback: Option[AbstractResponse] => Unit ): Unit = { - val principalSerde = request.context.principalSerde.asScala.getOrElse( - throw new IllegalArgumentException(s"Cannot deserialize principal from request $request " + - "since there is no serde defined") - ) - val serializedPrincipal = principalSerde.serialize(request.context.principal) - val forwardRequestBuffer = request.buffer.duplicate() - forwardRequestBuffer.flip() - val envelopeRequest = new EnvelopeRequest.Builder( - forwardRequestBuffer, - serializedPrincipal, - request.context.clientAddress.getAddress - ) + val requestBuffer = request.buffer.duplicate() + requestBuffer.flip() + val envelopeRequest = ForwardingManager.buildEnvelopeRequest(request.context, requestBuffer) class ForwardingResponseHandler extends ControllerRequestCompletionHandler { override def onComplete(clientResponse: ClientResponse): Unit = { diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5aad1f6cc7d1b..42a203c4a570c 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -1113,7 +1113,7 @@ class KafkaApis(val requestChannel: RequestChannel, val nonExistingTopics = topics.diff(topicResponses.map(_.name).toSet) val nonExistingTopicResponses = if (allowAutoTopicCreation) { val controllerMutationQuota = quotas.controllerMutation.newPermissiveQuotaFor(request) - autoTopicCreationManager.createTopics(nonExistingTopics, controllerMutationQuota) + autoTopicCreationManager.createTopics(nonExistingTopics, controllerMutationQuota, Some(request.context)) } else { nonExistingTopics.map { topic => val error = try { @@ -1341,7 +1341,7 @@ class KafkaApis(val requestChannel: RequestChannel, if (topicMetadata.headOption.isEmpty) { val controllerMutationQuota = quotas.controllerMutation.newPermissiveQuotaFor(request) - autoTopicCreationManager.createTopics(Seq(internalTopicName).toSet, controllerMutationQuota) + autoTopicCreationManager.createTopics(Seq(internalTopicName).toSet, controllerMutationQuota, None) requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createFindCoordinatorResponse( Errors.COORDINATOR_NOT_AVAILABLE, Node.noNode, requestThrottleMs)) } else { diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index 0bd97c36e258d..39f1f9e453737 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -17,25 +17,32 @@ package kafka.server -import java.util.Properties +import java.net.InetAddress +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicBoolean +import java.util.{Collections, Optional, Properties} import kafka.controller.KafkaController import kafka.coordinator.group.GroupCoordinator import kafka.coordinator.transaction.TransactionCoordinator import kafka.utils.TestUtils import kafka.utils.TestUtils.createBroker +import org.apache.kafka.clients.{ClientResponse, NodeApiVersions, RequestCompletionHandler} import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic.{GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME} -import org.apache.kafka.common.message.CreateTopicsRequestData +import org.apache.kafka.common.message.{ApiVersionsResponseData, CreateTopicsRequestData} import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic -import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.network.{ClientInformation, ListenerName} +import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests._ -import org.junit.jupiter.api.Assertions.assertEquals +import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerde, SecurityProtocol} +import org.apache.kafka.common.utils.{SecurityUtils, Utils} +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} import org.mockito.ArgumentMatchers.any import org.mockito.invocation.InvocationOnMock -import org.mockito.{ArgumentMatchers, Mockito} +import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} import scala.collection.{Map, Seq} @@ -219,6 +226,118 @@ class AutoTopicCreationManagerTest { testErrorWithCreationInZk(Errors.UNKNOWN_TOPIC_OR_PARTITION, Topic.TRANSACTION_STATE_TOPIC_NAME, isInternal = true) } + @Test + def testTopicCreationWithMetadataContextPassPrincipal(): Unit = { + val topicName = "topic" + + val userPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "user") + val serializeIsCalled = new AtomicBoolean(false) + val principalSerde = new KafkaPrincipalSerde { + override def serialize(principal: KafkaPrincipal): Array[Byte] = { + assertEquals(principal, userPrincipal) + serializeIsCalled.set(true) + Utils.utf8(principal.toString) + } + override def deserialize(bytes: Array[Byte]): KafkaPrincipal = SecurityUtils.parseKafkaPrincipal(Utils.utf8(bytes)) + } + + val requestContext = initializeRequestContext(topicName, userPrincipal, Optional.of(principalSerde)) + + autoTopicCreationManager.createTopics( + Set(topicName), UnboundedControllerMutationQuota, Some(requestContext)) + + assertTrue(serializeIsCalled.get()) + + val argumentCaptor = ArgumentCaptor.forClass(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]) + Mockito.verify(brokerToController).sendRequest( + argumentCaptor.capture(), + any(classOf[ControllerRequestCompletionHandler])) + val capturedRequest = argumentCaptor.getValue.asInstanceOf[EnvelopeRequest.Builder].build(ApiKeys.ENVELOPE.latestVersion()) + assertEquals(userPrincipal, SecurityUtils.parseKafkaPrincipal(Utils.utf8(capturedRequest.requestPrincipal))) + } + + @Test + def testTopicCreationWithMetadataContextWhenPrincipalSerdeNotDefined(): Unit = { + val topicName = "topic" + + val requestContext = initializeRequestContext(topicName, KafkaPrincipal.ANONYMOUS, Optional.empty()) + + // Throw upon undefined principal serde when building the forward request + assertThrows(classOf[IllegalArgumentException], () => autoTopicCreationManager.createTopics( + Set(topicName), UnboundedControllerMutationQuota, Some(requestContext))) + } + + @Test + def testTopicCreationWithMetadataContextNoRetryUponUnsupportedVersion(): Unit = { + val topicName = "topic" + + val principalSerde = new KafkaPrincipalSerde { + override def serialize(principal: KafkaPrincipal): Array[Byte] = { + Utils.utf8(principal.toString) + } + override def deserialize(bytes: Array[Byte]): KafkaPrincipal = SecurityUtils.parseKafkaPrincipal(Utils.utf8(bytes)) + } + + val requestContext = initializeRequestContext(topicName, KafkaPrincipal.ANONYMOUS, Optional.of(principalSerde)) + autoTopicCreationManager.createTopics( + Set(topicName), UnboundedControllerMutationQuota, Some(requestContext)) + autoTopicCreationManager.createTopics( + Set(topicName), UnboundedControllerMutationQuota, Some(requestContext)) + + // Should only trigger once + val argumentCaptor = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) + Mockito.verify(brokerToController).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Complete with unsupported version will not trigger a retry, but cleanup the inflight topics instead + val header = new RequestHeader(ApiKeys.ENVELOPE, 0, "client", 1) + val response = new EnvelopeResponse(ByteBuffer.allocate(0), Errors.UNSUPPORTED_VERSION) + val clientResponse = new ClientResponse(header, null, null, + 0, 0, false, null, null, response) + argumentCaptor.getValue.asInstanceOf[RequestCompletionHandler].onComplete(clientResponse) + Mockito.verify(brokerToController, Mockito.times(1)).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + + // Could do the send again as inflight topics are cleared. + autoTopicCreationManager.createTopics( + Set(topicName), UnboundedControllerMutationQuota, Some(requestContext)) + Mockito.verify(brokerToController, Mockito.times(2)).sendRequest( + any(classOf[AbstractRequest.Builder[_ <: AbstractRequest]]), + argumentCaptor.capture()) + } + + private def initializeRequestContext(topicName: String, + kafkaPrincipal: KafkaPrincipal, + principalSerde: Optional[KafkaPrincipalSerde]): RequestContext = { + + autoTopicCreationManager = new DefaultAutoTopicCreationManager( + config, + Some(brokerToController), + Some(adminManager), + Some(controller), + groupCoordinator, + transactionCoordinator) + + val topicsCollection = new CreateTopicsRequestData.CreatableTopicCollection + topicsCollection.add(getNewTopic(topicName)) + val createTopicApiVersion = new ApiVersionsResponseData.ApiVersion() + .setApiKey(ApiKeys.CREATE_TOPICS.id) + .setMinVersion(0) + .setMaxVersion(0) + Mockito.when(brokerToController.controllerApiVersions()) + .thenReturn(Some(NodeApiVersions.create(Collections.singleton(createTopicApiVersion)))) + + Mockito.when(controller.isActive).thenReturn(false) + + val requestHeader = new RequestHeader(ApiKeys.METADATA, ApiKeys.METADATA.latestVersion, + "clientId", 0) + new RequestContext(requestHeader, "1", InetAddress.getLocalHost, + kafkaPrincipal, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), + SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY, false, principalSerde) + } + private def testErrorWithCreationInZk(error: Errors, topicName: String, isInternal: Boolean, @@ -261,9 +380,10 @@ class AutoTopicCreationManagerTest { private def createTopicAndVerifyResult(error: Errors, topicName: String, - isInternal: Boolean): Unit = { + isInternal: Boolean, + metadataContext: Option[RequestContext] = None): Unit = { val topicResponses = autoTopicCreationManager.createTopics( - Set(topicName), UnboundedControllerMutationQuota) + Set(topicName), UnboundedControllerMutationQuota, metadataContext) val expectedResponses = Seq(new MetadataResponseTopic() .setErrorCode(error.code()) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 177a33a9344aa..8280380b712d0 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -892,7 +892,7 @@ class KafkaApisTest { val capturedResponse = expectNoThrottling(request) - verifyTopicCreation(topicName, true, true, request) + val capturedRequest = verifyTopicCreation(topicName, true, true, request) EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, authorizer, autoTopicCreationManager, forwardingManager, controller, clientControllerQuotaManager, groupCoordinator, txnCoordinator) @@ -903,6 +903,8 @@ class KafkaApisTest { val response = capturedResponse.getValue.asInstanceOf[FindCoordinatorResponse] assertEquals(Errors.COORDINATOR_NOT_AVAILABLE, response.error()) + assertTrue(capturedRequest.getValue.isEmpty) + verify(authorizer, autoTopicCreationManager) } @@ -993,7 +995,7 @@ class KafkaApisTest { val capturedResponse = expectNoThrottling(request) - verifyTopicCreation(topicName, enableAutoTopicCreation, isInternal, request) + val capturedRequest = verifyTopicCreation(topicName, enableAutoTopicCreation, isInternal, request) EasyMock.replay(replicaManager, clientRequestQuotaManager, requestChannel, authorizer, autoTopicCreationManager, forwardingManager, clientControllerQuotaManager, groupCoordinator, txnCoordinator) @@ -1011,26 +1013,34 @@ class KafkaApisTest { assertEquals(expectedMetadataResponse, response.topicMetadata()) + if (enableAutoTopicCreation) { + assertTrue(capturedRequest.getValue.isDefined) + assertEquals(request.context, capturedRequest.getValue.get) + } + verify(authorizer, autoTopicCreationManager) } private def verifyTopicCreation(topicName: String, enableAutoTopicCreation: Boolean, isInternal: Boolean, - request: RequestChannel.Request) = { + request: RequestChannel.Request): Capture[Option[RequestContext]] = { + val capturedRequest = EasyMock.newCapture[Option[RequestContext]]() if (enableAutoTopicCreation) { EasyMock.expect(clientControllerQuotaManager.newPermissiveQuotaFor(EasyMock.eq(request))) .andReturn(UnboundedControllerMutationQuota) EasyMock.expect(autoTopicCreationManager.createTopics( EasyMock.eq(Set(topicName)), - EasyMock.eq(UnboundedControllerMutationQuota))).andReturn( + EasyMock.eq(UnboundedControllerMutationQuota), + EasyMock.capture(capturedRequest))).andReturn( Seq(new MetadataResponseTopic() .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) .setIsInternal(isInternal) .setName(topicName)) ).once() } + capturedRequest } private def setupBrokerMetadata(hasEnoughLiveBrokers: Boolean, numBrokersNeeded: Int): Unit = { From 0a8921517f8e7bdfd5318e7576f6feacc4f3e6f8 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 5 Apr 2021 22:20:23 -0700 Subject: [PATCH 006/155] MINOR: Jenkinsfile's `post` needs `agent` to be set (#10479) The `node` block achieves that. Tested that an email was sent to the mailing list for: https://github.com/apache/kafka/pull/10479/commits/592a0c31d5dec5b2d33b6239f6243831b7cca361 Added back the condition not to send emails for PR builds after such test. Reviewers: Gwen Shapira --- Jenkinsfile | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index 07db6e4b648ee..c9ea085b8f3c5 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -274,12 +274,14 @@ pipeline { post { always { - script { - if (!isChangeRequest(env)) { - step([$class: 'Mailer', - notifyEveryUnstableBuild: true, - recipients: "dev@kafka.apache.org", - sendToIndividuals: false]) + node('ubuntu') { + script { + if (!isChangeRequest(env)) { + step([$class: 'Mailer', + notifyEveryUnstableBuild: true, + recipients: "dev@kafka.apache.org", + sendToIndividuals: false]) + } } } } From 267b73616d801e946bb12746787b2793f0639e7b Mon Sep 17 00:00:00 2001 From: Luke Chen <43372967+showuon@users.noreply.github.com> Date: Tue, 6 Apr 2021 15:46:00 +0800 Subject: [PATCH 007/155] MINOR: update GroupMetadataManager#getMagic docs (#10442) Reviewers: Chia-Ping Tsai --- .../scala/kafka/coordinator/group/GroupMetadataManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index fb6d07e52f61e..c054234abb082 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -934,7 +934,7 @@ class GroupMetadataManager(brokerId: Int, } /** - * Check if the replica is local and return the message format version and timestamp + * Check if the replica is local and return the message format version * * @param partition Partition of GroupMetadataTopic * @return Some(MessageFormatVersion) if replica is local, None otherwise From 2f3600198722dd5a01a210bc78b7d43b33967c7f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 6 Apr 2021 08:54:53 -0700 Subject: [PATCH 008/155] KAFKA-12579: Remove various deprecated clients classes/methods for 3.0 (#10438) * Remove `ExtendedSerializer` and `ExtendedDeserializer`, deprecated since 2.1. The extra functionality was also made available in `Serializer` and `Deserializer`. * Remove `close(long, TimeUnit)` from the producer, consumer and admin client, deprecated since 2.0 for the consumer and 2.2 for the rest. The replacement is `close(Duration)`. * Remove `ConsumerConfig.addDeserializerToConfig` and `ProducerConfig.addSerializerToConfig`, deprecated since 2.7 with no replacement. These methods were not intended to be public API and are likely not used much (if at all). * Remove `NoOffsetForPartitionException.partition()`, deprecated since 0.11. `partitions()` should be used instead. * Remove `MessageFormatter.init(Properties)`, deprecated since 2.7. The `configure(Map)` method should be used instead. * Remove `kafka.common.MessageFormatter`, deprecated since 2.7. `org.apache.kafka.common.MessageFormatter` should be used instead. Reviewers: Chia-Ping Tsai , David Jacot --- .../org/apache/kafka/clients/admin/Admin.java | 22 +----- .../kafka/clients/consumer/Consumer.java | 7 -- .../clients/consumer/ConsumerConfig.java | 32 +------- .../kafka/clients/consumer/KafkaConsumer.java | 23 ------ .../kafka/clients/consumer/MockConsumer.java | 9 +-- .../NoOffsetForPartitionException.java | 10 --- .../kafka/clients/producer/Producer.java | 6 -- .../clients/producer/ProducerConfig.java | 25 ------ .../apache/kafka/common/MessageFormatter.java | 19 +---- .../serialization/ExtendedDeserializer.java | 79 ------------------- .../serialization/ExtendedSerializer.java | 79 ------------------- .../clients/consumer/ConsumerConfigTest.java | 28 ------- .../clients/consumer/KafkaConsumerTest.java | 14 ---- .../clients/producer/KafkaProducerTest.java | 7 -- .../clients/producer/ProducerConfigTest.java | 31 -------- .../scala/kafka/common/MessageFormatter.scala | 28 ------- .../kafka/api/PlaintextConsumerTest.scala | 10 --- docs/upgrade.html | 37 ++++++--- .../processor/internals/StreamThread.java | 4 +- 19 files changed, 36 insertions(+), 434 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ExtendedDeserializer.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ExtendedSerializer.java delete mode 100644 core/src/main/scala/kafka/common/MessageFormatter.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index 3fe717a6e9f17..e7704f52cff32 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -24,7 +24,6 @@ import java.util.Optional; import java.util.Properties; import java.util.Set; -import java.util.concurrent.TimeUnit; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.ElectionType; @@ -147,28 +146,11 @@ static Admin create(Map conf) { /** * Close the Admin and release all associated resources. *

- * See {@link #close(long, TimeUnit)} + * See {@link #close(Duration)} */ @Override default void close() { - close(Long.MAX_VALUE, TimeUnit.MILLISECONDS); - } - - /** - * Close the Admin and release all associated resources. - *

- * The close operation has a grace period during which current operations will be allowed to - * complete, specified by the given duration and time unit. - * New operations will not be accepted during the grace period. Once the grace period is over, - * all operations that have not yet been completed will be aborted with a {@link org.apache.kafka.common.errors.TimeoutException}. - * - * @param duration The duration to use for the wait time. - * @param unit The time unit to use for the wait time. - * @deprecated Since 2.2. Use {@link #close(Duration)} or {@link #close()}. - */ - @Deprecated - default void close(long duration, TimeUnit unit) { - close(Duration.ofMillis(unit.toMillis(duration))); + close(Duration.ofMillis(Long.MAX_VALUE)); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index b324773093aad..0dd7dd8d78fd5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -28,7 +28,6 @@ import java.util.Map; import java.util.OptionalLong; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; /** @@ -264,12 +263,6 @@ public interface Consumer extends Closeable { */ void close(); - /** - * @see KafkaConsumer#close(long, TimeUnit) - */ - @Deprecated - void close(long timeout, TimeUnit unit); - /** * @see KafkaConsumer#close(Duration) */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 77443538085ea..93ff024031e51 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -578,19 +578,9 @@ private void maybeOverrideClientId(Map configs) { } } - /** - * @deprecated Since 2.7.0. This will be removed in a future major release. - */ - @Deprecated - public static Map addDeserializerToConfig(Map configs, - Deserializer keyDeserializer, - Deserializer valueDeserializer) { - return appendDeserializerToConfig(configs, keyDeserializer, valueDeserializer); - } - - static Map appendDeserializerToConfig(Map configs, - Deserializer keyDeserializer, - Deserializer valueDeserializer) { + protected static Map appendDeserializerToConfig(Map configs, + Deserializer keyDeserializer, + Deserializer valueDeserializer) { Map newConfigs = new HashMap<>(configs); if (keyDeserializer != null) newConfigs.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass()); @@ -599,22 +589,6 @@ static Map appendDeserializerToConfig(Map config return newConfigs; } - /** - * @deprecated Since 2.7.0. This will be removed in a future major release. - */ - @Deprecated - public static Properties addDeserializerToConfig(Properties properties, - Deserializer keyDeserializer, - Deserializer valueDeserializer) { - Properties newProperties = new Properties(); - newProperties.putAll(properties); - if (keyDeserializer != null) - newProperties.put(KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer.getClass().getName()); - if (valueDeserializer != null) - newProperties.put(VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass().getName()); - return newProperties; - } - boolean maybeOverrideEnableAutoCommit() { Optional groupId = Optional.ofNullable(getString(CommonClientConfigs.GROUP_ID_CONFIG)); boolean enableAutoCommit = getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index ad0f159c85129..460383c563e5c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -2308,29 +2308,6 @@ public void close() { close(Duration.ofMillis(DEFAULT_CLOSE_TIMEOUT_MS)); } - /** - * Tries to close the consumer cleanly within the specified timeout. This method waits up to - * {@code timeout} for the consumer to complete pending commits and leave the group. - * If auto-commit is enabled, this will commit the current offsets if possible within the - * timeout. If the consumer is unable to complete offset commits and gracefully leave the group - * before the timeout expires, the consumer is force closed. Note that {@link #wakeup()} cannot be - * used to interrupt close. - * - * @param timeout The maximum time to wait for consumer to close gracefully. The value must be - * non-negative. Specifying a timeout of zero means do not wait for pending requests to complete. - * @param timeUnit The time unit for the {@code timeout} - * @throws IllegalArgumentException If the {@code timeout} is negative. - * @throws InterruptException If the thread is interrupted before or while this function is called - * @throws org.apache.kafka.common.KafkaException for any other error during close - * - * @deprecated Since 2.0. Use {@link #close(Duration)} or {@link #close()}. - */ - @Deprecated - @Override - public void close(long timeout, TimeUnit timeUnit) { - close(Duration.ofMillis(timeUnit.toMillis(timeout))); - } - /** * Tries to close the consumer cleanly within the specified timeout. This method waits up to * {@code timeout} for the consumer to complete pending commits and leave the group. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index b5e0c1d93fc70..345cdec00dd8e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -40,7 +40,6 @@ import java.util.OptionalLong; import java.util.Queue; import java.util.Set; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -447,13 +446,7 @@ public synchronized Map endOffsets(Collection partitions) { this.partitions = Collections.unmodifiableSet(new HashSet<>(partitions)); } - /** - * returns the first partition (out of {@link #partitions}) for which no offset is defined. - * @deprecated please use {@link #partitions} - * @return a partition with no offset - */ - @Deprecated - public TopicPartition partition() { - return partitions.isEmpty() ? null : partitions.iterator().next(); - } - /** * returns all partitions for which no offests are defined. * @return all partitions without offsets diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 15a62cd232cb5..6a0cb8a8179d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -29,7 +29,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; /** * The interface for the {@link KafkaProducer} @@ -100,11 +99,6 @@ void sendOffsetsToTransaction(Map offsets, */ void close(); - @Deprecated - default void close(long timeout, TimeUnit unit) { - close(Duration.ofMillis(unit.toMillis(timeout))); - } - /** * See {@link KafkaProducer#close(Duration)} */ diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 284ac0be37480..dcbb6d22199b2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -490,15 +490,6 @@ private static String parseAcks(String acksString) { } } - /** - * @deprecated Since 2.7.0. This will be removed in a future major release. - */ - @Deprecated - public static Map addSerializerToConfig(Map configs, - Serializer keySerializer, Serializer valueSerializer) { - return appendSerializerToConfig(configs, keySerializer, valueSerializer); - } - static Map appendSerializerToConfig(Map configs, Serializer keySerializer, Serializer valueSerializer) { @@ -510,22 +501,6 @@ static Map appendSerializerToConfig(Map configs, return newConfigs; } - /** - * @deprecated Since 2.7.0. This will be removed in a future major release. - */ - @Deprecated - public static Properties addSerializerToConfig(Properties properties, - Serializer keySerializer, - Serializer valueSerializer) { - Properties newProperties = new Properties(); - newProperties.putAll(properties); - if (keySerializer != null) - newProperties.put(KEY_SERIALIZER_CLASS_CONFIG, keySerializer.getClass().getName()); - if (valueSerializer != null) - newProperties.put(VALUE_SERIALIZER_CLASS_CONFIG, valueSerializer.getClass().getName()); - return newProperties; - } - public ProducerConfig(Properties props) { super(CONFIG, props); } diff --git a/clients/src/main/java/org/apache/kafka/common/MessageFormatter.java b/clients/src/main/java/org/apache/kafka/common/MessageFormatter.java index c4a255f315571..390aa61ca7250 100644 --- a/clients/src/main/java/org/apache/kafka/common/MessageFormatter.java +++ b/clients/src/main/java/org/apache/kafka/common/MessageFormatter.java @@ -19,7 +19,6 @@ import java.io.Closeable; import java.io.PrintStream; import java.util.Map; -import java.util.Properties; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -34,33 +33,21 @@ */ public interface MessageFormatter extends Configurable, Closeable { - /** - * Initialises the MessageFormatter - * @param props Properties to configure the formatter - * @deprecated Use {@link #configure(Map)} instead, this method is for backward compatibility with the older Formatter interface - */ - @Deprecated - default public void init(Properties props) {} - /** * Configures the MessageFormatter * @param configs Map to configure the formatter */ - default public void configure(Map configs) { - Properties properties = new Properties(); - properties.putAll(configs); - init(properties); - } + default void configure(Map configs) {} /** * Parses and formats a record for display * @param consumerRecord the record to format * @param output the print stream used to output the record */ - public void writeTo(ConsumerRecord consumerRecord, PrintStream output); + void writeTo(ConsumerRecord consumerRecord, PrintStream output); /** * Closes the formatter */ - default public void close() {} + default void close() {} } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ExtendedDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ExtendedDeserializer.java deleted file mode 100644 index 2f4a012fc8f06..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ExtendedDeserializer.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.common.serialization; - -import java.util.Map; - -import org.apache.kafka.common.header.Headers; - -/** - * A Deserializer that has access to the headers associated with the record. - * - * Prefer {@link Deserializer} if access to the headers is not required. Once Kafka drops support for Java 7, the - * {@code deserialize()} method introduced by this interface will be added to Deserializer with a default implementation - * so that backwards compatibility is maintained. This interface may be deprecated once that happens. - * - * A class that implements this interface is expected to have a constructor with no parameters. - * @param - * @deprecated This class has been deprecated and will be removed in a future release. Please use {@link Deserializer} instead. - */ -@Deprecated -public interface ExtendedDeserializer extends Deserializer { - - /** - * Deserialize a record value from a byte array into a value or object. - * @param topic topic associated with the data - * @param headers headers associated with the record; may be empty. - * @param data serialized bytes; may be null; implementations are recommended to handle null by returning a value or null rather than throwing an exception. - * @return deserialized typed data; may be null - */ - T deserialize(String topic, Headers headers, byte[] data); - - class Wrapper implements ExtendedDeserializer { - - private final Deserializer deserializer; - - public Wrapper(Deserializer deserializer) { - this.deserializer = deserializer; - } - - - @Override - public T deserialize(String topic, Headers headers, byte[] data) { - return deserialize(topic, data); - } - - @Override - public void configure(Map configs, boolean isKey) { - deserializer.configure(configs, isKey); - } - - @Override - public T deserialize(String topic, byte[] data) { - return deserializer.deserialize(topic, data); - } - - @Override - public void close() { - deserializer.close(); - } - - public static ExtendedDeserializer ensureExtended(Deserializer deserializer) { - return deserializer == null ? null : deserializer instanceof ExtendedDeserializer ? (ExtendedDeserializer) deserializer : new ExtendedDeserializer.Wrapper<>(deserializer); - } - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ExtendedSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ExtendedSerializer.java deleted file mode 100644 index 8c949807a03a5..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/serialization/ExtendedSerializer.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.common.serialization; - -import java.util.Map; - -import org.apache.kafka.common.header.Headers; - -/** - * A Serializer that has access to the headers associated with the record. - * - * Prefer {@link Serializer} if access to the headers is not required. Once Kafka drops support for Java 7, the - * {@code serialize()} method introduced by this interface will be added to Serializer with a default implementation - * so that backwards compatibility is maintained. This interface may be deprecated once that happens. - * - * A class that implements this interface is expected to have a constructor with no parameters. - * @param - * @deprecated This class has been deprecated and will be removed in a future release. Please use {@link Serializer} instead. - */ -@Deprecated -public interface ExtendedSerializer extends Serializer { - - /** - * Convert {@code data} into a byte array. - * - * @param topic topic associated with data - * @param headers headers associated with the record - * @param data typed data - * @return serialized bytes - */ - byte[] serialize(String topic, Headers headers, T data); - - class Wrapper implements ExtendedSerializer { - - private final Serializer serializer; - - public Wrapper(Serializer serializer) { - this.serializer = serializer; - } - - @Override - public byte[] serialize(String topic, Headers headers, T data) { - return serialize(topic, data); - } - - @Override - public void configure(Map configs, boolean isKey) { - serializer.configure(configs, isKey); - } - - @Override - public byte[] serialize(String topic, T data) { - return serializer.serialize(topic, data); - } - - @Override - public void close() { - serializer.close(); - } - - public static ExtendedSerializer ensureExtended(Serializer serializer) { - return serializer == null ? null : serializer instanceof ExtendedSerializer ? (ExtendedSerializer) serializer : new ExtendedSerializer.Wrapper<>(serializer); - } - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java index ef8981769716d..0ea9f85f1a808 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java @@ -68,34 +68,6 @@ public void testOverrideEnableAutoCommit() { } } - @SuppressWarnings("deprecation") - @Test - public void testDeserializerToPropertyConfig() { - Properties properties = new Properties(); - properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClassName); - properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClassName); - Properties newProperties = ConsumerConfig.addDeserializerToConfig(properties, null, null); - assertEquals(newProperties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), keyDeserializerClassName); - assertEquals(newProperties.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), valueDeserializerClassName); - - properties.clear(); - properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClassName); - newProperties = ConsumerConfig.addDeserializerToConfig(properties, keyDeserializer, null); - assertEquals(newProperties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), keyDeserializerClassName); - assertEquals(newProperties.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), valueDeserializerClassName); - - properties.clear(); - properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClassName); - newProperties = ConsumerConfig.addDeserializerToConfig(properties, null, valueDeserializer); - assertEquals(newProperties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), keyDeserializerClassName); - assertEquals(newProperties.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), valueDeserializerClassName); - - properties.clear(); - newProperties = ConsumerConfig.addDeserializerToConfig(properties, keyDeserializer, valueDeserializer); - assertEquals(newProperties.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG), keyDeserializerClassName); - assertEquals(newProperties.get(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG), valueDeserializerClassName); - } - @Test public void testAppendDeserializerToConfig() { Map configs = new HashMap<>(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 251170747cf9f..b125a91f70cad 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -142,11 +142,6 @@ 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.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.Mockito.doCallRealMethod; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; public class KafkaConsumerTest { private final String topic = "test"; @@ -2441,15 +2436,6 @@ private static class FetchInfo { } } - @Test - @SuppressWarnings("deprecation") - public void testCloseWithTimeUnit() { - KafkaConsumer consumer = mock(KafkaConsumer.class); - doCallRealMethod().when(consumer).close(anyLong(), any()); - consumer.close(1, TimeUnit.SECONDS); - verify(consumer).close(Duration.ofSeconds(1)); - } - @Test public void testSubscriptionOnInvalidTopic() { Time time = new MockTime(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 27afb09683ace..721db4d41f68d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -639,13 +639,6 @@ public void testTopicExpiryInMetadata() throws InterruptedException { } } - @SuppressWarnings("unchecked") - @Test - @Deprecated - public void testHeadersWithExtendedClasses() { - doTestHeaders(org.apache.kafka.common.serialization.ExtendedSerializer.class); - } - @SuppressWarnings("unchecked") @Test public void testHeaders() { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java index 1f5776fdfbbc0..a2f318bebc7a0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java @@ -23,7 +23,6 @@ import java.util.HashMap; import java.util.Map; -import java.util.Properties; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -31,39 +30,9 @@ public class ProducerConfigTest { private final Serializer keySerializer = new ByteArraySerializer(); private final Serializer valueSerializer = new StringSerializer(); - private final String keySerializerClassName = keySerializer.getClass().getName(); - private final String valueSerializerClassName = valueSerializer.getClass().getName(); private final Object keySerializerClass = keySerializer.getClass(); private final Object valueSerializerClass = valueSerializer.getClass(); - @SuppressWarnings("deprecation") - @Test - public void testSerializerToPropertyConfig() { - Properties properties = new Properties(); - properties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClassName); - properties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClassName); - Properties newProperties = ProducerConfig.addSerializerToConfig(properties, null, null); - assertEquals(newProperties.get(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), keySerializerClassName); - assertEquals(newProperties.get(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), valueSerializerClassName); - - properties.clear(); - properties.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClassName); - newProperties = ProducerConfig.addSerializerToConfig(properties, keySerializer, null); - assertEquals(newProperties.get(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), keySerializerClassName); - assertEquals(newProperties.get(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), valueSerializerClassName); - - properties.clear(); - properties.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClassName); - newProperties = ProducerConfig.addSerializerToConfig(properties, null, valueSerializer); - assertEquals(newProperties.get(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), keySerializerClassName); - assertEquals(newProperties.get(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), valueSerializerClassName); - - properties.clear(); - newProperties = ProducerConfig.addSerializerToConfig(properties, keySerializer, valueSerializer); - assertEquals(newProperties.get(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG), keySerializerClassName); - assertEquals(newProperties.get(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG), valueSerializerClassName); - } - @Test public void testAppendSerializerToConfig() { Map configs = new HashMap<>(); diff --git a/core/src/main/scala/kafka/common/MessageFormatter.scala b/core/src/main/scala/kafka/common/MessageFormatter.scala deleted file mode 100644 index 7826eb7264a13..0000000000000 --- a/core/src/main/scala/kafka/common/MessageFormatter.scala +++ /dev/null @@ -1,28 +0,0 @@ -/** - * 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 kafka.common - -/** - * Typical implementations of this interface convert a `ConsumerRecord` into a type that can then be passed to - * a `PrintStream`. - * - * This is used by the `ConsoleConsumer`. - */ -@deprecated("This class is deprecated and will be replaced by org.apache.kafka.common.MessageFormatter.", "2.7.0") -trait MessageFormatter extends org.apache.kafka.common.MessageFormatter { -} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index d4c8492e9a45c..a62476615e915 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -137,16 +137,6 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(Set(tp, tp2), consumer.assignment().asScala) } - @deprecated("Serializer now includes a default method that provides the headers", since = "2.1") - @Test - def testHeadersExtendedSerializerDeserializer(): Unit = { - val extendedSerializer = new ExtendedSerializer[Array[Byte]] with SerializerImpl - - val extendedDeserializer = new ExtendedDeserializer[Array[Byte]] with DeserializerImpl - - testHeadersSerializeDeserialize(extendedSerializer, extendedDeserializer) - } - @Test def testHeadersSerializerDeserializer(): Unit = { val extendedSerializer = new Serializer[Array[Byte]] with SerializerImpl diff --git a/docs/upgrade.html b/docs/upgrade.html index a486b6c4c910f..1eeb6e64baa33 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -27,18 +27,31 @@

Notable changes in 3 or updating the application not to use internal classes.
  • The Streams API removed all deprecated APIs that were deprecated in version 2.5.0 or earlier. For a complete list of removed APIs compare the detailed Kafka Streams upgrade notes.
  • -
  • The deprecated Scala Authorizer, SimpleAclAuthorizer and related classes have been removed. Please use the Java Authorizer - and AclAuthorizer instead.
  • -
  • The deprecated Metric#value() method was removed (KAFKA-12573).
  • -
  • Deprecated security classes were removed: PrincipalBuilder, DefaultPrincipalBuilder and ResourceFilter. - Furthermore, deprecated constants and constructors were removed from SslConfigs, SaslConfigs, - AclBinding and AclBindingFilter.
  • -
  • The deprecated Admin.electedPreferredLeaders() methods were removed. Please use Admin.electLeaders instead.
  • -
  • The deprecated kafka-preferred-replica-election command line tool was removed. Please use kafka-leader-election instead.
  • -
  • The deprecated ConfigEntry constructor was removed (KAFKA-12577). - Please use the remaining public constructor instead.
  • -
  • The deprecated config value default for the client config client.dns.lookup has been removed. In the unlikely - event that you set this config explicitly, we recommend leaving the config unset (use_all_dns_ips is used by default).
  • +
  • A number of deprecated classes, methods and tools have been removed from the clients, core and tools modules:
  • +
      +
    • The Scala Authorizer, SimpleAclAuthorizer and related classes have been removed. Please use the Java Authorizer + and AclAuthorizer instead.
    • +
    • The Metric#value() method was removed (KAFKA-12573).
    • +
    • The PrincipalBuilder, DefaultPrincipalBuilder and ResourceFilter classes were removed. +
    • Various constants and constructors were removed from SslConfigs, SaslConfigs, AclBinding and + AclBindingFilter.
    • +
    • The Admin.electedPreferredLeaders() methods were removed. Please use Admin.electLeaders instead.
    • +
    • The kafka-preferred-replica-election command line tool was removed. Please use kafka-leader-election instead.
    • +
    • The ConfigEntry constructor was removed (KAFKA-12577). + Please use the remaining public constructor instead.
    • +
    • The config value default for the client config client.dns.lookup has been removed. In the unlikely + event that you set this config explicitly, we recommend leaving the config unset (use_all_dns_ips is used by default).
    • +
    • The ExtendedDeserializer and ExtendedSerializer classes have been removed. Please use Deserializer + and Serializer instead.
    • +
    • The close(long, TimeUnit) method was removed from the producer, consumer and admin client. Please use + close(Duration).
    • +
    • The ConsumerConfig.addDeserializerToConfig and ProducerConfig.addSerializerToConfig methods + were removed. These methods were not intended to be public API and there is no replacement.
    • +
    • The NoOffsetForPartitionException.partition() method was removed. Please use partitions() + instead.
    • +
    • The Scala kafka.common.MessageFormatter was removed. Plese use the Java org.apache.kafka.common.MessageFormatter.
    • +
    • The MessageFormatter.init(Properties) method was removed. Please use configure(Map) instead.
    • +
    Notable changes in 2.8.0
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 0d56087a3759d..c378de6526e68 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -530,10 +530,10 @@ public StreamThread(final Time time, this.numIterations = 1; } - @SuppressWarnings("deprecation") private static final class InternalConsumerConfig extends ConsumerConfig { private InternalConsumerConfig(final Map props) { - super(ConsumerConfig.addDeserializerToConfig(props, new ByteArrayDeserializer(), new ByteArrayDeserializer()), false); + super(ConsumerConfig.appendDeserializerToConfig(props, new ByteArrayDeserializer(), + new ByteArrayDeserializer()), false); } } From 137491c7248c90327b72eda88fa26d18c4a589f0 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 6 Apr 2021 09:49:14 -0700 Subject: [PATCH 009/155] MINOR: Support ExponentialBackoff without jitter (#10455) It is useful to allow ExponentialBackoff to be configured to work without jitter, in order to make unit tests more repeatable. Reviewers: David Arthur --- .../apache/kafka/common/utils/ExponentialBackoff.java | 3 ++- .../kafka/common/utils/ExponentialBackoffTest.java | 9 +++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/ExponentialBackoff.java b/clients/src/main/java/org/apache/kafka/common/utils/ExponentialBackoff.java index 73972c834ac8a..7550184ba39d0 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/ExponentialBackoff.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/ExponentialBackoff.java @@ -47,7 +47,8 @@ public long backoff(long attempts) { } double exp = Math.min(attempts, this.expMax); double term = initialInterval * Math.pow(multiplier, exp); - double randomFactor = ThreadLocalRandom.current().nextDouble(1 - jitter, 1 + jitter); + double randomFactor = jitter < Double.MIN_NORMAL ? 1.0 : + ThreadLocalRandom.current().nextDouble(1 - jitter, 1 + jitter); return (long) (randomFactor * term); } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/ExponentialBackoffTest.java b/clients/src/test/java/org/apache/kafka/common/utils/ExponentialBackoffTest.java index efa7d0cae845a..4e843863ab5c7 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/ExponentialBackoffTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/ExponentialBackoffTest.java @@ -45,4 +45,13 @@ public void testExponentialBackoff() { } } } + + @Test + public void testExponentialBackoffWithoutJitter() { + ExponentialBackoff exponentialBackoff = new ExponentialBackoff(100, 2, 400, 0.0); + assertEquals(100, exponentialBackoff.backoff(0)); + assertEquals(200, exponentialBackoff.backoff(1)); + assertEquals(400, exponentialBackoff.backoff(2)); + assertEquals(400, exponentialBackoff.backoff(3)); + } } From 7bc84d6ced71056dbb4cecdc9abbdbd7d8a5aa10 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 6 Apr 2021 10:18:06 -0700 Subject: [PATCH 010/155] KAFKA-12467: Implement QuorumController snapshot generation (#10366) Implement controller-side snapshot generation.Implement QuorumController snapshot generation. Note that this PR does not handle KRaft integration, just the internal snapshot record generation and consumption logic. Reading a snapshot is relatively straightforward. When the QuorumController starts up, it loads the most recent snapshot. This is just a series of records that we replay, plus a log offset ("snapshot epoch") that we advance to. Writing a snapshot is more complex. There are several components: the SnapshotWriter which persists the snapshot, the SnapshotGenerator which manages writing each batch of records, and the SnapshotGeneratorManager which interfaces the preceding two classes with the event queue. Controller snapshots are done incrementally. In order to avoid blocking the controller thread for a long time, we pull a few record batches at a time from our record batch iterators. These iterators are implemented by controller manager classes such as ReplicationControlManager, ClusterControlManager, etc. Finally, this PR adds ControllerTestUtils#deepSortRecords and ControllerTestUtils#assertBatchIteratorContains, which make it easier to write unit tests. Since records are often constructed from unsorted data structures, it is often useful to sort them before comparing them. Reviewers: David Arthur --- checkstyle/suppressions.xml | 2 + .../test/java/kafka/test/MockController.java | 5 + .../controller/ClientQuotaControlManager.java | 58 +++++- .../controller/ClusterControlManager.java | 67 ++++++- .../ConfigurationControlManager.java | 40 ++++ .../apache/kafka/controller/Controller.java | 8 + .../kafka/controller/EmptySnapshotReader.java | 52 +++++ .../controller/FeatureControlManager.java | 51 +++-- .../kafka/controller/QuorumController.java | 187 ++++++++++++++++-- .../controller/ReplicationControlManager.java | 44 +++++ .../kafka/controller/SnapshotGenerator.java | 146 ++++++++++++++ .../kafka/controller/SnapshotReader.java | 37 ++++ .../kafka/controller/SnapshotWriter.java | 55 ++++++ .../kafka/timeline/SnapshotRegistry.java | 4 + .../ClientQuotaControlManagerTest.java | 48 ++++- .../controller/ClusterControlManagerTest.java | 63 +++++- .../ConfigurationControlManagerTest.java | 10 +- .../kafka/controller/ControllerTestUtils.java | 51 +++++ .../controller/FeatureControlManagerTest.java | 26 ++- .../kafka/controller/MockSnapshotReader.java | 55 ++++++ .../kafka/controller/MockSnapshotWriter.java | 94 +++++++++ .../kafka/controller/NoOpSnapshotWriter.java | 49 +++++ .../controller/NoOpSnapshotWriterBuilder.java | 28 +++ .../controller/QuorumControllerTest.java | 146 ++++++++++++++ .../ReplicationControlManagerTest.java | 11 ++ .../controller/SnapshotGeneratorTest.java | 110 +++++++++++ 26 files changed, 1394 insertions(+), 53 deletions(-) create mode 100644 metadata/src/main/java/org/apache/kafka/controller/EmptySnapshotReader.java create mode 100644 metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java create mode 100644 metadata/src/main/java/org/apache/kafka/controller/SnapshotReader.java create mode 100644 metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java create mode 100644 metadata/src/test/java/org/apache/kafka/controller/MockSnapshotReader.java create mode 100644 metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java create mode 100644 metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriter.java create mode 100644 metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java create mode 100644 metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 5473b035ea919..2ed1fac5df95d 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -268,6 +268,8 @@ files="(ReplicationControlManager|ReplicationControlManagerTest).java"/> + waitForReadyBrokers(int minBrokers) { throw new UnsupportedOperationException(); } + @Override + public CompletableFuture beginWritingSnapshot() { + throw new UnsupportedOperationException(); + } + @Override public void beginShutdown() { this.active = false; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java index 9b8e2d683b650..f6a24973b58f2 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.config.internals.QuotaConfigs; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.metadata.QuotaRecord; +import org.apache.kafka.common.metadata.QuotaRecord.EntityData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; @@ -35,18 +36,20 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.Objects; import java.util.function.Supplier; import java.util.stream.Collectors; public class ClientQuotaControlManager { - private final SnapshotRegistry snapshotRegistry; - final TimelineHashMap> clientQuotaData; + final TimelineHashMap> clientQuotaData; ClientQuotaControlManager(SnapshotRegistry snapshotRegistry) { this.snapshotRegistry = snapshotRegistry; @@ -98,7 +101,7 @@ public void replay(QuotaRecord record) { Map entityMap = new HashMap<>(2); record.entity().forEach(entityData -> entityMap.put(entityData.entityType(), entityData.entityName())); ClientQuotaEntity entity = new ClientQuotaEntity(entityMap); - Map quotas = clientQuotaData.get(entity); + TimelineHashMap quotas = clientQuotaData.get(entity); if (quotas == null) { quotas = new TimelineHashMap<>(snapshotRegistry, 0); clientQuotaData.put(entity, quotas); @@ -136,14 +139,15 @@ private void alterClientQuotaEntity( } // Don't share objects between different records - Supplier> recordEntitySupplier = () -> - validatedEntityMap.entrySet().stream().map(mapEntry -> new QuotaRecord.EntityData() + Supplier> recordEntitySupplier = () -> + validatedEntityMap.entrySet().stream().map(mapEntry -> new EntityData() .setEntityType(mapEntry.getKey()) .setEntityName(mapEntry.getValue())) .collect(Collectors.toList()); List newRecords = new ArrayList<>(newQuotaConfigs.size()); - Map currentQuotas = clientQuotaData.getOrDefault(entity, Collections.emptyMap()); + Map currentQuotas = clientQuotaData.containsKey(entity) ? + clientQuotaData.get(entity) : Collections.emptyMap(); newQuotaConfigs.forEach((key, newValue) -> { if (newValue == null) { if (currentQuotas.containsKey(key)) { @@ -249,7 +253,7 @@ private ApiError validateEntity(ClientQuotaEntity entity, Map va return new ApiError(Errors.INVALID_REQUEST, "Invalid empty client quota entity"); } - for (Map.Entry entityEntry : entity.entries().entrySet()) { + for (Entry entityEntry : entity.entries().entrySet()) { String entityType = entityEntry.getKey(); String entityName = entityEntry.getValue(); if (validatedEntityMap.containsKey(entityType)) { @@ -272,4 +276,44 @@ private ApiError validateEntity(ClientQuotaEntity entity, Map va return ApiError.NONE; } + + class ClientQuotaControlIterator implements Iterator> { + private final long epoch; + private final Iterator>> iterator; + + ClientQuotaControlIterator(long epoch) { + this.epoch = epoch; + this.iterator = clientQuotaData.entrySet(epoch).iterator(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + if (!hasNext()) throw new NoSuchElementException(); + Entry> entry = iterator.next(); + ClientQuotaEntity entity = entry.getKey(); + List records = new ArrayList<>(); + for (Entry quotaEntry : entry.getValue().entrySet(epoch)) { + QuotaRecord record = new QuotaRecord(); + for (Entry entityEntry : entity.entries().entrySet()) { + record.entity().add(new EntityData(). + setEntityType(entityEntry.getKey()). + setEntityName(entityEntry.getValue())); + } + record.setKey(quotaEntry.getKey()); + record.setValue(quotaEntry.getValue()); + record.setRemove(false); + records.add(new ApiMessageAndVersion(record, (short) 0)); + } + return records; + } + } + + ClientQuotaControlIterator iterator(long epoch) { + return new ClientQuotaControlIterator(epoch); + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 4748d195986ab..05380fc49e421 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -24,6 +24,10 @@ import org.apache.kafka.common.message.BrokerRegistrationRequestData; import org.apache.kafka.common.metadata.FenceBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpointCollection; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerFeature; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerFeatureCollection; import org.apache.kafka.common.metadata.UnfenceBrokerRecord; import org.apache.kafka.common.metadata.UnregisterBrokerRecord; import org.apache.kafka.common.security.auth.SecurityProtocol; @@ -40,8 +44,11 @@ import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -184,7 +191,7 @@ public ControllerResult registerBroker( setBrokerEpoch(brokerEpoch). setRack(request.rack()); for (BrokerRegistrationRequestData.Listener listener : request.listeners()) { - record.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint(). + record.endPoints().add(new BrokerEndpoint(). setHost(listener.host()). setName(listener.name()). setPort(listener.port()). @@ -199,7 +206,7 @@ public ControllerResult registerBroker( "the broker has an unsupported version of " + feature.name()); } } - record.features().add(new RegisterBrokerRecord.BrokerFeature(). + record.features().add(new BrokerFeature(). setName(feature.name()). setMinSupportedVersion(feature.minSupportedVersion()). setMaxSupportedVersion(feature.maxSupportedVersion())); @@ -219,13 +226,13 @@ public ControllerResult registerBroker( public void replay(RegisterBrokerRecord record) { int brokerId = record.brokerId(); List listeners = new ArrayList<>(); - for (RegisterBrokerRecord.BrokerEndpoint endpoint : record.endPoints()) { + for (BrokerEndpoint endpoint : record.endPoints()) { listeners.add(new Endpoint(endpoint.name(), SecurityProtocol.forId(endpoint.securityProtocol()), endpoint.host(), endpoint.port())); } Map features = new HashMap<>(); - for (RegisterBrokerRecord.BrokerFeature feature : record.features()) { + for (BrokerFeature feature : record.features()) { features.put(feature.name(), new VersionRange( feature.minSupportedVersion(), feature.maxSupportedVersion())); } @@ -343,4 +350,56 @@ public void addReadyBrokersFuture(CompletableFuture future, int minBrokers readyBrokersFuture = Optional.empty(); } } + + class ClusterControlIterator implements Iterator> { + private final Iterator> iterator; + + ClusterControlIterator(long epoch) { + this.iterator = brokerRegistrations.entrySet(epoch).iterator(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + if (!hasNext()) throw new NoSuchElementException(); + Entry entry = iterator.next(); + int brokerId = entry.getKey(); + BrokerRegistration registration = entry.getValue(); + BrokerEndpointCollection endpoints = new BrokerEndpointCollection(); + for (Entry endpointEntry : registration.listeners().entrySet()) { + endpoints.add(new BrokerEndpoint().setName(endpointEntry.getKey()). + setHost(endpointEntry.getValue().host()). + setPort(endpointEntry.getValue().port()). + setSecurityProtocol(endpointEntry.getValue().securityProtocol().id)); + } + BrokerFeatureCollection features = new BrokerFeatureCollection(); + for (Entry featureEntry : registration.supportedFeatures().entrySet()) { + features.add(new BrokerFeature().setName(featureEntry.getKey()). + setMaxSupportedVersion(featureEntry.getValue().max()). + setMinSupportedVersion(featureEntry.getValue().min())); + } + List batch = new ArrayList<>(); + batch.add(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerId(brokerId). + setIncarnationId(registration.incarnationId()). + setBrokerEpoch(registration.epoch()). + setEndPoints(endpoints). + setFeatures(features). + setRack(registration.rack().orElse(null)), (short) 0)); + if (!registration.fenced()) { + batch.add(new ApiMessageAndVersion(new UnfenceBrokerRecord(). + setId(brokerId). + setEpoch(registration.epoch()), (short) 0)); + } + return batch; + } + } + + ClusterControlIterator iterator(long epoch) { + return new ClusterControlIterator(epoch); + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index dcfe92d46a54b..3e9e9e1d593c8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -40,10 +40,12 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.Objects; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.APPEND; + public class ConfigurationControlManager { private final Logger log; private final SnapshotRegistry snapshotRegistry; @@ -316,6 +318,9 @@ public void replay(ConfigRecord record) { } else { configs.put(record.name(), record.value()); } + if (configs.isEmpty()) { + configData.remove(configResource); + } log.info("{}: set configuration {} to {}", configResource, record.name(), record.value()); } @@ -368,4 +373,39 @@ public Map>> describeConfigs( void deleteTopicConfigs(String name) { configData.remove(new ConfigResource(Type.TOPIC, name)); } + + class ConfigurationControlIterator implements Iterator> { + private final long epoch; + private final Iterator>> iterator; + + ConfigurationControlIterator(long epoch) { + this.epoch = epoch; + this.iterator = configData.entrySet(epoch).iterator(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + if (!hasNext()) throw new NoSuchElementException(); + List records = new ArrayList<>(); + Entry> entry = iterator.next(); + ConfigResource resource = entry.getKey(); + for (Entry configEntry : entry.getValue().entrySet(epoch)) { + records.add(new ApiMessageAndVersion(new ConfigRecord(). + setResourceName(resource.name()). + setResourceType(resource.type().id()). + setName(configEntry.getKey()). + setValue(configEntry.getValue()), (short) 0)); + } + return records; + } + } + + ConfigurationControlIterator iterator(long epoch) { + return new ConfigurationControlIterator(epoch); + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java index 26394632e6a94..7892c1991ee9a 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java @@ -187,6 +187,14 @@ CompletableFuture> alterClientQuotas( Collection quotaAlterations, boolean validateOnly ); + /** + * Begin writing a controller snapshot. If there was already an ongoing snapshot, it + * simply returns information about that snapshot rather than starting a new one. + * + * @return A future yielding the epoch of the snapshot. + */ + CompletableFuture beginWritingSnapshot(); + /** * Begin shutting down, but don't block. You must still call close to clean up all * resources. diff --git a/metadata/src/main/java/org/apache/kafka/controller/EmptySnapshotReader.java b/metadata/src/main/java/org/apache/kafka/controller/EmptySnapshotReader.java new file mode 100644 index 0000000000000..cf334c75ec0ee --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/EmptySnapshotReader.java @@ -0,0 +1,52 @@ +/* + * 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.controller; + +import java.util.List; +import java.util.NoSuchElementException; + +import org.apache.kafka.common.protocol.ApiMessage; + + +public class EmptySnapshotReader implements SnapshotReader { + private final long epoch; + + public EmptySnapshotReader(long epoch) { + this.epoch = epoch; + } + + @Override + public long epoch() { + return epoch; + } + + @Override + public void close() { + // Nothing to do + } + + @Override + public boolean hasNext() { + return false; + } + + @Override + public List next() { + throw new NoSuchElementException(); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index 99874ac3c5ef7..fb6d7dbcce740 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -18,13 +18,16 @@ package org.apache.kafka.controller; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map.Entry; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Set; import java.util.TreeMap; + import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; @@ -34,12 +37,11 @@ import org.apache.kafka.metadata.VersionRange; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; -import org.apache.kafka.timeline.TimelineHashSet; public class FeatureControlManager { /** - * The features supported by this controller's software. + * An immutable map containing the features supported by this controller's software. */ private final Map supportedFeatures; @@ -48,16 +50,10 @@ public class FeatureControlManager { */ private final TimelineHashMap finalizedVersions; - /** - * The latest feature epoch. - */ - private final TimelineHashSet epoch; - FeatureControlManager(Map supportedFeatures, SnapshotRegistry snapshotRegistry) { this.supportedFeatures = supportedFeatures; this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0); - this.epoch = new TimelineHashSet<>(snapshotRegistry, 0); } ControllerResult> updateFeatures( @@ -120,18 +116,39 @@ FeatureMapAndEpoch finalizedFeatures(long lastCommittedOffset) { for (Entry entry : finalizedVersions.entrySet(lastCommittedOffset)) { features.put(entry.getKey(), entry.getValue()); } - long currentEpoch = -1; - Iterator iterator = epoch.iterator(lastCommittedOffset); - if (iterator.hasNext()) { - currentEpoch = iterator.next(); - } - return new FeatureMapAndEpoch(new FeatureMap(features), currentEpoch); + return new FeatureMapAndEpoch(new FeatureMap(features), lastCommittedOffset); } - void replay(FeatureLevelRecord record, long offset) { + public void replay(FeatureLevelRecord record) { finalizedVersions.put(record.name(), new VersionRange(record.minFeatureLevel(), record.maxFeatureLevel())); - epoch.clear(); - epoch.add(offset); + } + + class FeatureControlIterator implements Iterator> { + private final Iterator> iterator; + + FeatureControlIterator(long epoch) { + this.iterator = finalizedVersions.entrySet(epoch).iterator(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + if (!hasNext()) throw new NoSuchElementException(); + Entry entry = iterator.next(); + VersionRange versions = entry.getValue(); + return Collections.singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName(entry.getKey()). + setMinFeatureLevel(versions.min()). + setMaxFeatureLevel(versions.max()), (short) 0)); + } + } + + FeatureControlIterator iterator(long epoch) { + return new FeatureControlIterator(epoch); } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 4f2b708bfe5a4..9730f0ca898af 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -17,16 +17,19 @@ package org.apache.kafka.controller; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map.Entry; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Random; import java.util.concurrent.CompletableFuture; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.kafka.clients.admin.AlterConfigOp.OpType; @@ -60,9 +63,11 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.requests.ApiError; +import org.apache.kafka.common.utils.ExponentialBackoff; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.controller.SnapshotGenerator.Section; import org.apache.kafka.metadata.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; @@ -113,6 +118,8 @@ static public class Builder { private int defaultNumPartitions = 1; private ReplicaPlacementPolicy replicaPlacementPolicy = new SimpleReplicaPlacementPolicy(new Random()); + private Function snapshotWriterBuilder; + private SnapshotReader snapshotReader; private long sessionTimeoutNs = NANOSECONDS.convert(18, TimeUnit.SECONDS); private ControllerMetrics controllerMetrics = null; @@ -165,6 +172,16 @@ public Builder setReplicaPlacementPolicy(ReplicaPlacementPolicy replicaPlacement return this; } + public Builder setSnapshotWriterBuilder(Function snapshotWriterBuilder) { + this.snapshotWriterBuilder = snapshotWriterBuilder; + return this; + } + + public Builder setSnapshotReader(SnapshotReader snapshotReader) { + this.snapshotReader = snapshotReader; + return this; + } + public Builder setSessionTimeoutNs(long sessionTimeoutNs) { this.sessionTimeoutNs = sessionTimeoutNs; return this; @@ -175,6 +192,7 @@ public Builder setMetrics(ControllerMetrics controllerMetrics) { return this; } + @SuppressWarnings("unchecked") public QuorumController build() throws Exception { if (logManager == null) { throw new RuntimeException("You must set a metadata log manager."); @@ -189,16 +207,25 @@ public QuorumController build() throws Exception { controllerMetrics = (ControllerMetrics) Class.forName( "org.apache.kafka.controller.MockControllerMetrics").getConstructor().newInstance(); } + if (snapshotWriterBuilder == null) { + snapshotWriterBuilder = (Function) Class.forName( + "org.apache.kafka.controller.NoOpSnapshotWriterBuilder").getConstructor().newInstance(); + } + if (snapshotReader == null) { + snapshotReader = new EmptySnapshotReader(-1); + } KafkaEventQueue queue = null; try { queue = new KafkaEventQueue(time, logContext, threadNamePrefix); return new QuorumController(logContext, nodeId, queue, time, configDefs, - logManager, supportedFeatures, defaultReplicationFactor, - defaultNumPartitions, replicaPlacementPolicy, sessionTimeoutNs, - controllerMetrics); + logManager, supportedFeatures, defaultReplicationFactor, + defaultNumPartitions, replicaPlacementPolicy, snapshotWriterBuilder, + snapshotReader, sessionTimeoutNs, controllerMetrics); } catch (Exception e) { Utils.closeQuietly(queue, "event queue"); throw e; + } finally { + Utils.closeQuietly(snapshotReader, "snapshotReader"); } } } @@ -301,6 +328,93 @@ private void appendControlEvent(String name, Runnable handler) { queue.append(event); } + private static final String GENERATE_SNAPSHOT = "generateSnapshot"; + + private static final int MAX_BATCHES_PER_GENERATE_CALL = 10; + + class SnapshotGeneratorManager implements Runnable { + private final Function writerBuilder; + private final ExponentialBackoff exponentialBackoff = + new ExponentialBackoff(10, 2, 5000, 0); + private SnapshotGenerator generator = null; + + SnapshotGeneratorManager(Function writerBuilder) { + this.writerBuilder = writerBuilder; + } + + void createSnapshotGenerator(long epoch) { + if (generator != null) { + throw new RuntimeException("Snapshot generator already exists."); + } + if (!snapshotRegistry.hasSnapshot(epoch)) { + throw new RuntimeException("Can't generate a snapshot at epoch " + epoch + + " because no such epoch exists in the snapshot registry."); + } + generator = new SnapshotGenerator(logContext, + writerBuilder.apply(epoch), + MAX_BATCHES_PER_GENERATE_CALL, + exponentialBackoff, + Arrays.asList( + new Section("features", featureControl.iterator(epoch)), + new Section("cluster", clusterControl.iterator(epoch)), + new Section("replication", replicationControl.iterator(epoch)), + new Section("configuration", configurationControl.iterator(epoch)), + new Section("clientQuotas", clientQuotaControlManager.iterator(epoch)))); + reschedule(0); + } + + void cancel() { + if (generator == null) return; + log.error("Cancelling snapshot {}", generator.epoch()); + generator.writer().close(); + generator = null; + queue.cancelDeferred(GENERATE_SNAPSHOT); + } + + void reschedule(long delayNs) { + ControlEvent event = new ControlEvent(GENERATE_SNAPSHOT, this); + queue.scheduleDeferred(event.name, + new EarliestDeadlineFunction(time.nanoseconds() + delayNs), event); + } + + @Override + public void run() { + if (generator == null) { + log.debug("No snapshot is in progress."); + return; + } + OptionalLong nextDelay; + try { + nextDelay = generator.generateBatches(); + } catch (Exception e) { + log.error("Error while generating snapshot {}", generator.epoch(), e); + generator.writer().close(); + generator = null; + return; + } + if (!nextDelay.isPresent()) { + try { + generator.writer().completeSnapshot(); + log.info("Finished generating snapshot {}.", generator.epoch()); + } catch (Exception e) { + log.error("Error while completing snapshot {}", generator.epoch(), e); + } finally { + generator.writer().close(); + generator = null; + } + return; + } + reschedule(nextDelay.getAsLong()); + } + + long snapshotEpoch() { + if (generator == null) { + return Long.MAX_VALUE; + } + return generator.epoch(); + } + } + /** * A controller event that reads the committed internal state in order to expose it * to an API. @@ -451,7 +565,7 @@ public void run() throws Exception { writeOffset = offset; resultAndOffset = ControllerResultAndOffset.of(offset, result); for (ApiMessageAndVersion message : result.records()) { - replay(message.message(), offset); + replay(message.message(), -1, offset); } snapshotRegistry.createSnapshot(offset); log.debug("Read-write operation {} will be completed when the log " + @@ -516,7 +630,7 @@ public void handleCommits(long offset, List messages) { } } for (ApiMessage message : messages) { - replay(message, offset); + replay(message, -1, offset); } } else { // If the controller is active, the records were already replayed, @@ -526,9 +640,11 @@ public void handleCommits(long offset, List messages) { // Complete any events in the purgatory that were waiting for this offset. purgatory.completeUpTo(offset); - // Delete all snapshots older than the offset. - // TODO: add an exception here for when we're writing out a log snapshot - snapshotRegistry.deleteSnapshotsUpTo(offset); + // Delete all the in-memory snapshots that we no longer need. + // If we are writing a new snapshot, then we need to keep that around; + // otherwise, we should delete up to the current committed offset. + snapshotRegistry.deleteSnapshotsUpTo( + Math.min(offset, snapshotGeneratorManager.snapshotEpoch())); } lastCommittedOffset = offset; }); @@ -545,7 +661,7 @@ public void handleNewLeader(MetaLogLeader newLeader) { newEpoch + ", but we never renounced controller epoch " + curEpoch); } - log.info("Becoming active at controller epoch {}.", newEpoch); + log.warn("Becoming active at controller epoch {}.", newEpoch); curClaimEpoch = newEpoch; controllerMetrics.setActive(true); writeOffset = lastCommittedOffset; @@ -558,9 +674,9 @@ public void handleNewLeader(MetaLogLeader newLeader) { public void handleRenounce(long oldEpoch) { appendControlEvent("handleRenounce[" + oldEpoch + "]", () -> { if (curClaimEpoch == oldEpoch) { - log.info("Renouncing the leadership at oldEpoch {} due to a metadata " + - "log event. Reverting to last committed offset {}.", curClaimEpoch, - lastCommittedOffset); + log.warn("Renouncing the leadership at oldEpoch {} due to a metadata " + + "log event. Reverting to last committed offset {}.", curClaimEpoch, + lastCommittedOffset); renounce(); } }); @@ -626,7 +742,7 @@ private void cancelMaybeFenceReplicas() { } @SuppressWarnings("unchecked") - private void replay(ApiMessage message, long offset) { + private void replay(ApiMessage message, long snapshotEpoch, long offset) { try { MetadataRecordType type = MetadataRecordType.fromId(message.apiKey()); switch (type) { @@ -658,7 +774,7 @@ private void replay(ApiMessage message, long offset) { replicationControl.replay((RemoveTopicRecord) message); break; case FEATURE_LEVEL_RECORD: - featureControl.replay((FeatureLevelRecord) message, offset); + featureControl.replay((FeatureLevelRecord) message); break; case QUOTA_RECORD: clientQuotaControlManager.replay((QuotaRecord) message); @@ -667,10 +783,18 @@ private void replay(ApiMessage message, long offset) { throw new RuntimeException("Unhandled record type " + type); } } catch (Exception e) { - log.error("Error replaying record {}", message.toString(), e); + if (snapshotEpoch < 0) { + log.error("Error replaying record {} at offset {}.", + message.toString(), offset, e); + } else { + log.error("Error replaying record {} from snapshot {} at index {}.", + message.toString(), snapshotEpoch, offset, e); + } } } + private final LogContext logContext; + private final Logger log; /** @@ -735,6 +859,11 @@ private void replay(ApiMessage message, long offset) { */ private final ReplicationControlManager replicationControl; + /** + * Manages generating controller snapshots. + */ + private final SnapshotGeneratorManager snapshotGeneratorManager; + /** * The interface that we use to mutate the Raft log. */ @@ -773,15 +902,17 @@ private QuorumController(LogContext logContext, short defaultReplicationFactor, int defaultNumPartitions, ReplicaPlacementPolicy replicaPlacementPolicy, + Function snapshotWriterBuilder, + SnapshotReader snapshotReader, long sessionTimeoutNs, ControllerMetrics controllerMetrics) throws Exception { + this.logContext = logContext; this.log = logContext.logger(QuorumController.class); this.nodeId = nodeId; this.queue = queue; this.time = time; this.controllerMetrics = controllerMetrics; this.snapshotRegistry = new SnapshotRegistry(logContext); - snapshotRegistry.createSnapshot(-1); this.purgatory = new ControllerPurgatory(); this.configurationControl = new ConfigurationControlManager(logContext, snapshotRegistry, configDefs); @@ -789,14 +920,24 @@ private QuorumController(LogContext logContext, this.clusterControl = new ClusterControlManager(logContext, time, snapshotRegistry, sessionTimeoutNs, replicaPlacementPolicy); this.featureControl = new FeatureControlManager(supportedFeatures, snapshotRegistry); + this.snapshotGeneratorManager = new SnapshotGeneratorManager(snapshotWriterBuilder); this.replicationControl = new ReplicationControlManager(snapshotRegistry, logContext, defaultReplicationFactor, defaultNumPartitions, configurationControl, clusterControl); this.logManager = logManager; this.metaLogListener = new QuorumMetaLogListener(); this.curClaimEpoch = -1L; - this.lastCommittedOffset = -1L; + this.lastCommittedOffset = snapshotReader.epoch(); this.writeOffset = -1L; + + while (snapshotReader.hasNext()) { + List batch = snapshotReader.next(); + long index = 0; + for (ApiMessage message : batch) { + replay(message, snapshotReader.epoch(), index++); + } + } + snapshotRegistry.createSnapshot(lastCommittedOffset); this.logManager.register(metaLogListener); } @@ -954,6 +1095,18 @@ public CompletableFuture> alterClientQuotas( }); } + @Override + public CompletableFuture beginWritingSnapshot() { + CompletableFuture future = new CompletableFuture<>(); + appendControlEvent("beginWritingSnapshot", () -> { + if (snapshotGeneratorManager.generator == null) { + snapshotGeneratorManager.createSnapshotGenerator(lastCommittedOffset); + } + future.complete(snapshotGeneratorManager.generator.epoch()); + }); + return future; + } + @Override public CompletableFuture waitForReadyBrokers(int minBrokers) { final CompletableFuture future = new CompletableFuture<>(); diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index 97dee1c590887..10e849091931f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -71,6 +71,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.Objects; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; @@ -1005,4 +1006,47 @@ ControllerResult maybeFenceStaleBrokers() { } return ControllerResult.of(records, null); } + + class ReplicationControlIterator implements Iterator> { + private final long epoch; + private final Iterator iterator; + + ReplicationControlIterator(long epoch) { + this.epoch = epoch; + this.iterator = topics.values(epoch).iterator(); + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + if (!hasNext()) throw new NoSuchElementException(); + TopicControlInfo topic = iterator.next(); + List records = new ArrayList<>(); + records.add(new ApiMessageAndVersion(new TopicRecord(). + setName(topic.name). + setTopicId(topic.id), (short) 0)); + for (Entry entry : topic.parts.entrySet(epoch)) { + PartitionControlInfo partition = entry.getValue(); + records.add(new ApiMessageAndVersion(new PartitionRecord(). + setPartitionId(entry.getKey()). + setTopicId(topic.id). + setReplicas(Replicas.toList(partition.replicas)). + setIsr(Replicas.toList(partition.isr)). + setRemovingReplicas(Replicas.toList(partition.removingReplicas)). + setAddingReplicas(Replicas.toList(partition.addingReplicas)). + setLeader(partition.leader). + setLeaderEpoch(partition.leaderEpoch). + setPartitionEpoch(partition.partitionEpoch), (short) 0)); + } + return records; + } + } + + ReplicationControlIterator iterator(long epoch) { + return new ReplicationControlIterator(epoch); + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java b/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java new file mode 100644 index 0000000000000..91295c2a2f769 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java @@ -0,0 +1,146 @@ +/* + * 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.controller; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.OptionalLong; + +import org.apache.kafka.common.utils.ExponentialBackoff; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.slf4j.Logger; + + +final class SnapshotGenerator { + static class Section { + private final String name; + private final Iterator> iterator; + + Section(String name, Iterator> iterator) { + this.name = name; + this.iterator = iterator; + } + + String name() { + return name; + } + + Iterator> iterator() { + return iterator; + } + } + + private final Logger log; + private final SnapshotWriter writer; + private final int maxBatchesPerGenerateCall; + private final ExponentialBackoff exponentialBackoff; + private final List
    sections; + private final Iterator
    sectionIterator; + private Iterator> batchIterator; + private List batch; + private Section section; + private long numRecords; + private long numWriteTries; + + SnapshotGenerator(LogContext logContext, + SnapshotWriter writer, + int maxBatchesPerGenerateCall, + ExponentialBackoff exponentialBackoff, + List
    sections) { + this.log = logContext.logger(SnapshotGenerator.class); + this.writer = writer; + this.maxBatchesPerGenerateCall = maxBatchesPerGenerateCall; + this.exponentialBackoff = exponentialBackoff; + this.sections = sections; + this.sectionIterator = this.sections.iterator(); + this.batchIterator = Collections.emptyIterator(); + this.batch = null; + this.section = null; + this.numRecords = 0; + this.numWriteTries = 0; + } + + /** + * Returns the epoch of the snapshot that we are generating. + */ + long epoch() { + return writer.epoch(); + } + + SnapshotWriter writer() { + return writer; + } + + /** + * Generate the next batch of records. + * + * @return 0 if a batch was sent to the writer, + * -1 if there are no more batches to generate, + * or the number of times we tried to write and the writer + * was busy, otherwise. + */ + private long generateBatch() throws Exception { + if (batch == null) { + while (!batchIterator.hasNext()) { + if (section != null) { + log.info("Generated {} record(s) for the {} section of snapshot {}.", + numRecords, section.name(), writer.epoch()); + section = null; + numRecords = 0; + } + if (!sectionIterator.hasNext()) { + writer.completeSnapshot(); + return -1; + } + section = sectionIterator.next(); + log.info("Generating records for the {} section of snapshot {}.", + section.name(), writer.epoch()); + batchIterator = section.iterator(); + } + batch = batchIterator.next(); + } + if (writer.writeBatch(batch)) { + numRecords += batch.size(); + numWriteTries = 0; + batch = null; + return 0; + } else { + return ++numWriteTries; + } + } + + /** + * Generate the next few batches of records. + * + * @return The number of nanoseconds to delay before rescheduling the + * generateBatches event, or empty if the snapshot is done. + */ + OptionalLong generateBatches() throws Exception { + for (int numBatches = 0; numBatches < maxBatchesPerGenerateCall; numBatches++) { + long result = generateBatch(); + if (result < 0) { + return OptionalLong.empty(); + } else if (result > 0) { + return OptionalLong.of(exponentialBackoff.backoff(result - 1)); + } + } + return OptionalLong.of(0); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/controller/SnapshotReader.java b/metadata/src/main/java/org/apache/kafka/controller/SnapshotReader.java new file mode 100644 index 0000000000000..1a994ed41edab --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/SnapshotReader.java @@ -0,0 +1,37 @@ +/* + * 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.controller; + +import org.apache.kafka.common.protocol.ApiMessage; + +import java.util.Iterator; +import java.util.List; + + +interface SnapshotReader extends Iterator>, AutoCloseable { + /** + * Returns the snapshot epoch, which is the offset of this snapshot within the log. + */ + long epoch(); + + /** + * Invoked when the snapshot reader is no longer needed. This should clean + * up all reader resources. + */ + void close(); +} diff --git a/metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java b/metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java new file mode 100644 index 0000000000000..595f4c181b2fa --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java @@ -0,0 +1,55 @@ +/* + * 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.controller; + +import java.io.IOException; +import java.util.List; +import org.apache.kafka.metadata.ApiMessageAndVersion; + + +interface SnapshotWriter extends AutoCloseable { + /** + * @return The epoch of the snapshot we are writing. + **/ + long epoch(); + + /** + * Write a batch of records. The writer may do more batching than is + * requested-- in other words, small batches may be coalesced into large + * ones, if that is convenient. + * + * @param batch The batch of records. + * + * @return True if the data was received; false if the writer is + * not ready for the data yet. If the writer returns false + * the snapshot generator will try again after a delay. + */ + boolean writeBatch(List batch) throws IOException; + + /** + * Invoked when the snapshot writer is no longer needed. This should clean + * up all writer resources. If this is called prior to completing the snapshot, + * any partial snapshot on disk should be deleted. + */ + void close(); + + /** + * Invoked to finish writing the snapshot to disk. + */ + void completeSnapshot() throws IOException; +} diff --git a/metadata/src/main/java/org/apache/kafka/timeline/SnapshotRegistry.java b/metadata/src/main/java/org/apache/kafka/timeline/SnapshotRegistry.java index 29a79b60faf95..90be3e5e45ed7 100644 --- a/metadata/src/main/java/org/apache/kafka/timeline/SnapshotRegistry.java +++ b/metadata/src/main/java/org/apache/kafka/timeline/SnapshotRegistry.java @@ -151,6 +151,10 @@ public List epochsList() { return result; } + public boolean hasSnapshot(long epoch) { + return snapshots.containsKey(epoch); + } + /** * Gets the snapshot for a specific epoch. */ diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java index c4e8da87104f3..f24853a16a7b3 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java @@ -19,16 +19,19 @@ import org.apache.kafka.common.config.internals.QuotaConfigs; import org.apache.kafka.common.metadata.QuotaRecord; +import org.apache.kafka.common.metadata.QuotaRecord.EntityData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.metadata.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -151,7 +154,7 @@ public void testAlterAndRemove() { } @Test - public void testEntityTypes() { + public void testEntityTypes() throws Exception { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); ClientQuotaControlManager manager = new ClientQuotaControlManager(snapshotRegistry); @@ -179,10 +182,51 @@ public void testEntityTypes() { quotasToTest.put(clientEntity("client-id-2"), quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 60.60)); - List alters = new ArrayList<>(); quotasToTest.forEach((entity, quota) -> entityQuotaToAlterations(entity, quota, alters::add)); alterQuotas(alters, manager); + + ControllerTestUtils.assertBatchIteratorContains(Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-1"), + new EntityData().setEntityType("client-id").setEntityName("client-id-1"))). + setKey("request_percentage").setValue(50.5).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-2"), + new EntityData().setEntityType("client-id").setEntityName("client-id-1"))). + setKey("request_percentage").setValue(51.51).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-3"), + new EntityData().setEntityType("client-id").setEntityName("client-id-2"))). + setKey("request_percentage").setValue(52.52).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName(null), + new EntityData().setEntityType("client-id").setEntityName("client-id-1"))). + setKey("request_percentage").setValue(53.53).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-1"), + new EntityData().setEntityType("client-id").setEntityName(null))). + setKey("request_percentage").setValue(54.54).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-3"), + new EntityData().setEntityType("client-id").setEntityName(null))). + setKey("request_percentage").setValue(55.55).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-1"))). + setKey("request_percentage").setValue(56.56).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-2"))). + setKey("request_percentage").setValue(57.57).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName("user-3"))). + setKey("request_percentage").setValue(58.58).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("user").setEntityName(null))). + setKey("request_percentage").setValue(59.59).setRemove(false), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new QuotaRecord().setEntity(Arrays.asList( + new EntityData().setEntityType("client-id").setEntityName("client-id-2"))). + setKey("request_percentage").setValue(60.60).setRemove(false), (short) 0))), + manager.iterator(Long.MAX_VALUE)); } static void entityQuotaToAlterations(ClientQuotaEntity entity, Map quota, diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index c410a686344d2..009e8811e1514 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.controller; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -26,11 +27,14 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.StaleBrokerEpochException; import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpointCollection; import org.apache.kafka.common.metadata.UnfenceBrokerRecord; import org.apache.kafka.common.metadata.UnregisterBrokerRecord; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.metadata.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; @@ -58,7 +62,7 @@ public void testReplay() { assertFalse(clusterControl.unfenced(0)); RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(1); - brokerRecord.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint(). + brokerRecord.endPoints().add(new BrokerEndpoint(). setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9092). setName("PLAINTEXT"). @@ -86,7 +90,7 @@ public void testUnregister() throws Exception { setBrokerEpoch(100). setIncarnationId(Uuid.fromString("fPZv1VBsRFmnlRvmGcOW9w")). setRack("arack"); - brokerRecord.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint(). + brokerRecord.endPoints().add(new BrokerEndpoint(). setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9092). setName("PLAINTEXT"). @@ -122,7 +126,7 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception { for (int i = 0; i < numUsableBrokers; i++) { RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(i); - brokerRecord.endPoints().add(new RegisterBrokerRecord.BrokerEndpoint(). + brokerRecord.endPoints().add(new BrokerEndpoint(). setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9092). setName("PLAINTEXT"). @@ -147,4 +151,57 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception { } } } + + @Test + public void testIterator() throws Exception { + MockTime time = new MockTime(0, 0, 0); + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + ClusterControlManager clusterControl = new ClusterControlManager( + new LogContext(), time, snapshotRegistry, 1000, + new SimpleReplicaPlacementPolicy(new Random())); + clusterControl.activate(); + assertFalse(clusterControl.unfenced(0)); + for (int i = 0; i < 3; i++) { + RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord(). + setBrokerEpoch(100).setBrokerId(i).setRack(null); + brokerRecord.endPoints().add(new BrokerEndpoint(). + setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). + setPort((short) 9092 + i). + setName("PLAINTEXT"). + setHost("example.com")); + clusterControl.replay(brokerRecord); + } + for (int i = 0; i < 2; i++) { + UnfenceBrokerRecord unfenceBrokerRecord = + new UnfenceBrokerRecord().setId(i).setEpoch(100); + clusterControl.replay(unfenceBrokerRecord); + } + ControllerTestUtils.assertBatchIteratorContains(Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerEpoch(100).setBrokerId(0).setRack(null). + setEndPoints(new BrokerEndpointCollection(Collections.singleton( + new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). + setPort((short) 9092). + setName("PLAINTEXT"). + setHost("example.com")).iterator())), (short) 0), + new ApiMessageAndVersion(new UnfenceBrokerRecord(). + setId(0).setEpoch(100), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerEpoch(100).setBrokerId(1).setRack(null). + setEndPoints(new BrokerEndpointCollection(Collections.singleton( + new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). + setPort((short) 9093). + setName("PLAINTEXT"). + setHost("example.com")).iterator())), (short) 0), + new ApiMessageAndVersion(new UnfenceBrokerRecord(). + setId(1).setEpoch(100), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerEpoch(100).setBrokerId(2).setRack(null). + setEndPoints(new BrokerEndpointCollection(Collections.singleton( + new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). + setPort((short) 9094). + setName("PLAINTEXT"). + setHost("example.com")).iterator())), (short) 0))), + clusterControl.iterator(Long.MAX_VALUE)); + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 561a25b2d0ee2..608b428f68a85 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -81,7 +81,7 @@ static Entry entry(A a, B b) { } @Test - public void testReplay() { + public void testReplay() throws Exception { SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); ConfigurationControlManager manager = new ConfigurationControlManager(new LogContext(), snapshotRegistry, CONFIGS); @@ -103,6 +103,14 @@ public void testReplay() { setName("def").setValue("blah")); assertEquals(toMap(entry("abc", "x,y,z"), entry("def", "blah")), manager.getConfigs(MYTOPIC)); + ControllerTestUtils.assertBatchIteratorContains(Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new ConfigRecord(). + setResourceType(TOPIC.id()).setResourceName("mytopic"). + setName("abc").setValue("x,y,z"), (short) 0), + new ApiMessageAndVersion(new ConfigRecord(). + setResourceType(TOPIC.id()).setResourceName("mytopic"). + setName("def").setValue("blah"), (short) 0))), + manager.iterator(Long.MAX_VALUE)); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java b/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java index 746c7efb55836..10def7ecb352a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java @@ -18,14 +18,21 @@ package org.apache.kafka.controller; import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.Message; +import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; import org.apache.kafka.metadata.ApiMessageAndVersion; +import java.lang.reflect.Field; import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Comparator; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; +import static org.junit.jupiter.api.Assertions.assertEquals; + public class ControllerTestUtils { public static void replayAll(Object target, @@ -48,4 +55,48 @@ public static Set iteratorToSet(Iterator iterator) { } return set; } + + public static void assertBatchIteratorContains(List> batches, + Iterator> iterator) throws Exception { + List> actual = new ArrayList<>(); + while (iterator.hasNext()) { + actual.add(new ArrayList<>(iterator.next())); + } + deepSortRecords(actual); + List> expected = new ArrayList<>(); + for (List batch : batches) { + expected.add(new ArrayList<>(batch)); + } + deepSortRecords(expected); + assertEquals(expected, actual); + } + + @SuppressWarnings("unchecked") + public static void deepSortRecords(Object o) throws Exception { + if (o == null) { + return; + } else if (o instanceof List) { + List list = (List) o; + for (Object entry : list) { + if (entry != null) { + if (Number.class.isAssignableFrom(entry.getClass())) { + return; + } + deepSortRecords(entry); + } + } + list.sort(Comparator.comparing(Object::toString)); + } else if (o instanceof ImplicitLinkedHashCollection) { + ImplicitLinkedHashCollection coll = (ImplicitLinkedHashCollection) o; + for (Object entry : coll) { + deepSortRecords(entry); + } + coll.sort(Comparator.comparing(Object::toString)); + } else if (o instanceof Message || o instanceof ApiMessageAndVersion) { + for (Field field : o.getClass().getDeclaredFields()) { + field.setAccessible(true); + deepSortRecords(field.get(o)); + } + } + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index 0670984e52876..0ee5ee1eb3f26 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.controller; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -88,7 +89,7 @@ public void testReplay() { snapshotRegistry.createSnapshot(-1); FeatureControlManager manager = new FeatureControlManager( rangeMap("foo", 1, 2), snapshotRegistry); - manager.replay(record, 123); + manager.replay(record); snapshotRegistry.createSnapshot(123); assertEquals(new FeatureMapAndEpoch(new FeatureMap(rangeMap("foo", 1, 2)), 123), manager.finalizedFeatures(123)); @@ -121,7 +122,7 @@ public void testUpdateFeaturesErrorCases() { ControllerResult> result = manager.updateFeatures( rangeMap("foo", 1, 3), Collections.emptySet(), Collections.emptyMap()); assertEquals(Collections.singletonMap("foo", ApiError.NONE), result.response()); - manager.replay((FeatureLevelRecord) result.records().get(0).message(), 3); + manager.replay((FeatureLevelRecord) result.records().get(0).message()); snapshotRegistry.createSnapshot(3); assertEquals(ControllerResult.atomicOf(Collections.emptyList(), Collections. @@ -151,4 +152,25 @@ public void testUpdateFeaturesErrorCases() { ) ); } + + @Test + public void testFeatureControlIterator() throws Exception { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + FeatureControlManager manager = new FeatureControlManager( + rangeMap("foo", 1, 5, "bar", 1, 2), snapshotRegistry); + ControllerResult> result = manager. + updateFeatures(rangeMap("foo", 1, 5, "bar", 1, 1), + Collections.emptySet(), Collections.emptyMap()); + ControllerTestUtils.replayAll(manager, result.records()); + ControllerTestUtils.assertBatchIteratorContains(Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName("foo"). + setMinFeatureLevel((short) 1). + setMaxFeatureLevel((short) 5), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new FeatureLevelRecord(). + setName("bar"). + setMinFeatureLevel((short) 1). + setMaxFeatureLevel((short) 1), (short) 0))), + manager.iterator(Long.MAX_VALUE)); + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotReader.java b/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotReader.java new file mode 100644 index 0000000000000..31ef31b927165 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotReader.java @@ -0,0 +1,55 @@ +/* + * 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.controller; + +import org.apache.kafka.common.protocol.ApiMessage; + +import java.util.Iterator; +import java.util.List; + + +public class MockSnapshotReader implements SnapshotReader { + private final long epoch; + private final Iterator> iterator; + + public MockSnapshotReader(long epoch, + Iterator> iterator) { + this.epoch = epoch; + this.iterator = iterator; + } + + @Override + public long epoch() { + return epoch; + } + + @Override + public void close() { + // nothing to do + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + return iterator.next(); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java b/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java new file mode 100644 index 0000000000000..76b4934482390 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java @@ -0,0 +1,94 @@ +/* + * 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.controller; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.metadata.ApiMessageAndVersion; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + + +class MockSnapshotWriter implements SnapshotWriter { + private final long epoch; + private boolean ready = true; + private boolean completed = false; + private boolean closed = false; + private final List> batches = new ArrayList<>(); + + public MockSnapshotWriter(long epoch) { + this.epoch = epoch; + } + + @Override + public long epoch() { + return epoch; + } + + @Override + public synchronized boolean writeBatch(List batch) throws IOException { + if (completed) throw new RuntimeException("writer has been completed"); + if (closed) throw new RuntimeException("writer is closed"); + if (!ready) return false; + batches.add(batch); + return true; + } + + synchronized void setReady(boolean ready) { + this.ready = ready; + } + + @Override + public synchronized void close() { + this.closed = true; + } + + @Override + public synchronized void completeSnapshot() throws IOException { + if (closed) throw new RuntimeException("writer is closed"); + this.notifyAll(); + this.completed = true; + } + + synchronized void waitForCompletion() throws InterruptedException { + while (!completed) { + this.wait(); + } + } + + synchronized boolean completed() { + return completed; + } + + synchronized List> batches() { + return batches; + } + + public MockSnapshotReader toReader() { + List> readerBatches = new ArrayList<>(); + for (List batch : batches) { + List readerBatch = new ArrayList<>(); + for (ApiMessageAndVersion messageAndVersion : batch) { + readerBatch.add(messageAndVersion.message()); + } + readerBatches.add(readerBatch); + } + return new MockSnapshotReader(epoch, readerBatches.iterator()); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriter.java b/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriter.java new file mode 100644 index 0000000000000..dd8da43f1aa91 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriter.java @@ -0,0 +1,49 @@ +/* + * 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.controller; + +import java.io.IOException; +import java.util.List; +import org.apache.kafka.metadata.ApiMessageAndVersion; + + +public final class NoOpSnapshotWriter implements SnapshotWriter { + private final long epoch; + + public NoOpSnapshotWriter(long epoch) { + this.epoch = epoch; + } + + @Override + public long epoch() { + return epoch; + } + + @Override + public boolean writeBatch(List batch) throws IOException { + return true; + } + + @Override + public void close() { + } + + @Override + public void completeSnapshot() throws IOException { + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java b/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java new file mode 100644 index 0000000000000..62d1f8da37673 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java @@ -0,0 +1,28 @@ +/* + * 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.controller; + +import java.util.function.Function; + + +public final class NoOpSnapshotWriterBuilder implements Function { + @Override + public SnapshotWriter apply(Long epoch) { + return new NoOpSnapshotWriter(epoch); + } +} diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index e82081147ee94..d0588a0d27038 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -17,22 +17,38 @@ package org.apache.kafka.controller; +import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.function.Function; + import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData.Listener; import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection; import org.apache.kafka.common.message.BrokerRegistrationRequestData; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignmentCollection; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection; import org.apache.kafka.common.message.CreateTopicsRequestData; +import org.apache.kafka.common.message.CreateTopicsResponseData; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; +import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpointCollection; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.metadata.UnfenceBrokerRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.controller.BrokersToIsrs.TopicIdPartition; +import org.apache.kafka.metadata.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metalog.LocalLogManagerTestEnv; @@ -177,4 +193,134 @@ public void testUnregisterBroker() throws Throwable { } } } + + static class MockSnapshotWriterBuilder implements Function { + final LinkedBlockingDeque writers = new LinkedBlockingDeque<>(); + + @Override + public SnapshotWriter apply(Long epoch) { + MockSnapshotWriter writer = new MockSnapshotWriter(epoch); + writers.add(writer); + return writer; + } + } + + @Test + public void testSnapshotSaveAndLoad() throws Throwable { + MockSnapshotWriterBuilder snapshotWriterBuilder = new MockSnapshotWriterBuilder(); + final int numBrokers = 4; + MockSnapshotWriter writer = null; + Map brokerEpochs = new HashMap<>(); + Uuid fooId; + try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(3)) { + try (QuorumControllerTestEnv controlEnv = + new QuorumControllerTestEnv(logEnv, b -> b.setConfigDefs(CONFIGS). + setSnapshotWriterBuilder(snapshotWriterBuilder))) { + QuorumController active = controlEnv.activeController(); + for (int i = 0; i < numBrokers; i++) { + BrokerRegistrationReply reply = active.registerBroker( + new BrokerRegistrationRequestData(). + setBrokerId(i). + setRack(null). + setClusterId("06B-K3N1TBCNYFgruEVP0Q"). + setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)). + setListeners(new ListenerCollection(Arrays.asList(new Listener(). + setName("PLAINTEXT").setHost("localhost"). + setPort(9092 + i)).iterator()))).get(); + brokerEpochs.put(i, reply.epoch()); + } + for (int i = 0; i < numBrokers - 1; i++) { + assertEquals(new BrokerHeartbeatReply(true, false, false, false), + active.processBrokerHeartbeat(new BrokerHeartbeatRequestData(). + setWantFence(false).setBrokerEpoch(brokerEpochs.get(i)). + setBrokerId(i).setCurrentMetadataOffset(100000L)).get()); + } + CreateTopicsResponseData fooData = active.createTopics( + new CreateTopicsRequestData().setTopics( + new CreatableTopicCollection(Collections.singleton( + new CreatableTopic().setName("foo").setNumPartitions(-1). + setReplicationFactor((short) -1). + setAssignments(new CreatableReplicaAssignmentCollection( + Arrays.asList(new CreatableReplicaAssignment(). + setPartitionIndex(0). + setBrokerIds(Arrays.asList(0, 1, 2)), + new CreatableReplicaAssignment(). + setPartitionIndex(1). + setBrokerIds(Arrays.asList(1, 2, 0))). + iterator()))).iterator()))).get(); + fooId = fooData.topics().find("foo").topicId(); + long snapshotEpoch = active.beginWritingSnapshot().get(); + writer = snapshotWriterBuilder.writers.takeFirst(); + assertEquals(snapshotEpoch, writer.epoch()); + writer.waitForCompletion(); + checkSnapshotContents(fooId, brokerEpochs, writer.batches().iterator()); + } + + final MockSnapshotReader reader = writer.toReader(); + try (QuorumControllerTestEnv controlEnv = + new QuorumControllerTestEnv(logEnv, b -> b.setConfigDefs(CONFIGS). + setSnapshotReader(reader). + setSnapshotWriterBuilder(snapshotWriterBuilder))) { + QuorumController active = controlEnv.activeController(); + long snapshotEpoch = active.beginWritingSnapshot().get(); + writer = snapshotWriterBuilder.writers.takeFirst(); + assertEquals(snapshotEpoch, writer.epoch()); + writer.waitForCompletion(); + checkSnapshotContents(fooId, brokerEpochs, writer.batches().iterator()); + } + } + } + + private void checkSnapshotContents(Uuid fooId, + Map brokerEpochs, + Iterator> iterator) throws Exception { + ControllerTestUtils.assertBatchIteratorContains(Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new TopicRecord(). + setName("foo").setTopicId(fooId), (short) 0), + new ApiMessageAndVersion(new PartitionRecord().setPartitionId(0). + setTopicId(fooId).setReplicas(Arrays.asList(0, 1, 2)). + setIsr(Arrays.asList(0, 1, 2)).setRemovingReplicas(null). + setAddingReplicas(null).setLeader(0).setLeaderEpoch(0). + setPartitionEpoch(0), (short) 0), + new ApiMessageAndVersion(new PartitionRecord().setPartitionId(1). + setTopicId(fooId).setReplicas(Arrays.asList(1, 2, 0)). + setIsr(Arrays.asList(1, 2, 0)).setRemovingReplicas(null). + setAddingReplicas(null).setLeader(1).setLeaderEpoch(0). + setPartitionEpoch(0), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0)). + setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB0")). + setEndPoints(new BrokerEndpointCollection(Arrays.asList( + new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + setPort(9092).setSecurityProtocol((short) 0)).iterator())). + setRack(null), (short) 0), + new ApiMessageAndVersion(new UnfenceBrokerRecord(). + setId(0).setEpoch(brokerEpochs.get(0)), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerId(1).setBrokerEpoch(brokerEpochs.get(1)). + setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB1")). + setEndPoints(new BrokerEndpointCollection(Arrays.asList( + new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + setPort(9093).setSecurityProtocol((short) 0)).iterator())). + setRack(null), (short) 0), + new ApiMessageAndVersion(new UnfenceBrokerRecord(). + setId(1).setEpoch(brokerEpochs.get(1)), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerId(2).setBrokerEpoch(brokerEpochs.get(2)). + setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB2")). + setEndPoints(new BrokerEndpointCollection(Arrays.asList( + new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + setPort(9094).setSecurityProtocol((short) 0)).iterator())). + setRack(null), (short) 0), + new ApiMessageAndVersion(new UnfenceBrokerRecord(). + setId(2).setEpoch(brokerEpochs.get(2)), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + setBrokerId(3).setBrokerEpoch(brokerEpochs.get(3)). + setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB3")). + setEndPoints(new BrokerEndpointCollection(Arrays.asList( + new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + setPort(9095).setSecurityProtocol((short) 0)).iterator())). + setRack(null), (short) 0))), + iterator); + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index fa7fc0094e748..6739944a02988 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection; import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; +import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.TopicRecord; import org.apache.kafka.common.protocol.Errors; @@ -162,6 +163,16 @@ public void testCreateTopics() throws Exception { setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()). setErrorMessage(Errors.TOPIC_ALREADY_EXISTS.exception().getMessage())); assertEquals(expectedResponse3, result3.response()); + Uuid fooId = result2.response().topics().find("foo").topicId(); + ControllerTestUtils.assertBatchIteratorContains(Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new PartitionRecord(). + setPartitionId(0).setTopicId(fooId). + setReplicas(Arrays.asList(2, 0, 1)).setIsr(Arrays.asList(2, 0, 1)). + setRemovingReplicas(null).setAddingReplicas(null).setLeader(2). + setLeaderEpoch(0).setPartitionEpoch(0), (short) 0), + new ApiMessageAndVersion(new TopicRecord(). + setTopicId(fooId).setName("foo"), (short) 0))), + ctx.replicationControl.iterator(Long.MAX_VALUE)); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java new file mode 100644 index 0000000000000..e8346211f1614 --- /dev/null +++ b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java @@ -0,0 +1,110 @@ +/* + * 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.controller; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.metadata.ConfigRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.utils.ExponentialBackoff; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.controller.SnapshotGenerator.Section; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.util.Arrays; +import java.util.List; +import java.util.OptionalLong; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +@Timeout(40) +public class SnapshotGeneratorTest { + private static final List> BATCHES; + + static { + BATCHES = Arrays.asList( + Arrays.asList(new ApiMessageAndVersion(new TopicRecord(). + setName("foo").setTopicId(Uuid.randomUuid()), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new TopicRecord(). + setName("bar").setTopicId(Uuid.randomUuid()), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new TopicRecord(). + setName("baz").setTopicId(Uuid.randomUuid()), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new ConfigRecord(). + setResourceName("foo").setResourceType(ConfigResource.Type.TOPIC.id()). + setName("retention.ms").setValue("10000000"), (short) 0), + new ApiMessageAndVersion(new ConfigRecord(). + setResourceName("foo").setResourceType(ConfigResource.Type.TOPIC.id()). + setName("max.message.bytes").setValue("100000000"), (short) 0)), + Arrays.asList(new ApiMessageAndVersion(new ConfigRecord(). + setResourceName("bar").setResourceType(ConfigResource.Type.TOPIC.id()). + setName("retention.ms").setValue("5000000"), (short) 0))); + } + + @Test + public void testGenerateBatches() throws Exception { + MockSnapshotWriter writer = new MockSnapshotWriter(123); + ExponentialBackoff exponentialBackoff = + new ExponentialBackoff(100, 2, 400, 0.0); + List
    sections = Arrays.asList(new Section("replication", + Arrays.asList(BATCHES.get(0), BATCHES.get(1), BATCHES.get(2)).iterator()), + new Section("configuration", + Arrays.asList(BATCHES.get(3), BATCHES.get(4)).iterator())); + SnapshotGenerator generator = new SnapshotGenerator(new LogContext(), + writer, 2, exponentialBackoff, sections); + assertFalse(writer.completed()); + assertEquals(123L, generator.epoch()); + assertEquals(writer, generator.writer()); + assertEquals(OptionalLong.of(0L), generator.generateBatches()); + assertEquals(OptionalLong.of(0L), generator.generateBatches()); + assertFalse(writer.completed()); + assertEquals(OptionalLong.empty(), generator.generateBatches()); + assertTrue(writer.completed()); + } + + @Test + public void testGenerateBatchesWithBackoff() throws Exception { + MockSnapshotWriter writer = new MockSnapshotWriter(123); + ExponentialBackoff exponentialBackoff = + new ExponentialBackoff(100, 2, 400, 0.0); + List
    sections = Arrays.asList(new Section("replication", + Arrays.asList(BATCHES.get(0), BATCHES.get(1), BATCHES.get(2)).iterator()), + new Section("configuration", + Arrays.asList(BATCHES.get(3), BATCHES.get(4)).iterator())); + SnapshotGenerator generator = new SnapshotGenerator(new LogContext(), + writer, 2, exponentialBackoff, sections); + assertEquals(123L, generator.epoch()); + assertEquals(writer, generator.writer()); + assertFalse(writer.completed()); + assertEquals(OptionalLong.of(0L), generator.generateBatches()); + writer.setReady(false); + assertEquals(OptionalLong.of(100L), generator.generateBatches()); + assertEquals(OptionalLong.of(200L), generator.generateBatches()); + assertEquals(OptionalLong.of(400L), generator.generateBatches()); + assertEquals(OptionalLong.of(400L), generator.generateBatches()); + writer.setReady(true); + assertFalse(writer.completed()); + assertEquals(OptionalLong.of(0L), generator.generateBatches()); + assertEquals(OptionalLong.empty(), generator.generateBatches()); + assertTrue(writer.completed()); + } +} \ No newline at end of file From 12e655f45c4f8002a19c5847363a15d974bba7e9 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 6 Apr 2021 12:10:55 -0700 Subject: [PATCH 011/155] MINOR: Enable scala/java joint compilation consistently for `core` module (#10485) We were doing it only for test files previously. Reviewers: Chia-Ping Tsai , Jose Sancio --- build.gradle | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index e1a05e3c0a170..f4b0bf44e5c33 100644 --- a/build.gradle +++ b/build.gradle @@ -1029,9 +1029,13 @@ project(':core') { } sourceSets { + // Set java/scala source folders in the `scala` block to enable joint compilation main { java { - srcDirs = ["src/generated/java", "src/main/java"] + srcDirs = [] + } + scala { + srcDirs = ["src/generated/java", "src/main/java", "src/main/scala"] } } test { From ff77864a699df46a01fc76114f662c609f4fd7a6 Mon Sep 17 00:00:00 2001 From: John Roesler Date: Tue, 6 Apr 2021 15:08:24 -0500 Subject: [PATCH 012/155] KAFKA-12602: Fix LICENSE file (#10474) Fixes the LICENSE files that we ship with our releases: * the source-distribution license included wrong and unnecessary dependencies * the binary-distribution license was missing most of our actual dependencies Reviewers: A. Sophie Blee-Goldman , Ewen Cheslack-Postava , Justin Mclean --- LICENSE | 347 ---------- LICENSE-binary | 320 +++++++++ NOTICE | 9 +- build.gradle | 4 +- licenses/CDDL+GPL-1.1 | 760 ++++++++++++++++++++++ licenses/DWTFYWTPL | 14 + licenses/argparse-MIT | 23 + licenses/eclipse-distribution-license-1.0 | 13 + licenses/eclipse-public-license-2.0 | 87 +++ licenses/jopt-simple-MIT | 24 + licenses/paranamer-BSD-3-clause | 29 + licenses/slf4j-MIT | 24 + licenses/zstd-jni-BSD-2-clause | 26 + 13 files changed, 1331 insertions(+), 349 deletions(-) create mode 100644 LICENSE-binary create mode 100644 licenses/CDDL+GPL-1.1 create mode 100644 licenses/DWTFYWTPL create mode 100644 licenses/argparse-MIT create mode 100644 licenses/eclipse-distribution-license-1.0 create mode 100644 licenses/eclipse-public-license-2.0 create mode 100644 licenses/jopt-simple-MIT create mode 100644 licenses/paranamer-BSD-3-clause create mode 100644 licenses/slf4j-MIT create mode 100644 licenses/zstd-jni-BSD-2-clause diff --git a/LICENSE b/LICENSE index b0115dabb4d0f..d645695673349 100644 --- a/LICENSE +++ b/LICENSE @@ -200,350 +200,3 @@ 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. - ------------------------------------------------------------------------------------- -This distribution has a binary dependency on jersey, which is available under the EPLv2 -License as described below. - -Eclipse Public License - v 2.0 - - THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE - PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION - OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. - - 1. DEFINITIONS - - "Contribution" means: - - a) in the case of the initial Contributor, the initial content - Distributed under this Agreement, and - - b) in the case of each subsequent Contributor: - i) changes to the Program, and - ii) additions to the Program; - where such changes and/or additions to the Program originate from - and are Distributed by that particular Contributor. A Contribution - "originates" from a Contributor if it was added to the Program by - such Contributor itself or anyone acting on such Contributor's behalf. - Contributions do not include changes or additions to the Program that - are not Modified Works. - - "Contributor" means any person or entity that Distributes the Program. - - "Licensed Patents" mean patent claims licensable by a Contributor which - are necessarily infringed by the use or sale of its Contribution alone - or when combined with the Program. - - "Program" means the Contributions Distributed in accordance with this - Agreement. - - "Recipient" means anyone who receives the Program under this Agreement - or any Secondary License (as applicable), including Contributors. - - "Derivative Works" shall mean any work, whether in Source Code or other - form, that is based on (or derived from) the Program and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. - - "Modified Works" shall mean any work in Source Code or other form that - results from an addition to, deletion from, or modification of the - contents of the Program, including, for purposes of clarity any new file - in Source Code form that contains any contents of the Program. Modified - Works shall not include works that contain only declarations, - interfaces, types, classes, structures, or files of the Program solely - in each case in order to link to, bind by name, or subclass the Program - or Modified Works thereof. - - "Distribute" means the acts of a) distributing or b) making available - in any manner that enables the transfer of a copy. - - "Source Code" means the form of a Program preferred for making - modifications, including but not limited to software source code, - documentation source, and configuration files. - - "Secondary License" means either the GNU General Public License, - Version 2.0, or any later versions of that license, including any - exceptions or additional permissions as identified by the initial - Contributor. - - 2. GRANT OF RIGHTS - - a) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free copyright - license to reproduce, prepare Derivative Works of, publicly display, - publicly perform, Distribute and sublicense the Contribution of such - Contributor, if any, and such Derivative Works. - - b) Subject to the terms of this Agreement, each Contributor hereby - grants Recipient a non-exclusive, worldwide, royalty-free patent - license under Licensed Patents to make, use, sell, offer to sell, - import and otherwise transfer the Contribution of such Contributor, - if any, in Source Code or other form. This patent license shall - apply to the combination of the Contribution and the Program if, at - the time the Contribution is added by the Contributor, such addition - of the Contribution causes such combination to be covered by the - Licensed Patents. The patent license shall not apply to any other - combinations which include the Contribution. No hardware per se is - licensed hereunder. - - c) Recipient understands that although each Contributor grants the - licenses to its Contributions set forth herein, no assurances are - provided by any Contributor that the Program does not infringe the - patent or other intellectual property rights of any other entity. - Each Contributor disclaims any liability to Recipient for claims - brought by any other entity based on infringement of intellectual - property rights or otherwise. As a condition to exercising the - rights and licenses granted hereunder, each Recipient hereby - assumes sole responsibility to secure any other intellectual - property rights needed, if any. For example, if a third party - patent license is required to allow Recipient to Distribute the - Program, it is Recipient's responsibility to acquire that license - before distributing the Program. - - d) Each Contributor represents that to its knowledge it has - sufficient copyright rights in its Contribution, if any, to grant - the copyright license set forth in this Agreement. - - e) Notwithstanding the terms of any Secondary License, no - Contributor makes additional grants to any Recipient (other than - those set forth in this Agreement) as a result of such Recipient's - receipt of the Program under the terms of a Secondary License - (if permitted under the terms of Section 3). - - 3. REQUIREMENTS - - 3.1 If a Contributor Distributes the Program in any form, then: - - a) the Program must also be made available as Source Code, in - accordance with section 3.2, and the Contributor must accompany - the Program with a statement that the Source Code for the Program - is available under this Agreement, and informs Recipients how to - obtain it in a reasonable manner on or through a medium customarily - used for software exchange; and - - b) the Contributor may Distribute the Program under a license - different than this Agreement, provided that such license: - i) effectively disclaims on behalf of all other Contributors all - warranties and conditions, express and implied, including - warranties or conditions of title and non-infringement, and - implied warranties or conditions of merchantability and fitness - for a particular purpose; - - ii) effectively excludes on behalf of all other Contributors all - liability for damages, including direct, indirect, special, - incidental and consequential damages, such as lost profits; - - iii) does not attempt to limit or alter the recipients' rights - in the Source Code under section 3.2; and - - iv) requires any subsequent distribution of the Program by any - party to be under a license that satisfies the requirements - of this section 3. - - 3.2 When the Program is Distributed as Source Code: - - a) it must be made available under this Agreement, or if the - Program (i) is combined with other material in a separate file or - files made available under a Secondary License, and (ii) the initial - Contributor attached to the Source Code the notice described in - Exhibit A of this Agreement, then the Program may be made available - under the terms of such Secondary Licenses, and - - b) a copy of this Agreement must be included with each copy of - the Program. - - 3.3 Contributors may not remove or alter any copyright, patent, - trademark, attribution notices, disclaimers of warranty, or limitations - of liability ("notices") contained within the Program from any copy of - the Program which they Distribute, provided that Contributors may add - their own appropriate notices. - - 4. COMMERCIAL DISTRIBUTION - - Commercial distributors of software may accept certain responsibilities - with respect to end users, business partners and the like. While this - license is intended to facilitate the commercial use of the Program, - the Contributor who includes the Program in a commercial product - offering should do so in a manner which does not create potential - liability for other Contributors. Therefore, if a Contributor includes - the Program in a commercial product offering, such Contributor - ("Commercial Contributor") hereby agrees to defend and indemnify every - other Contributor ("Indemnified Contributor") against any losses, - damages and costs (collectively "Losses") arising from claims, lawsuits - and other legal actions brought by a third party against the Indemnified - Contributor to the extent caused by the acts or omissions of such - Commercial Contributor in connection with its distribution of the Program - in a commercial product offering. The obligations in this section do not - apply to any claims or Losses relating to any actual or alleged - intellectual property infringement. In order to qualify, an Indemnified - Contributor must: a) promptly notify the Commercial Contributor in - writing of such claim, and b) allow the Commercial Contributor to control, - and cooperate with the Commercial Contributor in, the defense and any - related settlement negotiations. The Indemnified Contributor may - participate in any such claim at its own expense. - - For example, a Contributor might include the Program in a commercial - product offering, Product X. That Contributor is then a Commercial - Contributor. If that Commercial Contributor then makes performance - claims, or offers warranties related to Product X, those performance - claims and warranties are such Commercial Contributor's responsibility - alone. Under this section, the Commercial Contributor would have to - defend claims against the other Contributors related to those performance - claims and warranties, and if a court requires any other Contributor to - pay any damages as a result, the Commercial Contributor must pay - those damages. - - 5. NO WARRANTY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN "AS IS" - BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR - IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF - TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR - PURPOSE. Each Recipient is solely responsible for determining the - appropriateness of using and distributing the Program and assumes all - risks associated with its exercise of rights under this Agreement, - including but not limited to the risks and costs of program errors, - compliance with applicable laws, damage to or loss of data, programs - or equipment, and unavailability or interruption of operations. - - 6. DISCLAIMER OF LIABILITY - - EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT - PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS - SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, - EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST - PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN - CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) - ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE - EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE - POSSIBILITY OF SUCH DAMAGES. - - 7. GENERAL - - If any provision of this Agreement is invalid or unenforceable under - applicable law, it shall not affect the validity or enforceability of - the remainder of the terms of this Agreement, and without further - action by the parties hereto, such provision shall be reformed to the - minimum extent necessary to make such provision valid and enforceable. - - If Recipient institutes patent litigation against any entity - (including a cross-claim or counterclaim in a lawsuit) alleging that the - Program itself (excluding combinations of the Program with other software - or hardware) infringes such Recipient's patent(s), then such Recipient's - rights granted under Section 2(b) shall terminate as of the date such - litigation is filed. - - All Recipient's rights under this Agreement shall terminate if it - fails to comply with any of the material terms or conditions of this - Agreement and does not cure such failure in a reasonable period of - time after becoming aware of such noncompliance. If all Recipient's - rights under this Agreement terminate, Recipient agrees to cease use - and distribution of the Program as soon as reasonably practicable. - However, Recipient's obligations under this Agreement and any licenses - granted by Recipient relating to the Program shall continue and survive. - - Everyone is permitted to copy and distribute copies of this Agreement, - but in order to avoid inconsistency the Agreement is copyrighted and - may only be modified in the following manner. The Agreement Steward - reserves the right to publish new versions (including revisions) of - this Agreement from time to time. No one other than the Agreement - Steward has the right to modify this Agreement. The Eclipse Foundation - is the initial Agreement Steward. The Eclipse Foundation may assign the - responsibility to serve as the Agreement Steward to a suitable separate - entity. Each new version of the Agreement will be given a distinguishing - version number. The Program (including Contributions) may always be - Distributed subject to the version of the Agreement under which it was - received. In addition, after a new version of the Agreement is published, - Contributor may elect to Distribute the Program (including its - Contributions) under the new version. - - Except as expressly stated in Sections 2(a) and 2(b) above, Recipient - receives no rights or licenses to the intellectual property of any - Contributor under this Agreement, whether expressly, by implication, - estoppel or otherwise. All rights in the Program not expressly granted - under this Agreement are reserved. Nothing in this Agreement is intended - to be enforceable by any entity that is not a Contributor or Recipient. - No third-party beneficiary rights are created under this Agreement. - - Exhibit A - Form of Secondary Licenses Notice - - "This Source Code may also be made available under the following - Secondary Licenses when the conditions for such availability set forth - in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), - version(s), and exceptions or additional permissions here}." - - Simply including a copy of this Agreement, including this Exhibit A - is not sufficient to license the Source Code under Secondary Licenses. - - If it is not possible or desirable to put the notice in a particular - file, then You may include the notice in a location (such as a LICENSE - file in a relevant directory) where a recipient would be likely to - look for such a notice. - - You may add additional accurate notices of copyright ownership. - ------------------------------------------------------------------------------------- -This distribution has a binary dependency on zstd, which is available under the BSD 3-Clause License as described below. - -BSD License - -For Zstandard software - -Copyright (c) 2016-present, Facebook, Inc. All rights reserved. - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - - * Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - - * Redistributions in binary form must reproduce the above copyright notice, - this list of conditions and the following disclaimer in the documentation - and/or other materials provided with the distribution. - - * Neither the name Facebook nor the names of its contributors may be used to - endorse or promote products derived from this software without specific - prior written permission. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON -ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - ------------------------------------------------------------------------------------- -This distribution has a binary dependency on zstd-jni, which is available under the BSD 2-Clause License -as described below. - -Zstd-jni: JNI bindings to Zstd Library - -Copyright (c) 2015-2016, Luben Karavelov/ All rights reserved. - -BSD License - -Redistribution and use in source and binary forms, with or without modification, -are permitted provided that the following conditions are met: - -* Redistributions of source code must retain the above copyright notice, this - list of conditions and the following disclaimer. - -* Redistributions in binary form must reproduce the above copyright notice, this - list of conditions and the following disclaimer in the documentation and/or - other materials provided with the distribution. - -THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND -ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED -WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE -DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR -ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES -(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; -LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON -ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/LICENSE-binary b/LICENSE-binary new file mode 100644 index 0000000000000..4d8e2c76a41c8 --- /dev/null +++ b/LICENSE-binary @@ -0,0 +1,320 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + +------------------------------------------------------------------------------- +This project bundles some components that are also licensed under the Apache +License Version 2.0: + +audience-annotations-0.5.0 +commons-cli-1.4 +commons-lang3-3.8.1 +jackson-annotations-2.10.5 +jackson-core-2.10.5 +jackson-databind-2.10.5.1 +jackson-dataformat-csv-2.10.5 +jackson-datatype-jdk8-2.10.5 +jackson-jaxrs-base-2.10.5 +jackson-jaxrs-json-provider-2.10.5 +jackson-module-jaxb-annotations-2.10.5 +jackson-module-paranamer-2.10.5 +jackson-module-scala_2.13-2.10.5 +jakarta.validation-api-2.0.2 +javassist-3.27.0-GA +jetty-client-9.4.38.v20210224 +jetty-continuation-9.4.38.v20210224 +jetty-http-9.4.38.v20210224 +jetty-io-9.4.38.v20210224 +jetty-security-9.4.38.v20210224 +jetty-server-9.4.38.v20210224 +jetty-servlet-9.4.38.v20210224 +jetty-servlets-9.4.38.v20210224 +jetty-util-9.4.38.v20210224 +jetty-util-ajax-9.4.38.v20210224 +jersey-common-2.31 +jersey-server-2.31 +log4j-1.2.17 +lz4-java-1.7.1 +maven-artifact-3.6.3 +metrics-core-2.2.0 +netty-buffer-4.1.59.Final +netty-codec-4.1.59.Final +netty-common-4.1.59.Final +netty-handler-4.1.59.Final +netty-resolver-4.1.59.Final +netty-transport-4.1.59.Final +netty-transport-native-epoll-4.1.59.Final +netty-transport-native-epoll-4.1.59.Final +netty-transport-native-unix-common-4.1.59.Final +plexus-utils-3.2.1 +rocksdbjni-5.18.4 +scala-collection-compat_2.13-2.3.0 +scala-library-2.13.5 +scala-logging_2.13-3.9.2 +scala-reflect-2.13.5 +scala-java8-compat_2.13-0.9.1 +snappy-java-1.1.8.1 +zookeeper-3.5.9 +zookeeper-jute-3.5.9 + +=============================================================================== +This product bundles various third-party components under other open source +licenses. This section summarizes those components and their licenses. +See licenses/ for text of these licenses. + +--------------------------------------- +Eclipse Distribution License - v 1.0 +see: licenses/eclipse-distribution-license-1.0 + +jakarta.activation-api-1.2.1 +jakarta.xml.bind-api-2.3.2 + +--------------------------------------- +Eclipse Public License - v 2.0 +see: licenses/eclipse-public-license-2.0 + +jakarta.annotation-api-1.3.5 +jakarta.ws.rs-api-2.1.6 +javax.ws.rs-api-2.1.1 +hk2-api-2.6.1 +hk2-locator-2.6.1 +hk2-utils-2.6.1 +osgi-resource-locator-1.0.3 +aopalliance-repackaged-2.6.1 +jakarta.inject-2.6.1 +jersey-container-servlet-2.31 +jersey-container-servlet-core-2.31 +jersey-client-2.31 +jersey-hk2-2.31 +jersey-media-jaxb-2.31 + +--------------------------------------- +CDDL 1.1 + GPLv2 with classpath exception +see: licenses/CDDL+GPL-1.1 + +javax.servlet-api-3.1.0 +jaxb-api-2.3.0 +activation-1.1.1 + +--------------------------------------- +MIT License + +argparse4j-0.7.0, see: licenses/argparse-MIT +jopt-simple-5.0.4, see: licenses/jopt-simple-MIT +slf4j-api-1.7.30, see: licenses/slf4j-MIT +slf4j-log4j12-1.7.30, see: licenses/slf4j-MIT + +--------------------------------------- +BSD 2-Clause + +zstd-jni-1.4.9-1, see: licenses/zstd-jni-BSD-2-clause + +--------------------------------------- +BSD 3-Clause + +paranamer-2.8, see: licenses/paranamer-BSD-3-clause + +--------------------------------------- +Do What The F*ck You Want To Public License +see: licenses/DWTFYWTPL + +reflections-0.9.12 \ No newline at end of file diff --git a/NOTICE b/NOTICE index 02f5a418ff10b..526e040085dd7 100644 --- a/NOTICE +++ b/NOTICE @@ -13,4 +13,11 @@ https://github.com/jlink/jqwik. The streams-scala (streams/streams-scala) module was donated by Lightbend and the original code was copyrighted by them: Copyright (C) 2018 Lightbend Inc. -Copyright (C) 2017-2018 Alexis Seigneurin. \ No newline at end of file +Copyright (C) 2017-2018 Alexis Seigneurin. + +This project contains the following code copied from Apache Hadoop: +clients/src/main/java/org/apache/kafka/common/utils/PureJavaCrc32C.java +Some portions of this file Copyright (c) 2004-2006 Intel Corporation and licensed under the BSD license. + +This project contains the following code copied from Apache Hive: +streams/src/main/java/org/apache/kafka/streams/state/internals/Murmur3.java \ No newline at end of file diff --git a/build.gradle b/build.gradle index f4b0bf44e5c33..a733b1af085d2 100644 --- a/build.gradle +++ b/build.gradle @@ -158,6 +158,7 @@ if (file('.git').exists()) { 'checkstyle/suppressions.xml', 'streams/quickstart/java/src/test/resources/projects/basic/goal.txt', 'streams/streams-scala/logs/*', + 'licenses/*', '**/generated/**' ]) } @@ -967,7 +968,8 @@ project(':core') { compression = Compression.GZIP from(project.file("$rootDir/bin")) { into "bin/" } from(project.file("$rootDir/config")) { into "config/" } - from "$rootDir/LICENSE" + from(project.file("$rootDir/licenses")) { into "licenses/" } + from "$rootDir/LICENSE-binary" rename {String filename -> filename.replace("-binary", "")} from "$rootDir/NOTICE" from(configurations.runtimeClasspath) { into("libs/") } from(configurations.archives.artifacts.files) { into("libs/") } diff --git a/licenses/CDDL+GPL-1.1 b/licenses/CDDL+GPL-1.1 new file mode 100644 index 0000000000000..4b156e6e78b43 --- /dev/null +++ b/licenses/CDDL+GPL-1.1 @@ -0,0 +1,760 @@ +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) Version 1.1 + +1. Definitions. + + 1.1. "Contributor" means each individual or entity that creates or + contributes to the creation of Modifications. + + 1.2. "Contributor Version" means the combination of the Original + Software, prior Modifications used by a Contributor (if any), and + the Modifications made by that particular Contributor. + + 1.3. "Covered Software" means (a) the Original Software, or (b) + Modifications, or (c) the combination of files containing Original + Software with files containing Modifications, in each case including + portions thereof. + + 1.4. "Executable" means the Covered Software in any form other than + Source Code. + + 1.5. "Initial Developer" means the individual or entity that first + makes Original Software available under this License. + + 1.6. "Larger Work" means a work which combines Covered Software or + portions thereof with code not governed by the terms of this License. + + 1.7. "License" means this document. + + 1.8. "Licensable" means having the right to grant, to the maximum + extent possible, whether at the time of the initial grant or + subsequently acquired, any and all of the rights conveyed herein. + + 1.9. "Modifications" means the Source Code and Executable form of + any of the following: + + A. Any file that results from an addition to, deletion from or + modification of the contents of a file containing Original Software + or previous Modifications; + + B. Any new file that contains any part of the Original Software or + previous Modification; or + + C. Any new file that is contributed or otherwise made available + under the terms of this License. + + 1.10. "Original Software" means the Source Code and Executable form + of computer software code that is originally released under this + License. + + 1.11. "Patent Claims" means any patent claim(s), now owned or + hereafter acquired, including without limitation, method, process, + and apparatus claims, in any patent Licensable by grantor. + + 1.12. "Source Code" means (a) the common form of computer software + code in which modifications are made and (b) associated + documentation included in or with such code. + + 1.13. "You" (or "Your") means an individual or a legal entity + exercising rights under, and complying with all of the terms of, + this License. For legal entities, "You" includes any entity which + controls, is controlled by, or is under common control with You. For + purposes of this definition, "control" means (a) the power, direct + or indirect, to cause the direction or management of such entity, + whether by contract or otherwise, or (b) ownership of more than + fifty percent (50%) of the outstanding shares or beneficial + ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, the Initial Developer + hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Initial Developer, to use, reproduce, + modify, display, perform, sublicense and distribute the Original + Software (or portions thereof), with or without Modifications, + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of + Original Software, to make, have made, use, practice, sell, and + offer for sale, and/or otherwise dispose of the Original Software + (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on + the date Initial Developer first distributes or otherwise makes the + Original Software available to a third party under the terms of this + License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is + granted: (1) for code that You delete from the Original Software, or + (2) for infringements caused by: (i) the modification of the + Original Software, or (ii) the combination of the Original Software + with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject + to third party intellectual property claims, each Contributor hereby + grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or + trademark) Licensable by Contributor to use, reproduce, modify, + display, perform, sublicense and distribute the Modifications + created by such Contributor (or portions thereof), either on an + unmodified basis, with other Modifications, as Covered Software + and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling + of Modifications made by that Contributor either alone and/or in + combination with its Contributor Version (or portions of such + combination), to make, use, sell, offer for sale, have made, and/or + otherwise dispose of: (1) Modifications made by that Contributor (or + portions thereof); and (2) the combination of Modifications made by + that Contributor with its Contributor Version (or portions of such + combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective + on the date Contributor first distributes or otherwise makes the + Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is + granted: (1) for any code that Contributor has deleted from the + Contributor Version; (2) for infringements caused by: (i) third + party modifications of Contributor Version, or (ii) the combination + of Modifications made by that Contributor with other software + (except as part of the Contributor Version) or other devices; or (3) + under Patent Claims infringed by Covered Software in the absence of + Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available + in Executable form must also be made available in Source Code form + and that Source Code form must be distributed only under the terms + of this License. You must include a copy of this License with every + copy of the Source Code form of the Covered Software You distribute + or otherwise make available. You must inform recipients of any such + Covered Software in Executable form as to how they can obtain such + Covered Software in Source Code form in a reasonable manner on or + through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are + governed by the terms of this License. You represent that You + believe Your Modifications are Your original creation(s) and/or You + have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that + identifies You as the Contributor of the Modification. You may not + remove or alter any copyright, patent or trademark notices contained + within the Covered Software, or any notices of licensing or any + descriptive text giving attribution to any Contributor or the + Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in + Source Code form that alters or restricts the applicable version of + this License or the recipients' rights hereunder. You may choose to + offer, and to charge a fee for, warranty, support, indemnity or + liability obligations to one or more recipients of Covered Software. + However, you may do so only on Your own behalf, and not on behalf of + the Initial Developer or any Contributor. You must make it + absolutely clear that any such warranty, support, indemnity or + liability obligation is offered by You alone, and You hereby agree + to indemnify the Initial Developer and every Contributor for any + liability incurred by the Initial Developer or such Contributor as a + result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under + the terms of this License or under the terms of a license of Your + choice, which may contain terms different from this License, + provided that You are in compliance with the terms of this License + and that the license for the Executable form does not attempt to + limit or alter the recipient's rights in the Source Code form from + the rights set forth in this License. If You distribute the Covered + Software in Executable form under a different license, You must make + it absolutely clear that any terms which differ from this License + are offered by You alone, not by the Initial Developer or + Contributor. You hereby agree to indemnify the Initial Developer and + every Contributor for any liability incurred by the Initial + Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with + other code not governed by the terms of this License and distribute + the Larger Work as a single product. In such a case, You must make + sure the requirements of this License are fulfilled for the Covered + Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or + new versions of this License from time to time. Each version will be + given a distinguishing version number. Except as provided in Section + 4.3, no one other than the license steward has the right to modify + this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the + Covered Software available under the terms of the version of the + License under which You originally received the Covered Software. If + the Initial Developer includes a notice in the Original Software + prohibiting it from being distributed or otherwise made available + under any subsequent version of the License, You must distribute and + make the Covered Software available under the terms of the version + of the License under which You originally received the Covered + Software. Otherwise, You may also choose to use, distribute or + otherwise make the Covered Software available under the terms of any + subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new + license for Your Original Software, You may create and use a + modified version of this License if You: (a) rename the license and + remove any references to the name of the license steward (except to + note that the license differs from this License); and (b) otherwise + make it clear that the license contains terms which differ from this + License. + +5. DISCLAIMER OF WARRANTY. + + COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN "AS IS" BASIS, + WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, + INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE + IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR + NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF + THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE + DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY + OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, + REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN + ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS + AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. + +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate + automatically if You fail to comply with terms herein and fail to + cure such breach within 30 days of becoming aware of the breach. + Provisions which, by their nature, must remain in effect beyond the + termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding + declaratory judgment actions) against Initial Developer or a + Contributor (the Initial Developer or Contributor against whom You + assert such claim is referred to as "Participant") alleging that the + Participant Software (meaning the Contributor Version where the + Participant is a Contributor or the Original Software where the + Participant is the Initial Developer) directly or indirectly + infringes any patent, then any and all rights granted directly or + indirectly to You by such Participant, the Initial Developer (if the + Initial Developer is not the Participant) and all Contributors under + Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice + from Participant terminate prospectively and automatically at the + expiration of such 60 day notice period, unless if within such 60 + day period You withdraw Your claim with respect to the Participant + Software against such Participant either unilaterally or pursuant to + a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant + alleging that the Participant Software directly or indirectly + infringes any patent where such claim is resolved (such as by + license or settlement) prior to the initiation of patent + infringement litigation, then the reasonable value of the licenses + granted by such Participant under Sections 2.1 or 2.2 shall be taken + into account in determining the amount or value of any payment or + license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, + all end user licenses that have been validly granted by You or any + distributor hereunder prior to termination (excluding licenses + granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + + UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT + (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE + INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF + COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE + TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR + CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT + LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER + FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR + LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE + POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT + APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH + PARTY'S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH + LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR + LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION + AND LIMITATION MAY NOT APPLY TO YOU. + +8. U.S. GOVERNMENT END USERS. + + The Covered Software is a "commercial item," as that term is defined + in 48 C.F.R. 2.101 (Oct. 1995), consisting of "commercial computer + software" (as that term is defined at 48 C.F.R. § + 252.227-7014(a)(1)) and "commercial computer software documentation" + as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent + with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 + (June 1995), all U.S. Government End Users acquire Covered Software + with only those rights set forth herein. This U.S. Government Rights + clause is in lieu of, and supersedes, any other FAR, DFAR, or other + clause or provision that addresses Government rights in computer + software under this License. + +9. MISCELLANEOUS. + + This License represents the complete agreement concerning subject + matter hereof. If any provision of this License is held to be + unenforceable, such provision shall be reformed only to the extent + necessary to make it enforceable. This License shall be governed by + the law of the jurisdiction specified in a notice contained within + the Original Software (except to the extent applicable law, if any, + provides otherwise), excluding such jurisdiction's conflict-of-law + provisions. Any litigation relating to this License shall be subject + to the jurisdiction of the courts located in the jurisdiction and + venue specified in a notice contained within the Original Software, + with the losing party responsible for costs, including, without + limitation, court costs and reasonable attorneys' fees and expenses. + The application of the United Nations Convention on Contracts for + the International Sale of Goods is expressly excluded. Any law or + regulation which provides that the language of a contract shall be + construed against the drafter shall not apply to this License. You + agree that You alone are responsible for compliance with the United + States export administration regulations (and the export control + laws and regulation of any other countries) when You use, distribute + or otherwise make available any Covered Software. + +10. RESPONSIBILITY FOR CLAIMS. + + As between Initial Developer and the Contributors, each party is + responsible for claims and damages arising, directly or indirectly, + out of its utilization of rights under this License and You agree to + work with Initial Developer and Contributors to distribute such + responsibility on an equitable basis. Nothing herein is intended or + shall be deemed to constitute any admission of liability. + +------------------------------------------------------------------------ + +NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION +LICENSE (CDDL) + +The code released under the CDDL shall be governed by the laws of the +State of California (excluding conflict-of-law provisions). Any +litigation relating to this License shall be subject to the jurisdiction +of the Federal Courts of the Northern District of California and the +state courts of the State of California, with venue lying in Santa Clara +County, California. + + + + The GNU General Public License (GPL) Version 2, June 1991 + +Copyright (C) 1989, 1991 Free Software Foundation, Inc. +51 Franklin Street, Fifth Floor +Boston, MA 02110-1335 +USA + +Everyone is permitted to copy and distribute verbatim copies +of this license document, but changing it is not allowed. + +Preamble + +The licenses for most software are designed to take away your freedom to +share and change it. By contrast, the GNU General Public License is +intended to guarantee your freedom to share and change free software--to +make sure the software is free for all its users. This General Public +License applies to most of the Free Software Foundation's software and +to any other program whose authors commit to using it. (Some other Free +Software Foundation software is covered by the GNU Library General +Public License instead.) You can apply it to your programs, too. + +When we speak of free software, we are referring to freedom, not price. +Our General Public Licenses are designed to make sure that you have the +freedom to distribute copies of free software (and charge for this +service if you wish), that you receive source code or can get it if you +want it, that you can change the software or use pieces of it in new +free programs; and that you know you can do these things. + +To protect your rights, we need to make restrictions that forbid anyone +to deny you these rights or to ask you to surrender the rights. These +restrictions translate to certain responsibilities for you if you +distribute copies of the software, or if you modify it. + +For example, if you distribute copies of such a program, whether gratis +or for a fee, you must give the recipients all the rights that you have. +You must make sure that they, too, receive or can get the source code. +And you must show them these terms so they know their rights. + +We protect your rights with two steps: (1) copyright the software, and +(2) offer you this license which gives you legal permission to copy, +distribute and/or modify the software. + +Also, for each author's protection and ours, we want to make certain +that everyone understands that there is no warranty for this free +software. If the software is modified by someone else and passed on, we +want its recipients to know that what they have is not the original, so +that any problems introduced by others will not reflect on the original +authors' reputations. + +Finally, any free program is threatened constantly by software patents. +We wish to avoid the danger that redistributors of a free program will +individually obtain patent licenses, in effect making the program +proprietary. To prevent this, we have made it clear that any patent must +be licensed for everyone's free use or not licensed at all. + +The precise terms and conditions for copying, distribution and +modification follow. + +TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + +0. This License applies to any program or other work which contains a +notice placed by the copyright holder saying it may be distributed under +the terms of this General Public License. The "Program", below, refers +to any such program or work, and a "work based on the Program" means +either the Program or any derivative work under copyright law: that is +to say, a work containing the Program or a portion of it, either +verbatim or with modifications and/or translated into another language. +(Hereinafter, translation is included without limitation in the term +"modification".) Each licensee is addressed as "you". + +Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of running +the Program is not restricted, and the output from the Program is +covered only if its contents constitute a work based on the Program +(independent of having been made by running the Program). Whether that +is true depends on what the Program does. + +1. You may copy and distribute verbatim copies of the Program's source +code as you receive it, in any medium, provided that you conspicuously +and appropriately publish on each copy an appropriate copyright notice +and disclaimer of warranty; keep intact all the notices that refer to +this License and to the absence of any warranty; and give any other +recipients of the Program a copy of this License along with the Program. + +You may charge a fee for the physical act of transferring a copy, and +you may at your option offer warranty protection in exchange for a fee. + +2. You may modify your copy or copies of the Program or any portion of +it, thus forming a work based on the Program, and copy and distribute +such modifications or work under the terms of Section 1 above, provided +that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any part + thereof, to be licensed as a whole at no charge to all third parties + under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a notice + that there is no warranty (or else, saying that you provide a + warranty) and that users may redistribute the program under these + conditions, and telling the user how to view a copy of this License. + (Exception: if the Program itself is interactive but does not + normally print such an announcement, your work based on the Program + is not required to print an announcement.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Program, and +can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based on +the Program, the distribution of the whole must be on the terms of this +License, whose permissions for other licensees extend to the entire +whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Program. + +In addition, mere aggregation of another work not based on the Program +with the Program (or with a work based on the Program) on a volume of a +storage or distribution medium does not bring the other work under the +scope of this License. + +3. You may copy and distribute the Program (or a work based on it, +under Section 2) in object code or executable form under the terms of +Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections 1 + and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your cost + of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer to + distribute corresponding source code. (This alternative is allowed + only for noncommercial distribution and only if you received the + program in object code or executable form with such an offer, in + accord with Subsection b above.) + +The source code for a work means the preferred form of the work for +making modifications to it. For an executable work, complete source code +means all the source code for all modules it contains, plus any +associated interface definition files, plus the scripts used to control +compilation and installation of the executable. However, as a special +exception, the source code distributed need not include anything that is +normally distributed (in either source or binary form) with the major +components (compiler, kernel, and so on) of the operating system on +which the executable runs, unless that component itself accompanies the +executable. + +If distribution of executable or object code is made by offering access +to copy from a designated place, then offering equivalent access to copy +the source code from the same place counts as distribution of the source +code, even though third parties are not compelled to copy the source +along with the object code. + +4. You may not copy, modify, sublicense, or distribute the Program +except as expressly provided under this License. Any attempt otherwise +to copy, modify, sublicense or distribute the Program is void, and will +automatically terminate your rights under this License. However, parties +who have received copies, or rights, from you under this License will +not have their licenses terminated so long as such parties remain in +full compliance. + +5. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Program or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Program (or any work based on the +Program), you indicate your acceptance of this License to do so, and all +its terms and conditions for copying, distributing or modifying the +Program or works based on it. + +6. Each time you redistribute the Program (or any work based on the +Program), the recipient automatically receives a license from the +original licensor to copy, distribute or modify the Program subject to +these terms and conditions. You may not impose any further restrictions +on the recipients' exercise of the rights granted herein. You are not +responsible for enforcing compliance by third parties to this License. + +7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot distribute +so as to satisfy simultaneously your obligations under this License and +any other pertinent obligations, then as a consequence you may not +distribute the Program at all. For example, if a patent license would +not permit royalty-free redistribution of the Program by all those who +receive copies directly or indirectly through you, then the only way you +could satisfy both it and this License would be to refrain entirely from +distribution of the Program. + +If any portion of this section is held invalid or unenforceable under +any particular circumstance, the balance of the section is intended to +apply and the section as a whole is intended to apply in other +circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system, which is implemented +by public license practices. Many people have made generous +contributions to the wide range of software distributed through that +system in reliance on consistent application of that system; it is up to +the author/donor to decide if he or she is willing to distribute +software through any other system and a licensee cannot impose that choice. + +This section is intended to make thoroughly clear what is believed to be +a consequence of the rest of this License. + +8. If the distribution and/or use of the Program is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Program under this License may +add an explicit geographical distribution limitation excluding those +countries, so that distribution is permitted only in or among countries +not thus excluded. In such case, this License incorporates the +limitation as if written in the body of this License. + +9. The Free Software Foundation may publish revised and/or new +versions of the General Public License from time to time. Such new +versions will be similar in spirit to the present version, but may +differ in detail to address new problems or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any +later version", you have the option of following the terms and +conditions either of that version or of any later version published by +the Free Software Foundation. If the Program does not specify a version +number of this License, you may choose any version ever published by the +Free Software Foundation. + +10. If you wish to incorporate parts of the Program into other free +programs whose distribution conditions are different, write to the +author to ask for permission. For software which is copyrighted by the +Free Software Foundation, write to the Free Software Foundation; we +sometimes make exceptions for this. Our decision will be guided by the +two goals of preserving the free status of all derivatives of our free +software and of promoting the sharing and reuse of software generally. + +NO WARRANTY + +11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO +WARRANTY FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. +EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR +OTHER PARTIES PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, +EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE +ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH +YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL +NECESSARY SERVICING, REPAIR OR CORRECTION. + +12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN +WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY +AND/OR REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR +DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL +DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE PROGRAM +(INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING RENDERED +INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A FAILURE OF +THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER OR +OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +END OF TERMS AND CONDITIONS + +How to Apply These Terms to Your New Programs + +If you develop a new program, and you want it to be of the greatest +possible use to the public, the best way to achieve this is to make it +free software which everyone can redistribute and change under these terms. + +To do so, attach the following notices to the program. It is safest to +attach them to the start of each source file to most effectively convey +the exclusion of warranty; and each file should have at least the +"copyright" line and a pointer to where the full notice is found. + + One line to give the program's name and a brief idea of what it does. + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, but + WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1335 USA + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this +when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type + `show w'. This is free software, and you are welcome to redistribute + it under certain conditions; type `show c' for details. + +The hypothetical commands `show w' and `show c' should show the +appropriate parts of the General Public License. Of course, the commands +you use may be called something other than `show w' and `show c'; they +could even be mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the program, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + program `Gnomovision' (which makes passes at compilers) written by + James Hacker. + + signature of Ty Coon, 1 April 1989 + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program +into proprietary programs. If your program is a subroutine library, you +may consider it more useful to permit linking proprietary applications +with the library. If this is what you want to do, use the GNU Library +General Public License instead of this License. + +# + +Certain source files distributed by Oracle America, Inc. and/or its +affiliates are subject to the following clarification and special +exception to the GPLv2, based on the GNU Project exception for its +Classpath libraries, known as the GNU Classpath Exception, but only +where Oracle has expressly included in the particular source file's +header the words "Oracle designates this particular file as subject to +the "Classpath" exception as provided by Oracle in the LICENSE file +that accompanied this code." + +You should also note that Oracle includes multiple, independent +programs in this software package. Some of those programs are provided +under licenses deemed incompatible with the GPLv2 by the Free Software +Foundation and others. For example, the package includes programs +licensed under the Apache License, Version 2.0. Such programs are +licensed to you under their original licenses. + +Oracle facilitates your further distribution of this package by adding +the Classpath Exception to the necessary parts of its GPLv2 code, which +permits you to use that code in combination with other independent +modules not licensed under the GPLv2. However, note that this would +not permit you to commingle code under an incompatible license with +Oracle's GPLv2 licensed code by, for example, cutting and pasting such +code into a file also containing Oracle's GPLv2 licensed code and then +distributing the result. Additionally, if you were to remove the +Classpath Exception from any of the files to which it applies and +distribute the result, you would likely be required to license some or +all of the other code in that distribution under the GPLv2 as well, and +since the GPLv2 is incompatible with the license terms of some items +included in the distribution by Oracle, removing the Classpath +Exception could therefore effectively compromise your ability to +further distribute the package. + +Proceed with caution and we recommend that you obtain the advice of a +lawyer skilled in open source matters before removing the Classpath +Exception or making modifications to this package which may +subsequently be redistributed and/or involve the use of third party +software. + +CLASSPATH EXCEPTION +Linking this library statically or dynamically with other modules is +making a combined work based on this library. Thus, the terms and +conditions of the GNU General Public License version 2 cover the whole +combination. + +As a special exception, the copyright holders of this library give you +permission to link this library with independent modules to produce an +executable, regardless of the license terms of these independent +modules, and to copy and distribute the resulting executable under +terms of your choice, provided that you also meet, for each linked +independent module, the terms and conditions of the license of that +module. An independent module is a module which is not derived from or +based on this library. If you modify this library, you may extend this +exception to your version of the library, but you are not obligated to +do so. If you do not wish to do so, delete this exception statement +from your version. + diff --git a/licenses/DWTFYWTPL b/licenses/DWTFYWTPL new file mode 100644 index 0000000000000..5a8e332545f66 --- /dev/null +++ b/licenses/DWTFYWTPL @@ -0,0 +1,14 @@ + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + Version 2, December 2004 + + Copyright (C) 2004 Sam Hocevar + + Everyone is permitted to copy and distribute verbatim or modified + copies of this license document, and changing it is allowed as long + as the name is changed. + + DO WHAT THE FUCK YOU WANT TO PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. You just DO WHAT THE FUCK YOU WANT TO. + diff --git a/licenses/argparse-MIT b/licenses/argparse-MIT new file mode 100644 index 0000000000000..773b0df0e3591 --- /dev/null +++ b/licenses/argparse-MIT @@ -0,0 +1,23 @@ +/* + * Copyright (C) 2011-2017 Tatsuhiro Tsujikawa + * + * Permission is hereby granted, free of charge, to any person + * obtaining a copy of this software and associated documentation + * files (the "Software"), to deal in the Software without + * restriction, including without limitation the rights to use, copy, + * modify, merge, publish, distribute, sublicense, and/or sell copies + * of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be + * included in all copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + * EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + * MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + * NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS + * BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN + * ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN + * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ diff --git a/licenses/eclipse-distribution-license-1.0 b/licenses/eclipse-distribution-license-1.0 new file mode 100644 index 0000000000000..5f06513abf5fe --- /dev/null +++ b/licenses/eclipse-distribution-license-1.0 @@ -0,0 +1,13 @@ +Eclipse Distribution License - v 1.0 + +Copyright (c) 2007, Eclipse Foundation, Inc. and its licensors. + +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. +* Neither the name of the Eclipse Foundation, Inc. nor the names of its contributors may be used to endorse or promote products derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses/eclipse-public-license-2.0 b/licenses/eclipse-public-license-2.0 new file mode 100644 index 0000000000000..c9f1425f82d02 --- /dev/null +++ b/licenses/eclipse-public-license-2.0 @@ -0,0 +1,87 @@ +Eclipse Public License - v 2.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE (“AGREEMENT”). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. +1. DEFINITIONS + +“Contribution” means: + + a) in the case of the initial Contributor, the initial content Distributed under this Agreement, and + b) in the case of each subsequent Contributor: + i) changes to the Program, and + ii) additions to the Program; + where such changes and/or additions to the Program originate from and are Distributed by that particular Contributor. A Contribution “originates” from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include changes or additions to the Program that are not Modified Works. + +“Contributor” means any person or entity that Distributes the Program. + +“Licensed Patents” mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program. + +“Program” means the Contributions Distributed in accordance with this Agreement. + +“Recipient” means anyone who receives the Program under this Agreement or any Secondary License (as applicable), including Contributors. + +“Derivative Works” shall mean any work, whether in Source Code or other form, that is based on (or derived from) the Program and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. + +“Modified Works” shall mean any work in Source Code or other form that results from an addition to, deletion from, or modification of the contents of the Program, including, for purposes of clarity any new file in Source Code form that contains any contents of the Program. Modified Works shall not include works that contain only declarations, interfaces, types, classes, structures, or files of the Program solely in each case in order to link to, bind by name, or subclass the Program or Modified Works thereof. + +“Distribute” means the acts of a) distributing or b) making available in any manner that enables the transfer of a copy. + +“Source Code” means the form of a Program preferred for making modifications, including but not limited to software source code, documentation source, and configuration files. + +“Secondary License” means either the GNU General Public License, Version 2.0, or any later versions of that license, including any exceptions or additional permissions as identified by the initial Contributor. +2. GRANT OF RIGHTS + + a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, Distribute and sublicense the Contribution of such Contributor, if any, and such Derivative Works. + b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in Source Code or other form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder. + c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to Distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program. + d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement. + e) Notwithstanding the terms of any Secondary License, no Contributor makes additional grants to any Recipient (other than those set forth in this Agreement) as a result of such Recipient's receipt of the Program under the terms of a Secondary License (if permitted under the terms of Section 3). + +3. REQUIREMENTS + +3.1 If a Contributor Distributes the Program in any form, then: + + a) the Program must also be made available as Source Code, in accordance with section 3.2, and the Contributor must accompany the Program with a statement that the Source Code for the Program is available under this Agreement, and informs Recipients how to obtain it in a reasonable manner on or through a medium customarily used for software exchange; and + b) the Contributor may Distribute the Program under a license different than this Agreement, provided that such license: + i) effectively disclaims on behalf of all other Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose; + ii) effectively excludes on behalf of all other Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits; + iii) does not attempt to limit or alter the recipients' rights in the Source Code under section 3.2; and + iv) requires any subsequent distribution of the Program by any party to be under a license that satisfies the requirements of this section 3. + +3.2 When the Program is Distributed as Source Code: + + a) it must be made available under this Agreement, or if the Program (i) is combined with other material in a separate file or files made available under a Secondary License, and (ii) the initial Contributor attached to the Source Code the notice described in Exhibit A of this Agreement, then the Program may be made available under the terms of such Secondary Licenses, and + b) a copy of this Agreement must be included with each copy of the Program. + +3.3 Contributors may not remove or alter any copyright, patent, trademark, attribution notices, disclaimers of warranty, or limitations of liability (‘notices’) contained within the Program from any copy of the Program which they Distribute, provided that Contributors may add their own appropriate notices. +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor (“Commercial Contributor”) hereby agrees to defend and indemnify every other Contributor (“Indemnified Contributor”) against any losses, damages and costs (collectively “Losses”) arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages. +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT PERMITTED BY APPLICABLE LAW, THE PROGRAM IS PROVIDED ON AN “AS IS” BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement, including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations. +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, AND TO THE EXTENT PERMITTED BY APPLICABLE LAW, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be Distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to Distribute the Program (including its Contributions) under the new version. + +Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved. Nothing in this Agreement is intended to be enforceable by any entity that is not a Contributor or Recipient. No third-party beneficiary rights are created under this Agreement. +Exhibit A – Form of Secondary Licenses Notice + +“This Source Code may also be made available under the following Secondary Licenses when the conditions for such availability set forth in the Eclipse Public License, v. 2.0 are satisfied: {name license(s), version(s), and exceptions or additional permissions here}.” + + Simply including a copy of this Agreement, including this Exhibit A is not sufficient to license the Source Code under Secondary Licenses. + + If it is not possible or desirable to put the notice in a particular file, then You may include the notice in a location (such as a LICENSE file in a relevant directory) where a recipient would be likely to look for such a notice. + + You may add additional accurate notices of copyright ownership. + diff --git a/licenses/jopt-simple-MIT b/licenses/jopt-simple-MIT new file mode 100644 index 0000000000000..54b27325bb6b2 --- /dev/null +++ b/licenses/jopt-simple-MIT @@ -0,0 +1,24 @@ +/* + The MIT License + + Copyright (c) 2004-2016 Paul R. Holser, Jr. + + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, + distribute, sublicense, and/or sell copies of the Software, and to + permit persons to whom the Software is furnished to do so, subject to + the following conditions: + + The above copyright notice and this permission notice shall be + included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE + LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION + OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION + WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. +*/ diff --git a/licenses/paranamer-BSD-3-clause b/licenses/paranamer-BSD-3-clause new file mode 100644 index 0000000000000..9eab87918636a --- /dev/null +++ b/licenses/paranamer-BSD-3-clause @@ -0,0 +1,29 @@ +[ ParaNamer used to be 'Pubic Domain', but since it includes a small piece of ASM it is now the same license as that: BSD ] + + Portions copyright (c) 2006-2018 Paul Hammant & ThoughtWorks Inc + Portions copyright (c) 2000-2007 INRIA, France Telecom + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions + are met: + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + 3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" + AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE + LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR + CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF + SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS + INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN + CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF + THE POSSIBILITY OF SUCH DAMAGE. diff --git a/licenses/slf4j-MIT b/licenses/slf4j-MIT new file mode 100644 index 0000000000000..315bd4979f155 --- /dev/null +++ b/licenses/slf4j-MIT @@ -0,0 +1,24 @@ +Copyright (c) 2004-2017 QOS.ch +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + + + diff --git a/licenses/zstd-jni-BSD-2-clause b/licenses/zstd-jni-BSD-2-clause new file mode 100644 index 0000000000000..66abb8ae782c8 --- /dev/null +++ b/licenses/zstd-jni-BSD-2-clause @@ -0,0 +1,26 @@ +Zstd-jni: JNI bindings to Zstd Library + +Copyright (c) 2015-present, Luben Karavelov/ All rights reserved. + +BSD License + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. From 33d0445b8408289800352de7822340028782a154 Mon Sep 17 00:00:00 2001 From: Marco Aurelio Lotz Date: Wed, 7 Apr 2021 02:47:52 +0200 Subject: [PATCH 013/155] KAFKA-5146: remove Connect dependency from Streams module (#10131) Reviewers: A. Sophie Blee-Goldman , Ismael Juma , Matthias J. Sax --- build.gradle | 19 ++++++++----------- docs/streams/upgrade-guide.html | 7 +++++++ docs/upgrade.html | 2 ++ 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/build.gradle b/build.gradle index a733b1af085d2..98ab70d8ca825 100644 --- a/build.gradle +++ b/build.gradle @@ -1430,18 +1430,9 @@ project(':streams') { dependencies { api project(':clients') - - // use `api` dependency for `connect-json` for compatibility (e.g. users who use `JsonSerializer`/`JsonDeserializer` - // at compile-time without an explicit dependency on `connect-json`) - // this dependency should be removed after we unify data API - api(project(':connect:json')) { - // this transitive dependency is not used in Streams, and it breaks SBT builds - exclude module: 'javax.ws.rs-api' - } - // `org.rocksdb.Options` is part of Kafka Streams public api via `RocksDBConfigSetter` api libs.rocksDBJni - + implementation libs.slf4jApi implementation libs.jacksonAnnotations implementation libs.jacksonDatabind @@ -1647,8 +1638,14 @@ project(':streams:examples') { archivesBaseName = "kafka-streams-examples" dependencies { + // this dependency should be removed after we unify data API + implementation(project(':connect:json')) { + // this transitive dependency is not used in Streams, and it breaks SBT builds + exclude module: 'javax.ws.rs-api' + } + implementation project(':streams') - implementation project(':connect:json') // this dependency should be removed after we unify data API + implementation libs.slf4jlog4j testImplementation project(':streams:test-utils') diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 470e591ab4a45..88f700d628502 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -108,6 +108,13 @@

    Streams API
  • Overloaded KafkaStreams#metadataForKey: deprecated in Kafka 2.5.0 (KIP-535).
  • Overloaded KafkaStreams#store: deprecated in Kafka 2.5.0 (KIP-562).
  • +

    + The following dependencies were removed from Kafka Streams: +

    +
      +
    • Connect-json: As of Kafka Streams no longer has a compile time dependency on "connect:json" module (KAFKA-5146). + Projects that were relying on this transitive dependency will have to explicitly declare it.
    • +

    Streams API changes in 2.8.0

    diff --git a/docs/upgrade.html b/docs/upgrade.html index 1eeb6e64baa33..a0bc5af2c1ae6 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -51,6 +51,8 @@

    Notable changes in 3 instead.
  • The Scala kafka.common.MessageFormatter was removed. Plese use the Java org.apache.kafka.common.MessageFormatter.
  • The MessageFormatter.init(Properties) method was removed. Please use configure(Map) instead.
  • +
  • Kafka Streams no longer has a compile time dependency on "connect:json" module (KAFKA-5146). + Projects that were relying on this transitive dependency will have to explicitly declare it.
  • From 174f0f9ea110897797b7dd0bb503f74786f1aa66 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 7 Apr 2021 11:29:14 +0800 Subject: [PATCH 014/155] KAFKA-12384: stabilize ListOffsetsRequestTest#testResponseIncludesLeaderEpoch (#10389) Reviewers: Luke Chen , dengziming , Ismael Juma --- .../kafka/server/ListOffsetsRequestTest.scala | 24 ++++++++++++------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index d52d38469f056..6bbf0a0d46169 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -16,16 +16,16 @@ */ package kafka.server -import java.util.Optional - import kafka.utils.TestUtils import org.apache.kafka.common.message.ListOffsetsRequestData.{ListOffsetsPartition, ListOffsetsTopic} +import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test +import java.util.Optional import scala.jdk.CollectionConverters._ class ListOffsetsRequestTest extends BaseRequestTest { @@ -123,10 +123,9 @@ class ListOffsetsRequestTest extends BaseRequestTest { assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch - 1)) } - // -1 indicate "latest" - def fetchOffsetAndEpoch(serverId: Int, - timestamp: Long, - version: Short): (Long, Int) = { + private[this] def sendRequest(serverId: Int, + timestamp: Long, + version: Short): ListOffsetsPartitionResponse = { val targetTimes = List(new ListOffsetsTopic() .setName(topic) .setPartitions(List(new ListOffsetsPartition() @@ -139,9 +138,15 @@ class ListOffsetsRequestTest extends BaseRequestTest { val request = if (version == -1) builder.build() else builder.build(version) - val response = sendRequest(serverId, request) - val partitionData = response.topics.asScala.find(_.name == topic).get + sendRequest(serverId, request).topics.asScala.find(_.name == topic).get .partitions.asScala.find(_.partitionIndex == partition.partition).get + } + + // -1 indicate "latest" + private[this] def fetchOffsetAndEpoch(serverId: Int, + timestamp: Long, + version: Short): (Long, Int) = { + val partitionData = sendRequest(serverId, timestamp, version) if (version == 0) { if (partitionData.oldStyleOffsets().isEmpty) @@ -166,6 +171,9 @@ class ListOffsetsRequestTest extends BaseRequestTest { // Kill the first leader so that we can verify the epoch change when fetching the latest offset killBroker(firstLeaderId) val secondLeaderId = TestUtils.awaitLeaderChange(servers, partition, firstLeaderId) + // make sure high watermark of new leader has caught up + TestUtils.waitUntilTrue(() => sendRequest(secondLeaderId, 0L, -1).errorCode() != Errors.OFFSET_NOT_AVAILABLE.code(), + "the second leader does not sync to follower") val secondLeaderEpoch = TestUtils.findLeaderEpoch(secondLeaderId, partition, servers) // No changes to written data From be592dfbed716f2ce39e8f8e719a70d0a9d56ae1 Mon Sep 17 00:00:00 2001 From: "high.lee" Date: Wed, 7 Apr 2021 12:39:48 +0900 Subject: [PATCH 015/155] =?UTF-8?q?KAFKA-10769=20Remove=20JoinGroupRequest?= =?UTF-8?q?#containsValidPattern=20as=20it=20is=20dup=E2=80=A6=20(#9851)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Chia-Ping Tsai --- .../apache/kafka/common/internals/Topic.java | 25 +++++++++------ .../common/requests/JoinGroupRequest.java | 31 +++---------------- .../common/requests/JoinGroupRequestTest.java | 12 ------- 3 files changed, 20 insertions(+), 48 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java index a5ef3357dd135..7a5fefb3d9f23 100644 --- a/clients/src/main/java/org/apache/kafka/common/internals/Topic.java +++ b/clients/src/main/java/org/apache/kafka/common/internals/Topic.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.Set; +import java.util.function.Consumer; public class Topic { @@ -34,15 +35,21 @@ public class Topic { private static final int MAX_NAME_LENGTH = 249; public static void validate(String topic) { - if (topic.isEmpty()) - throw new InvalidTopicException("Topic name is illegal, it can't be empty"); - if (topic.equals(".") || topic.equals("..")) - throw new InvalidTopicException("Topic name cannot be \".\" or \"..\""); - if (topic.length() > MAX_NAME_LENGTH) - throw new InvalidTopicException("Topic name is illegal, it can't be longer than " + MAX_NAME_LENGTH + - " characters, topic name: " + topic); - if (!containsValidPattern(topic)) - throw new InvalidTopicException("Topic name \"" + topic + "\" is illegal, it contains a character other than " + + validate(topic, "Topic name", message -> { + throw new InvalidTopicException(message); + }); + } + + public static void validate(String name, String logPrefix, Consumer throwableConsumer) { + if (name.isEmpty()) + throwableConsumer.accept(logPrefix + " is illegal, it can't be empty"); + if (".".equals(name) || "..".equals(name)) + throwableConsumer.accept(logPrefix + " cannot be \".\" or \"..\""); + if (name.length() > MAX_NAME_LENGTH) + throwableConsumer.accept(logPrefix + " is illegal, it can't be longer than " + MAX_NAME_LENGTH + + " characters, " + logPrefix + ": " + name); + if (!containsValidPattern(name)) + throwableConsumer.accept(logPrefix + " \"" + name + "\" is illegal, it contains a character other than " + "ASCII alphanumerics, '.', '_' and '-'"); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java index 35155a085b6f7..220a59d183428 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.protocol.ApiKeys; @@ -59,38 +60,14 @@ public String toString() { public static final int UNKNOWN_GENERATION_ID = -1; public static final String UNKNOWN_PROTOCOL_NAME = ""; - private static final int MAX_GROUP_INSTANCE_ID_LENGTH = 249; - /** * Ported from class Topic in {@link org.apache.kafka.common.internals} to restrict the charset for * static member id. */ public static void validateGroupInstanceId(String id) { - if (id.equals("")) - throw new InvalidConfigurationException("Group instance id must be non-empty string"); - if (id.equals(".") || id.equals("..")) - throw new InvalidConfigurationException("Group instance id cannot be \".\" or \"..\""); - if (id.length() > MAX_GROUP_INSTANCE_ID_LENGTH) - throw new InvalidConfigurationException("Group instance id can't be longer than " + MAX_GROUP_INSTANCE_ID_LENGTH + - " characters: " + id); - if (!containsValidPattern(id)) - throw new InvalidConfigurationException("Group instance id \"" + id + "\" is illegal, it contains a character other than " + - "ASCII alphanumerics, '.', '_' and '-'"); - } - - /** - * Valid characters for Consumer group.instance.id are the ASCII alphanumerics, '.', '_', and '-' - */ - static boolean containsValidPattern(String topic) { - for (int i = 0; i < topic.length(); ++i) { - char c = topic.charAt(i); - - boolean validChar = (c >= 'a' && c <= 'z') || (c >= '0' && c <= '9') || (c >= 'A' && c <= 'Z') || c == '.' || - c == '_' || c == '-'; - if (!validChar) - return false; - } - return true; + Topic.validate(id, "Group instance id", message -> { + throw new InvalidConfigurationException(message); + }); } public JoinGroupRequest(JoinGroupRequestData data, short version) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java index f007bd3f9f3b7..ebf6ef25ebe63 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/JoinGroupRequestTest.java @@ -27,7 +27,6 @@ import java.util.Arrays; 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.fail; @@ -58,17 +57,6 @@ public void shouldThrowOnInvalidGroupInstanceIds() { } } } - - @Test - public void shouldRecognizeInvalidCharactersInGroupInstanceIds() { - char[] invalidChars = {'/', '\\', ',', '\u0000', ':', '"', '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '='}; - - for (char c : invalidChars) { - String instanceId = "Is " + c + "illegal"; - assertFalse(JoinGroupRequest.containsValidPattern(instanceId)); - } - } - @Test public void testRequestVersionCompatibilityFailBuild() { assertThrows(UnsupportedVersionException.class, () -> new JoinGroupRequest.Builder( From e28924062f6cf5702080ca8b8473019bcb09ecae Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 6 Apr 2021 22:29:26 -0700 Subject: [PATCH 016/155] MINOR: move NoOpSnapshotWriter to main (#10496) Move NoOpSnapshotWriter and NoOpSnapshotWriterBuilder out of the test directory and into the main directory, until we implement the KRaft integration. Reviewers: Ismael Juma , Chia-Ping Tsai --- .../org/apache/kafka/controller/NoOpSnapshotWriter.java | 6 ++++++ .../apache/kafka/controller/NoOpSnapshotWriterBuilder.java | 0 .../java/org/apache/kafka/controller/QuorumController.java | 3 +-- 3 files changed, 7 insertions(+), 2 deletions(-) rename metadata/src/{test => main}/java/org/apache/kafka/controller/NoOpSnapshotWriter.java (89%) rename metadata/src/{test => main}/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java (100%) diff --git a/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriter.java b/metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriter.java similarity index 89% rename from metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriter.java rename to metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriter.java index dd8da43f1aa91..0263dd556a58e 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriter.java +++ b/metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriter.java @@ -22,6 +22,12 @@ import org.apache.kafka.metadata.ApiMessageAndVersion; +/** + * The no-op snapshot writer which does nothing. + * + * TODO: This will be moved to the test/ directory once we have the KRaft + * implementation of the snapshot writer. + */ public final class NoOpSnapshotWriter implements SnapshotWriter { private final long epoch; diff --git a/metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java similarity index 100% rename from metadata/src/test/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java rename to metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriterBuilder.java diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 9730f0ca898af..db141c60f1375 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -208,8 +208,7 @@ public QuorumController build() throws Exception { "org.apache.kafka.controller.MockControllerMetrics").getConstructor().newInstance(); } if (snapshotWriterBuilder == null) { - snapshotWriterBuilder = (Function) Class.forName( - "org.apache.kafka.controller.NoOpSnapshotWriterBuilder").getConstructor().newInstance(); + snapshotWriterBuilder = new NoOpSnapshotWriterBuilder(); } if (snapshotReader == null) { snapshotReader = new EmptySnapshotReader(-1); From 999fcba3ec16526c32949836549ae8059ea61ffa Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Wed, 7 Apr 2021 10:34:51 -0700 Subject: [PATCH 017/155] MINOR: refactor ControllerApis#createTopics (#10465) Refactor ControllerApis#createTopics to be easier to unit test. Add unit tests for various invalid request and permission denied scenarios. In ControllerApisTest, statically import all the Errors enums. Implement MockController#createTopics. Reviewers: Chia-Ping Tsai --- .../scala/kafka/server/ControllerApis.scala | 91 ++++++++++--------- .../test/java/kafka/test/MockController.java | 30 +++++- .../kafka/server/ControllerApisTest.scala | 84 +++++++++++------ 3 files changed, 133 insertions(+), 72 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index 1d9fd44e7fc46..cc47ae9ca69e2 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -32,11 +32,10 @@ import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_A import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{ApiException, ClusterAuthorizationException, InvalidRequestException, TopicDeletionDisabledException} import org.apache.kafka.common.internals.FatalExitError -import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection} import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseBroker -import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, BrokerHeartbeatResponseData, BrokerRegistrationResponseData, CreateTopicsResponseData, DeleteTopicsRequestData, DeleteTopicsResponseData, DescribeQuorumResponseData, EndQuorumEpochResponseData, FetchResponseData, MetadataResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, UnregisterBrokerResponseData, VoteResponseData} +import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, BrokerHeartbeatResponseData, BrokerRegistrationResponseData, CreateTopicsRequestData, CreateTopicsResponseData, DeleteTopicsRequestData, DeleteTopicsResponseData, DescribeQuorumResponseData, EndQuorumEpochResponseData, FetchResponseData, MetadataResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, UnregisterBrokerResponseData, VoteResponseData} import org.apache.kafka.common.protocol.Errors.{INVALID_REQUEST, TOPIC_AUTHORIZATION_FAILED} import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors} import org.apache.kafka.common.requests._ @@ -48,7 +47,6 @@ import org.apache.kafka.controller.Controller import org.apache.kafka.metadata.{ApiMessageAndVersion, BrokerHeartbeatReply, BrokerRegistrationReply, VersionRange} import org.apache.kafka.server.authorizer.Authorizer -import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -309,52 +307,57 @@ class ControllerApis(val requestChannel: RequestChannel, } def handleCreateTopics(request: RequestChannel.Request): Unit = { - val createTopicRequest = request.body[CreateTopicsRequest] - val (authorizedCreateRequest, unauthorizedTopics) = - if (authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME)) { - (createTopicRequest.data, Seq.empty) - } else { - val duplicate = createTopicRequest.data.duplicate - val authorizedTopics = new CreatableTopicCollection() - val unauthorizedTopics = mutable.Buffer.empty[String] - - createTopicRequest.data.topics.forEach { topicData => - if (authHelper.authorize(request.context, CREATE, TOPIC, topicData.name)) { - authorizedTopics.add(topicData) - } else { - unauthorizedTopics += topicData.name - } - } - (duplicate.setTopics(authorizedTopics), unauthorizedTopics) - } + val responseData = createTopics(request.body[CreateTopicsRequest].data(), + authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME), + names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity)) + requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { + responseData.setThrottleTimeMs(throttleTimeMs) + new CreateTopicsResponse(responseData) + }) + } - def sendResponse(response: CreateTopicsResponseData): Unit = { - unauthorizedTopics.foreach { topic => - val result = new CreatableTopicResult() - .setName(topic) - .setErrorCode(TOPIC_AUTHORIZATION_FAILED.code) - response.topics.add(result) + def createTopics(request: CreateTopicsRequestData, + hasClusterAuth: Boolean, + getCreatableTopics: Iterable[String] => Set[String]): CreateTopicsResponseData = { + val topicNames = new util.HashSet[String]() + val duplicateTopicNames = new util.HashSet[String]() + request.topics().forEach { topicData => + if (!duplicateTopicNames.contains(topicData.name())) { + if (!topicNames.add(topicData.name())) { + topicNames.remove(topicData.name()) + duplicateTopicNames.add(topicData.name()) + } } - - requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { - response.setThrottleTimeMs(throttleTimeMs) - new CreateTopicsResponse(response) - }) } - - if (authorizedCreateRequest.topics.isEmpty) { - sendResponse(new CreateTopicsResponseData()) + val authorizedTopicNames = if (hasClusterAuth) { + topicNames.asScala } else { - val future = controller.createTopics(authorizedCreateRequest) - future.whenComplete((responseData, exception) => { - val response = if (exception != null) { - createTopicRequest.getErrorResponse(exception).asInstanceOf[CreateTopicsResponse].data - } else { - responseData - } - sendResponse(response) - }) + getCreatableTopics.apply(topicNames.asScala) + } + val effectiveRequest = request.duplicate() + val iterator = effectiveRequest.topics().iterator() + while (iterator.hasNext) { + val creatableTopic = iterator.next() + if (duplicateTopicNames.contains(creatableTopic.name()) || + !authorizedTopicNames.contains(creatableTopic.name())) { + iterator.remove() + } + } + val response = controller.createTopics(effectiveRequest).get() + duplicateTopicNames.forEach { name => + response.topics().add(new CreatableTopicResult(). + setName(name). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Found multiple entries for this topic.")) + } + topicNames.forEach { name => + if (!authorizedTopicNames.contains(name)) { + response.topics().add(new CreatableTopicResult(). + setName(name). + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code())) + } } + response } def handleApiVersionsRequest(request: RequestChannel.Request): Unit = { diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java index 2de88ab961261..5bfddd49df4f5 100644 --- a/core/src/test/java/kafka/test/MockController.java +++ b/core/src/test/java/kafka/test/MockController.java @@ -26,7 +26,9 @@ import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData; import org.apache.kafka.common.message.CreateTopicsRequestData; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; import org.apache.kafka.common.message.CreateTopicsResponseData; +import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; import org.apache.kafka.common.message.ElectLeadersRequestData; import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.protocol.Errors; @@ -43,12 +45,15 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; public class MockController implements Controller { private final static NotControllerException NOT_CONTROLLER_EXCEPTION = new NotControllerException("This is not the correct controller for this cluster."); + private final AtomicLong nextTopicId = new AtomicLong(1); + public static class Builder { private final Map initialTopics = new HashMap<>(); @@ -77,8 +82,29 @@ public CompletableFuture alterIsr(AlterIsrRequestData requ } @Override - public CompletableFuture createTopics(CreateTopicsRequestData request) { - throw new UnsupportedOperationException(); + synchronized public CompletableFuture + createTopics(CreateTopicsRequestData request) { + CreateTopicsResponseData response = new CreateTopicsResponseData(); + for (CreatableTopic topic : request.topics()) { + if (topicNameToId.containsKey(topic.name())) { + response.topics().add(new CreatableTopicResult(). + setName(topic.name()). + setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code())); + } else { + long topicId = nextTopicId.getAndIncrement(); + Uuid topicUuid = new Uuid(0, topicId); + topicNameToId.put(topic.name(), topicUuid); + topics.put(topicUuid, new MockTopic(topic.name(), topicUuid)); + response.topics().add(new CreatableTopicResult(). + setName(topic.name()). + setErrorCode(Errors.NONE.code()). + setTopicId(topicUuid)); + // For a better mock, we might want to return configs, replication factor, + // etc. Right now, the tests that use MockController don't need these + // things. + } + } + return CompletableFuture.completedFuture(response); } @Override diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 6c1f3493e44fa..71b107c8973c9 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -32,11 +32,14 @@ import org.apache.kafka.common.Uuid.ZERO_UUID import org.apache.kafka.common.errors.{InvalidRequestException, NotControllerException, TopicDeletionDisabledException} import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.ApiMessageType.ListenerType +import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} +import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.DeleteTopicsRequestData.DeleteTopicState import org.apache.kafka.common.message.DeleteTopicsResponseData.DeletableTopicResult -import org.apache.kafka.common.message.{BrokerRegistrationRequestData, DeleteTopicsRequestData} +import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreateTopicsRequestData, DeleteTopicsRequestData} import org.apache.kafka.common.network.{ClientInformation, ListenerName} -import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.protocol.Errors.{CLUSTER_AUTHORIZATION_FAILED, INVALID_REQUEST, NONE, TOPIC_AUTHORIZATION_FAILED, UNKNOWN_TOPIC_ID, UNKNOWN_TOPIC_OR_PARTITION} +import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerRegistrationRequest, BrokerRegistrationResponse, RequestContext, RequestHeader, RequestTestUtils} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.controller.Controller @@ -144,10 +147,39 @@ class ControllerApisTest { assertNotNull(capturedResponse.getValue) val brokerRegistrationResponse = capturedResponse.getValue.asInstanceOf[BrokerRegistrationResponse] - assertEquals(Map(Errors.CLUSTER_AUTHORIZATION_FAILED -> 1), + assertEquals(Map(CLUSTER_AUTHORIZATION_FAILED -> 1), brokerRegistrationResponse.errorCounts().asScala) } + @Test + def testCreateTopics(): Unit = { + val controller = new MockController.Builder().build() + val controllerApis = createControllerApis(None, controller) + val request = new CreateTopicsRequestData().setTopics(new CreatableTopicCollection( + util.Arrays.asList(new CreatableTopic().setName("foo").setNumPartitions(1).setReplicationFactor(3), + new CreatableTopic().setName("foo").setNumPartitions(2).setReplicationFactor(3), + new CreatableTopic().setName("bar").setNumPartitions(2).setReplicationFactor(3), + new CreatableTopic().setName("bar").setNumPartitions(2).setReplicationFactor(3), + new CreatableTopic().setName("bar").setNumPartitions(2).setReplicationFactor(3), + new CreatableTopic().setName("baz").setNumPartitions(2).setReplicationFactor(3), + new CreatableTopic().setName("quux").setNumPartitions(2).setReplicationFactor(3), + ).iterator())) + val expectedResponse = Set(new CreatableTopicResult().setName("foo"). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Found multiple entries for this topic."), + new CreatableTopicResult().setName("bar"). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Found multiple entries for this topic."), + new CreatableTopicResult().setName("baz"). + setErrorCode(NONE.code()). + setTopicId(new Uuid(0L, 1L)), + new CreatableTopicResult().setName("quux"). + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code())) + assertEquals(expectedResponse, controllerApis.createTopics(request, + false, + _ => Set("baz")).topics().asScala.toSet) + } + @Test def testDeleteTopicsByName(): Unit = { val fooId = Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q") @@ -156,10 +188,10 @@ class ControllerApisTest { val request = new DeleteTopicsRequestData().setTopicNames( util.Arrays.asList("foo", "bar", "quux", "quux")) val expectedResponse = Set(new DeletableTopicResult().setName("quux"). - setErrorCode(Errors.INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code()). setErrorMessage("Duplicate topic name."), new DeletableTopicResult().setName("bar"). - setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()). + setErrorCode(UNKNOWN_TOPIC_OR_PARTITION.code()). setErrorMessage("This server does not host this topic-partition."), new DeletableTopicResult().setName("foo").setTopicId(fooId)) assertEquals(expectedResponse, controllerApis.deleteTopics(request, @@ -182,10 +214,10 @@ class ControllerApisTest { request.topics().add(new DeleteTopicState().setName(null).setTopicId(quuxId)) request.topics().add(new DeleteTopicState().setName(null).setTopicId(quuxId)) val response = Set(new DeletableTopicResult().setName(null).setTopicId(quuxId). - setErrorCode(Errors.INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code()). setErrorMessage("Duplicate topic id."), new DeletableTopicResult().setName(null).setTopicId(barId). - setErrorCode(Errors.UNKNOWN_TOPIC_ID.code()). + setErrorCode(UNKNOWN_TOPIC_ID.code()). setErrorMessage("This server does not host this topic ID."), new DeletableTopicResult().setName("foo").setTopicId(fooId)) assertEquals(response, controllerApis.deleteTopics(request, @@ -216,19 +248,19 @@ class ControllerApisTest { request.topics().add(new DeleteTopicState().setName(null).setTopicId(bazId)) request.topics().add(new DeleteTopicState().setName(null).setTopicId(bazId)) val response = Set(new DeletableTopicResult().setName(null).setTopicId(ZERO_UUID). - setErrorCode(Errors.INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code()). setErrorMessage("Neither topic name nor id were specified."), new DeletableTopicResult().setName("foo").setTopicId(fooId). - setErrorCode(Errors.INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code()). setErrorMessage("You may not specify both topic name and topic id."), new DeletableTopicResult().setName("bar").setTopicId(barId). - setErrorCode(Errors.INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code()). setErrorMessage("The provided topic name maps to an ID that was already supplied."), new DeletableTopicResult().setName("quux").setTopicId(ZERO_UUID). - setErrorCode(Errors.INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code()). setErrorMessage("Duplicate topic name."), new DeletableTopicResult().setName(null).setTopicId(bazId). - setErrorCode(Errors.INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code()). setErrorMessage("Duplicate topic id.")) assertEquals(response, controllerApis.deleteTopics(request, ApiKeys.DELETE_TOPICS.latestVersion().toInt, @@ -255,17 +287,17 @@ class ControllerApisTest { request.topics().add(new DeleteTopicState().setName("baz").setTopicId(ZERO_UUID)) request.topics().add(new DeleteTopicState().setName("quux").setTopicId(ZERO_UUID)) val response = Set(new DeletableTopicResult().setName(null).setTopicId(barId). - setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code). - setErrorMessage(Errors.TOPIC_AUTHORIZATION_FAILED.message), + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code). + setErrorMessage(TOPIC_AUTHORIZATION_FAILED.message), new DeletableTopicResult().setName("quux").setTopicId(ZERO_UUID). - setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code). - setErrorMessage(Errors.TOPIC_AUTHORIZATION_FAILED.message), + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code). + setErrorMessage(TOPIC_AUTHORIZATION_FAILED.message), new DeletableTopicResult().setName("baz").setTopicId(ZERO_UUID). - setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code). - setErrorMessage(Errors.TOPIC_AUTHORIZATION_FAILED.message), + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code). + setErrorMessage(TOPIC_AUTHORIZATION_FAILED.message), new DeletableTopicResult().setName("foo").setTopicId(fooId). - setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code). - setErrorMessage(Errors.TOPIC_AUTHORIZATION_FAILED.message)) + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code). + setErrorMessage(TOPIC_AUTHORIZATION_FAILED.message)) assertEquals(response, controllerApis.deleteTopics(request, ApiKeys.DELETE_TOPICS.latestVersion().toInt, false, @@ -283,14 +315,14 @@ class ControllerApisTest { request.topics().add(new DeleteTopicState().setName("bar").setTopicId(ZERO_UUID)) request.topics().add(new DeleteTopicState().setName(null).setTopicId(barId)) val expectedResponse = Set(new DeletableTopicResult().setName("foo"). - setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code). - setErrorMessage(Errors.UNKNOWN_TOPIC_OR_PARTITION.message), + setErrorCode(UNKNOWN_TOPIC_OR_PARTITION.code). + setErrorMessage(UNKNOWN_TOPIC_OR_PARTITION.message), new DeletableTopicResult().setName("bar"). - setErrorCode(Errors.TOPIC_AUTHORIZATION_FAILED.code). - setErrorMessage(Errors.TOPIC_AUTHORIZATION_FAILED.message), + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code). + setErrorMessage(TOPIC_AUTHORIZATION_FAILED.message), new DeletableTopicResult().setName(null).setTopicId(barId). - setErrorCode(Errors.UNKNOWN_TOPIC_ID.code). - setErrorMessage(Errors.UNKNOWN_TOPIC_ID.message)) + setErrorCode(UNKNOWN_TOPIC_ID.code). + setErrorMessage(UNKNOWN_TOPIC_ID.message)) assertEquals(expectedResponse, controllerApis.deleteTopics(request, ApiKeys.DELETE_TOPICS.latestVersion().toInt, false, From 37493d1e18d9ec85b392e852032c85a57bbc3d28 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Sergio=20Pe=C3=B1a?= Date: Wed, 7 Apr 2021 15:35:12 -0500 Subject: [PATCH 018/155] KAFKA-10847: Add new RocksDBTimeOrderedWindowStore that persists (time-key)-value records (#10331) This new store is more efficient when calling range queries with only time parameters, like `fetch(from, to)`. For range queries using key ranges, then the current RocksDBWindowStore should be used. Reviewers: Guozhang Wang --- ...ngeLoggingTimestampedWindowBytesStore.java | 2 +- .../ChangeLoggingWindowBytesStore.java | 12 +- .../RocksDBTimeOrderedWindowStore.java | 236 ++++++++++++++++++ .../RocksDbWindowBytesStoreSupplier.java | 79 ++++-- .../state/internals/TimeOrderedKeySchema.java | 215 ++++++++++++++++ .../TimeOrderedWindowStoreBuilder.java | 84 +++++++ .../state/internals/WindowStoreBuilder.java | 6 +- ...bstractRocksDBSegmentedBytesStoreTest.java | 17 +- .../AbstractWindowBytesStoreTest.java | 84 ++++++- .../ChangeLoggingWindowBytesStoreTest.java | 4 +- .../RocksDBTimeOrderedWindowStoreTest.java | 70 ++++++ .../internals/TimeOrderedKeySchemaTest.java | 177 +++++++++++++ .../state/internals/WindowKeySchemaTest.java | 2 +- 13 files changed, 954 insertions(+), 34 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java index 37141500d5791..85846165c2864 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStore.java @@ -26,7 +26,7 @@ class ChangeLoggingTimestampedWindowBytesStore extends ChangeLoggingWindowBytesS ChangeLoggingTimestampedWindowBytesStore(final WindowStore bytesStore, final boolean retainDuplicates) { - super(bytesStore, retainDuplicates); + super(bytesStore, retainDuplicates, WindowKeySchema::toStoreKeyBinary); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 47f088e7c564f..6a367b1c3cfc1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import static java.util.Objects.requireNonNull; import static org.apache.kafka.streams.processor.internals.ProcessorContextUtils.asInternalProcessorContext; /** @@ -36,14 +37,21 @@ class ChangeLoggingWindowBytesStore extends WrappedStateStore, byte[], byte[]> implements WindowStore { + interface ChangeLoggingKeySerializer { + Bytes serialize(final Bytes key, final long timestamp, final int seqnum); + } + private final boolean retainDuplicates; InternalProcessorContext context; private int seqnum = 0; + private final ChangeLoggingKeySerializer keySerializer; ChangeLoggingWindowBytesStore(final WindowStore bytesStore, - final boolean retainDuplicates) { + final boolean retainDuplicates, + final ChangeLoggingKeySerializer keySerializer) { super(bytesStore); this.retainDuplicates = retainDuplicates; + this.keySerializer = requireNonNull(keySerializer, "keySerializer"); } @Deprecated @@ -138,7 +146,7 @@ public void put(final Bytes key, final byte[] value, final long windowStartTimestamp) { wrapped().put(key, value, windowStartTimestamp); - log(WindowKeySchema.toStoreKeyBinary(key, windowStartTimestamp, maybeUpdateSeqnumForDups()), value); + log(keySerializer.serialize(key, windowStartTimestamp, maybeUpdateSeqnumForDups()), value); } void log(final Bytes key, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java new file mode 100644 index 0000000000000..9316d0cbbf6bc --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java @@ -0,0 +1,236 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; + +/** + * A persistent (time-key)-value store based on RocksDB. + * + * The store uses the {@link TimeOrderedKeySchema} to serialize the record key bytes to generate the + * combined (time-key) store key. This key schema is efficient when doing time range queries in + * the store (i.e. fetchAll(from, to) ). + * + * For key range queries, like fetch(key, fromTime, toTime), use the {@link RocksDBWindowStore} + * which uses the {@link WindowKeySchema} to serialize the record bytes for efficient key queries. + */ +public class RocksDBTimeOrderedWindowStore + extends WrappedStateStore + implements WindowStore { + + private final boolean retainDuplicates; + private final long windowSize; + + private InternalProcessorContext context; + private int seqnum = 0; + + RocksDBTimeOrderedWindowStore(final SegmentedBytesStore bytesStore, + final boolean retainDuplicates, + final long windowSize) { + super(bytesStore); + this.retainDuplicates = retainDuplicates; + this.windowSize = windowSize; + } + + @Deprecated + @Override + public void init(final ProcessorContext context, final StateStore root) { + this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; + super.init(context, root); + } + + @Override + public void init(final StateStoreContext context, final StateStore root) { + this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; + super.init(context, root); + } + + @Deprecated + @Override + public void put(final Bytes key, final byte[] value) { + put(key, value, context != null ? context.timestamp() : 0L); + } + + @Override + public void put(final Bytes key, final byte[] value, final long timestamp) { + // Skip if value is null and duplicates are allowed since this delete is a no-op + if (!(value == null && retainDuplicates)) { + maybeUpdateSeqnumForDups(); + wrapped().put(TimeOrderedKeySchema.toStoreKeyBinary(key, timestamp, seqnum), value); + } + } + + @Override + public byte[] fetch(final Bytes key, final long timestamp) { + return wrapped().get(TimeOrderedKeySchema.toStoreKeyBinary(key, timestamp, seqnum)); + } + + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed + @Override + public WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo) { + final KeyValueIterator bytesIterator = wrapped().fetch(key, timeFrom, timeTo); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); + } + + @Override + public WindowStoreIterator backwardFetch(final Bytes key, final long timeFrom, final long timeTo) { + final KeyValueIterator bytesIterator = wrapped().backwardFetch(key, timeFrom, timeTo); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); + } + + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed + @Override + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo, + final long timeFrom, + final long timeTo) { + final KeyValueIterator bytesIterator = wrapped().fetch(keyFrom, keyTo, timeFrom, timeTo); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + + @Override + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo, + final long timeFrom, + final long timeTo) { + final KeyValueIterator bytesIterator = wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + + @Override + public KeyValueIterator, byte[]> all() { + final KeyValueIterator bytesIterator = wrapped().all(); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + + @Override + public KeyValueIterator, byte[]> backwardAll() { + final KeyValueIterator bytesIterator = wrapped().backwardAll(); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + + @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed + @Override + public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { + final KeyValueIterator bytesIterator = wrapped().fetchAll(timeFrom, timeTo); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + + @Override + public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, final long timeTo) { + final KeyValueIterator bytesIterator = wrapped().backwardFetchAll(timeFrom, timeTo); + return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + } + + private void maybeUpdateSeqnumForDups() { + if (retainDuplicates) { + seqnum = (seqnum + 1) & 0x7FFFFFFF; + } + } + + static class TimeOrderedWindowStoreIteratorWrapper { + private final KeyValueIterator bytesIterator; + private final long windowSize; + + TimeOrderedWindowStoreIteratorWrapper(final KeyValueIterator bytesIterator, + final long windowSize) { + this.bytesIterator = bytesIterator; + this.windowSize = windowSize; + } + + public WindowStoreIterator valuesIterator() { + return new WrappedWindowStoreIterator(bytesIterator); + } + + public KeyValueIterator, byte[]> keyValueIterator() { + return new WrappedKeyValueIterator(bytesIterator, windowSize); + } + + private static class WrappedWindowStoreIterator implements WindowStoreIterator { + final KeyValueIterator bytesIterator; + + WrappedWindowStoreIterator( + final KeyValueIterator bytesIterator) { + this.bytesIterator = bytesIterator; + } + + @Override + public Long peekNextKey() { + return TimeOrderedKeySchema.extractStoreTimestamp(bytesIterator.peekNextKey().get()); + } + + @Override + public boolean hasNext() { + return bytesIterator.hasNext(); + } + + @Override + public KeyValue next() { + final KeyValue next = bytesIterator.next(); + final long timestamp = TimeOrderedKeySchema.extractStoreTimestamp(next.key.get()); + return KeyValue.pair(timestamp, next.value); + } + + @Override + public void close() { + bytesIterator.close(); + } + } + + private static class WrappedKeyValueIterator implements KeyValueIterator, byte[]> { + final KeyValueIterator bytesIterator; + final long windowSize; + + WrappedKeyValueIterator(final KeyValueIterator bytesIterator, + final long windowSize) { + this.bytesIterator = bytesIterator; + this.windowSize = windowSize; + } + + @Override + public Windowed peekNextKey() { + final byte[] nextKey = bytesIterator.peekNextKey().get(); + return TimeOrderedKeySchema.fromStoreBytesKey(nextKey, windowSize); + } + + @Override + public boolean hasNext() { + return bytesIterator.hasNext(); + } + + @Override + public KeyValue, byte[]> next() { + final KeyValue next = bytesIterator.next(); + return KeyValue.pair(TimeOrderedKeySchema.fromStoreBytesKey(next.key.get(), windowSize), next.value); + } + + @Override + public void close() { + bytesIterator.close(); + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java index 5ccdd7a174653..937e70e7036c5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDbWindowBytesStoreSupplier.java @@ -21,12 +21,18 @@ import org.apache.kafka.streams.state.WindowStore; public class RocksDbWindowBytesStoreSupplier implements WindowBytesStoreSupplier { + public enum WindowStoreTypes { + DEFAULT_WINDOW_STORE, + TIMESTAMPED_WINDOW_STORE, + TIME_ORDERED_WINDOW_STORE + } + private final String name; private final long retentionPeriod; private final long segmentInterval; private final long windowSize; private final boolean retainDuplicates; - private final boolean returnTimestampedStore; + private final WindowStoreTypes windowStoreType; public RocksDbWindowBytesStoreSupplier(final String name, final long retentionPeriod, @@ -34,12 +40,24 @@ public RocksDbWindowBytesStoreSupplier(final String name, final long windowSize, final boolean retainDuplicates, final boolean returnTimestampedStore) { + this(name, retentionPeriod, segmentInterval, windowSize, retainDuplicates, + returnTimestampedStore + ? WindowStoreTypes.TIMESTAMPED_WINDOW_STORE + : WindowStoreTypes.DEFAULT_WINDOW_STORE); + } + + public RocksDbWindowBytesStoreSupplier(final String name, + final long retentionPeriod, + final long segmentInterval, + final long windowSize, + final boolean retainDuplicates, + final WindowStoreTypes windowStoreType) { this.name = name; this.retentionPeriod = retentionPeriod; this.segmentInterval = segmentInterval; this.windowSize = windowSize; this.retainDuplicates = retainDuplicates; - this.returnTimestampedStore = returnTimestampedStore; + this.windowStoreType = windowStoreType; } @Override @@ -49,26 +67,41 @@ public String name() { @Override public WindowStore get() { - if (!returnTimestampedStore) { - return new RocksDBWindowStore( - new RocksDBSegmentedBytesStore( - name, - metricsScope(), - retentionPeriod, - segmentInterval, - new WindowKeySchema()), - retainDuplicates, - windowSize); - } else { - return new RocksDBTimestampedWindowStore( - new RocksDBTimestampedSegmentedBytesStore( - name, - metricsScope(), - retentionPeriod, - segmentInterval, - new WindowKeySchema()), - retainDuplicates, - windowSize); + switch (windowStoreType) { + case DEFAULT_WINDOW_STORE: + return new RocksDBWindowStore( + new RocksDBSegmentedBytesStore( + name, + metricsScope(), + retentionPeriod, + segmentInterval, + new WindowKeySchema()), + retainDuplicates, + windowSize); + case TIMESTAMPED_WINDOW_STORE: + return new RocksDBTimestampedWindowStore( + new RocksDBTimestampedSegmentedBytesStore( + name, + metricsScope(), + retentionPeriod, + segmentInterval, + new WindowKeySchema()), + retainDuplicates, + windowSize); + case TIME_ORDERED_WINDOW_STORE: + return new RocksDBTimeOrderedWindowStore( + new RocksDBSegmentedBytesStore( + name, + metricsScope(), + retentionPeriod, + segmentInterval, + new TimeOrderedKeySchema() + ), + retainDuplicates, + windowSize + ); + default: + throw new IllegalArgumentException("invalid window store type: " + windowStoreType); } } @@ -105,7 +138,7 @@ public String toString() { ", segmentInterval=" + segmentInterval + ", windowSize=" + windowSize + ", retainDuplicates=" + retainDuplicates + - ", returnTimestampedStore=" + returnTimestampedStore + + ", windowStoreType=" + windowStoreType + '}'; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java new file mode 100644 index 0000000000000..b834b3a1ab1a8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java @@ -0,0 +1,215 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.TimeWindow; +import org.apache.kafka.streams.state.StateSerdes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.List; + +/** + * A {@link RocksDBSegmentedBytesStore.KeySchema} to serialize/deserialize a RocksDB store + * key into a schema combined of (time,seq,key). This key schema is more efficient when doing + * range queries between a time interval. For key range queries better use {@link WindowKeySchema}. + */ +public class TimeOrderedKeySchema implements RocksDBSegmentedBytesStore.KeySchema { + private static final Logger LOG = LoggerFactory.getLogger(TimeOrderedKeySchema.class); + + private static final int TIMESTAMP_SIZE = 8; + private static final int SEQNUM_SIZE = 4; + private static final int PREFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE; + + /** + * {@inheritdoc} + * + * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key + * range queries may be slower. If better performance on key range queries are necessary, then + * use the {@link WindowKeySchema}. + */ + @Override + public Bytes upperRange(final Bytes key, final long to) { + return toStoreKeyBinary(key.get(), to, Integer.MAX_VALUE); + } + + /** + * {@inheritdoc} + * + * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key + * range queries may be slower. If better performance on key range queries are necessary, then + * use the {@link WindowKeySchema}. + */ + @Override + public Bytes lowerRange(final Bytes key, final long from) { + return toStoreKeyBinary(key.get(), from, 0); + } + + /** + * {@inheritdoc} + * + * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key + * range queries may be slower. If better performance on key range queries are necessary, then + * use the {@link WindowKeySchema}. + */ + @Override + public Bytes upperRangeFixedSize(final Bytes key, final long to) { + return toStoreKeyBinary(key, to, Integer.MAX_VALUE); + } + + /** + * {@inheritdoc} + * + * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key + * range queries may be slower. If better performance on key range queries are necessary, then + * use the {@link WindowKeySchema}. + */ + @Override + public Bytes lowerRangeFixedSize(final Bytes key, final long from) { + return toStoreKeyBinary(key, Math.max(0, from), 0); + } + + @Override + public long segmentTimestamp(final Bytes key) { + return extractStoreTimestamp(key.get()); + } + + /** + * {@inheritdoc} + * + * This method is not optimized for {@link TimeOrderedKeySchema}. The method may do unnecessary + * checks to find the next record. + */ + @Override + public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, final Bytes binaryKeyTo, final long from, final long to) { + return iterator -> { + while (iterator.hasNext()) { + final Bytes bytes = iterator.peekNextKey(); + final Bytes keyBytes = Bytes.wrap(extractStoreKeyBytes(bytes.get())); + final long time = extractStoreTimestamp(bytes.get()); + if ((binaryKeyFrom == null || keyBytes.compareTo(binaryKeyFrom) >= 0) + && (binaryKeyTo == null || keyBytes.compareTo(binaryKeyTo) <= 0) + && time >= from + && time <= to) { + return true; + } + iterator.next(); + } + return false; + }; + } + + @Override + public List segmentsToSearch(final Segments segments, final long from, final long to, final boolean forward) { + return segments.segments(from, to, forward); + } + + public static Bytes toStoreKeyBinary(final Bytes key, + final long timestamp, + final int seqnum) { + final byte[] serializedKey = key.get(); + return toStoreKeyBinary(serializedKey, timestamp, seqnum); + } + + public static Bytes toStoreKeyBinary(final Windowed timeKey, + final int seqnum) { + final byte[] bytes = timeKey.key().get(); + return toStoreKeyBinary(bytes, timeKey.window().start(), seqnum); + } + + public static Bytes toStoreKeyBinary(final Windowed timeKey, + final int seqnum, + final StateSerdes serdes) { + final byte[] serializedKey = serdes.rawKey(timeKey.key()); + return toStoreKeyBinary(serializedKey, timeKey.window().start(), seqnum); + } + + // package private for testing + static Bytes toStoreKeyBinary(final byte[] serializedKey, + final long timestamp, + final int seqnum) { + final ByteBuffer buf = ByteBuffer.allocate(TIMESTAMP_SIZE + serializedKey.length + SEQNUM_SIZE); + buf.putLong(timestamp); + buf.putInt(seqnum); + buf.put(serializedKey); + return Bytes.wrap(buf.array()); + } + + static byte[] extractStoreKeyBytes(final byte[] binaryKey) { + final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - SEQNUM_SIZE]; + System.arraycopy(binaryKey, PREFIX_SIZE, bytes, 0, bytes.length); + return bytes; + } + + static K extractStoreKey(final byte[] binaryKey, + final StateSerdes serdes) { + final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - SEQNUM_SIZE]; + System.arraycopy(binaryKey, PREFIX_SIZE, bytes, 0, bytes.length); + return serdes.keyFrom(bytes); + } + + static long extractStoreTimestamp(final byte[] binaryKey) { + return ByteBuffer.wrap(binaryKey).getLong(0); + } + + static int extractStoreSequence(final byte[] binaryKey) { + return ByteBuffer.wrap(binaryKey).getInt(TIMESTAMP_SIZE); + } + + static Windowed fromStoreKey(final byte[] binaryKey, + final long windowSize, + final Deserializer deserializer, + final String topic) { + final K key = deserializer.deserialize(topic, extractStoreKeyBytes(binaryKey)); + final Window window = extractStoreWindow(binaryKey, windowSize); + return new Windowed<>(key, window); + } + + static Windowed fromStoreBytesKey(final byte[] binaryKey, + final long windowSize) { + final Bytes key = Bytes.wrap(extractStoreKeyBytes(binaryKey)); + final Window window = extractStoreWindow(binaryKey, windowSize); + return new Windowed<>(key, window); + } + + static Window extractStoreWindow(final byte[] binaryKey, + final long windowSize) { + final ByteBuffer buffer = ByteBuffer.wrap(binaryKey); + final long start = buffer.getLong(0); + return timeWindowForSize(start, windowSize); + } + + /** + * Safely construct a time window of the given size, + * taking care of bounding endMs to Long.MAX_VALUE if necessary + */ + static TimeWindow timeWindowForSize(final long startMs, + final long windowSize) { + long endMs = startMs + windowSize; + + if (endMs < 0) { + LOG.warn("Warning: window end time was truncated to Long.MAX"); + endMs = Long.MAX_VALUE; + } + return new TimeWindow(startMs, endMs); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java new file mode 100644 index 0000000000000..78c6774edf2dd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java @@ -0,0 +1,84 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; +import org.apache.kafka.streams.state.WindowStore; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; + +public class TimeOrderedWindowStoreBuilder extends AbstractStoreBuilder> { + private final Logger log = LoggerFactory.getLogger(WindowStoreBuilder.class); + + private final WindowBytesStoreSupplier storeSupplier; + + public TimeOrderedWindowStoreBuilder(final WindowBytesStoreSupplier storeSupplier, + final Serde keySerde, + final Serde valueSerde, + final Time time) { + super(storeSupplier.name(), keySerde, valueSerde, time); + Objects.requireNonNull(storeSupplier, "storeSupplier can't be null"); + Objects.requireNonNull(storeSupplier.metricsScope(), "storeSupplier's metricsScope can't be null"); + this.storeSupplier = storeSupplier; + } + + @Override + public WindowStore build() { + if (storeSupplier.retainDuplicates() && enableCaching) { + log.warn("Disabling caching for {} since store was configured to retain duplicates", storeSupplier.name()); + enableCaching = false; + } + + return new MeteredWindowStore<>( + maybeWrapCaching(maybeWrapLogging(storeSupplier.get())), + storeSupplier.windowSize(), + storeSupplier.metricsScope(), + time, + keySerde, + valueSerde); + } + + private WindowStore maybeWrapCaching(final WindowStore inner) { + if (!enableCaching) { + return inner; + } + return new CachingWindowStore( + inner, + storeSupplier.windowSize(), + storeSupplier.segmentIntervalMs()); + } + + private WindowStore maybeWrapLogging(final WindowStore inner) { + if (!enableLogging) { + return inner; + } + return new ChangeLoggingWindowBytesStore( + inner, + storeSupplier.retainDuplicates(), + TimeOrderedKeySchema::toStoreKeyBinary + ); + } + + public long retentionPeriod() { + return storeSupplier.retentionPeriod(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java index 5876f78c64e97..c866936e7dccc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java @@ -71,7 +71,11 @@ private WindowStore maybeWrapLogging(final WindowStore @Parameters(name = "{0}") public static Object[] getKeySchemas() { - return new Object[] {new SessionKeySchema(), new WindowKeySchema()}; + return new Object[] {new SessionKeySchema(), new WindowKeySchema(), new TimeOrderedKeySchema()}; } @Before @@ -109,7 +109,7 @@ public void before() { // expire it. nextSegmentWindow = new SessionWindow(segmentInterval + retention, segmentInterval + retention); } - if (schema instanceof WindowKeySchema) { + if (schema instanceof WindowKeySchema || schema instanceof TimeOrderedKeySchema) { windows[0] = timeWindowForSize(10L, windowSizeForTimeWindow); windows[1] = timeWindowForSize(500L, windowSizeForTimeWindow); windows[2] = timeWindowForSize(1_000L, windowSizeForTimeWindow); @@ -482,6 +482,8 @@ private Bytes serializeKey(final Windowed key) { final StateSerdes stateSerdes = StateSerdes.withBuiltinTypes("dummy", String.class, Long.class); if (schema instanceof SessionKeySchema) { return Bytes.wrap(SessionKeySchema.toBinary(key, stateSerdes.keySerializer(), "dummy")); + } else if (schema instanceof TimeOrderedKeySchema) { + return TimeOrderedKeySchema.toStoreKeyBinary(key, 0, stateSerdes); } else { return WindowKeySchema.toStoreKeyBinary(key, 0, stateSerdes); } @@ -507,6 +509,17 @@ private List, Long>> toList(final KeyValueIterator, Long> deserialized = KeyValue.pair( + TimeOrderedKeySchema.fromStoreKey( + next.key.get(), + windowSizeForTimeWindow, + stateSerdes.keyDeserializer(), + stateSerdes.topic() + ), + stateSerdes.valueDeserializer().deserialize("dummy", next.value) + ); + results.add(deserialized); } else { final KeyValue, Long> deserialized = KeyValue.pair( SessionKeySchema.from(next.key.get(), stateSerdes.keyDeserializer(), "dummy"), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 6aeb28d54b267..aa4a412747245 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -278,6 +278,77 @@ public void shouldGetAll() { ); } + @Test + public void shouldGetAllNonDeletedRecords() { + final long startTime = SEGMENT_INTERVAL - 4L; + + // Add some records + windowStore.put(0, "zero", startTime + 0); + windowStore.put(1, "one", startTime + 1); + windowStore.put(2, "two", startTime + 2); + windowStore.put(3, "three", startTime + 3); + windowStore.put(4, "four", startTime + 4); + + // Delete some records + windowStore.put(1, null, startTime + 1); + windowStore.put(3, null, startTime + 3); + + // Only non-deleted records should appear in the all() iterator + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + + assertEquals( + asList(zero, two, four), + toList(windowStore.all()) + ); + } + + @Test + public void shouldGetAllReturnTimestampOrderedRecords() { + final long startTime = SEGMENT_INTERVAL - 4L; + + // Add some records in different order + windowStore.put(4, "four", startTime + 4); + windowStore.put(0, "zero", startTime + 0); + windowStore.put(2, "two", startTime + 2); + windowStore.put(3, "three", startTime + 3); + windowStore.put(1, "one", startTime + 1); + + // Only non-deleted records should appear in the all() iterator + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> three = windowedPair(3, "three", startTime + 3); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + + assertEquals( + asList(zero, one, two, three, four), + toList(windowStore.all()) + ); + } + + @Test + public void shouldEarlyClosedIteratorStillGetAllRecords() { + final long startTime = SEGMENT_INTERVAL - 4L; + + windowStore.put(0, "zero", startTime + 0); + windowStore.put(1, "one", startTime + 1); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + + final KeyValueIterator, String> it = windowStore.all(); + assertEquals(zero, it.next()); + it.close(); + + // A new all() iterator after a previous all() iterator was closed should return all elements. + assertEquals( + asList(zero, one), + toList(windowStore.all()) + ); + } + @Test public void shouldGetBackwardAll() { final long startTime = SEGMENT_INTERVAL - 4L; @@ -1173,14 +1244,23 @@ private void putSecondBatch(final WindowStore store, store.put(2, "two+6"); } + long extractStoreTimestamp(final byte[] binaryKey) { + return WindowKeySchema.extractStoreTimestamp(binaryKey); + } + + K extractStoreKey(final byte[] binaryKey, + final StateSerdes serdes) { + return WindowKeySchema.extractStoreKey(binaryKey, serdes); + } + private Map> entriesByKey(final List> changeLog, @SuppressWarnings("SameParameterValue") final long startTime) { final HashMap> entriesByKey = new HashMap<>(); for (final KeyValue entry : changeLog) { - final long timestamp = WindowKeySchema.extractStoreTimestamp(entry.key); + final long timestamp = extractStoreTimestamp(entry.key); - final Integer key = WindowKeySchema.extractStoreKey(entry.key, serdes); + final Integer key = extractStoreKey(entry.key, serdes); final String value = entry.value == null ? null : serdes.valueFrom(entry.value); final Set entries = entriesByKey.computeIfAbsent(key, k -> new HashSet<>()); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index c2c31e0729880..b19b99ab98947 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -51,7 +51,7 @@ public class ChangeLoggingWindowBytesStoreTest { @Before public void setUp() { - store = new ChangeLoggingWindowBytesStore(inner, false); + store = new ChangeLoggingWindowBytesStore(inner, false, WindowKeySchema::toStoreKeyBinary); } private void init() { @@ -154,7 +154,7 @@ public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { @Test @SuppressWarnings("deprecation") public void shouldRetainDuplicatesWhenSet() { - store = new ChangeLoggingWindowBytesStore(inner, true); + store = new ChangeLoggingWindowBytesStore(inner, true, WindowKeySchema::toStoreKeyBinary); inner.put(bytesKey, value, 0); EasyMock.expectLastCall().times(2); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java new file mode 100644 index 0000000000000..37f98f1a43d4f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java @@ -0,0 +1,70 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.WindowStore; + +import static org.apache.kafka.streams.state.internals.RocksDbWindowBytesStoreSupplier.WindowStoreTypes; + +public class RocksDBTimeOrderedWindowStoreTest extends RocksDBWindowStoreTest { + private static final String STORE_NAME = "rocksDB window store"; + + @Override + WindowStore buildWindowStore(final long retentionPeriod, + final long windowSize, + final boolean retainDuplicates, + final Serde keySerde, + final Serde valueSerde) { + return new TimeOrderedWindowStoreBuilder<>( + new RocksDbWindowBytesStoreSupplier( + STORE_NAME, + retentionPeriod, + Math.max(retentionPeriod / 2, 60_000L), + windowSize, + retainDuplicates, + WindowStoreTypes.TIME_ORDERED_WINDOW_STORE), + keySerde, + valueSerde, + Time.SYSTEM) + .build(); + } + + @Override + String getMetricsScope() { + return new RocksDbWindowBytesStoreSupplier(null, 0, 0, 0, false, WindowStoreTypes.TIME_ORDERED_WINDOW_STORE).metricsScope(); + } + + @Override + void setClassLoggerToDebug() { + LogCaptureAppender.setClassLoggerToDebug(AbstractRocksDBSegmentedBytesStore.class); + } + + @Override + long extractStoreTimestamp(final byte[] binaryKey) { + return TimeOrderedKeySchema.extractStoreTimestamp(binaryKey); + } + + @Override + K extractStoreKey(final byte[] binaryKey, + final StateSerdes serdes) { + return TimeOrderedKeySchema.extractStoreKey(binaryKey, serdes); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java new file mode 100644 index 0000000000000..5a5c4fc47a5a4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java @@ -0,0 +1,177 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.TimeWindow; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.KeyValueIteratorStub; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +public class TimeOrderedKeySchemaTest { + final private String key = "key"; + final private long startTime = 50L; + final private long endTime = 100L; + final private Serde serde = Serdes.String(); + + final private Window window = new TimeWindow(startTime, endTime); + final private Windowed windowedKey = new Windowed<>(key, window); + final private TimeOrderedKeySchema timeOrderedKeySchema = new TimeOrderedKeySchema(); + final private StateSerdes stateSerdes = new StateSerdes<>("dummy", serde, Serdes.ByteArray()); + + @Test + public void testHasNextConditionUsingNullKeys() { + final List> keys = Arrays.asList( + KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0}), new TimeWindow(0, 1)), 0), 1), + KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0}), new TimeWindow(0, 1)), 0), 2), + KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0, 0}), new TimeWindow(0, 1)), 0), 3), + KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0}), new TimeWindow(10, 20)), 4), 4), + KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0}), new TimeWindow(10, 20)), 5), 5), + KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0, 0}), new TimeWindow(10, 20)), 6), 6)); + final DelegatingPeekingKeyValueIterator iterator = new DelegatingPeekingKeyValueIterator<>("foo", new KeyValueIteratorStub<>(keys.iterator())); + + final HasNextCondition hasNextCondition = timeOrderedKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE); + final List results = new ArrayList<>(); + while (hasNextCondition.hasNext(iterator)) { + results.add(iterator.next().value); + } + assertThat(results, equalTo(Arrays.asList(1, 2, 3, 4, 5, 6))); + } + + @Test + public void testUpperBoundWithLargeTimestamps() { + final Bytes upper = timeOrderedKeySchema.upperRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), Long.MAX_VALUE); + + assertThat( + "shorter key with max timestamp should be in range", + upper.compareTo( + TimeOrderedKeySchema.toStoreKeyBinary( + new byte[] {0xA}, + Long.MAX_VALUE, + Integer.MAX_VALUE + ) + ) >= 0 + ); + + assertThat( + "shorter key with max timestamp should be in range", + upper.compareTo( + TimeOrderedKeySchema.toStoreKeyBinary( + new byte[] {0xA, 0xB}, + Long.MAX_VALUE, + Integer.MAX_VALUE + ) + ) >= 0 + ); + + assertThat(upper, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, Long.MAX_VALUE, Integer.MAX_VALUE))); + } + + @Test + public void testUpperBoundWithZeroTimestamp() { + final Bytes upper = timeOrderedKeySchema.upperRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 0); + assertThat(upper, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, Integer.MAX_VALUE))); + } + + @Test + public void testLowerBoundWithZeroTimestamp() { + final Bytes lower = timeOrderedKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 0); + assertThat(lower, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, 0))); + } + + @Test + public void testLowerBoundWithNonZeroTimestamp() { + final Bytes lower = timeOrderedKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 42); + assertThat(lower, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 42, 0))); + } + + @Test + public void testLowerBoundMatchesTrailingZeros() { + final Bytes lower = timeOrderedKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), Long.MAX_VALUE - 1); + + assertThat( + "appending zeros to key should still be in range", + lower.compareTo( + TimeOrderedKeySchema.toStoreKeyBinary( + new byte[] {0xA, 0xB, 0xC, 0, 0, 0, 0, 0, 0, 0, 0}, + Long.MAX_VALUE - 1, + 0 + ) + ) < 0 + ); + + assertThat(lower, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, Long.MAX_VALUE - 1, 0))); + } + + @Test + public void shouldConvertToBinaryAndBack() { + final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); + final Windowed result = TimeOrderedKeySchema.fromStoreKey(serialized.get(), endTime - startTime, stateSerdes.keyDeserializer(), stateSerdes.topic()); + assertEquals(windowedKey, result); + } + + @Test + public void shouldExtractSequenceFromBinary() { + final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); + assertEquals(0, TimeOrderedKeySchema.extractStoreSequence(serialized.get())); + } + + @Test + public void shouldExtractStartTimeFromBinary() { + final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); + assertEquals(startTime, TimeOrderedKeySchema.extractStoreTimestamp(serialized.get())); + } + + @Test + public void shouldExtractWindowFromBinary() { + final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); + assertEquals(window, TimeOrderedKeySchema.extractStoreWindow(serialized.get(), endTime - startTime)); + } + + @Test + public void shouldExtractKeyBytesFromBinary() { + final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); + assertArrayEquals(key.getBytes(), TimeOrderedKeySchema.extractStoreKeyBytes(serialized.get())); + } + + @Test + public void shouldExtractKeyFromBinary() { + final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); + assertEquals(windowedKey, TimeOrderedKeySchema.fromStoreKey(serialized.get(), endTime - startTime, stateSerdes.keyDeserializer(), stateSerdes.topic())); + } + + @Test + public void shouldExtractBytesKeyFromBinary() { + final Windowed windowedBytesKey = new Windowed<>(Bytes.wrap(key.getBytes()), window); + final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedBytesKey, 0); + assertEquals(windowedBytesKey, TimeOrderedKeySchema.fromStoreBytesKey(serialized.get(), endTime - startTime)); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java index f6e92e35627cb..bc17abf749efa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WindowKeySchemaTest.java @@ -242,7 +242,7 @@ public void shouldConvertToBinaryAndBack() { } @Test - public void shouldExtractEndTimeFromBinary() { + public void shouldExtractSequenceFromBinary() { final Bytes serialized = WindowKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); assertEquals(0, WindowKeySchema.extractStoreSequence(serialized.get())); } From 00ec646c5a1762bdce28ed9878cfa46961de37b6 Mon Sep 17 00:00:00 2001 From: "high.lee" Date: Thu, 8 Apr 2021 05:36:39 +0900 Subject: [PATCH 019/155] KAFKA-7785: move internal DefaultPartitionGrouper (#10302) Reviewers: Guozhang Wang --- .../apache/kafka/streams/StreamsConfig.java | 17 ------ .../streams/processor/PartitionGrouper.java | 55 ------------------- .../PartitionGrouper.java} | 12 ++-- .../internals/StreamsPartitionAssignor.java | 4 +- .../assignment/AssignorConfiguration.java | 8 --- .../kafka/streams/StreamsConfigTest.java | 20 ------- .../PartitionGrouperTest.java} | 12 ++-- .../SingleGroupPartitionGrouperStub.java | 45 --------------- .../StreamsPartitionAssignorTest.java | 35 ------------ 9 files changed, 13 insertions(+), 195 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java rename streams/src/main/java/org/apache/kafka/streams/processor/{DefaultPartitionGrouper.java => internals/PartitionGrouper.java} (90%) rename streams/src/test/java/org/apache/kafka/streams/processor/{DefaultPartitionGrouperTest.java => internals/PartitionGrouperTest.java} (92%) delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index f7f26aadfd52a..4aaede745dad1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -561,15 +561,6 @@ public class StreamsConfig extends AbstractConfig { public static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG = "windowstore.changelog.additional.retention.ms"; private static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC = "Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. Default is 1 day"; - // deprecated - - /** {@code partition.grouper} */ - @SuppressWarnings("WeakerAccess") - @Deprecated - public static final String PARTITION_GROUPER_CLASS_CONFIG = "partition.grouper"; - private static final String PARTITION_GROUPER_CLASS_DOC = "Partition grouper class that implements the org.apache.kafka.streams.processor.PartitionGrouper interface." + - " WARNING: This config is deprecated and will be removed in 3.0.0 release."; - /** * {@code topology.optimization} * @deprecated since 2.7; use {@link #TOPOLOGY_OPTIMIZATION_CONFIG} instead @@ -773,11 +764,6 @@ public class StreamsConfig extends AbstractConfig { atLeast(0), Importance.LOW, CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) - .define(PARTITION_GROUPER_CLASS_CONFIG, - Type.CLASS, - org.apache.kafka.streams.processor.DefaultPartitionGrouper.class.getName(), - Importance.LOW, - PARTITION_GROUPER_CLASS_DOC) .define(POLL_MS_CONFIG, Type.LONG, 100L, @@ -1025,9 +1011,6 @@ protected StreamsConfig(final Map props, final boolean doLog) { super(CONFIG, props, doLog); eosEnabled = StreamThread.eosEnabled(this); - if (props.containsKey(PARTITION_GROUPER_CLASS_CONFIG)) { - log.warn("Configuration parameter `{}` is deprecated and will be removed in 3.0.0 release.", PARTITION_GROUPER_CLASS_CONFIG); - } if (props.containsKey(RETRIES_CONFIG)) { log.warn("Configuration parameter `{}` is deprecated and will be removed in 3.0.0 release.", RETRIES_CONFIG); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java deleted file mode 100644 index ecbc3c6e74d3d..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.streams.processor; - -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.TopicPartition; - -import java.util.Map; -import java.util.Set; - -/** - * A partition grouper that generates partition groups given the list of topic-partitions. - * - * This grouper also acts as the stream task creation function along with partition distribution - * such that each generated partition group is assigned with a distinct {@link TaskId}; - * the created task ids will then be assigned to Kafka Streams instances that host the stream - * processing application. - * - * @deprecated since 2.4 release; will be removed in 3.0.0 via KAFKA-7785 - */ -@Deprecated -public interface PartitionGrouper { - - /** - * Returns a map of task ids to groups of partitions. A partition group forms a task, thus, partitions that are - * expected to be processed together must be in the same group. - * - * Note that the grouping of partitions need to be sticky such that for a given partition, its assigned - * task should always be the same regardless of the input parameters to this function. This is to ensure task's - * local state stores remain valid through workload rebalances among Kafka Streams instances. - * - * The default partition grouper implements this interface by assigning all partitions across different topics with the same - * partition id into the same task. See {@link DefaultPartitionGrouper} for more information. - * - * @param topicGroups The map from the topic group id to topics - * @param metadata Metadata of the consuming cluster - * @return a map of task ids to groups of partitions - */ - Map> partitionGroups(Map> topicGroups, Cluster metadata); - -} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java similarity index 90% rename from streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java rename to streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java index c6dfdb9665c95..8dcb8233913a6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.processor; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.TaskId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,18 +32,15 @@ import java.util.Set; /** - * Default implementation of the {@link PartitionGrouper} interface that groups partitions by the partition id. + * Groups partitions by the partition id. * * Join operations requires that topics of the joining entities are copartitoned, i.e., being partitioned by the same key and having the same * number of partitions. Copartitioning is ensured by having the same number of partitions on * joined topics, and by using the serialization and Producer's default partitioner. - * - * @deprecated since 2.4 release; will be removed in 3.0.0 via KAFKA-7785 */ -@Deprecated -public class DefaultPartitionGrouper implements PartitionGrouper { +public class PartitionGrouper { - private static final Logger log = LoggerFactory.getLogger(DefaultPartitionGrouper.class); + private static final Logger log = LoggerFactory.getLogger(PartitionGrouper.class); /** * Generate tasks with the assigned topic partitions. * diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index c1fcd7159912e..ec31d168ffcb2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -174,7 +174,7 @@ public String toString() { private TaskManager taskManager; private StreamsMetadataState streamsMetadataState; @SuppressWarnings("deprecation") - private org.apache.kafka.streams.processor.PartitionGrouper partitionGrouper; + private PartitionGrouper partitionGrouper; private AtomicInteger assignmentErrorCode; private AtomicLong nextScheduledRebalanceMs; private Time time; @@ -213,7 +213,7 @@ public void configure(final Map configs) { nextScheduledRebalanceMs = referenceContainer.nextScheduledRebalanceMs; time = Objects.requireNonNull(referenceContainer.time, "Time was not specified"); assignmentConfigs = assignorConfiguration.assignmentConfigs(); - partitionGrouper = assignorConfiguration.partitionGrouper(); + partitionGrouper = new PartitionGrouper(); userEndPoint = assignorConfiguration.userEndPoint(); internalTopicManager = assignorConfiguration.internalTopicManager(); copartitionedTopicsEnforcer = assignorConfiguration.copartitionedTopicsEnforcer(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java index eef7a8d0cba0a..8e4794eaf7088 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignorConfiguration.java @@ -155,14 +155,6 @@ public int configuredMetadataVersion(final int priorVersion) { return priorVersion; } - @SuppressWarnings("deprecation") - public org.apache.kafka.streams.processor.PartitionGrouper partitionGrouper() { - return streamsConfig.getConfiguredInstance( - StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, - org.apache.kafka.streams.processor.PartitionGrouper.class - ); - } - public String userEndPoint() { final String configuredUserEndpoint = streamsConfig.getString(StreamsConfig.APPLICATION_SERVER_CONFIG); if (configuredUserEndpoint != null && !configuredUserEndpoint.isEmpty()) { diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index a54e70cf3a2b9..85a1872a8ba90 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -907,26 +907,6 @@ public void shouldThrowConfigExceptionWhenOptimizationConfigNotValueInRange() { assertThrows(ConfigException.class, () -> new StreamsConfig(props)); } - @SuppressWarnings("deprecation") - @Test - public void shouldLogWarningWhenPartitionGrouperIsUsed() { - props.put( - StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, - org.apache.kafka.streams.processor.DefaultPartitionGrouper.class - ); - - LogCaptureAppender.setClassLoggerToDebug(StreamsConfig.class); - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StreamsConfig.class)) { - new StreamsConfig(props); - - assertThat( - appender.getMessages(), - hasItem("Configuration parameter `" + StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG + - "` is deprecated and will be removed in 3.0.0 release.") - ); - } - } - @SuppressWarnings("deprecation") @Test public void shouldLogWarningWhenRetriesIsUsed() { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java similarity index 92% rename from streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java rename to streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java index e52b701cab85a..669f710ab97c4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.processor; +package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.processor.TaskId; import org.junit.Test; import java.util.Arrays; @@ -33,8 +34,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; -@SuppressWarnings("deprecation") -public class DefaultPartitionGrouperTest { +public class PartitionGrouperTest { private final List infos = Arrays.asList( new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]), @@ -53,7 +53,7 @@ public class DefaultPartitionGrouperTest { @Test public void shouldComputeGroupingForTwoGroups() { - final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final PartitionGrouper grouper = new PartitionGrouper(); final Map> expectedPartitionsForTask = new HashMap<>(); final Map> topicGroups = new HashMap<>(); @@ -73,7 +73,7 @@ public void shouldComputeGroupingForTwoGroups() { @Test public void shouldComputeGroupingForSingleGroupWithMultipleTopics() { - final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final PartitionGrouper grouper = new PartitionGrouper(); final Map> expectedPartitionsForTask = new HashMap<>(); final Map> topicGroups = new HashMap<>(); @@ -95,7 +95,7 @@ public void shouldComputeGroupingForSingleGroupWithMultipleTopics() { @Test public void shouldNotCreateAnyTasksBecauseOneTopicHasUnknownPartitions() { - final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final PartitionGrouper grouper = new PartitionGrouper(); final Map> topicGroups = new HashMap<>(); final int topicGroupId = 0; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java deleted file mode 100644 index 43e26d0b7043f..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SingleGroupPartitionGrouperStub.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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.streams.processor.internals; - -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.streams.processor.TaskId; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - -/** - * Used for testing the assignment of a subset of a topology group, not the entire topology - */ -@SuppressWarnings("deprecation") -public class SingleGroupPartitionGrouperStub implements org.apache.kafka.streams.processor.PartitionGrouper { - private org.apache.kafka.streams.processor.PartitionGrouper defaultPartitionGrouper = - new org.apache.kafka.streams.processor.DefaultPartitionGrouper(); - - @Override - public Map> partitionGroups(final Map> topicGroups, final Cluster metadata) { - final Map> includedTopicGroups = new HashMap<>(); - - for (final Map.Entry> entry : topicGroups.entrySet()) { - includedTopicGroups.put(entry.getKey(), entry.getValue()); - break; // arbitrarily use the first entry only - } - return defaultPartitionGrouper.partitionGroups(includedTopicGroups, metadata); - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 8bb8b864b25cd..7e58a8ae61d0a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -652,41 +652,6 @@ public void shouldAssignEvenlyAcrossConsumersOneClientMultipleThreads() { assertEquals(expectedInfo11TaskIds, info11.activeTasks()); } - @SuppressWarnings("deprecation") - @Test - public void testAssignWithPartialTopology() { - builder.addSource(null, "source1", null, null, null, "topic1"); - builder.addProcessor("processor1", new MockApiProcessorSupplier<>(), "source1"); - builder.addStateStore(new MockKeyValueStoreBuilder("store1", false), "processor1"); - builder.addSource(null, "source2", null, null, null, "topic2"); - builder.addProcessor("processor2", new MockApiProcessorSupplier<>(), "source2"); - builder.addStateStore(new MockKeyValueStoreBuilder("store2", false), "processor2"); - final List topics = asList("topic1", "topic2"); - final Set allTasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); - - createDefaultMockTaskManager(); - adminClient = createMockAdminClientForAssignor(getTopicPartitionOffsetsMap( - singletonList(APPLICATION_ID + "-store1-changelog"), - singletonList(3)) - ); - configurePartitionAssignorWith(Collections.singletonMap(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, SingleGroupPartitionGrouperStub.class)); - - // will throw exception if it fails - subscriptions.put("consumer10", - new Subscription( - topics, - defaultSubscriptionInfo.encode() - )); - final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); - - // check assignment info - final AssignmentInfo info10 = checkAssignment(mkSet("topic1"), assignments.get("consumer10")); - final Set allActiveTasks = new HashSet<>(info10.activeTasks()); - - assertEquals(3, allActiveTasks.size()); - assertEquals(allTasks, new HashSet<>(allActiveTasks)); - } - @Test public void testAssignEmptyMetadata() { builder.addSource(null, "source1", null, null, null, "topic1"); From 6e1723b4834b75bc0e16749afe31e5b690bec698 Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Thu, 8 Apr 2021 02:22:50 +0530 Subject: [PATCH 020/155] MINOR Moved tiered storage API classes from clients module to a new storage-api wmodule. (#10489) Moved tiered storage API classes from clients module to a new storage-api module. Created storage and storage-api modules. All the remote storage API classes are moved to storage-api module. All the remote storage implementation classes will be added to storage module. Reviewers: Jun Rao --- build.gradle | 128 +++++++++++++++++- settings.gradle | 2 + .../log/remote/storage/LogSegmentData.java | 0 .../storage/RemoteLogMetadataManager.java | 0 .../remote/storage/RemoteLogSegmentId.java | 0 .../storage/RemoteLogSegmentMetadata.java | 0 .../RemoteLogSegmentMetadataUpdate.java | 0 .../remote/storage/RemoteLogSegmentState.java | 0 .../RemotePartitionDeleteMetadata.java | 0 .../storage/RemotePartitionDeleteState.java | 0 .../RemoteResourceNotFoundException.java | 0 .../storage/RemoteStorageException.java | 0 .../remote/storage/RemoteStorageManager.java | 0 13 files changed, 129 insertions(+), 1 deletion(-) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/LogSegmentData.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentId.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteResourceNotFoundException.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java (100%) rename {clients => storage/api}/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java (100%) diff --git a/build.gradle b/build.gradle index 98ab70d8ca825..ac8f9e3d47154 100644 --- a/build.gradle +++ b/build.gradle @@ -775,6 +775,7 @@ project(':core') { implementation project(':metadata') implementation project(':raft') + implementation project(':storage') implementation libs.argparse4j implementation libs.jacksonDatabind @@ -992,6 +993,10 @@ project(':core') { from(project(':connect:mirror').configurations.runtimeClasspath) { into("libs/") } from(project(':connect:mirror-client').jar) { into("libs/") } from(project(':connect:mirror-client').configurations.runtimeClasspath) { into("libs/") } + from(project(':storage').jar) { into("libs/") } + from(project(':storage').configurations.runtimeClasspath) { into("libs/") } + from(project(':storage:api').jar) { into("libs/") } + from(project(':storage:api').configurations.runtimeClasspath) { into("libs/") } from(project(':streams').jar) { into("libs/") } from(project(':streams').configurations.runtimeClasspath) { into("libs/") } from(project(':streams:streams-scala').jar) { into("libs/") } @@ -1245,7 +1250,6 @@ project(':clients') { include "**/org/apache/kafka/server/authorizer/*" include "**/org/apache/kafka/server/policy/*" include "**/org/apache/kafka/server/quota/*" - include "**/org/apache/kafka/server/log/remote/storage/*" } } @@ -1331,6 +1335,128 @@ project(':raft') { } } +project(':storage:api') { + archivesBaseName = "kafka-storage-api" + + dependencies { + implementation project(':clients') + implementation libs.slf4jApi + + testImplementation project(':clients') + testImplementation project(':clients').sourceSets.test.output + testImplementation libs.junitJupiter + testImplementation libs.mockitoCore + + testRuntime libs.slf4jlog4j + } + + task createVersionFile(dependsOn: determineCommitId) { + ext.receiptFile = file("$buildDir/kafka/$buildVersionFileName") + outputs.file receiptFile + outputs.upToDateWhen { false } + doLast { + def data = [ + commitId: commitId, + version: version, + ] + + receiptFile.parentFile.mkdirs() + def content = data.entrySet().collect { "$it.key=$it.value" }.sort().join("\n") + receiptFile.setText(content, "ISO-8859-1") + } + } + + sourceSets { + main { + java { + srcDirs = ["src/main/java"] + } + } + test { + java { + srcDirs = ["src/test/java"] + } + } + } + + jar { + dependsOn createVersionFile + from("$buildDir") { + include "kafka/$buildVersionFileName" + } + } + + clean.doFirst { + delete "$buildDir/kafka/" + } + + javadoc { + include "**/org/apache/kafka/server/log/remote/storage/*" + } +} + +project(':storage') { + archivesBaseName = "kafka-storage" + + dependencies { + implementation project(':storage:api') + implementation project(':clients') + implementation libs.slf4jApi + implementation libs.jacksonDatabind + + testImplementation project(':clients') + testImplementation project(':clients').sourceSets.test.output + testImplementation libs.junitJupiter + testImplementation libs.mockitoCore + + testRuntime libs.slf4jlog4j + } + + task createVersionFile(dependsOn: determineCommitId) { + ext.receiptFile = file("$buildDir/kafka/$buildVersionFileName") + outputs.file receiptFile + outputs.upToDateWhen { false } + doLast { + def data = [ + commitId: commitId, + version: version, + ] + + receiptFile.parentFile.mkdirs() + def content = data.entrySet().collect { "$it.key=$it.value" }.sort().join("\n") + receiptFile.setText(content, "ISO-8859-1") + } + } + + sourceSets { + main { + java { + srcDirs = ["src/main/java"] + } + } + test { + java { + srcDirs = ["src/test/java"] + } + } + } + + jar { + dependsOn createVersionFile + from("$buildDir") { + include "kafka/$buildVersionFileName" + } + } + + clean.doFirst { + delete "$buildDir/kafka/" + } + + javadoc { + enabled = false + } +} + project(':tools') { archivesBaseName = "kafka-tools" diff --git a/settings.gradle b/settings.gradle index fedfa9a650cc4..91c71f52eb8d3 100644 --- a/settings.gradle +++ b/settings.gradle @@ -30,6 +30,8 @@ include 'clients', 'metadata', 'raft', 'shell', + 'storage', + 'storage:api', 'streams', 'streams:examples', 'streams:streams-scala', diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/LogSegmentData.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/LogSegmentData.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/LogSegmentData.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/LogSegmentData.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentId.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentId.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentId.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentId.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteResourceNotFoundException.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteResourceNotFoundException.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteResourceNotFoundException.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteResourceNotFoundException.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java diff --git a/clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java similarity index 100% rename from clients/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java rename to storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java From 04f47c54c26525f6aa7ee4097dc1f60525fad83a Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 7 Apr 2021 13:57:12 -0700 Subject: [PATCH 021/155] KAFKA-12527: Remove deprecated PartitionGrouper annotation (#10380) A quick follow-up rebased on https://github.com/apache/kafka/pull/10302 to remove deprecated annotation. --- .../streams/processor/internals/StreamsPartitionAssignor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index ec31d168ffcb2..ea6a7834a4d3d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -173,7 +173,6 @@ public String toString() { private Admin adminClient; private TaskManager taskManager; private StreamsMetadataState streamsMetadataState; - @SuppressWarnings("deprecation") private PartitionGrouper partitionGrouper; private AtomicInteger assignmentErrorCode; private AtomicLong nextScheduledRebalanceMs; From 3ca5a3bb78ec2324391c026d3dd1b6773cf9c772 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 7 Apr 2021 17:38:43 -0700 Subject: [PATCH 022/155] KAFKA-12568: Remove deprecated APIs in KStream, KTable and Joined (#10421) This is related to KIP-307 / KIP-372 / KIP-479. Reviewers: John Roesler --- docs/streams/upgrade-guide.html | 3 + .../apache/kafka/streams/kstream/Joined.java | 27 -- .../apache/kafka/streams/kstream/KStream.java | 318 ------------------ .../apache/kafka/streams/kstream/KTable.java | 34 -- .../kafka/streams/kstream/Serialized.java | 87 ----- .../kafka/streams/kstream/StreamJoined.java | 4 + .../kstream/internals/JoinedInternal.java | 2 - .../kstream/internals/KStreamImpl.java | 79 ----- .../streams/kstream/internals/KTableImpl.java | 10 - .../kstream/internals/SerializedInternal.java | 34 -- .../kafka/streams/StreamsBuilderTest.java | 9 +- .../kstream/RepartitionTopicNamingTest.java | 12 +- .../kstream/internals/KStreamImplTest.java | 193 +---------- 13 files changed, 17 insertions(+), 795 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 88f700d628502..85d5ff9765f24 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -105,6 +105,9 @@

    Streams API
  • WindowBytesStoreSupplier#segments: deprecated in Kafka 2.1.0 (KIP-319).
  • segments, until, maintainMs on TimeWindows, JoinWindows, and SessionWindows: deprecated in Kafka 2.1.0 (KIP-328).
  • Overloaded JoinWindows#of, before, after, SessionWindows#with, TimeWindows#of, advanceBy, UnlimitedWindows#startOn and KafkaStreams#close with long typed parameters: deprecated in Kafka 2.1.0 (KIP-358).
  • +
  • Overloaded KStream#groupBy, groupByKey and KTable#groupBy with Serialized parameter: deprecated in Kafka 2.1.0 (KIP-372).
  • +
  • Joined#named, name: deprecated in Kafka 2.3.0 (KIP-307).
  • +
  • Overloaded KStream#join, leftJoin, outerJoin with KStream and Joined parameters: deprecated in Kafka 2.4.0 (KIP-479).
  • Overloaded KafkaStreams#metadataForKey: deprecated in Kafka 2.5.0 (KIP-535).
  • Overloaded KafkaStreams#store: deprecated in Kafka 2.5.0 (KIP-562).
  • diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java index 15978373a86db..a2793afce16a4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Joined.java @@ -129,24 +129,6 @@ public static Joined otherValueSerde(final Serde otherV return new Joined<>(null, null, otherValueSerde, null); } - /** - * Create an instance of {@code Joined} with base name for all components of the join, this may - * include any repartition topics created to complete the join. - * - * @param name the name used as the base for naming components of the join including any - * repartition topics - * @param key type - * @param value type - * @param other value type - * @return new {@code Joined} instance configured with the name - * - * @deprecated use {@link #as(String)} instead - */ - @Deprecated - public static Joined named(final String name) { - return new Joined<>(null, null, null, name); - } - /** * Create an instance of {@code Joined} with base name for all components of the join, this may * include any repartition topics created to complete the join. @@ -221,13 +203,4 @@ public Serde valueSerde() { public Serde otherValueSerde() { return otherValueSerde; } - - /** - * @deprecated this method will be removed in a in a future release - */ - @Deprecated - public String name() { - return name; - } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index cfe4a408f5772..00b9ee8991477 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -1083,39 +1083,6 @@ KTable toTable(final Named named, */ KGroupedStream groupBy(final KeyValueMapper keySelector); - /** - * Group the records of this {@code KStream} on a new key that is selected using the provided {@link KeyValueMapper} - * and {@link Serde}s as specified by {@link Serialized}. - * {@link KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. - * Grouping a stream on the record key is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedStream}). - * The {@link KeyValueMapper} selects a new key (which may or may not be of the same type) while preserving the - * original values. - * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. - *

    - * Because a new key is selected, an internal repartitioning topic may need to be created in Kafka if a - * later operator depends on the newly selected key. - * This topic will be as "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. - *

    - * You can retrieve all generated internal topic names via {@link Topology#describe()}. - *

    - * All data of this stream will be redistributed through the repartitioning topic by writing all records to it, - * and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned on the new key. - *

    - * This operation is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link #groupByKey()}. - * - * @param keySelector a {@link KeyValueMapper} that computes a new key for grouping - * @param the key type of the result {@link KGroupedStream} - * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} - * - * @deprecated since 2.1. Use {@link org.apache.kafka.streams.kstream.KStream#groupBy(KeyValueMapper, Grouped)} instead - */ - @Deprecated - KGroupedStream groupBy(final KeyValueMapper keySelector, - final Serialized serialized); - /** * Group the records of this {@code KStream} on a new key that is selected using the provided {@link KeyValueMapper} * and {@link Serde}s as specified by {@link Grouped}. @@ -1178,36 +1145,6 @@ KGroupedStream groupBy(final KeyValueMapper groupByKey(); - /** - * Group the records by their current key into a {@link KGroupedStream} while preserving the original values - * and using the serializers as defined by {@link Serialized}. - * {@link KGroupedStream} can be further grouped with other streams to form a {@link CogroupedKStream}. - * Grouping a stream on the record key is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedStream}). - * If a record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. - *

    - * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)} or - * {@link #transform(TransformerSupplier, String...)}) an internal repartitioning topic may need to be created in - * Kafka if a later operator depends on the newly selected key. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "<name>" is an internally generated name, and "-repartition" is a fixed suffix. - *

    - * You can retrieve all generated internal topic names via {@link Topology#describe()}. - *

    - * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned - * correctly on its key. - * - * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} - * @see #groupBy(KeyValueMapper) - * - * @deprecated since 2.1. Use {@link org.apache.kafka.streams.kstream.KStream#groupByKey(Grouped)} instead - */ - @Deprecated - KGroupedStream groupByKey(final Serialized serialized); - /** * Group the records by their current key into a {@link KGroupedStream} while preserving the original values * and using the serializers as defined by {@link Grouped}. @@ -1391,89 +1328,6 @@ KStream join(final KStream otherStream, final ValueJoinerWithKey joiner, final JoinWindows windows); - - /** - * Join records of this stream with another {@code KStream}'s records using windowed inner equi join using the - * {@link Joined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value serde}, - * and {@link Serde the other stream's value serde}. - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

    - * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. - *

    - * Example (assuming all input records belong to the correct windows): - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
    thisotherresult
    <K1:A>
    <K2:B><K2:b><K2:ValueJoiner(B,b)>
    <K3:c>
    - * Both input streams (or to be more precise, their underlying source topics) need to have the same number of - * partitions. - * If this is not the case, you would need to call {@link #repartition(Repartitioned)} (for one input stream) before - * doing the join and specify the "correct" number of partitions via {@link Repartitioned} parameter. - * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). - * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an - * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. - * The repartitioning topic will be named "${applicationId}-<name>-repartition", where "applicationId" is - * user-specified in {@link StreamsConfig} via parameter - * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is an internally generated - * name, and "-repartition" is a fixed suffix. - *

    - * Repartitioning can happen for one or both of the joining {@code KStream}s. - * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned - * correctly on its key. - *

    - * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. - * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. - * The changelog topic will be named "${applicationId}-<storename>-changelog", where "applicationId" is user-specified - * in {@link StreamsConfig} via parameter - * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an - * internally generated name, and "-changelog" is a fixed suffix. - *

    - * You can retrieve all generated internal topic names via {@link Topology#describe()}. - * - * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param windows the specification of the {@link JoinWindows} - * @param joined a {@link Joined} instance that defines the serdes to - * be used to serialize/deserialize inputs and outputs of the joined streams - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key and within the joining window intervals - * @see #leftJoin(KStream, ValueJoiner, JoinWindows, Joined) - * @see #outerJoin(KStream, ValueJoiner, JoinWindows, Joined) - * @deprecated since 2.4. Use {@link KStream#join(KStream, ValueJoiner, JoinWindows, StreamJoined)} instead. - */ - @Deprecated - KStream join(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final Joined joined); - /** * Join records of this stream with another {@code KStream}'s records using windowed inner equi join using the * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value @@ -1794,92 +1648,6 @@ KStream leftJoin(final KStream otherStream, final ValueJoinerWithKey joiner, final JoinWindows windows); - /** - * Join records of this stream with another {@code KStream}'s records using windowed left equi join using the - * {@link Joined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value serde}, - * and {@link Serde the other stream's value serde}. - * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join}, all records from this stream will - * produce at least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

    - * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of this {@code KStream} that does not satisfy the join predicate the provided - * {@link ValueJoiner} will be called with a {@code null} value for the other stream. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. - *

    - * Example (assuming all input records belong to the correct windows): - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
    thisotherresult
    <K1:A><K1:ValueJoiner(A,null)>
    <K2:B><K2:b><K2:ValueJoiner(B,b)>
    <K3:c>
    - * Both input streams (or to be more precise, their underlying source topics) need to have the same number of - * partitions. - * If this is not the case, you would need to call {@link #repartition(Repartitioned)} (for one input stream) before - * doing the join and specify the "correct" number of partitions via {@link Repartitioned} parameter. - * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). - * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an - * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. - * The repartitioning topic will be named "${applicationId}-<name>-repartition", where "applicationId" is - * user-specified in {@link StreamsConfig} via parameter - * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is an internally generated - * name, and "-repartition" is a fixed suffix. - *

    - * Repartitioning can happen for one or both of the joining {@code KStream}s. - * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned - * correctly on its key. - *

    - * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. - * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. - * The changelog topic will be named "${applicationId}-<storename>-changelog", where "applicationId" is user-specified - * in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "storeName" is an internally generated name, and "-changelog" is a fixed suffix. - *

    - * You can retrieve all generated internal topic names via {@link Topology#describe()}. - * - * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param windows the specification of the {@link JoinWindows} - * @param joined a {@link Joined} instance that defines the serdes to - * be used to serialize/deserialize inputs and outputs of the joined streams - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * this {@code KStream} and within the joining window intervals - * @see #join(KStream, ValueJoiner, JoinWindows, Joined) - * @see #outerJoin(KStream, ValueJoiner, JoinWindows, Joined) - * @deprecated since 2.4. Use {@link KStream#leftJoin(KStream, ValueJoiner, JoinWindows, StreamJoined)} instead. - */ - @Deprecated - KStream leftJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final Joined joined); - /** * Join records of this stream with another {@code KStream}'s records using windowed left equi join using the * {@link StreamJoined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value @@ -2209,92 +1977,6 @@ KStream outerJoin(final KStream otherStream, KStream outerJoin(final KStream otherStream, final ValueJoinerWithKey joiner, final JoinWindows windows); - /** - * Join records of this stream with another {@code KStream}'s records using windowed outer equi join using the - * {@link Joined} instance for configuration of the {@link Serde key serde}, {@link Serde this stream's value serde}, - * and {@link Serde the other stream's value serde}. - * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join} or - * {@link #leftJoin(KStream, ValueJoiner, JoinWindows) left-join}, all records from both streams will produce at - * least one output record (cf. below). - * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. - * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given - * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. - *

    - * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute - * a value (with arbitrary type) for the result record. - * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of both {@code KStream}s that does not satisfy the join predicate the provided - * {@link ValueJoiner} will be called with a {@code null} value for this/other stream, respectively. - * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@code KStream}. - *

    - * Example (assuming all input records belong to the correct windows): - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
    thisotherresult
    <K1:A><K1:ValueJoiner(A,null)>
    <K2:B><K2:b><K2:ValueJoiner(null,b)>

    <K2:ValueJoiner(B,b)>
    <K3:c><K3:ValueJoiner(null,c)>
    - * Both input streams (or to be more precise, their underlying source topics) need to have the same number of - * partitions. - * If this is not the case, you would need to call {@link #repartition(Repartitioned)} (for one input stream) before - * doing the join and specify the "correct" number of partitions via {@link Repartitioned} parameter. - * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). - * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an - * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. - * The repartitioning topic will be named "${applicationId}-<name>-repartition", where "applicationId" is - * user-specified in {@link StreamsConfig} via parameter - * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is an internally generated - * name, and "-repartition" is a fixed suffix. - *

    - * Repartitioning can happen for one or both of the joining {@code KStream}s. - * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned - * correctly on its key. - *

    - * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. - * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. - * The changelog topic will be named "${applicationId}-<storename>-changelog", where "applicationId" is user-specified - * in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, - * "storeName" is an internally generated name, and "-changelog" is a fixed suffix. - *

    - * You can retrieve all generated internal topic names via {@link Topology#describe()}. - * - * @param otherStream the {@code KStream} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param windows the specification of the {@link JoinWindows} - * @param joined a {@link Joined} instance that defines the serdes to - * be used to serialize/deserialize inputs and outputs of the joined streams - * @param the value type of the other stream - * @param the value type of the result stream - * @return a {@code KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * both {@code KStream} and within the joining window intervals - * @see #join(KStream, ValueJoiner, JoinWindows, Joined) - * @see #leftJoin(KStream, ValueJoiner, JoinWindows, Joined) - * @deprecated since 2.4. Use {@link KStream#outerJoin(KStream, ValueJoiner, JoinWindows, StreamJoined)} instead. - */ - @Deprecated - KStream outerJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final Joined joined); /** * Join records of this stream with another {@code KStream}'s records using windowed outer equi join using the diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index b7f073b4acd26..04e55858bbc34 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -1062,40 +1062,6 @@ KTable transformValues(final ValueTransformerWithKeySupplier KGroupedTable groupBy(final KeyValueMapper> selector); - /** - * Re-groups the records of this {@code KTable} using the provided {@link KeyValueMapper} - * and {@link Serde}s as specified by {@link Serialized}. - * Each {@link KeyValue} pair of this {@code KTable} is mapped to a new {@link KeyValue} pair by applying the - * provided {@link KeyValueMapper}. - * Re-grouping a {@code KTable} is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedTable}). - * The {@link KeyValueMapper} selects a new key and value (with both maybe being the same type or a new type). - * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedTable} - *

    - * Because a new key is selected, an internal repartitioning topic will be created in Kafka. - * This topic will be named "${applicationId}-<name>-repartition", where "applicationId" is user-specified in - * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "<name>" is - * an internally generated name, and "-repartition" is a fixed suffix. - * - * You can retrieve all generated internal topic names via {@link Topology#describe()}. - * - *

    - * All data of this {@code KTable} will be redistributed through the repartitioning topic by writing all update - * records to and rereading all updated records from it, such that the resulting {@link KGroupedTable} is partitioned - * on the new key. - * - * @param selector a {@link KeyValueMapper} that computes a new grouping key and value to be aggregated - * @param serialized the {@link Serialized} instance used to specify {@link org.apache.kafka.common.serialization.Serdes} - * @param the key type of the result {@link KGroupedTable} - * @param the value type of the result {@link KGroupedTable} - * @return a {@link KGroupedTable} that contains the re-grouped records of the original {@code KTable} - * - * @deprecated since 2.1. Use {@link org.apache.kafka.streams.kstream.KTable#groupBy(KeyValueMapper, Grouped)} instead - */ - @Deprecated - KGroupedTable groupBy(final KeyValueMapper> selector, - final Serialized serialized); - /** * Re-groups the records of this {@code KTable} using the provided {@link KeyValueMapper} * and {@link Serde}s as specified by {@link Grouped}. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java deleted file mode 100644 index a34fe9d4accd9..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Serialized.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.streams.kstream; - -import org.apache.kafka.common.serialization.Serde; - -/** - * The class that is used to capture the key and value {@link Serde}s used when performing - * {@link KStream#groupBy(KeyValueMapper, Serialized)} and {@link KStream#groupByKey(Serialized)} operations. - * - * @param the key type - * @param the value type - * - * @deprecated since 2.1. Use {@link org.apache.kafka.streams.kstream.Grouped} instead - */ -@Deprecated -public class Serialized { - - protected final Serde keySerde; - protected final Serde valueSerde; - - private Serialized(final Serde keySerde, - final Serde valueSerde) { - this.keySerde = keySerde; - this.valueSerde = valueSerde; - } - - protected Serialized(final Serialized serialized) { - this(serialized.keySerde, serialized.valueSerde); - } - - /** - * Construct a {@code Serialized} instance with the provided key and value {@link Serde}s. - * If the {@link Serde} params are {@code null} the default serdes defined in the configs will be used. - * - * @param keySerde keySerde that will be used to materialize a stream - * if not specified the default serdes defined in the configs will be used - * @param valueSerde valueSerde that will be used to materialize a stream - * if not specified the default serdes defined in the configs will be used - * @param the key type - * @param the value type - * @return a new instance of {@link Serialized} configured with the provided serdes - */ - public static Serialized with(final Serde keySerde, - final Serde valueSerde) { - return new Serialized<>(keySerde, valueSerde); - } - - /** - * Construct a {@code Serialized} instance with the provided key {@link Serde}. - * If the {@link Serde} params are null the default serdes defined in the configs will be used. - * - * @param keySerde keySerde that will be used to materialize a stream - * if not specified the default serdes defined in the configs will be used - * @return a new instance of {@link Serialized} configured with the provided key serde - */ - public Serialized withKeySerde(final Serde keySerde) { - return new Serialized<>(keySerde, null); - } - - /** - * Construct a {@code Serialized} instance with the provided value {@link Serde}. - * If the {@link Serde} params are null the default serdes defined in the configs will be used. - * - * @param valueSerde valueSerde that will be used to materialize a stream - * if not specified the default serdes defined in the configs will be used - * @return a new instance of {@link Serialized} configured with the provided key serde - */ - public Serialized withValueSerde(final Serde valueSerde) { - return new Serialized<>(null, valueSerde); - } - -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/StreamJoined.java b/streams/src/main/java/org/apache/kafka/streams/kstream/StreamJoined.java index 6fef9a1e45cdb..72d09221eda6b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/StreamJoined.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/StreamJoined.java @@ -108,6 +108,10 @@ public static StreamJoined with(final WindowBytesStoreSup * or an outer join. The changelog topics will have the -changelog suffix. The user should note that even though the join stores will have a * specified name, the stores will remain unavailable for querying. * + * Please note that if you are using {@link StreamJoined} to replace deprecated {@link KStream#join} functions with + * {@link Joined} parameters in order to set the name for the join processors, you would need to create the {@link StreamJoined} + * object first and then call {@link StreamJoined#withName} + * * @param storeName The name to use for the store * @param The key type * @param This value type diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/JoinedInternal.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/JoinedInternal.java index 99f7a0fd740cb..eb5884042cd72 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/JoinedInternal.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/JoinedInternal.java @@ -37,8 +37,6 @@ public Serde otherValueSerde() { return otherValueSerde; } - @Override // TODO remove annotation when super.name() is removed - @SuppressWarnings("deprecation") // this method should not be removed if super.name() is removed public String name() { return name; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 9b5ed15539deb..bacddc5e70cfc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -777,18 +777,6 @@ public KGroupedStream groupBy(final KeyValueMapper KGroupedStream groupBy(final KeyValueMapper keySelector, - final org.apache.kafka.streams.kstream.Serialized serialized) { - Objects.requireNonNull(keySelector, "keySelector can't be null"); - Objects.requireNonNull(serialized, "serialized can't be null"); - - final SerializedInternal serializedInternal = new SerializedInternal<>(serialized); - - return groupBy(keySelector, Grouped.with(serializedInternal.keySerde(), serializedInternal.valueSerde())); - } - @Override public KGroupedStream groupBy(final KeyValueMapper keySelector, final Grouped grouped) { @@ -815,15 +803,6 @@ public KGroupedStream groupByKey() { return groupByKey(Grouped.with(keySerde, valueSerde)); } - @Override - @Deprecated - public KGroupedStream groupByKey(final org.apache.kafka.streams.kstream.Serialized serialized) { - Objects.requireNonNull(serialized, "serialized can't be null"); - - final SerializedInternal serializedInternal = new SerializedInternal<>(serialized); - return groupByKey(Grouped.with(serializedInternal.keySerde(), serializedInternal.valueSerde())); - } - @Override public KGroupedStream groupByKey(final Grouped grouped) { Objects.requireNonNull(grouped, "grouped can't be null"); @@ -853,26 +832,6 @@ public KStream join(final KStream otherStream, return join(otherStream, joiner, windows, StreamJoined.with(null, null, null)); } - @Override - @Deprecated - public KStream join(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final Joined joined) { - Objects.requireNonNull(joined, "joined can't be null"); - - final JoinedInternal joinedInternal = new JoinedInternal<>(joined); - final StreamJoined streamJoined = StreamJoined - .with( - joinedInternal.keySerde(), - joinedInternal.valueSerde(), - joinedInternal.otherValueSerde()) - .withName(joinedInternal.name()); - - return join(otherStream, joiner, windows, streamJoined); - - } - @Override public KStream join(final KStream otherStream, final ValueJoiner joiner, @@ -910,25 +869,6 @@ public KStream leftJoin(final KStream otherStream, return leftJoin(otherStream, joiner, windows, StreamJoined.with(null, null, null)); } - @Override - @Deprecated - public KStream leftJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final Joined joined) { - Objects.requireNonNull(joined, "joined can't be null"); - - final JoinedInternal joinedInternal = new JoinedInternal<>(joined); - final StreamJoined streamJoined = StreamJoined - .with( - joinedInternal.keySerde(), - joinedInternal.valueSerde(), - joinedInternal.otherValueSerde()) - .withName(joinedInternal.name()); - - return leftJoin(otherStream, joiner, windows, streamJoined); - } - @Override public KStream leftJoin(final KStream otherStream, final ValueJoiner joiner, @@ -969,25 +909,6 @@ public KStream outerJoin(final KStream otherStream, return outerJoin(otherStream, joiner, windows, StreamJoined.with(null, null, null)); } - @Override - @Deprecated - public KStream outerJoin(final KStream otherStream, - final ValueJoiner joiner, - final JoinWindows windows, - final Joined joined) { - Objects.requireNonNull(joined, "joined can't be null"); - - final JoinedInternal joinedInternal = new JoinedInternal<>(joined); - final StreamJoined streamJoined = StreamJoined - .with( - joinedInternal.keySerde(), - joinedInternal.valueSerde(), - joinedInternal.otherValueSerde()) - .withName(joinedInternal.name()); - - return outerJoin(otherStream, joiner, windows, streamJoined); - } - @Override public KStream outerJoin(final KStream otherStream, final ValueJoiner joiner, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 7173ac796e16f..98aee1304be8b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -781,16 +781,6 @@ public KGroupedTable groupBy(final KeyValueMapper KGroupedTable groupBy(final KeyValueMapper> selector, - final org.apache.kafka.streams.kstream.Serialized serialized) { - Objects.requireNonNull(selector, "selector can't be null"); - Objects.requireNonNull(serialized, "serialized can't be null"); - final SerializedInternal serializedInternal = new SerializedInternal<>(serialized); - return groupBy(selector, Grouped.with(serializedInternal.keySerde(), serializedInternal.valueSerde())); - } - @Override public KGroupedTable groupBy(final KeyValueMapper> selector, final Grouped grouped) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java deleted file mode 100644 index 09fd9e5bd0cef..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SerializedInternal.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.streams.kstream.internals; - -import org.apache.kafka.common.serialization.Serde; - -@Deprecated -public class SerializedInternal extends org.apache.kafka.streams.kstream.Serialized { - public SerializedInternal(final org.apache.kafka.streams.kstream.Serialized serialized) { - super(serialized); - } - - public Serde keySerde() { - return keySerde; - } - - public Serde valueSerde() { - return valueSerde; - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java index a4a9be6ae8be0..d8f3bf4f650ea 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -725,12 +725,11 @@ public void shouldUseSpecifiedNameForLeftJoinOperationBetweenKStreamAndKStream() } @Test - @Deprecated public void shouldUseGeneratedStoreNamesForLeftJoinOperationBetweenKStreamAndKStream() { final KStream streamOne = builder.stream(STREAM_TOPIC); final KStream streamTwo = builder.stream(STREAM_TOPIC_TWO); - streamOne.leftJoin(streamTwo, (value1, value2) -> value1, JoinWindows.of(Duration.ofHours(1)), Joined.as(STREAM_OPERATION_NAME)); + streamOne.leftJoin(streamTwo, (value1, value2) -> value1, JoinWindows.of(Duration.ofHours(1)), StreamJoined.with(Serdes.String(), Serdes.String(), Serdes.String()).withName(STREAM_OPERATION_NAME)); builder.build(); final ProcessorTopology topology = builder.internalTopologyBuilder.rewriteTopology(new StreamsConfig(props)).buildTopology(); @@ -772,12 +771,11 @@ public void shouldUseSpecifiedNameForJoinOperationBetweenKStreamAndKStream() { } @Test - @Deprecated public void shouldUseGeneratedNameForJoinOperationBetweenKStreamAndKStream() { final KStream streamOne = builder.stream(STREAM_TOPIC); final KStream streamTwo = builder.stream(STREAM_TOPIC_TWO); - streamOne.join(streamTwo, (value1, value2) -> value1, JoinWindows.of(Duration.ofHours(1)), Joined.as(STREAM_OPERATION_NAME)); + streamOne.join(streamTwo, (value1, value2) -> value1, JoinWindows.of(Duration.ofHours(1)), StreamJoined.with(Serdes.String(), Serdes.String(), Serdes.String()).withName(STREAM_OPERATION_NAME)); builder.build(); final ProcessorTopology topology = builder.internalTopologyBuilder.rewriteTopology(new StreamsConfig(props)).buildTopology(); @@ -818,12 +816,11 @@ public void shouldUseSpecifiedNameForOuterJoinOperationBetweenKStreamAndKStream( } @Test - @Deprecated public void shouldUseGeneratedStoreNamesForOuterJoinOperationBetweenKStreamAndKStream() { final KStream streamOne = builder.stream(STREAM_TOPIC); final KStream streamTwo = builder.stream(STREAM_TOPIC_TWO); - streamOne.outerJoin(streamTwo, (value1, value2) -> value1, JoinWindows.of(Duration.ofHours(1)), Joined.as(STREAM_OPERATION_NAME)); + streamOne.outerJoin(streamTwo, (value1, value2) -> value1, JoinWindows.of(Duration.ofHours(1)), StreamJoined.with(Serdes.String(), Serdes.String(), Serdes.String()).withName(STREAM_OPERATION_NAME)); builder.build(); final ProcessorTopology topology = builder.internalTopologyBuilder.rewriteTopology(new StreamsConfig(props)).buildTopology(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java index c286b14245984..3091a6f9e24c8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java @@ -39,7 +39,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -@SuppressWarnings("deprecation") public class RepartitionTopicNamingTest { private final KeyValueMapper kvMapper = (k, v) -> k + v; @@ -208,11 +207,12 @@ public void shouldFailWithSameRepartitionTopicNameInJoin() { final KStream stream3 = builder.stream("topic3").selectKey((k, v) -> k); final KStream joined = stream1.join(stream2, (v1, v2) -> v1 + v2, - JoinWindows.of(Duration.ofMillis(30L)), - Joined.named("join-repartition")); + JoinWindows.of(Duration.ofMillis(30L)), + StreamJoined.as("join-store").withName("join-repartition")); joined.join(stream3, (v1, v2) -> v1 + v2, JoinWindows.of(Duration.ofMillis(30L)), - Joined.named("join-repartition")); + StreamJoined.as("join-store").withName("join-repartition")); + builder.build(); fail("Should not build re-using repartition topic name"); } catch (final TopologyException te) { @@ -446,8 +446,8 @@ private String buildStreamJoin(final boolean includeOtherOperations) { } final String joinRepartitionTopicName = "my-join"; - updatedStreamOne.join(updatedStreamTwo, (v1, v2) -> v1 + v2, - JoinWindows.of(Duration.ofMillis(1000L)), Joined.with(Serdes.String(), Serdes.String(), Serdes.String(), joinRepartitionTopicName)); + updatedStreamOne.join(updatedStreamTwo, (v1, v2) -> v1 + v2, JoinWindows.of(Duration.ofMillis(1000L)), + StreamJoined.with(Serdes.String(), Serdes.String(), Serdes.String()).withName(joinRepartitionTopicName)); return builder.build().describe().toString(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index e184f6d364154..ac5db68b5403b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -102,10 +102,6 @@ public class KStreamImplTest { private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String()); - @SuppressWarnings("deprecation") - private final org.apache.kafka.streams.kstream.Serialized stringSerialized = - org.apache.kafka.streams.kstream.Serialized.with(Serdes.String(), Serdes.String()); - private final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); private final TransformerSupplier> transformerSupplier = () -> new Transformer>() { @@ -650,15 +646,6 @@ public void shouldNotAllowNullSelectorOnGroupBy() { assertThat(exception.getMessage(), equalTo("keySelector can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullSelectorOnGroupByWithSerialized() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.groupBy(null, stringSerialized)); - assertThat(exception.getMessage(), equalTo("keySelector can't be null")); - } - @Test public void shouldNotAllowNullSelectorOnGroupByWithGrouped() { final NullPointerException exception = assertThrows( @@ -667,15 +654,6 @@ public void shouldNotAllowNullSelectorOnGroupByWithGrouped() { assertThat(exception.getMessage(), equalTo("keySelector can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullSerializedOnGroupBy() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.groupBy((k, v) -> k, (org.apache.kafka.streams.kstream.Serialized) null)); - assertThat(exception.getMessage(), equalTo("serialized can't be null")); - } - @Test public void shouldNotAllowNullGroupedOnGroupBy() { final NullPointerException exception = assertThrows( @@ -684,15 +662,6 @@ public void shouldNotAllowNullGroupedOnGroupBy() { assertThat(exception.getMessage(), equalTo("grouped can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullSerializedOnGroupByKey() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.groupByKey((org.apache.kafka.streams.kstream.Serialized) null)); - assertThat(exception.getMessage(), equalTo("serialized can't be null")); - } - @Test public void shouldNotAllowNullGroupedOnGroupByKey() { final NullPointerException exception = assertThrows( @@ -741,19 +710,6 @@ public void shouldNotAllowNullOtherStreamOnJoin() { assertThat(exception.getMessage(), equalTo("otherStream can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullOtherStreamOnJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.join( - null, - MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(10)), - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("otherStream can't be null")); - } - @Test public void shouldNotAllowNullOtherStreamOnJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -782,20 +738,6 @@ public void shouldNotAllowNullValueJoinerWithKeyOnJoin() { assertThat(exception.getMessage(), equalTo("joiner can't be null")); } - - @Deprecated - @Test - public void shouldNotAllowNullValueJoinerOnJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.join( - testStream, - null, - JoinWindows.of(ofMillis(10)), - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("joiner can't be null")); - } - @Test public void shouldNotAllowNullValueJoinerOnJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -828,19 +770,6 @@ public void shouldNotAllowNullJoinWindowsOnJoin() { assertThat(exception.getMessage(), equalTo("windows can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullJoinWindowsOnJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.join( - testStream, - MockValueJoiner.TOSTRING_JOINER, - null, - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("windows can't be null")); - } - @Test public void shouldNotAllowNullJoinWindowsOnJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -853,19 +782,6 @@ public void shouldNotAllowNullJoinWindowsOnJoinWithStreamJoined() { assertThat(exception.getMessage(), equalTo("windows can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullJoinedOnJoin() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.join( - testStream, - MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(10)), - (Joined) null)); - assertThat(exception.getMessage(), equalTo("joined can't be null")); - } - @Test public void shouldNotAllowNullStreamJoinedOnJoin() { final NullPointerException exception = assertThrows( @@ -886,19 +802,6 @@ public void shouldNotAllowNullOtherStreamOnLeftJoin() { assertThat(exception.getMessage(), equalTo("otherStream can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullOtherStreamOnLeftJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.leftJoin( - null, - MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(10)), - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("otherStream can't be null")); - } - @Test public void shouldNotAllowNullOtherStreamOnLeftJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -927,19 +830,6 @@ public void shouldNotAllowNullValueJoinerWithKeyOnLeftJoin() { assertThat(exception.getMessage(), equalTo("joiner can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullValueJoinerOnLeftJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.leftJoin( - testStream, - null, - JoinWindows.of(ofMillis(10)), - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("joiner can't be null")); - } - @Test public void shouldNotAllowNullValueJoinerOnLeftJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -973,19 +863,6 @@ public void shouldNotAllowNullJoinWindowsOnLeftJoin() { assertThat(exception.getMessage(), equalTo("windows can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullJoinWindowsOnLeftJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.leftJoin( - testStream, - MockValueJoiner.TOSTRING_JOINER, - null, - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("windows can't be null")); - } - @Test public void shouldNotAllowNullJoinWindowsOnLeftJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -998,19 +875,6 @@ public void shouldNotAllowNullJoinWindowsOnLeftJoinWithStreamJoined() { assertThat(exception.getMessage(), equalTo("windows can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullJoinedOnLeftJoin() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.leftJoin( - testStream, - MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(10)), - (Joined) null)); - assertThat(exception.getMessage(), equalTo("joined can't be null")); - } - @Test public void shouldNotAllowNullStreamJoinedOnLeftJoin() { final NullPointerException exception = assertThrows( @@ -1031,19 +895,6 @@ public void shouldNotAllowNullOtherStreamOnOuterJoin() { assertThat(exception.getMessage(), equalTo("otherStream can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullOtherStreamOnOuterJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.outerJoin( - null, - MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(10)), - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("otherStream can't be null")); - } - @Test public void shouldNotAllowNullOtherStreamOnOuterJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -1072,19 +923,6 @@ public void shouldNotAllowNullValueJoinerWithKeyOnOuterJoin() { assertThat(exception.getMessage(), equalTo("joiner can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullValueJoinerOnOuterJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.outerJoin( - testStream, - null, - JoinWindows.of(ofMillis(10)), - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("joiner can't be null")); - } - @Test public void shouldNotAllowNullValueJoinerOnOuterJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -1117,19 +955,6 @@ public void shouldNotAllowNullJoinWindowsOnOuterJoin() { assertThat(exception.getMessage(), equalTo("windows can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullJoinWindowsOnOuterJoinWithJoined() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.outerJoin( - testStream, - MockValueJoiner.TOSTRING_JOINER, - null, - Joined.as("name"))); - assertThat(exception.getMessage(), equalTo("windows can't be null")); - } - @Test public void shouldNotAllowNullJoinWindowsOnOuterJoinWithStreamJoined() { final NullPointerException exception = assertThrows( @@ -1142,19 +967,6 @@ public void shouldNotAllowNullJoinWindowsOnOuterJoinWithStreamJoined() { assertThat(exception.getMessage(), equalTo("windows can't be null")); } - @Deprecated - @Test - public void shouldNotAllowNullJoinedOnOuterJoin() { - final NullPointerException exception = assertThrows( - NullPointerException.class, - () -> testStream.outerJoin( - testStream, - MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(10)), - (Joined) null)); - assertThat(exception.getMessage(), equalTo("joined can't be null")); - } - @Test public void shouldNotAllowNullStreamJoinedOnOuterJoin() { final NullPointerException exception = assertThrows( @@ -1697,7 +1509,6 @@ public void shouldSendDataToDynamicTopics() { assertThat(mockProcessors.get(1).processed(), equalTo(Collections.singletonList(new KeyValueTimestamp<>("b", "v1", 0)))); } - @SuppressWarnings("deprecation") // specifically testing the deprecated variant @Test public void shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningTopicCreatedWithRetention() { final StreamsBuilder builder = new StreamsBuilder(); @@ -1709,9 +1520,7 @@ public void shouldUseRecordMetadataTimestampExtractorWhenInternalRepartitioningT stream.join(kStream, valueJoiner, JoinWindows.of(ofMillis(windowSize)).grace(ofMillis(3 * windowSize)), - Joined.with(Serdes.String(), - Serdes.String(), - Serdes.String())) + StreamJoined.with(Serdes.String(), Serdes.String(), Serdes.String())) .to("output-topic", Produced.with(Serdes.String(), Serdes.String())); final ProcessorTopology topology = TopologyWrapper.getInternalTopologyBuilder(builder.build()).setApplicationId("X").buildTopology(); From e9c5a3995abcf5ddda16ff4ae4ad19be97035e9a Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Wed, 7 Apr 2021 21:13:32 -0700 Subject: [PATCH 023/155] MINOR: un-deprecate StreamsConfig overloads to support dependency injection (#10484) In #5344 it came to our attention that the StreamsConfig overloads of the KafkaStreams constructors are actually quite useful for dependency injection, providing a cleaner way to configure dependencies and better type safety. Reviewers: Matthias J. Sax --- .../apache/kafka/streams/KafkaStreams.java | 33 +++++++++++++++---- 1 file changed, 27 insertions(+), 6 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 28d58d6a28754..c1b4369d503a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -744,18 +744,32 @@ public KafkaStreams(final Topology topology, } /** - * @deprecated use {@link #KafkaStreams(Topology, Properties)} instead + * Create a {@code KafkaStreams} instance. + *

    + * Note: even if you never call {@link #start()} on a {@code KafkaStreams} instance, + * you still must {@link #close()} it to avoid resource leaks. + * + * @param topology the topology specifying the computational logic + * @param config configs for Kafka Streams + * @throws StreamsException if any fatal error occurs */ - @Deprecated public KafkaStreams(final Topology topology, final StreamsConfig config) { this(topology, config, new DefaultKafkaClientSupplier()); } /** - * @deprecated use {@link #KafkaStreams(Topology, Properties, KafkaClientSupplier)} instead + * Create a {@code KafkaStreams} instance. + *

    + * Note: even if you never call {@link #start()} on a {@code KafkaStreams} instance, + * you still must {@link #close()} it to avoid resource leaks. + * + * @param topology the topology specifying the computational logic + * @param config configs for Kafka Streams + * @param clientSupplier the Kafka clients supplier which provides underlying producer and consumer clients + * for the new {@code KafkaStreams} instance + * @throws StreamsException if any fatal error occurs */ - @Deprecated public KafkaStreams(final Topology topology, final StreamsConfig config, final KafkaClientSupplier clientSupplier) { @@ -763,9 +777,16 @@ public KafkaStreams(final Topology topology, } /** - * @deprecated use {@link #KafkaStreams(Topology, Properties, Time)} instead + * Create a {@code KafkaStreams} instance. + *

    + * Note: even if you never call {@link #start()} on a {@code KafkaStreams} instance, + * you still must {@link #close()} it to avoid resource leaks. + * + * @param topology the topology specifying the computational logic + * @param config configs for Kafka Streams + * @param time {@code Time} implementation; cannot be null + * @throws StreamsException if any fatal error occurs */ - @Deprecated public KafkaStreams(final Topology topology, final StreamsConfig config, final Time time) { From a97f0f515d02d743ab2fcc48fba024c2e115c243 Mon Sep 17 00:00:00 2001 From: Alok Thatikunta Date: Thu, 8 Apr 2021 01:27:07 -0700 Subject: [PATCH 024/155] MINOR: Update Crc32Test#testUpdate method with correct Crc32 (#10406) Reviewers: Chia-Ping Tsai --- .../test/java/org/apache/kafka/common/utils/Crc32Test.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/utils/Crc32Test.java b/clients/src/test/java/org/apache/kafka/common/utils/Crc32Test.java index a34c9a80a4a71..a358210366c6c 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/Crc32Test.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/Crc32Test.java @@ -29,9 +29,9 @@ public void testUpdate() { final byte[] bytes = "Any String you want".getBytes(); final int len = bytes.length; - Checksum crc1 = Crc32C.create(); - Checksum crc2 = Crc32C.create(); - Checksum crc3 = Crc32C.create(); + Checksum crc1 = new Crc32(); + Checksum crc2 = new Crc32(); + Checksum crc3 = new Crc32(); crc1.update(bytes, 0, len); for (int i = 0; i < len; i++) From 232e5150b56ca0c4253f1281db7de693e00fa11e Mon Sep 17 00:00:00 2001 From: CHUN-HAO TANG Date: Thu, 8 Apr 2021 16:35:15 +0800 Subject: [PATCH 025/155] MINOR: Remove redundant code from ReplicaManagerTest#testFetchMessagesWhenNotFollowerForOnePartition (#10501) Reviewers: Chia-Ping Tsai --- core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index ced71864bc892..4bbe7012c46cc 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -778,7 +778,6 @@ class ReplicaManagerTest { assertTrue(tp0Log.isDefined) assertEquals(1, tp0Log.get.highWatermark, "hw should be incremented") - replicaManager.localLog(tp1) val tp1Replica = replicaManager.localLog(tp1) assertTrue(tp1Replica.isDefined) assertEquals(0, tp1Replica.get.highWatermark, "hw should not be incremented") From aa0f450dada9ac47c5e9874c44eba801051ed899 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 8 Apr 2021 08:39:51 -0700 Subject: [PATCH 026/155] KAFKA-12630: Remove deprecated KafkaClientSupplier#getAdminClient in Streams (#10502) Reviewers: A. Sophie Blee-Goldman --- .../kafka/streams/KafkaClientSupplier.java | 18 +----------------- .../internals/DefaultKafkaClientSupplier.java | 7 ------- 2 files changed, 1 insertion(+), 24 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java b/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java index 2482ae3b3418c..fc96ca701c6ac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams; import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.streams.kstream.GlobalKTable; @@ -31,29 +30,14 @@ * @see KafkaStreams#KafkaStreams(Topology, java.util.Properties, KafkaClientSupplier) */ public interface KafkaClientSupplier { - /** - * Create an {@link AdminClient} which is used for internal topic management. - * - * @param config Supplied by the {@link java.util.Properties} given to the {@link KafkaStreams} - * @return an instance of {@link AdminClient} - * @deprecated Not called by Kafka Streams, which now uses {@link #getAdmin} instead. - */ - @Deprecated - default AdminClient getAdminClient(final Map config) { - throw new UnsupportedOperationException("Direct use of this method is deprecated. " + - "Implementations of KafkaClientSupplier should implement the getAdmin() method instead. " + - "The method will be removed in a future release."); - } - /** * Create an {@link Admin} which is used for internal topic management. * * @param config Supplied by the {@link java.util.Properties} given to the {@link KafkaStreams} * @return an instance of {@link Admin} */ - @SuppressWarnings("deprecation") default Admin getAdmin(final Map config) { - return getAdminClient(config); + throw new UnsupportedOperationException("Implementations of KafkaClientSupplier should implement the getAdmin() method."); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java index 803aef401ab82..15fec7645d47f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultKafkaClientSupplier.java @@ -19,7 +19,6 @@ import java.util.Map; import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; @@ -29,12 +28,6 @@ import org.apache.kafka.streams.KafkaClientSupplier; public class DefaultKafkaClientSupplier implements KafkaClientSupplier { - @Deprecated - @Override - public AdminClient getAdminClient(final Map config) { - return (AdminClient) getAdmin(config); - } - @Override public Admin getAdmin(final Map config) { // create a new client upon each call; but expect this call to be only triggered once so this should be fine From d44e01be4895bee9e19a8b9f0727e44af79ad940 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 8 Apr 2021 08:49:33 -0700 Subject: [PATCH 027/155] HOTFIX: add missing upgrade docs --- docs/streams/upgrade-guide.html | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 85d5ff9765f24..073badd95d35e 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -107,6 +107,7 @@

    Streams API
  • Overloaded JoinWindows#of, before, after, SessionWindows#with, TimeWindows#of, advanceBy, UnlimitedWindows#startOn and KafkaStreams#close with long typed parameters: deprecated in Kafka 2.1.0 (KIP-358).
  • Overloaded KStream#groupBy, groupByKey and KTable#groupBy with Serialized parameter: deprecated in Kafka 2.1.0 (KIP-372).
  • Joined#named, name: deprecated in Kafka 2.3.0 (KIP-307).
  • +
  • KafkaClientSupplier#getAdminClient: deprecated in Kafka 2.4.0 (KIP-476).
  • Overloaded KStream#join, leftJoin, outerJoin with KStream and Joined parameters: deprecated in Kafka 2.4.0 (KIP-479).
  • Overloaded KafkaStreams#metadataForKey: deprecated in Kafka 2.5.0 (KIP-535).
  • Overloaded KafkaStreams#store: deprecated in Kafka 2.5.0 (KIP-562).
  • From c2ea0c2e1d44332639214b921ddf3dc1edf478c0 Mon Sep 17 00:00:00 2001 From: Justine Olshan Date: Thu, 8 Apr 2021 10:24:23 -0700 Subject: [PATCH 028/155] KAFKA-12457; Add sentinel ID to metadata topic (#10492) KIP-516 introduces topic IDs to topics, but there is a small issue with how the KIP-500 metadata topic will interact with topic IDs. For example, https://github.com/apache/kafka/pull/9944 aims to replace topic names in the Fetch request with topic IDs. In order to get these IDs, brokers must fetch from the metadata topic. This leads to a sort of "chicken and the egg" problem concerning how we find out the metadata topic's topic ID. This PR adds the a special sentinel topic ID for the metadata topic, which gets around this problem. More information can be found in the [JIRA](https://issues.apache.org/jira/browse/KAFKA-12457) and in [KIP-516](https://cwiki.apache.org/confluence/display/KAFKA/KIP-516%3A+Topic+Identifiers). Reviewers: Jason Gustafson --- .../src/main/java/org/apache/kafka/common/Uuid.java | 8 ++++++-- .../test/java/org/apache/kafka/common/UuidTest.java | 4 +--- .../src/main/scala/kafka/raft/KafkaMetadataLog.scala | 12 ++++++++++-- core/src/main/scala/kafka/raft/RaftManager.scala | 4 +++- .../main/scala/kafka/server/KafkaRaftServer.scala | 4 +++- core/src/main/scala/kafka/tools/TestRaftServer.scala | 7 +++++-- .../test/java/kafka/testkit/KafkaClusterTestKit.java | 4 ++-- .../test/scala/kafka/raft/KafkaMetadataLogTest.scala | 8 ++++++++ raft/README.md | 2 +- .../java/org/apache/kafka/raft/ReplicatedLog.java | 6 ++++++ .../src/test/java/org/apache/kafka/raft/MockLog.java | 10 +++++++++- .../test/java/org/apache/kafka/raft/MockLogTest.java | 9 ++++++++- .../org/apache/kafka/raft/RaftClientTestContext.java | 2 +- .../apache/kafka/raft/RaftEventSimulationTest.java | 2 +- 14 files changed, 64 insertions(+), 18 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/Uuid.java b/clients/src/main/java/org/apache/kafka/common/Uuid.java index 6240070798708..a639f3edd7fc6 100644 --- a/clients/src/main/java/org/apache/kafka/common/Uuid.java +++ b/clients/src/main/java/org/apache/kafka/common/Uuid.java @@ -27,7 +27,11 @@ */ public class Uuid implements Comparable { - private static final java.util.UUID SENTINEL_ID_INTERNAL = new java.util.UUID(0L, 1L); + /** + * A UUID for the metadata topic in KRaft mode. Will never be returned by the randomUuid method. + */ + public static final Uuid METADATA_TOPIC_ID = new Uuid(0L, 1L); + private static final java.util.UUID METADATA_TOPIC_ID_INTERNAL = new java.util.UUID(0L, 1L); /** * A UUID that represents a null or empty UUID. Will never be returned by the randomUuid method. @@ -52,7 +56,7 @@ public Uuid(long mostSigBits, long leastSigBits) { */ public static Uuid randomUuid() { java.util.UUID uuid = java.util.UUID.randomUUID(); - while (uuid.equals(SENTINEL_ID_INTERNAL) || uuid.equals(ZERO_ID_INTERNAL)) { + while (uuid.equals(METADATA_TOPIC_ID_INTERNAL) || uuid.equals(ZERO_ID_INTERNAL)) { uuid = java.util.UUID.randomUUID(); } return new Uuid(uuid.getMostSignificantBits(), uuid.getLeastSignificantBits()); diff --git a/clients/src/test/java/org/apache/kafka/common/UuidTest.java b/clients/src/test/java/org/apache/kafka/common/UuidTest.java index be11af3f7577c..232b9927c896d 100644 --- a/clients/src/test/java/org/apache/kafka/common/UuidTest.java +++ b/clients/src/test/java/org/apache/kafka/common/UuidTest.java @@ -75,11 +75,9 @@ public void testStringConversion() { @Test public void testRandomUuid() { Uuid randomID = Uuid.randomUuid(); - // reservedSentinel is based on the value of SENTINEL_ID_INTERNAL in Uuid. - Uuid reservedSentinel = new Uuid(0L, 1L); assertNotEquals(randomID, Uuid.ZERO_UUID); - assertNotEquals(randomID, reservedSentinel); + assertNotEquals(randomID, Uuid.METADATA_TOPIC_ID); } @Test diff --git a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala index a917be0d9b2e9..d01f1c97915cb 100644 --- a/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala +++ b/core/src/main/scala/kafka/raft/KafkaMetadataLog.scala @@ -27,7 +27,7 @@ import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchLogEnd, LogDirFa import kafka.utils.{Logging, Scheduler} import org.apache.kafka.common.record.{MemoryRecords, Records} import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{KafkaException, TopicPartition} +import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.raft.{Isolation, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog} import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} @@ -220,6 +220,13 @@ final class KafkaMetadataLog private ( topicPartition } + /** + * Return the topic ID associated with the log. + */ + override def topicId(): Uuid = { + log.topicId.get + } + override def createSnapshot(snapshotId: OffsetAndEpoch): RawSnapshotWriter = { // Do not let the state machine create snapshots older than the latest snapshot latestSnapshotId().ifPresent { latest => @@ -321,6 +328,7 @@ object KafkaMetadataLog { def apply( topicPartition: TopicPartition, + topicId: Uuid, dataDir: File, time: Time, scheduler: Scheduler, @@ -346,7 +354,7 @@ object KafkaMetadataLog { producerIdExpirationCheckIntervalMs = Int.MaxValue, logDirFailureChannel = new LogDirFailureChannel(5), lastShutdownClean = false, - topicId = None, + topicId = Some(topicId), keepPartitionMetadataFile = false ) diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 50e67f43d3dca..ee06a95a78ab4 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -28,7 +28,7 @@ import kafka.server.{KafkaConfig, MetaProperties} import kafka.utils.timer.SystemTimer import kafka.utils.{KafkaScheduler, Logging, ShutdownableThread} import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, NetworkClient} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ChannelBuilders, ListenerName, NetworkReceive, Selectable, Selector} import org.apache.kafka.common.protocol.ApiMessage @@ -108,6 +108,7 @@ class KafkaRaftManager[T]( config: KafkaConfig, recordSerde: RecordSerde[T], topicPartition: TopicPartition, + topicId: Uuid, time: Time, metrics: Metrics, threadNamePrefixOpt: Option[String], @@ -244,6 +245,7 @@ class KafkaRaftManager[T]( private def buildMetadataLog(): KafkaMetadataLog = { KafkaMetadataLog( topicPartition, + topicId, dataDir, time, scheduler, diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index 965da861c7196..e1fc81f09ab8e 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -25,7 +25,7 @@ import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} import kafka.raft.KafkaRaftManager import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.utils.{AppInfoParser, Time} import org.apache.kafka.metadata.ApiMessageAndVersion import org.apache.kafka.raft.RaftConfig @@ -67,6 +67,7 @@ class KafkaRaftServer( config, new MetadataRecordSerde, KafkaRaftServer.MetadataPartition, + KafkaRaftServer.MetadataTopicId, time, metrics, threadNamePrefix, @@ -133,6 +134,7 @@ class KafkaRaftServer( object KafkaRaftServer { val MetadataTopic = "@metadata" val MetadataPartition = new TopicPartition(MetadataTopic, 0) + val MetadataTopicId = Uuid.METADATA_TOPIC_ID sealed trait ProcessRole case object BrokerRole extends ProcessRole diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index e52a960168927..c0c1c65df349c 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -42,7 +42,7 @@ import scala.jdk.CollectionConverters._ /** * This is an experimental server which is intended for testing the performance - * of the Raft implementation. It uses a hard-coded `__cluster_metadata` topic. + * of the Raft implementation. It uses a hard-coded `__raft_performance_test` topic. */ class TestRaftServer( val config: KafkaConfig, @@ -51,7 +51,9 @@ class TestRaftServer( ) extends Logging { import kafka.tools.TestRaftServer._ - private val partition = new TopicPartition("__cluster_metadata", 0) + private val partition = new TopicPartition("__raft_performance_test", 0) + // The topic ID must be constant. This value was chosen as to not conflict with the topic ID used for @metadata. + private val topicId = new Uuid(0L, 2L) private val time = Time.SYSTEM private val metrics = new Metrics(time) private val shutdownLatch = new CountDownLatch(1) @@ -82,6 +84,7 @@ class TestRaftServer( config, new ByteArraySerde, partition, + topicId, time, metrics, Some(threadNamePrefix), diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 6e3cebb525276..77e6a54fff8b9 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -173,7 +173,7 @@ public KafkaClusterTestKit build() throws Exception { MetaProperties metaProperties = MetaProperties.apply(nodes.clusterId().toString(), node.id()); TopicPartition metadataPartition = new TopicPartition(KafkaRaftServer.MetadataTopic(), 0); KafkaRaftManager raftManager = new KafkaRaftManager<>( - metaProperties, config, new MetadataRecordSerde(), metadataPartition, + metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftServer.MetadataTopicId(), Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future); MetaLogManager metaLogShim = new MetaLogRaftShim(raftManager.kafkaRaftClient(), config.nodeId()); ControllerServer controller = new ControllerServer( @@ -226,7 +226,7 @@ metaProperties, config, new MetadataRecordSerde(), metadataPartition, MetaProperties metaProperties = MetaProperties.apply(nodes.clusterId().toString(), node.id()); TopicPartition metadataPartition = new TopicPartition(KafkaRaftServer.MetadataTopic(), 0); KafkaRaftManager raftManager = new KafkaRaftManager<>( - metaProperties, config, new MetadataRecordSerde(), metadataPartition, + metaProperties, config, new MetadataRecordSerde(), metadataPartition, KafkaRaftServer.MetadataTopicId(), Time.SYSTEM, new Metrics(), Option.apply(threadNamePrefix), connectFutureManager.future); MetaLogManager metaLogShim = new MetaLogRaftShim(raftManager.kafkaRaftClient(), config.nodeId()); BrokerServer broker = new BrokerServer( diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index ffbba2a141be8..c54fd4780b626 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -104,6 +104,13 @@ final class KafkaMetadataLogTest { } } + @Test + def testTopicId(): Unit = { + val log = buildMetadataLog(tempDir, mockTime) + + assertEquals(KafkaRaftServer.MetadataTopicId, log.topicId()) + } + @Test def testReadMissingSnapshot(): Unit = { val log = buildMetadataLog(tempDir, mockTime) @@ -680,6 +687,7 @@ object KafkaMetadataLogTest { val metadataLog = KafkaMetadataLog( KafkaRaftServer.MetadataPartition, + KafkaRaftServer.MetadataTopicId, logDir, time, time.scheduler, diff --git a/raft/README.md b/raft/README.md index 824bd4f70ef4c..f4c27293f4459 100644 --- a/raft/README.md +++ b/raft/README.md @@ -43,6 +43,6 @@ Open up 3 separate terminals, and run individual commands: bin/test-kraft-server-start.sh --config config/kraft-quorum-3.properties Once a leader is elected, it will begin writing to an internal -`__cluster_metadata` topic with a steady workload of random data. +`__raft_performance_test` topic with a steady workload of random data. You can control the workload using the `--throughput` and `--record-size` arguments passed to `test-kraft-server-start.sh`. diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java index d385e0b948f5e..198a2c630a79f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.record.Records; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; @@ -198,6 +199,11 @@ default ValidOffsetAndEpoch validateOffsetAndEpoch(long offset, int epoch) { */ TopicPartition topicPartition(); + /** + * Return the topic ID associated with the log. + */ + Uuid topicId(); + /** * Truncate to an offset and epoch. * diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index 30cb6a19e8194..3ca50b14d9657 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; @@ -58,13 +59,15 @@ public class MockLog implements ReplicatedLog { private final List batches = new ArrayList<>(); private final NavigableMap snapshots = new TreeMap<>(); private final TopicPartition topicPartition; + private final Uuid topicId; private long nextId = ID_GENERATOR.getAndIncrement(); private LogOffsetMetadata highWatermark = new LogOffsetMetadata(0, Optional.empty()); private long lastFlushedOffset = 0; - public MockLog(TopicPartition topicPartition) { + public MockLog(TopicPartition topicPartition, Uuid topicId) { this.topicPartition = topicPartition; + this.topicId = topicId; } @Override @@ -126,6 +129,11 @@ public TopicPartition topicPartition() { return topicPartition; } + @Override + public Uuid topicId() { + return topicId; + } + private Optional metadataForOffset(long offset) { if (offset == endOffset().offset) { return endOffset().metadata; diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java index 1c7f1310ac032..f219dea729a02 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.OffsetOutOfRangeException; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.record.CompressionType; @@ -52,10 +53,11 @@ public class MockLogTest { private MockLog log; private final TopicPartition topicPartition = new TopicPartition("mock-topic", 0); + private final Uuid topicId = Uuid.randomUuid(); @BeforeEach public void setup() { - log = new MockLog(topicPartition); + log = new MockLog(topicPartition, topicId); } @AfterEach @@ -68,6 +70,11 @@ public void testTopicPartition() { assertEquals(topicPartition, log.topicPartition()); } + @Test + public void testTopicId() { + assertEquals(topicId, log.topicId()); + } + @Test public void testAppendAsLeaderHelper() { int epoch = 2; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 39e9c34de54d8..0339cadd46061 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -125,7 +125,7 @@ public static final class Builder { private final MockTime time = new MockTime(); 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 MockLog log = new MockLog(METADATA_PARTITION, Uuid.METADATA_TOPIC_ID); private final Set voters; private final OptionalInt localId; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 94a1a9601737b..94b278e7cc4a1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -492,7 +492,7 @@ void runUntil(Supplier exitCondition) { private static class PersistentState { final MockQuorumStateStore store = new MockQuorumStateStore(); - final MockLog log = new MockLog(METADATA_PARTITION); + final MockLog log = new MockLog(METADATA_PARTITION, Uuid.METADATA_TOPIC_ID); } private static class Cluster { From d2c06c9c3c35803b9f5f0b6060b242789657f008 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 8 Apr 2021 10:42:30 -0700 Subject: [PATCH 029/155] KAFKA-12619; Raft leader should expose hw only after committing LeaderChange (#10481) KIP-595 describes an extra condition on commitment here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum#KIP595:ARaftProtocolfortheMetadataQuorum-Fetch. In order to ensure that a newly elected leader's committed entries cannot get lost, it must commit one record from its own epoch. This guarantees that its latest entry is larger (in terms of epoch/offset) than any previously written record which ensures that any future leader must also include it. This is the purpose of the `LeaderChange` record which is written to the log as soon as the leader gets elected. Although we had this check implemented, it was off by one. We only ensured that replication reached the epoch start offset, which does not reflect the appended `LeaderChange` record. This patch fixes the check and clarifies the point of the check. The rest of the patch is just fixing up test cases. Reviewers: dengziming , Guozhang Wang --- .../org/apache/kafka/raft/LeaderState.java | 13 +++- .../kafka/raft/KafkaRaftClientTest.java | 77 +++++++++++++------ .../apache/kafka/raft/LeaderStateTest.java | 60 +++++++-------- .../kafka/raft/RaftClientTestContext.java | 33 +++----- 4 files changed, 100 insertions(+), 83 deletions(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 95c5628bf09f4..762769b12c51f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -113,13 +113,18 @@ private boolean updateHighWatermark() { Optional highWatermarkUpdateOpt = followersByDescendingFetchOffset.get(indexOfHw).endOffset; if (highWatermarkUpdateOpt.isPresent()) { - // When a leader is first elected, it cannot know the high watermark of the previous - // leader. In order to avoid exposing a non-monotonically increasing value, we have - // to wait for followers to catch up to the start of the leader's epoch. + + // The KRaft protocol requires an extra condition on commitment after a leader + // election. The leader must commit one record from its own epoch before it is + // allowed to expose records from any previous epoch. This guarantees that its + // log will contain the largest record (in terms of epoch/offset) in any log + // which ensures that any future leader will have replicated this record as well + // as all records from previous epochs that the current leader has committed. + LogOffsetMetadata highWatermarkUpdateMetadata = highWatermarkUpdateOpt.get(); long highWatermarkUpdateOffset = highWatermarkUpdateMetadata.offset; - if (highWatermarkUpdateOffset >= epochStartOffset) { + if (highWatermarkUpdateOffset > epochStartOffset) { if (highWatermark.isPresent()) { LogOffsetMetadata currentHighWatermarkMetadata = highWatermark.get(); if (highWatermarkUpdateOffset > currentHighWatermarkMetadata.offset diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 61e3e86a4d4cf..54c38ddd3ef12 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -187,7 +187,8 @@ public void testEndQuorumEpochRetriesWhileResigned() throws Exception { .build(); context.pollUntilRequest(); - List requests = context.collectEndQuorumRequests(epoch, Utils.mkSet(voter1, voter2)); + List requests = context.collectEndQuorumRequests( + epoch, Utils.mkSet(voter1, voter2), Optional.empty()); assertEquals(2, requests.size()); // Respond to one of the requests so that we can verify that no additional @@ -203,7 +204,8 @@ public void testEndQuorumEpochRetriesWhileResigned() throws Exception { int nonRespondedId = requests.get(1).destinationId(); context.time.sleep(6000); context.pollUntilRequest(); - List retries = context.collectEndQuorumRequests(epoch, Utils.mkSet(nonRespondedId)); + List retries = context.collectEndQuorumRequests( + epoch, Utils.mkSet(nonRespondedId), Optional.empty()); assertEquals(1, retries.size()); } @@ -831,13 +833,14 @@ public void testListenerCommitCallbackAfterLeaderWrite() throws Exception { // First poll has no high watermark advance context.client.poll(); assertEquals(OptionalLong.empty(), context.client.highWatermark()); + assertEquals(1L, context.log.endOffset().offset); // Let follower send a fetch to initialize the high watermark, // note the offset 0 would be a control message for becoming the leader - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 0L, epoch, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 1L, epoch, 0)); context.pollUntilResponse(); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); - assertEquals(OptionalLong.of(0L), context.client.highWatermark()); + assertEquals(OptionalLong.of(1L), context.client.highWatermark()); List records = Arrays.asList("a", "b", "c"); long offset = context.client.scheduleAppend(epoch, records); @@ -1579,7 +1582,18 @@ public void testEndQuorumEpochSentBasedOnFetchOffset() throws Exception { RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); - context.buildFollowerSet(epoch, closeFollower, laggingFollower); + // The lagging follower fetches first + context.deliverRequest(context.fetchRequest(1, laggingFollower, 1L, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(1L, epoch); + + // Append some records, so that the close follower will be able to advance further. + context.client.scheduleAppend(epoch, Arrays.asList("foo", "bar")); + context.client.poll(); + + context.deliverRequest(context.fetchRequest(epoch, closeFollower, 3L, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(3L, epoch); // Now shutdown context.client.shutdown(context.electionTimeoutMs() * 2); @@ -1591,10 +1605,11 @@ public void testEndQuorumEpochSentBasedOnFetchOffset() throws Exception { context.pollUntilRequest(); assertTrue(context.client.isRunning()); - List endQuorumRequests = context.collectEndQuorumRequests( - 1, Utils.mkSet(closeFollower, laggingFollower)); - - assertEquals(2, endQuorumRequests.size()); + context.collectEndQuorumRequests( + epoch, + Utils.mkSet(closeFollower, laggingFollower), + Optional.of(Arrays.asList(closeFollower, laggingFollower)) + ); } @Test @@ -1607,22 +1622,27 @@ public void testDescribeQuorum() throws Exception { RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); - context.buildFollowerSet(epoch, closeFollower, laggingFollower); + context.deliverRequest(context.fetchRequest(1, laggingFollower, 1L, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(1L, epoch); + + context.client.scheduleAppend(epoch, Arrays.asList("foo", "bar")); + context.client.poll(); + + context.deliverRequest(context.fetchRequest(epoch, closeFollower, 3L, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(3L, epoch); // Create observer int observerId = 3; context.deliverRequest(context.fetchRequest(epoch, observerId, 0L, 0, 0)); - context.pollUntilResponse(); - - long highWatermark = 1L; - context.assertSentFetchPartitionResponse(highWatermark, epoch); + context.assertSentFetchPartitionResponse(3L, epoch); context.deliverRequest(DescribeQuorumRequest.singletonRequest(context.metadataPartition)); - context.pollUntilResponse(); - context.assertSentDescribeQuorumResponse(localId, epoch, highWatermark, + context.assertSentDescribeQuorumResponse(localId, epoch, 3L, Arrays.asList( new ReplicaState() .setReplicaId(localId) @@ -1631,10 +1651,10 @@ public void testDescribeQuorum() throws Exception { .setLogEndOffset(3L), new ReplicaState() .setReplicaId(laggingFollower) - .setLogEndOffset(0L), + .setLogEndOffset(1L), new ReplicaState() .setReplicaId(closeFollower) - .setLogEndOffset(1L)), + .setLogEndOffset(3)), singletonList( new ReplicaState() .setReplicaId(observerId) @@ -2099,6 +2119,10 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse context.becomeLeader(); context.client.poll(); + // After becoming leader, we expect the `LeaderChange` record to be appended + // in addition to the initial 9 records in the log. + assertEquals(10L, context.log.endOffset().offset); + // The high watermark is not known to the leader until the followers // begin fetching, so we should not have fired the `handleClaim` callback. assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); @@ -2114,8 +2138,8 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse // Now catch up to the start of the leader epoch so that the high // watermark advances and we can start sending committed data to the - // listener. - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 9L, 2, 500)); + // listener. Note that the `LeaderChange` control record is filtered. + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 500)); context.client.poll(); assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); assertEquals(3, context.listener.numCommittedBatches()); @@ -2150,11 +2174,12 @@ public void testLateRegisteredListenerCatchesUp() throws Exception { context.becomeLeader(); context.client.poll(); + assertEquals(10L, context.log.endOffset().offset); // Let the initial listener catch up - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 9L, 2, 500)); + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 0)); context.client.poll(); - assertEquals(OptionalLong.of(9L), context.client.highWatermark()); + assertEquals(OptionalLong.of(10L), context.client.highWatermark()); context.client.poll(); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); @@ -2280,16 +2305,18 @@ public void testHandleCommitCallbackFiresInCandidateState() throws Exception { // Start off as the leader and receive a fetch to initialize the high watermark context.becomeLeader(); - context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 9L, epoch, 500)); + assertEquals(10L, context.log.endOffset().offset); + + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 0)); context.pollUntilResponse(); - assertEquals(OptionalLong.of(9L), context.client.highWatermark()); + assertEquals(OptionalLong.of(10L), context.client.highWatermark()); context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); // Now we receive a vote request which transitions us to the 'unattached' state context.deliverRequest(context.voteRequest(epoch + 1, otherNodeId, epoch, 9L)); context.pollUntilResponse(); context.assertUnknownLeader(epoch + 1); - assertEquals(OptionalLong.of(9L), context.client.highWatermark()); + assertEquals(OptionalLong.of(10L), context.client.highWatermark()); // Timeout the election and become candidate int candidateEpoch = epoch + 2; diff --git a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java index c7490b5a913e9..29ff800c4afbe 100644 --- a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java @@ -81,27 +81,32 @@ public void testNonFollowerAcknowledgement() { public void testUpdateHighWatermarkQuorumSizeOne() { LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateLocalState(0, new LogOffsetMetadata(15L))); - assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); + assertFalse(state.updateLocalState(0, new LogOffsetMetadata(15L))); + assertEquals(emptySet(), state.nonAcknowledgingVoters()); + assertEquals(Optional.empty(), state.highWatermark()); + assertTrue(state.updateLocalState(0, new LogOffsetMetadata(16L))); + assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); + assertTrue(state.updateLocalState(0, new LogOffsetMetadata(20))); + assertEquals(Optional.of(new LogOffsetMetadata(20L)), state.highWatermark()); } @Test public void testNonMonotonicLocalEndOffsetUpdate() { LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateLocalState(0, new LogOffsetMetadata(15L))); - assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); + assertTrue(state.updateLocalState(0, new LogOffsetMetadata(16L))); + assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); assertThrows(IllegalStateException.class, - () -> state.updateLocalState(0, new LogOffsetMetadata(14L))); + () -> state.updateLocalState(0, new LogOffsetMetadata(15L))); } @Test public void testIdempotentEndOffsetUpdate() { LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateLocalState(0, new LogOffsetMetadata(15L))); - assertFalse(state.updateLocalState(0, new LogOffsetMetadata(15L))); - assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); + assertTrue(state.updateLocalState(0, new LogOffsetMetadata(16L))); + assertFalse(state.updateLocalState(0, new LogOffsetMetadata(16L))); + assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); } @Test @@ -109,11 +114,11 @@ public void testUpdateHighWatermarkMetadata() { LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); - LogOffsetMetadata initialHw = new LogOffsetMetadata(15L, Optional.of(new MockOffsetMetadata("foo"))); + LogOffsetMetadata initialHw = new LogOffsetMetadata(16L, Optional.of(new MockOffsetMetadata("bar"))); assertTrue(state.updateLocalState(0, initialHw)); assertEquals(Optional.of(initialHw), state.highWatermark()); - LogOffsetMetadata updateHw = new LogOffsetMetadata(15L, Optional.of(new MockOffsetMetadata("bar"))); + LogOffsetMetadata updateHw = new LogOffsetMetadata(16L, Optional.of(new MockOffsetMetadata("baz"))); assertTrue(state.updateLocalState(0, updateHw)); assertEquals(Optional.of(updateHw), state.highWatermark()); } @@ -122,25 +127,16 @@ public void testUpdateHighWatermarkMetadata() { public void testUpdateHighWatermarkQuorumSizeTwo() { int otherNodeId = 1; LeaderState state = newLeaderState(mkSet(localId, otherNodeId), 10L); - assertFalse(state.updateLocalState(0, new LogOffsetMetadata(15L))); - assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateReplicaState(otherNodeId, 0, new LogOffsetMetadata(10L))); - assertEquals(emptySet(), state.nonAcknowledgingVoters()); - assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); - assertTrue(state.updateReplicaState(otherNodeId, 0, new LogOffsetMetadata(15L))); - assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - } - - @Test - public void testHighWatermarkUnknownUntilStartOfLeaderEpoch() { - int otherNodeId = 1; - LeaderState state = newLeaderState(mkSet(localId, otherNodeId), 15L); - assertFalse(state.updateLocalState(0, new LogOffsetMetadata(20L))); + assertFalse(state.updateLocalState(0, new LogOffsetMetadata(13L))); + assertEquals(singleton(otherNodeId), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); assertFalse(state.updateReplicaState(otherNodeId, 0, new LogOffsetMetadata(10L))); + assertEquals(emptySet(), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateReplicaState(otherNodeId, 0, new LogOffsetMetadata(15L))); - assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); + assertTrue(state.updateReplicaState(otherNodeId, 0, new LogOffsetMetadata(11L))); + assertEquals(Optional.of(new LogOffsetMetadata(11L)), state.highWatermark()); + assertTrue(state.updateReplicaState(otherNodeId, 0, new LogOffsetMetadata(13L))); + assertEquals(Optional.of(new LogOffsetMetadata(13L)), state.highWatermark()); } @Test @@ -149,17 +145,21 @@ public void testUpdateHighWatermarkQuorumSizeThree() { int node2 = 2; LeaderState state = newLeaderState(mkSet(localId, node1, node2), 10L); assertFalse(state.updateLocalState(0, new LogOffsetMetadata(15L))); + assertEquals(mkSet(node1, node2), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); - assertTrue(state.updateReplicaState(node1, 0, new LogOffsetMetadata(10L))); + assertFalse(state.updateReplicaState(node1, 0, new LogOffsetMetadata(10L))); assertEquals(singleton(node2), state.nonAcknowledgingVoters()); - assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); + assertEquals(Optional.empty(), state.highWatermark()); + assertFalse(state.updateReplicaState(node2, 0, new LogOffsetMetadata(10L))); + assertEquals(emptySet(), state.nonAcknowledgingVoters()); + assertEquals(Optional.empty(), state.highWatermark()); assertTrue(state.updateReplicaState(node2, 0, new LogOffsetMetadata(15L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); assertFalse(state.updateLocalState(0, new LogOffsetMetadata(20L))); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); - assertTrue(state.updateReplicaState(node2, 0, new LogOffsetMetadata(20L))); + assertTrue(state.updateReplicaState(node1, 0, new LogOffsetMetadata(20L))); assertEquals(Optional.of(new LogOffsetMetadata(20L)), state.highWatermark()); - assertFalse(state.updateReplicaState(node1, 0, new LogOffsetMetadata(20L))); + assertFalse(state.updateReplicaState(node2, 0, new LogOffsetMetadata(20L))); assertEquals(Optional.of(new LogOffsetMetadata(20L)), state.highWatermark()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 0339cadd46061..024095af91de5 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -63,7 +63,6 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -584,13 +583,13 @@ void assertSentBeginQuorumEpochResponse( int assertSentEndQuorumEpochRequest(int epoch, int destinationId) { List endQuorumRequests = collectEndQuorumRequests( - epoch, Collections.singleton(destinationId)); + epoch, Collections.singleton(destinationId), Optional.empty()); assertEquals(1, endQuorumRequests.size()); return endQuorumRequests.get(0).correlationId(); } void assertSentEndQuorumEpochResponse( - Errors responseError + Errors responseError ) { List sentMessages = drainSentResponses(ApiKeys.END_QUORUM_EPOCH); assertEquals(1, sentMessages.size()); @@ -728,28 +727,11 @@ Optional assertSentFetchSnapshotRes return FetchSnapshotResponse.forTopicPartition(response, topicPartition); } - void buildFollowerSet( + List collectEndQuorumRequests( int epoch, - int closeFollower, - int laggingFollower - ) throws Exception { - // The lagging follower fetches first - deliverRequest(fetchRequest(1, laggingFollower, 0L, 0, 0)); - - pollUntilResponse(); - assertSentFetchPartitionResponse(0L, epoch); - - // Append some records, so that the close follower will be able to advance further. - client.scheduleAppend(epoch, Arrays.asList("foo", "bar")); - client.poll(); - - deliverRequest(fetchRequest(epoch, closeFollower, 1L, epoch, 0)); - - pollUntilResponse(); - assertSentFetchPartitionResponse(1L, epoch); - } - - List collectEndQuorumRequests(int epoch, Set destinationIdSet) { + Set destinationIdSet, + Optional> preferredSuccessorsOpt + ) { List endQuorumRequests = new ArrayList<>(); Set collectedDestinationIdSet = new HashSet<>(); for (RaftMessage raftMessage : channel.drainSendQueue()) { @@ -761,6 +743,9 @@ List collectEndQuorumRequests(int epoch, Set dest assertEquals(epoch, partitionRequest.leaderEpoch()); assertEquals(localIdOrThrow(), partitionRequest.leaderId()); + preferredSuccessorsOpt.ifPresent(preferredSuccessors -> { + assertEquals(preferredSuccessors, partitionRequest.preferredSuccessors()); + }); RaftRequest.Outbound outboundRequest = (RaftRequest.Outbound) raftMessage; collectedDestinationIdSet.add(outboundRequest.destinationId()); From 5964401bf9aab611bd4a072941bd1c927e044258 Mon Sep 17 00:00:00 2001 From: David Arthur Date: Thu, 8 Apr 2021 13:56:21 -0400 Subject: [PATCH 030/155] KAFKA-12406 Integrate client quotas with KRaft broker (#10254) --- .../message/DescribeClientQuotasRequest.json | 2 +- .../scala/kafka/server/BrokerServer.scala | 3 +- .../main/scala/kafka/server/KafkaApis.scala | 68 +++++---- .../scala/kafka/server/MetadataSupport.scala | 5 +- .../server/metadata/ClientQuotaCache.scala | 12 +- .../metadata/ClientQuotaMetadataManager.scala | 26 ++-- .../test/java/kafka/test/ClusterConfig.java | 4 +- .../test/junit/ClusterTestExtensions.java | 8 +- .../junit/RaftClusterInvocationContext.java | 4 +- .../junit/ZkClusterInvocationContext.java | 2 +- .../test/java/kafka/testkit/TestKitNodes.java | 2 +- .../kafka/server/RaftClusterTest.scala | 111 +++++++++++++- .../server/ClientQuotasRequestTest.scala | 142 +++++++++--------- .../unit/kafka/server/KafkaApisTest.scala | 6 +- .../scala/unit/kafka/utils/TestUtils.scala | 22 +++ .../controller/ClientQuotaControlManager.java | 36 +++-- .../ClientQuotaControlManagerTest.java | 22 ++- 17 files changed, 335 insertions(+), 140 deletions(-) diff --git a/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json b/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json index 5ada552e29cfe..d14cfc95733d3 100644 --- a/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json +++ b/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json @@ -16,7 +16,7 @@ { "apiKey": 48, "type": "request", - "listeners": ["zkBroker"], + "listeners": ["zkBroker", "broker"], "name": "DescribeClientQuotasRequest", // Version 1 enables flexible versions. "validVersions": "0-1", diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 65620d3ae5a3d..4790b9f8b186f 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -115,6 +115,7 @@ class BrokerServer( var metadataCache: RaftMetadataCache = null var quotaManagers: QuotaFactory.QuotaManagers = null + var quotaCache: ClientQuotaCache = null private var _brokerTopicStats: BrokerTopicStats = null @@ -323,7 +324,7 @@ class BrokerServer( // Start processing requests once we've caught up on the metadata log, recovered logs if necessary, // and started all services that we previously delayed starting. - val raftSupport = RaftSupport(forwardingManager, metadataCache) + val raftSupport = RaftSupport(forwardingManager, metadataCache, quotaCache) dataPlaneRequestProcessor = new KafkaApis(socketServer.dataPlaneRequestChannel, raftSupport, replicaManager, groupCoordinator, transactionCoordinator, autoTopicCreationManager, config.nodeId, config, configRepository, metadataCache, metrics, authorizer, quotaManagers, diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 42a203c4a570c..ef8476381d18e 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -59,6 +59,7 @@ import org.apache.kafka.common.message._ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.{ListenerName, Send} import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.quota.ClientQuotaEntity import org.apache.kafka.common.record._ import org.apache.kafka.common.replica.ClientMetadata import org.apache.kafka.common.replica.ClientMetadata.DefaultClientMetadata @@ -3030,37 +3031,50 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDescribeClientQuotasRequest(request: RequestChannel.Request): Unit = { - val zkSupport = metadataSupport.requireZkOrThrow(KafkaApis.notYetSupported(request)) val describeClientQuotasRequest = request.body[DescribeClientQuotasRequest] - if (authHelper.authorize(request.context, DESCRIBE_CONFIGS, CLUSTER, CLUSTER_NAME)) { - val result = zkSupport.adminManager.describeClientQuotas(describeClientQuotasRequest.filter) - - val entriesData = result.iterator.map { case (quotaEntity, quotaValues) => - val entityData = quotaEntity.entries.asScala.iterator.map { case (entityType, entityName) => - new DescribeClientQuotasResponseData.EntityData() - .setEntityType(entityType) - .setEntityName(entityName) - }.toBuffer - - val valueData = quotaValues.iterator.map { case (key, value) => - new DescribeClientQuotasResponseData.ValueData() - .setKey(key) - .setValue(value) - }.toBuffer - - new DescribeClientQuotasResponseData.EntryData() - .setEntity(entityData.asJava) - .setValues(valueData.asJava) - }.toBuffer - - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData() - .setThrottleTimeMs(requestThrottleMs) - .setEntries(entriesData.asJava))) - } else { + if (!authHelper.authorize(request.context, DESCRIBE_CONFIGS, CLUSTER, CLUSTER_NAME)) { requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => describeClientQuotasRequest.getErrorResponse(requestThrottleMs, Errors.CLUSTER_AUTHORIZATION_FAILED.exception)) + } else { + metadataSupport match { + case ZkSupport(adminManager, controller, zkClient, forwardingManager, metadataCache) => + val result = adminManager.describeClientQuotas(describeClientQuotasRequest.filter) + + val entriesData = result.iterator.map { case (quotaEntity, quotaValues) => + val entityData = quotaEntity.entries.asScala.iterator.map { case (entityType, entityName) => + new DescribeClientQuotasResponseData.EntityData() + .setEntityType(entityType) + .setEntityName(entityName) + }.toBuffer + + val valueData = quotaValues.iterator.map { case (key, value) => + new DescribeClientQuotasResponseData.ValueData() + .setKey(key) + .setValue(value) + }.toBuffer + + new DescribeClientQuotasResponseData.EntryData() + .setEntity(entityData.asJava) + .setValues(valueData.asJava) + }.toBuffer + + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData() + .setThrottleTimeMs(requestThrottleMs) + .setEntries(entriesData.asJava))) + case RaftSupport(fwdMgr, metadataCache, quotaCache) => + val result = quotaCache.describeClientQuotas( + describeClientQuotasRequest.filter().components().asScala.toSeq, + describeClientQuotasRequest.filter().strict()) + val resultAsJava = new util.HashMap[ClientQuotaEntity, util.Map[String, java.lang.Double]](result.size) + result.foreach { case (entity, quotas) => + resultAsJava.put(entity, quotas.map { case (key, quota) => key -> Double.box(quota)}.asJava) + } + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + DescribeClientQuotasResponse.fromQuotaEntities(resultAsJava, requestThrottleMs) + ) + } } } diff --git a/core/src/main/scala/kafka/server/MetadataSupport.scala b/core/src/main/scala/kafka/server/MetadataSupport.scala index 86390eaa094c7..3970f4988330c 100644 --- a/core/src/main/scala/kafka/server/MetadataSupport.scala +++ b/core/src/main/scala/kafka/server/MetadataSupport.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.controller.KafkaController import kafka.network.RequestChannel -import kafka.server.metadata.RaftMetadataCache +import kafka.server.metadata.{ClientQuotaCache, RaftMetadataCache} import kafka.zk.{AdminZkClient, KafkaZkClient} import org.apache.kafka.common.requests.AbstractResponse @@ -91,7 +91,8 @@ case class ZkSupport(adminManager: ZkAdminManager, override def controllerId: Option[Int] = metadataCache.getControllerId } -case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: RaftMetadataCache) extends MetadataSupport { +case class RaftSupport(fwdMgr: ForwardingManager, metadataCache: RaftMetadataCache, quotaCache: ClientQuotaCache) + extends MetadataSupport { override val forwardingManager: Option[ForwardingManager] = Some(fwdMgr) override def requireZkOrThrow(createException: => Exception): ZkSupport = throw createException override def requireRaftOrThrow(createException: => Exception): RaftSupport = this diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaCache.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaCache.scala index ad2378efc32cb..3a148e63bd95a 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaCache.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaCache.scala @@ -122,6 +122,14 @@ class ClientQuotaCache { entityFilters.put(entityType, entityMatch) } + // Special case for non-strict empty filter, match everything + if (filters.isEmpty && !strict) { + val allResults: Map[QuotaEntity, Map[String, Double]] = quotaCache.map { + entry => entry._1 -> entry._2.toMap + }.toMap + return allResults + } + if (entityFilters.isEmpty) { return Map.empty } @@ -130,7 +138,7 @@ class ClientQuotaCache { val matchingEntities: Set[QuotaEntity] = if (entityFilters.contains(ClientQuotaEntity.IP)) { if (entityFilters.size > 1) { throw new InvalidRequestException("Invalid entity filter component combination, IP filter component should " + - "not be used with user or clientId filter component.") + "not be used with User or ClientId filter component.") } val ipMatch = entityFilters.get(ClientQuotaEntity.IP) ipMatch.fold(Set.empty[QuotaEntity]) { @@ -294,4 +302,6 @@ class ClientQuotaCache { updateCacheIndexPartial(ipEntityIndex, DefaultIp) } } + + override def toString = s"ClientQuotaCache($quotaCache)" } diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index 76e0b54d90752..c7e0e9217434f 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -121,16 +121,20 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag return } - // Update the cache - quotaCache.updateQuotaCache(ipEntity, quotaRecord.key, quotaRecord.value, quotaRecord.remove) - // Convert the value to an appropriate Option for the quota manager val newValue = if (quotaRecord.remove()) { None } else { Some(quotaRecord.value).map(_.toInt) } - connectionQuotas.updateIpConnectionRateQuota(inetAddress, newValue) + try { + connectionQuotas.updateIpConnectionRateQuota(inetAddress, newValue) + } catch { + case t: Throwable => error(s"Failed to update IP quota $ipEntity", t) + } + + // Update the cache + quotaCache.updateQuotaCache(ipEntity, quotaRecord.key, quotaRecord.value, quotaRecord.remove) } def handleUserClientQuota(quotaEntity: QuotaEntity, quotaRecord: QuotaRecord): Unit = { @@ -163,11 +167,15 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag Some(new Quota(quotaRecord.value(), true)) } - manager.updateQuota( - sanitizedUser = sanitizedUser, - clientId = sanitizedClientId.map(Sanitizer.desanitize), - sanitizedClientId = sanitizedClientId, - quota = quotaValue) + try { + manager.updateQuota( + sanitizedUser = sanitizedUser, + clientId = sanitizedClientId.map(Sanitizer.desanitize), + sanitizedClientId = sanitizedClientId, + quota = quotaValue) + } catch { + case t: Throwable => error(s"Failed to update user-client quota $quotaEntity", t) + } quotaCache.updateQuotaCache(quotaEntity, quotaRecord.key, quotaRecord.value, quotaRecord.remove) } diff --git a/core/src/test/java/kafka/test/ClusterConfig.java b/core/src/test/java/kafka/test/ClusterConfig.java index db5f14e1a4e0a..e7e8bf5a0234a 100644 --- a/core/src/test/java/kafka/test/ClusterConfig.java +++ b/core/src/test/java/kafka/test/ClusterConfig.java @@ -119,8 +119,8 @@ public Optional trustStoreFile() { public Map nameTags() { Map tags = new LinkedHashMap<>(3); name().ifPresent(name -> tags.put("Name", name)); - tags.put("security", securityProtocol.name()); - listenerName().ifPresent(listener -> tags.put("listener", listener)); + tags.put("Security", securityProtocol.name()); + listenerName().ifPresent(listener -> tags.put("Listener", listener)); return tags; } diff --git a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java index ced1e6fcbd8b7..94eefbd91cb81 100644 --- a/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java +++ b/core/src/test/java/kafka/test/junit/ClusterTestExtensions.java @@ -98,14 +98,14 @@ public Stream provideTestTemplateInvocationContex // Process single @ClusterTest annotation ClusterTest clusterTestAnnot = context.getRequiredTestMethod().getDeclaredAnnotation(ClusterTest.class); if (clusterTestAnnot != null) { - processClusterTest(clusterTestAnnot, defaults, generatedContexts::add); + processClusterTest(context, clusterTestAnnot, defaults, generatedContexts::add); } // Process multiple @ClusterTest annotation within @ClusterTests ClusterTests clusterTestsAnnot = context.getRequiredTestMethod().getDeclaredAnnotation(ClusterTests.class); if (clusterTestsAnnot != null) { for (ClusterTest annot : clusterTestsAnnot.value()) { - processClusterTest(annot, defaults, generatedContexts::add); + processClusterTest(context, annot, defaults, generatedContexts::add); } } @@ -137,7 +137,7 @@ private void generateClusterConfigurations(ExtensionContext context, String gene ReflectionUtils.invokeMethod(method, testInstance, generator); } - private void processClusterTest(ClusterTest annot, ClusterTestDefaults defaults, + private void processClusterTest(ExtensionContext context, ClusterTest annot, ClusterTestDefaults defaults, Consumer testInvocations) { final Type type; if (annot.clusterType() == Type.DEFAULT) { @@ -182,6 +182,8 @@ private void processClusterTest(ClusterTest annot, ClusterTestDefaults defaults, ClusterConfig.Builder builder = ClusterConfig.clusterBuilder(type, brokers, controllers, autoStart, annot.securityProtocol()); if (!annot.name().isEmpty()) { builder.name(annot.name()); + } else { + builder.name(context.getDisplayName()); } if (!annot.listener().isEmpty()) { builder.listenerName(annot.listener()); diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java index aa3fa057b080d..b4e50c59bfe98 100644 --- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java @@ -68,7 +68,7 @@ public String getDisplayName(int invocationIndex) { String clusterDesc = clusterConfig.nameTags().entrySet().stream() .map(Object::toString) .collect(Collectors.joining(", ")); - return String.format("[Quorum %d] %s", invocationIndex, clusterDesc); + return String.format("[%d] Type=Raft, %s", invocationIndex, clusterDesc); } @Override @@ -77,7 +77,7 @@ public List getAdditionalExtensions() { (BeforeTestExecutionCallback) context -> { KafkaClusterTestKit.Builder builder = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setNumKip500BrokerNodes(clusterConfig.numBrokers()). + setNumBrokerNodes(clusterConfig.numBrokers()). setNumControllerNodes(clusterConfig.numControllers()).build()); // Copy properties into the TestKit builder diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java index 62cc80df27771..8d4660b77382e 100644 --- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java +++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java @@ -70,7 +70,7 @@ public String getDisplayName(int invocationIndex) { String clusterDesc = clusterConfig.nameTags().entrySet().stream() .map(Object::toString) .collect(Collectors.joining(", ")); - return String.format("[Zk %d] %s", invocationIndex, clusterDesc); + return String.format("[%d] Type=ZK, %s", invocationIndex, clusterDesc); } @Override diff --git a/core/src/test/java/kafka/testkit/TestKitNodes.java b/core/src/test/java/kafka/testkit/TestKitNodes.java index b2d95041588f9..2950887c29f60 100644 --- a/core/src/test/java/kafka/testkit/TestKitNodes.java +++ b/core/src/test/java/kafka/testkit/TestKitNodes.java @@ -81,7 +81,7 @@ public Builder setNumControllerNodes(int numControllerNodes) { return this; } - public Builder setNumKip500BrokerNodes(int numBrokerNodes) { + public Builder setNumBrokerNodes(int numBrokerNodes) { if (numBrokerNodes < 0) { throw new RuntimeException("Invalid negative value for numBrokerNodes"); } diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterTest.scala index b19fd5a0d3a03..1b5c008bdd7c2 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterTest.scala @@ -20,12 +20,14 @@ package kafka.server import kafka.testkit.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, NewTopic} +import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.metadata.BrokerState import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.Assertions._ import java.util import java.util.Collections +import scala.jdk.CollectionConverters._ @Timeout(120000) class RaftClusterTest { @@ -34,7 +36,7 @@ class RaftClusterTest { def testCreateClusterAndClose(): Unit = { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setNumKip500BrokerNodes(1). + setNumBrokerNodes(1). setNumControllerNodes(1).build()).build() try { cluster.format() @@ -48,7 +50,7 @@ class RaftClusterTest { def testCreateClusterAndWaitForBrokerInRunningState(): Unit = { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setNumKip500BrokerNodes(3). + setNumBrokerNodes(3). setNumControllerNodes(3).build()).build() try { cluster.format() @@ -73,7 +75,7 @@ class RaftClusterTest { def testCreateClusterAndCreateListDeleteTopic(): Unit = { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setNumKip500BrokerNodes(3). + setNumBrokerNodes(3). setNumControllerNodes(3).build()).build() try { cluster.format() @@ -131,7 +133,7 @@ class RaftClusterTest { def testCreateClusterAndCreateAndManyTopics(): Unit = { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setNumKip500BrokerNodes(3). + setNumBrokerNodes(3). setNumControllerNodes(3).build()).build() try { cluster.format() @@ -174,7 +176,7 @@ class RaftClusterTest { def testCreateClusterAndCreateAndManyTopicsWithManyPartitions(): Unit = { val cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder(). - setNumKip500BrokerNodes(3). + setNumBrokerNodes(3). setNumControllerNodes(3).build()).build() try { cluster.format() @@ -212,4 +214,103 @@ class RaftClusterTest { cluster.close() } } + + @Test + def testClientQuotas(): Unit = { + val cluster = new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1).build()).build() + try { + cluster.format() + cluster.startup() + TestUtils.waitUntilTrue(() => cluster.brokers().get(0).currentState() == BrokerState.RUNNING, + "Broker never made it to RUNNING state.") + val admin = Admin.create(cluster.clientProperties()) + try { + val entity = new ClientQuotaEntity(Map("user" -> "testkit").asJava) + var filter = ClientQuotaFilter.containsOnly( + List(ClientQuotaFilterComponent.ofEntity("user", "testkit")).asJava) + + def alterThenDescribe(entity: ClientQuotaEntity, + quotas: Seq[ClientQuotaAlteration.Op], + filter: ClientQuotaFilter, + expectCount: Int): java.util.Map[ClientQuotaEntity, java.util.Map[String, java.lang.Double]] = { + val alterResult = admin.alterClientQuotas(Seq(new ClientQuotaAlteration(entity, quotas.asJava)).asJava) + try { + alterResult.all().get() + } catch { + case t: Throwable => fail("AlterClientQuotas request failed", t) + } + + def describeOrFail(filter: ClientQuotaFilter): java.util.Map[ClientQuotaEntity, java.util.Map[String, java.lang.Double]] = { + try { + admin.describeClientQuotas(filter).entities().get() + } catch { + case t: Throwable => fail("DescribeClientQuotas request failed", t) + } + } + + val (describeResult, ok) = TestUtils.computeUntilTrue(describeOrFail(filter)) { + results => results.getOrDefault(entity, java.util.Collections.emptyMap[String, java.lang.Double]()).size() == expectCount + } + assertTrue(ok, "Broker never saw new client quotas") + describeResult + } + + var describeResult = alterThenDescribe(entity, + Seq(new ClientQuotaAlteration.Op("request_percentage", 0.99)), filter, 1) + assertEquals(0.99, describeResult.get(entity).get("request_percentage"), 1e-6) + + describeResult = alterThenDescribe(entity, Seq( + new ClientQuotaAlteration.Op("request_percentage", 0.97), + new ClientQuotaAlteration.Op("producer_byte_rate", 10000), + new ClientQuotaAlteration.Op("consumer_byte_rate", 10001) + ), filter, 3) + assertEquals(0.97, describeResult.get(entity).get("request_percentage"), 1e-6) + assertEquals(10000.0, describeResult.get(entity).get("producer_byte_rate"), 1e-6) + assertEquals(10001.0, describeResult.get(entity).get("consumer_byte_rate"), 1e-6) + + describeResult = alterThenDescribe(entity, Seq( + new ClientQuotaAlteration.Op("request_percentage", 0.95), + new ClientQuotaAlteration.Op("producer_byte_rate", null), + new ClientQuotaAlteration.Op("consumer_byte_rate", null) + ), filter, 1) + assertEquals(0.95, describeResult.get(entity).get("request_percentage"), 1e-6) + + describeResult = alterThenDescribe(entity, Seq( + new ClientQuotaAlteration.Op("request_percentage", null)), filter, 0) + + describeResult = alterThenDescribe(entity, + Seq(new ClientQuotaAlteration.Op("producer_byte_rate", 9999)), filter, 1) + assertEquals(9999.0, describeResult.get(entity).get("producer_byte_rate"), 1e-6) + + // Add another quota for a different entity with same user part + val entity2 = new ClientQuotaEntity(Map("user" -> "testkit", "client-id" -> "some-client").asJava) + filter = ClientQuotaFilter.containsOnly( + List( + ClientQuotaFilterComponent.ofEntity("user", "testkit"), + ClientQuotaFilterComponent.ofEntity("client-id", "some-client"), + ).asJava) + describeResult = alterThenDescribe(entity2, + Seq(new ClientQuotaAlteration.Op("producer_byte_rate", 9998)), filter, 1) + assertEquals(9998.0, describeResult.get(entity2).get("producer_byte_rate"), 1e-6) + + // non-strict match + filter = ClientQuotaFilter.contains( + List(ClientQuotaFilterComponent.ofEntity("user", "testkit")).asJava) + + TestUtils.tryUntilNoAssertionError(){ + val results = admin.describeClientQuotas(filter).entities().get() + assertEquals(2, results.size(), "Broker did not see two client quotas") + assertEquals(9999.0, results.get(entity).get("producer_byte_rate"), 1e-6) + assertEquals(9998.0, results.get(entity2).get("producer_byte_rate"), 1e-6) + } + } finally { + admin.close() + } + } finally { + cluster.close() + } + } } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index 61bd02a0634fe..e495f3e865d76 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -38,7 +38,7 @@ import kafka.test.junit.ClusterTestExtensions import scala.jdk.CollectionConverters._ -@ClusterTestDefaults(clusterType = Type.ZK) +@ClusterTestDefaults(clusterType = Type.BOTH) @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ClientQuotasRequestTest(cluster: ClusterInstance) { private val ConsumerByteRateProp = QuotaConfigs.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG @@ -174,7 +174,7 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { )) } - @ClusterTest + @ClusterTest(clusterType = Type.ZK) // No SCRAM for Raft yet def testClientQuotasForScramUsers(): Unit = { val userName = "user" @@ -208,23 +208,23 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { val allIpEntityFilter = ClientQuotaFilterComponent.ofEntityType(ClientQuotaEntity.IP) def verifyIpQuotas(entityFilter: ClientQuotaFilterComponent, expectedMatches: Map[ClientQuotaEntity, Double]): Unit = { - val result = describeClientQuotas(ClientQuotaFilter.containsOnly(List(entityFilter).asJava)) - assertEquals(expectedMatches.keySet, result.asScala.keySet) - result.asScala.foreach { case (entity, props) => - assertEquals(Set(IpConnectionRateProp), props.asScala.keySet) - assertEquals(expectedMatches(entity), props.get(IpConnectionRateProp)) - val entityName = entity.entries.get(ClientQuotaEntity.IP) - // ClientQuotaEntity with null name maps to default entity - val entityIp = if (entityName == null) - InetAddress.getByName(unknownHost) - else - InetAddress.getByName(entityName) - var currentServerQuota = 0 - TestUtils.waitUntilTrue( - () => { - currentServerQuota = cluster.brokerSocketServers().asScala.head.connectionQuotas.connectionRateForIp(entityIp) - Math.abs(expectedMatches(entity) - currentServerQuota) < 0.01 - }, s"Connection quota of $entity is not ${expectedMatches(entity)} but $currentServerQuota") + TestUtils.tryUntilNoAssertionError() { + val result = describeClientQuotas(ClientQuotaFilter.containsOnly(List(entityFilter).asJava)) + assertEquals(expectedMatches.keySet, result.asScala.keySet) + result.asScala.foreach { case (entity, props) => + assertEquals(Set(IpConnectionRateProp), props.asScala.keySet) + assertEquals(expectedMatches(entity), props.get(IpConnectionRateProp)) + val entityName = entity.entries.get(ClientQuotaEntity.IP) + // ClientQuotaEntity with null name maps to default entity + val entityIp = if (entityName == null) + InetAddress.getByName(unknownHost) + else + InetAddress.getByName(entityName) + var currentServerQuota = 0 + currentServerQuota = cluster.brokerSocketServers().asScala.head.connectionQuotas.connectionRateForIp(entityIp) + assertTrue(Math.abs(expectedMatches(entity) - currentServerQuota) < 0.01, + s"Connection quota of $entity is not ${expectedMatches(entity)} but $currentServerQuota") + } } } @@ -362,12 +362,14 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { // Test exact matches. matchUserClientEntities.foreach { case (e, v) => - val result = matchEntity(e) - assertEquals(1, result.size) - assertTrue(result.get(e) != null) - val value = result.get(e).get(RequestPercentageProp) - assertNotNull(value) - assertEquals(value, v, 1e-6) + TestUtils.tryUntilNoAssertionError() { + val result = matchEntity(e) + assertEquals(1, result.size) + assertTrue(result.get(e) != null) + val value = result.get(e).get(RequestPercentageProp) + assertNotNull(value) + assertEquals(value, v, 1e-6) + } } // Entities not contained in `matchEntityList`. @@ -396,31 +398,33 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { setupDescribeClientQuotasMatchTest() def testMatchEntities(filter: ClientQuotaFilter, expectedMatchSize: Int, partition: ClientQuotaEntity => Boolean): Unit = { - val result = describeClientQuotas(filter) - val (expectedMatches, _) = (matchUserClientEntities ++ matchIpEntities).partition(e => partition(e._1)) - assertEquals(expectedMatchSize, expectedMatches.size) // for test verification - assertEquals(expectedMatchSize, result.size) - val expectedMatchesMap = expectedMatches.toMap - matchUserClientEntities.foreach { case (entity, expectedValue) => - if (expectedMatchesMap.contains(entity)) { - val config = result.get(entity) - assertNotNull(config) - val value = config.get(RequestPercentageProp) - assertNotNull(value) - assertEquals(expectedValue, value, 1e-6) - } else { - assertNull(result.get(entity)) + TestUtils.tryUntilNoAssertionError() { + val result = describeClientQuotas(filter) + val (expectedMatches, _) = (matchUserClientEntities ++ matchIpEntities).partition(e => partition(e._1)) + assertEquals(expectedMatchSize, expectedMatches.size) // for test verification + assertEquals(expectedMatchSize, result.size, s"Failed to match $expectedMatchSize entities for $filter") + val expectedMatchesMap = expectedMatches.toMap + matchUserClientEntities.foreach { case (entity, expectedValue) => + if (expectedMatchesMap.contains(entity)) { + val config = result.get(entity) + assertNotNull(config) + val value = config.get(RequestPercentageProp) + assertNotNull(value) + assertEquals(expectedValue, value, 1e-6) + } else { + assertNull(result.get(entity)) + } } - } - matchIpEntities.foreach { case (entity, expectedValue) => - if (expectedMatchesMap.contains(entity)) { - val config = result.get(entity) - assertNotNull(config) - val value = config.get(IpConnectionRateProp) - assertNotNull(value) - assertEquals(expectedValue, value, 1e-6) - } else { - assertNull(result.get(entity)) + matchIpEntities.foreach { case (entity, expectedValue) => + if (expectedMatchesMap.contains(entity)) { + val config = result.get(entity) + assertNotNull(config) + val value = config.get(IpConnectionRateProp) + assertNotNull(value) + assertEquals(expectedValue, value, 1e-6) + } else { + assertNull(result.get(entity)) + } } } } @@ -533,23 +537,25 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { } private def verifyDescribeEntityQuotas(entity: ClientQuotaEntity, quotas: Map[String, Double]) = { - val components = entity.entries.asScala.map { case (entityType, entityName) => - Option(entityName).map{ name => ClientQuotaFilterComponent.ofEntity(entityType, name)} - .getOrElse(ClientQuotaFilterComponent.ofDefaultEntity(entityType) - ) - } - val describe = describeClientQuotas(ClientQuotaFilter.containsOnly(components.toList.asJava)) - if (quotas.isEmpty) { - assertEquals(0, describe.size) - } else { - assertEquals(1, describe.size) - val configs = describe.get(entity) - assertNotNull(configs) - assertEquals(quotas.size, configs.size) - quotas.foreach { case (k, v) => - val value = configs.get(k) - assertNotNull(value) - assertEquals(v, value, 1e-6) + TestUtils.tryUntilNoAssertionError(waitTime = 5000L) { + val components = entity.entries.asScala.map { case (entityType, entityName) => + Option(entityName).map{ name => ClientQuotaFilterComponent.ofEntity(entityType, name)} + .getOrElse(ClientQuotaFilterComponent.ofDefaultEntity(entityType) + ) + } + val describe = describeClientQuotas(ClientQuotaFilter.containsOnly(components.toList.asJava)) + if (quotas.isEmpty) { + assertEquals(0, describe.size) + } else { + assertEquals(1, describe.size) + val configs = describe.get(entity) + assertNotNull(configs) + assertEquals(quotas.size, configs.size) + quotas.foreach { case (k, v) => + val value = configs.get(k) + assertNotNull(value) + assertEquals(v, value, 1e-6) + } } } } @@ -570,7 +576,7 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { private def sendDescribeClientQuotasRequest(filter: ClientQuotaFilter): DescribeClientQuotasResponse = { val request = new DescribeClientQuotasRequest.Builder(filter).build() IntegrationTestUtils.connectAndReceive[DescribeClientQuotasResponse](request, - destination = cluster.anyControllerSocketServer(), + destination = cluster.anyBrokerSocketServer(), listenerName = cluster.clientListener()) } @@ -598,7 +604,7 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { private def sendAlterClientQuotasRequest(entries: Iterable[ClientQuotaAlteration], validateOnly: Boolean): AlterClientQuotasResponse = { val request = new AlterClientQuotasRequest.Builder(entries.asJavaCollection, validateOnly).build() IntegrationTestUtils.connectAndReceive[AlterClientQuotasResponse](request, - destination = cluster.anyControllerSocketServer(), + destination = cluster.anyBrokerSocketServer(), listenerName = cluster.clientListener()) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 8280380b712d0..7813c2dca1049 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -23,7 +23,6 @@ import java.util import java.util.Arrays.asList import java.util.concurrent.TimeUnit import java.util.{Collections, Optional, Properties, Random} - import kafka.api.{ApiVersion, KAFKA_0_10_2_IV0, KAFKA_2_2_IV1, LeaderAndIsr} import kafka.cluster.{Broker, Partition} import kafka.controller.{ControllerContext, KafkaController} @@ -33,7 +32,7 @@ import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinat import kafka.log.AppendOrigin import kafka.network.RequestChannel import kafka.server.QuotaFactory.QuotaManagers -import kafka.server.metadata.{CachedConfigRepository, ConfigRepository, RaftMetadataCache} +import kafka.server.metadata.{CachedConfigRepository, ClientQuotaCache, ConfigRepository, RaftMetadataCache} import kafka.utils.{MockTime, TestUtils} import kafka.zk.KafkaZkClient import org.apache.kafka.clients.admin.AlterConfigOp.OpType @@ -108,6 +107,7 @@ class KafkaApisTest { private val replicaQuotaManager: ReplicationQuotaManager = EasyMock.createNiceMock(classOf[ReplicationQuotaManager]) private val quotas = QuotaManagers(clientQuotaManager, clientQuotaManager, clientRequestQuotaManager, clientControllerQuotaManager, replicaQuotaManager, replicaQuotaManager, replicaQuotaManager, None) + private val quotaCache = new ClientQuotaCache() private val fetchManager: FetchManager = EasyMock.createNiceMock(classOf[FetchManager]) private val brokerTopicStats = new BrokerTopicStats private val clusterId = "clusterId" @@ -150,7 +150,7 @@ class KafkaApisTest { // with a RaftMetadataCache instance metadataCache match { case raftMetadataCache: RaftMetadataCache => - RaftSupport(forwardingManager, raftMetadataCache) + RaftSupport(forwardingManager, raftMetadataCache, quotaCache) case _ => throw new IllegalStateException("Test must set an instance of RaftMetadataCache") } } else { diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index b9e5667dc3b4d..7114d5dda99bd 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -878,6 +878,28 @@ object TestUtils extends Logging { throw new RuntimeException("unexpected error") } + /** + * Invoke `assertions` until no AssertionErrors are thrown or `waitTime` elapses. + * + * This method is useful in cases where there may be some expected delay in a particular test condition that is + * otherwise difficult to poll for. `computeUntilTrue` and `waitUntilTrue` should be preferred in cases where we can + * easily wait on a condition before evaluating the assertions. + */ + def tryUntilNoAssertionError(waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS, pause: Long = 100L)(assertions: => Unit) = { + val (error, success) = TestUtils.computeUntilTrue({ + try { + assertions + None + } catch { + case ae: AssertionError => Some(ae) + } + }, waitTime = waitTime, pause = pause)(_.isEmpty) + + if (!success) { + throw error.get + } + } + def isLeaderLocalOnBroker(topic: String, partitionId: Int, server: KafkaServer): Boolean = { server.replicaManager.onlinePartition(new TopicPartition(topic, partitionId)).exists(_.leaderLogIfLocal.isDefined) } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java index f6a24973b58f2..c1a98016f4e46 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java @@ -147,8 +147,10 @@ private void alterClientQuotaEntity( List newRecords = new ArrayList<>(newQuotaConfigs.size()); Map currentQuotas = clientQuotaData.containsKey(entity) ? - clientQuotaData.get(entity) : Collections.emptyMap(); - newQuotaConfigs.forEach((key, newValue) -> { + clientQuotaData.get(entity) : Collections.emptyMap(); + for (Map.Entry entry : newQuotaConfigs.entrySet()) { + String key = entry.getKey(); + Double newValue = entry.getValue(); if (newValue == null) { if (currentQuotas.containsKey(key)) { // Null value indicates removal @@ -161,6 +163,7 @@ private void alterClientQuotaEntity( ApiError validationError = validateQuotaKeyValue(configKeys, key, newValue); if (validationError.isFailure()) { outputResults.put(entity, validationError); + return; } else { final Double currentValue = currentQuotas.get(key); if (!Objects.equals(currentValue, newValue)) { @@ -172,7 +175,7 @@ private void alterClientQuotaEntity( } } } - }); + } outputRecords.addAll(newRecords); outputResults.put(entity, ApiError.NONE); @@ -186,18 +189,23 @@ private ApiError configKeysForEntityType(Map entity, Map configKeys; - if (hasUser && hasClientId && !hasIp) { + if (hasIp) { + if (hasUser || hasClientId) { + return new ApiError(Errors.INVALID_REQUEST, "Invalid quota entity combination, IP entity should" + + "not be combined with User or ClientId"); + } else { + if (isValidIpEntity(entity.get(ClientQuotaEntity.IP))) { + configKeys = QuotaConfigs.ipConfigs().configKeys(); + } else { + return new ApiError(Errors.INVALID_REQUEST, entity.get(ClientQuotaEntity.IP) + " is not a valid IP or resolvable host."); + } + } + } else if (hasUser && hasClientId) { configKeys = QuotaConfigs.userConfigs().configKeys(); - } else if (hasUser && !hasClientId && !hasIp) { + } else if (hasUser) { configKeys = QuotaConfigs.userConfigs().configKeys(); - } else if (!hasUser && hasClientId && !hasIp) { + } else if (hasClientId) { configKeys = QuotaConfigs.clientConfigs().configKeys(); - } else if (!hasUser && !hasClientId && hasIp) { - if (isValidIpEntity(entity.get(ClientQuotaEntity.IP))) { - configKeys = QuotaConfigs.ipConfigs().configKeys(); - } else { - return new ApiError(Errors.INVALID_REQUEST, entity.get(ClientQuotaEntity.IP) + " is not a valid IP or resolvable host."); - } } else { return new ApiError(Errors.INVALID_REQUEST, "Invalid empty client quota entity"); } @@ -218,6 +226,8 @@ private ApiError validateQuotaKeyValue(Map validKey switch (configKey.type()) { case DOUBLE: break; + case SHORT: + case INT: case LONG: Double epsilon = 1e-6; Long longValue = Double.valueOf(value + epsilon).longValue(); @@ -257,7 +267,7 @@ private ApiError validateEntity(ClientQuotaEntity entity, Map va String entityType = entityEntry.getKey(); String entityName = entityEntry.getValue(); if (validatedEntityMap.containsKey(entityType)) { - return new ApiError(Errors.INVALID_REQUEST, "Invalid empty client quota entity, duplicate entity entry " + entityType); + return new ApiError(Errors.INVALID_REQUEST, "Invalid client quota entity, duplicate entity entry " + entityType); } if (Objects.equals(entityType, ClientQuotaEntity.USER)) { validatedEntityMap.put(ClientQuotaEntity.USER, entityName); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java index f24853a16a7b3..47726964c2e1a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java @@ -71,8 +71,28 @@ public void testInvalidEntityTypes() { } private void assertInvalidEntity(ClientQuotaControlManager manager, ClientQuotaEntity entity) { + assertInvalidQuota(manager, entity, quotas(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10000.0)); + } + + @Test + public void testInvalidQuotaKeys() { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + ClientQuotaControlManager manager = new ClientQuotaControlManager(snapshotRegistry); + ClientQuotaEntity entity = entity(ClientQuotaEntity.USER, "user-1"); + + // Invalid + valid keys + assertInvalidQuota(manager, entity, quotas("not.a.quota.key", 0.0, QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 99.9)); + + // Valid + invalid keys + assertInvalidQuota(manager, entity, quotas(QuotaConfigs.REQUEST_PERCENTAGE_OVERRIDE_CONFIG, 99.9, "not.a.quota.key", 0.0)); + + // Null key + assertInvalidQuota(manager, entity, quotas(null, 99.9)); + } + + private void assertInvalidQuota(ClientQuotaControlManager manager, ClientQuotaEntity entity, Map quota) { List alters = new ArrayList<>(); - entityQuotaToAlterations(entity, quotas(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 10000.0), alters::add); + entityQuotaToAlterations(entity, quota, alters::add); ControllerResult> result = manager.alterClientQuotas(alters); assertEquals(Errors.INVALID_REQUEST, result.response().get(entity).error()); assertEquals(0, result.records().size()); From 8d356b034b5759e7baa740921a4804dbba678f49 Mon Sep 17 00:00:00 2001 From: cyc Date: Thu, 8 Apr 2021 23:55:52 -0400 Subject: [PATCH 031/155] KAFKA-12492: Fix the formatting of example RocksDBConfigSetter (#10486) Fix the formatting of example RocksDBConfigSetter due to the un-arranged spaces within
     tag.
    
    Reviewers: Anna Sophie Blee-Goldman 
    ---
     .../developer-guide/config-streams.html       | 56 ++++++++++---------
     1 file changed, 30 insertions(+), 26 deletions(-)
    
    diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html
    index a1ddd128379de..8ce2d0f87dee0 100644
    --- a/docs/streams/developer-guide/config-streams.html
    +++ b/docs/streams/developer-guide/config-streams.html
    @@ -704,34 +704,37 @@ 

    probing.rebalance.interval.ms

    The RocksDB configuration. Kafka Streams uses RocksDB as the default storage engine for persistent stores. To change the default configuration for RocksDB, you can implement RocksDBConfigSetter and provide your custom class via rocksdb.config.setter.

    Here is an example that adjusts the memory size consumed by RocksDB.

    -
        public static class CustomRocksDBConfig implements RocksDBConfigSetter {
    -                    // This object should be a member variable so it can be closed in RocksDBConfigSetter#close.
    -                    private org.rocksdb.Cache cache = new org.rocksdb.LRUCache(16 * 1024L * 1024L);
    +              
    +
    +public static class CustomRocksDBConfig implements RocksDBConfigSetter {
    +    // This object should be a member variable so it can be closed in RocksDBConfigSetter#close.
    +    private org.rocksdb.Cache cache = new org.rocksdb.LRUCache(16 * 1024L * 1024L);
     
    -                    @Override
    -                    public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
    -                      // See #1 below.
    -                      BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
    -                      tableConfig.setBlockCache(cache);
    -                      // See #2 below.
    -                      tableConfig.setBlockSize(16 * 1024L);
    -                      // See #3 below.
    -                      tableConfig.setCacheIndexAndFilterBlocks(true);
    -                      options.setTableFormatConfig(tableConfig);
    -                      // See #4 below.
    -                      options.setMaxWriteBufferNumber(2);
    -                    }
    +    @Override
    +    public void setConfig(final String storeName, final Options options, final Map<String, Object> configs) {
    +        // See #1 below.
    +        BlockBasedTableConfig tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();
    +        tableConfig.setBlockCache(cache);
    +        // See #2 below.
    +        tableConfig.setBlockSize(16 * 1024L);
    +        // See #3 below.
    +        tableConfig.setCacheIndexAndFilterBlocks(true);
    +        options.setTableFormatConfig(tableConfig);
    +        // See #4 below.
    +        options.setMaxWriteBufferNumber(2);
    +    }
     
    -                    @Override
    -                    public void close(final String storeName, final Options options) {
    -                      // See #5 below.
    -                      cache.close();
    -                    }
    -                    }
    +    @Override
    +    public void close(final String storeName, final Options options) {
    +        // See #5 below.
    +        cache.close();
    +    }
    +}
     
    -                    Properties streamsSettings = new Properties();
    -                    streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksDBConfig.class);
    -                    
    +Properties streamsSettings = new Properties(); +streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, CustomRocksDBConfig.class); +
    +
    Notes for example:
    @@ -744,7 +747,8 @@

    probing.rebalance.interval.ms

    - + + From ff3b2d67a1049c75b300547ca6ecf074b6a3ab28 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 9 Apr 2021 18:11:34 +0200 Subject: [PATCH 032/155] KAFKA-12591; Remove deprecated `quota.producer.default` and `quota.consumer.default` configurations (#10427) `quota.producer.default` and `quota.consumer.default` were deprecated in AK 0.11.0.0. Dynamic default quotas must be used instead. This patch removes them for AK 3.0. Reviewers: Rajini Sivaram , Ismael Juma --- .../kafka/server/ClientQuotaManager.scala | 14 +------ .../main/scala/kafka/server/KafkaConfig.scala | 12 ------ .../scala/kafka/server/QuotaFactory.scala | 28 ++------------ .../kafka/api/ClientIdQuotaTest.scala | 11 ++++-- .../kafka/api/CustomQuotaCallbackTest.scala | 2 - .../kafka/api/UserClientIdQuotaTest.scala | 2 - .../integration/kafka/api/UserQuotaTest.scala | 4 +- .../DynamicBrokerReconfigurationTest.scala | 8 +++- .../kafka/server/ClientQuotaManagerTest.scala | 38 +++++++++++++------ .../unit/kafka/server/KafkaConfigTest.scala | 2 - docs/upgrade.html | 2 + 11 files changed, 49 insertions(+), 74 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 1f5b752d614bc..ac67ca9360d29 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -44,21 +44,16 @@ case class ClientSensors(metricTags: Map[String, String], quotaSensor: Sensor, t /** * Configuration settings for quota management - * @param quotaDefault The default allocated to any client-id if - * dynamic defaults or user quotas are not set * @param numQuotaSamples The number of samples to retain in memory * @param quotaWindowSizeSeconds The time span of each sample * */ -case class ClientQuotaManagerConfig(quotaDefault: Long = - ClientQuotaManagerConfig.QuotaDefault, - numQuotaSamples: Int = +case class ClientQuotaManagerConfig(numQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples, quotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds) object ClientQuotaManagerConfig { - val QuotaDefault = Long.MaxValue // Always have 10 whole windows + 1 current window val DefaultNumQuotaSamples = 11 val DefaultQuotaWindowSizeSeconds = 1 @@ -195,15 +190,12 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, private val lock = new ReentrantReadWriteLock() private val sensorAccessor = new SensorAccess(lock, metrics) private val quotaCallback = clientQuotaCallback.getOrElse(new DefaultQuotaCallback) - private val staticConfigClientIdQuota = Quota.upperBound(config.quotaDefault.toDouble) private val clientQuotaType = QuotaType.toClientQuotaType(quotaType) @volatile private var quotaTypesEnabled = clientQuotaCallback match { case Some(_) => QuotaTypes.CustomQuotas - case None => - if (config.quotaDefault == Long.MaxValue) QuotaTypes.NoQuotas - else QuotaTypes.ClientIdQuotaEnabled + case None => QuotaTypes.NoQuotas } private val delayQueueSensor = metrics.sensor(quotaType.toString + "-delayQueue") @@ -620,8 +612,6 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, // /config/clients/ quota = overriddenQuotas.get(DefaultClientIdQuotaEntity) } - if (quota == null) - quota = staticConfigClientIdQuota } } if (quota == null) null else quota.bound diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index e8a3e6cf4e498..e25cce8ae1e90 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -205,8 +205,6 @@ object Defaults { val FetchMaxBytes = 55 * 1024 * 1024 /** ********* Quota Configuration ***********/ - val ProducerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaDefault - val ConsumerQuotaBytesPerSecondDefault = ClientQuotaManagerConfig.QuotaDefault val NumQuotaSamples: Int = ClientQuotaManagerConfig.DefaultNumQuotaSamples val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DefaultQuotaWindowSizeSeconds val NumReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DefaultNumQuotaSamples @@ -511,8 +509,6 @@ object KafkaConfig { val FetchMaxBytes = "fetch.max.bytes" /** ********* Quota Configuration ***********/ - val ProducerQuotaBytesPerSecondDefaultProp = "quota.producer.default" - val ConsumerQuotaBytesPerSecondDefaultProp = "quota.consumer.default" val NumQuotaSamplesProp = "quota.window.num" val NumReplicationQuotaSamplesProp = "replication.quota.window.num" val NumAlterLogDirsReplicationQuotaSamplesProp = "alter.log.dirs.replication.quota.window.num" @@ -919,10 +915,6 @@ object KafkaConfig { val FetchMaxBytesDoc = "The maximum number of bytes we will return for a fetch request. Must be at least 1024." /** ********* Quota Configuration ***********/ - val ProducerQuotaBytesPerSecondDefaultDoc = "DEPRECATED: Used only when dynamic default quotas are not configured for , or in Zookeeper. " + - "Any producer distinguished by clientId will get throttled if it produces more bytes than this value per-second" - val ConsumerQuotaBytesPerSecondDefaultDoc = "DEPRECATED: Used only when dynamic default quotas are not configured for or in Zookeeper. " + - "Any consumer distinguished by clientId/consumer group will get throttled if it fetches more bytes than this value per-second" val NumQuotaSamplesDoc = "The number of samples to retain in memory for client quotas" val NumReplicationQuotaSamplesDoc = "The number of samples to retain in memory for replication quotas" val NumAlterLogDirsReplicationQuotaSamplesDoc = "The number of samples to retain in memory for alter log dirs replication quotas" @@ -1232,8 +1224,6 @@ object KafkaConfig { .define(KafkaMetricsPollingIntervalSecondsProp, INT, Defaults.KafkaMetricsPollingIntervalSeconds, atLeast(1), LOW, KafkaMetricsPollingIntervalSecondsDoc) /** ********* Quota configuration ***********/ - .define(ProducerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ProducerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ProducerQuotaBytesPerSecondDefaultDoc) - .define(ConsumerQuotaBytesPerSecondDefaultProp, LONG, Defaults.ConsumerQuotaBytesPerSecondDefault, atLeast(1), HIGH, ConsumerQuotaBytesPerSecondDefaultDoc) .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) .define(NumReplicationQuotaSamplesProp, INT, Defaults.NumReplicationQuotaSamples, atLeast(1), LOW, NumReplicationQuotaSamplesDoc) .define(NumAlterLogDirsReplicationQuotaSamplesProp, INT, Defaults.NumAlterLogDirsReplicationQuotaSamples, atLeast(1), LOW, NumAlterLogDirsReplicationQuotaSamplesDoc) @@ -1742,8 +1732,6 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean, dynamicConfigO def passwordEncoderIterations = getInt(KafkaConfig.PasswordEncoderIterationsProp) /** ********* Quota Configuration **************/ - val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) - val consumerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) val numQuotaSamples = getInt(KafkaConfig.NumQuotaSamplesProp) val quotaWindowSizeSeconds = getInt(KafkaConfig.QuotaWindowSizeSecondsProp) val numReplicationQuotaSamples = getInt(KafkaConfig.NumReplicationQuotaSamplesProp) diff --git a/core/src/main/scala/kafka/server/QuotaFactory.scala b/core/src/main/scala/kafka/server/QuotaFactory.scala index 9d4443b904ea8..f3901f6b29b11 100644 --- a/core/src/main/scala/kafka/server/QuotaFactory.scala +++ b/core/src/main/scala/kafka/server/QuotaFactory.scala @@ -76,9 +76,9 @@ object QuotaFactory extends Logging { val clientQuotaCallback = Option(cfg.getConfiguredInstance(KafkaConfig.ClientQuotaCallbackClassProp, classOf[ClientQuotaCallback])) QuotaManagers( - new ClientQuotaManager(clientFetchConfig(cfg), metrics, Fetch, time, threadNamePrefix, clientQuotaCallback), - new ClientQuotaManager(clientProduceConfig(cfg), metrics, Produce, time, threadNamePrefix, clientQuotaCallback), - new ClientRequestQuotaManager(clientRequestConfig(cfg), metrics, time, threadNamePrefix, clientQuotaCallback), + new ClientQuotaManager(clientConfig(cfg), metrics, Fetch, time, threadNamePrefix, clientQuotaCallback), + new ClientQuotaManager(clientConfig(cfg), metrics, Produce, time, threadNamePrefix, clientQuotaCallback), + new ClientRequestQuotaManager(clientConfig(cfg), metrics, time, threadNamePrefix, clientQuotaCallback), new ControllerMutationQuotaManager(clientControllerMutationConfig(cfg), metrics, time, threadNamePrefix, clientQuotaCallback), new ReplicationQuotaManager(replicationConfig(cfg), metrics, LeaderReplication, time), @@ -88,27 +88,7 @@ object QuotaFactory extends Logging { ) } - def clientProduceConfig(cfg: KafkaConfig): ClientQuotaManagerConfig = { - if (cfg.producerQuotaBytesPerSecondDefault != Long.MaxValue) - warn(s"${KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp} has been deprecated in 0.11.0.0 and will be removed in a future release. Use dynamic quota defaults instead.") - ClientQuotaManagerConfig( - quotaDefault = cfg.producerQuotaBytesPerSecondDefault, - numQuotaSamples = cfg.numQuotaSamples, - quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds - ) - } - - def clientFetchConfig(cfg: KafkaConfig): ClientQuotaManagerConfig = { - if (cfg.consumerQuotaBytesPerSecondDefault != Long.MaxValue) - warn(s"${KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp} has been deprecated in 0.11.0.0 and will be removed in a future release. Use dynamic quota defaults instead.") - ClientQuotaManagerConfig( - quotaDefault = cfg.consumerQuotaBytesPerSecondDefault, - numQuotaSamples = cfg.numQuotaSamples, - quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds - ) - } - - def clientRequestConfig(cfg: KafkaConfig): ClientQuotaManagerConfig = { + def clientConfig(cfg: KafkaConfig): ClientQuotaManagerConfig = { ClientQuotaManagerConfig( numQuotaSamples = cfg.numQuotaSamples, quotaWindowSizeSeconds = cfg.quotaWindowSizeSeconds diff --git a/core/src/test/scala/integration/kafka/api/ClientIdQuotaTest.scala b/core/src/test/scala/integration/kafka/api/ClientIdQuotaTest.scala index 0d1ff497fdbaa..28b0b15ec9ae8 100644 --- a/core/src/test/scala/integration/kafka/api/ClientIdQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/ClientIdQuotaTest.scala @@ -14,7 +14,7 @@ package kafka.api -import kafka.server.{KafkaConfig, KafkaServer} +import kafka.server.KafkaServer import org.apache.kafka.common.security.auth.KafkaPrincipal import org.junit.jupiter.api.BeforeEach @@ -25,9 +25,14 @@ class ClientIdQuotaTest extends BaseQuotaTest { @BeforeEach override def setUp(): Unit = { - this.serverConfig.setProperty(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, defaultProducerQuota.toString) - this.serverConfig.setProperty(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp, defaultConsumerQuota.toString) super.setUp() + quotaTestClients.alterClientQuotas( + quotaTestClients.clientQuotaAlteration( + quotaTestClients.clientQuotaEntity(None, Some(QuotaTestClients.DefaultEntity)), + Some(defaultProducerQuota), Some(defaultConsumerQuota), Some(defaultRequestQuota) + ) + ) + quotaTestClients.waitForQuotaUpdate(defaultProducerQuota, defaultConsumerQuota, defaultRequestQuota) } override def createQuotaTestClients(topic: String, leaderNode: KafkaServer): QuotaTestClients = { diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index 7db050b45db05..bfc967a67aea2 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -63,8 +63,6 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { @BeforeEach override def setUp(): Unit = { startSasl(jaasSections(kafkaServerSaslMechanisms, Some("SCRAM-SHA-256"), KafkaSasl, JaasTestUtils.KafkaServerContextName)) - this.serverConfig.setProperty(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, Long.MaxValue.toString) - this.serverConfig.setProperty(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp, Long.MaxValue.toString) this.serverConfig.setProperty(KafkaConfig.ClientQuotaCallbackClassProp, classOf[GroupedUserQuotaCallback].getName) this.serverConfig.setProperty(s"${listenerName.configPrefix}${KafkaConfig.PrincipalBuilderClassProp}", classOf[GroupedUserPrincipalBuilder].getName) diff --git a/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala b/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala index 56b2c02bacc44..e442ddab1d72a 100644 --- a/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/UserClientIdQuotaTest.scala @@ -32,8 +32,6 @@ class UserClientIdQuotaTest extends BaseQuotaTest { @BeforeEach override def setUp(): Unit = { this.serverConfig.setProperty(KafkaConfig.SslClientAuthProp, "required") - this.serverConfig.setProperty(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, Long.MaxValue.toString) - this.serverConfig.setProperty(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp, Long.MaxValue.toString) super.setUp() quotaTestClients.alterClientQuotas( quotaTestClients.clientQuotaAlteration( diff --git a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala index 4917b4e2108bf..7a00f4067ac5b 100644 --- a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala @@ -16,7 +16,7 @@ package kafka.api import java.io.File -import kafka.server.{KafkaConfig, KafkaServer} +import kafka.server.KafkaServer import kafka.utils.JaasTestUtils import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.junit.jupiter.api.{AfterEach, BeforeEach} @@ -33,8 +33,6 @@ class UserQuotaTest extends BaseQuotaTest with SaslSetup { @BeforeEach override def setUp(): Unit = { startSasl(jaasSections(kafkaServerSaslMechanisms, Some("GSSAPI"), KafkaSasl, JaasTestUtils.KafkaServerContextName)) - this.serverConfig.setProperty(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, Long.MaxValue.toString) - this.serverConfig.setProperty(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp, Long.MaxValue.toString) super.setUp() quotaTestClients.alterClientQuotas( quotaTestClients.clientQuotaAlteration( diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index c37d376208ebe..08a55aedef101 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -26,7 +26,6 @@ import java.time.Duration import java.util import java.util.{Collections, Properties} import java.util.concurrent._ - import javax.management.ObjectName import com.yammer.metrics.core.MetricName import kafka.admin.ConfigCommand @@ -52,6 +51,7 @@ import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.config.provider.FileConfigProvider import org.apache.kafka.common.errors.{AuthenticationException, InvalidRequestException} import org.apache.kafka.common.internals.Topic +import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.metrics.{KafkaMetric, MetricsReporter} import org.apache.kafka.common.network.{ListenerName, Mode} import org.apache.kafka.common.network.CertStores.{KEYSTORE_PROPS, TRUSTSTORE_PROPS} @@ -121,7 +121,6 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet props.put(KafkaConfig.SaslEnabledMechanismsProp, kafkaServerSaslMechanisms.mkString(",")) props.put(KafkaConfig.LogSegmentBytesProp, "2000") // low value to test log rolling on config update props.put(KafkaConfig.NumReplicaFetchersProp, "2") // greater than one to test reducing threads - props.put(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp, "10000000") // non-default value to trigger a new metric props.put(KafkaConfig.PasswordEncoderSecretProp, "dynamic-config-secret") props.put(KafkaConfig.LogRetentionTimeMillisProp, 1680000000.toString) props.put(KafkaConfig.LogRetentionTimeHoursProp, 168.toString) @@ -870,7 +869,12 @@ class DynamicBrokerReconfigurationTest extends ZooKeeperTestHarness with SaslSet } assertEquals(servers.map(_.config.brokerId).toSet, TestMetricsReporter.configuredBrokers.toSet) + // non-default value to trigger a new metric val clientId = "test-client-1" + servers.foreach { server => + server.quotaManagers.produce.updateQuota(None, Some(clientId), Some(clientId), + Some(Quota.upperBound(10000000))) + } val (producerThread, consumerThread) = startProduceConsume(retries = 0, clientId) TestUtils.waitUntilTrue(() => consumerThread.received >= 5, "Messages not sent") diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index c9991b451ede6..4159a1b3c041a 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -28,7 +28,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { - private val config = ClientQuotaManagerConfig(quotaDefault = 500) + private val config = ClientQuotaManagerConfig() private def testQuotaParsing(config: ClientQuotaManagerConfig, client1: UserClient, client2: UserClient, randomClient: UserClient, defaultConfigClient: UserClient): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "") @@ -38,10 +38,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(2000, true))) clientQuotaManager.updateQuota(client2.configUser, client2.configClientId, client2.sanitizedConfigClientId, Some(new Quota(4000, true))) - assertEquals(config.quotaDefault.toDouble, - clientQuotaManager.quota(randomClient.user, randomClient.clientId).bound, 0.0, "Default producer quota should be " + config.quotaDefault) - assertEquals(2000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the overridden value (2000)") - assertEquals(4000, clientQuotaManager.quota(client2.user, client2.clientId).bound, 0.0, "Should return the overridden value (4000)") + assertEquals(Long.MaxValue.toDouble, clientQuotaManager.quota(randomClient.user, randomClient.clientId).bound, 0.0, + "Default producer quota should be " + Long.MaxValue.toDouble) + assertEquals(2000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, + "Should return the overridden value (2000)") + assertEquals(4000, clientQuotaManager.quota(client2.user, client2.clientId).bound, 0.0, + "Should return the overridden value (4000)") // p1 should be throttled using the overridden quota var throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 2500 * config.numQuotaSamples) @@ -98,7 +100,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val client2 = UserClient("User2", "p2", Some("User2"), None) val randomClient = UserClient("RandomUser", "random-client-id", None, None) val defaultConfigClient = UserClient("", "", Some(ConfigEntityName.Default), None) - val config = ClientQuotaManagerConfig(quotaDefault = Long.MaxValue) + val config = ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -112,7 +114,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val client2 = UserClient("User2", "p2", Some("User2"), Some("p2")) val randomClient = UserClient("RandomUser", "random-client-id", None, None) val defaultConfigClient = UserClient("", "", Some(ConfigEntityName.Default), Some(ConfigEntityName.Default)) - val config = ClientQuotaManagerConfig(quotaDefault = Long.MaxValue) + val config = ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -158,7 +160,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testGetMaxValueInQuotaWindowWithNonDefaultQuotaWindow(): Unit = { val numFullQuotaWindows = 3 // 3 seconds window (vs. 10 seconds default) - val nonDefaultConfig = ClientQuotaManagerConfig(quotaDefault = Long.MaxValue, numQuotaSamples = numFullQuotaWindows + 1) + val nonDefaultConfig = ClientQuotaManagerConfig(numQuotaSamples = numFullQuotaWindows + 1) val clientQuotaManager = new ClientQuotaManager(nonDefaultConfig, metrics, Fetch, time, "") val userSession = Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "userA"), InetAddress.getLocalHost) @@ -177,7 +179,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testSetAndRemoveDefaultUserQuota(): Unit = { // quotaTypesEnabled will be QuotaTypes.NoQuotas initially - val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault = Long.MaxValue), + val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(), metrics, Produce, time, "") try { @@ -199,7 +201,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testSetAndRemoveUserQuota(): Unit = { // quotaTypesEnabled will be QuotaTypes.NoQuotas initially - val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault = Long.MaxValue), + val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(), metrics, Produce, time, "") try { @@ -218,7 +220,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testSetAndRemoveUserClientQuota(): Unit = { // quotaTypesEnabled will be QuotaTypes.NoQuotas initially - val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault = Long.MaxValue), + val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(), metrics, Produce, time, "") try { @@ -236,7 +238,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testQuotaConfigPrecedence(): Unit = { - val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(quotaDefault=Long.MaxValue), + val clientQuotaManager = new ClientQuotaManager(ClientQuotaManagerConfig(), metrics, Produce, time, "") try { @@ -303,6 +305,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "") val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", "Produce", "")) try { + clientQuotaManager.updateQuota(None, Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), + Some(new Quota(500, true))) + // We have 10 second windows. Make sure that there is no quota violation // if we produce under the quota for (_ <- 0 until 10) { @@ -348,6 +353,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireThrottleTimeSensor(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "") try { + clientQuotaManager.updateQuota(None, Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), + Some(new Quota(500, true))) + maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) // remove the throttle time sensor metrics.removeSensor("ProduceThrottleTime-:client1") @@ -367,6 +375,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireQuotaSensors(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "") try { + clientQuotaManager.updateQuota(None, Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), + Some(new Quota(500, true))) + maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) // remove all the sensors metrics.removeSensor("ProduceThrottleTime-:client1") @@ -391,6 +402,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val clientQuotaManager = new ClientQuotaManager(config, metrics, Produce, time, "") val clientId = "client@#$%" try { + clientQuotaManager.updateQuota(None, Some(ConfigEntityName.Default), Some(ConfigEntityName.Default), + Some(new Quota(500, true))) + maybeRecord(clientQuotaManager, "ANONYMOUS", clientId, 100) // The metrics should use the raw client ID, even if the reporters internally sanitize them diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 1f803b64762fd..36613db0d97c2 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -743,8 +743,6 @@ class KafkaConfigTest { case KafkaConfig.TransactionsTopicPartitionsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case KafkaConfig.TransactionsTopicSegmentBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case KafkaConfig.TransactionsTopicReplicationFactorProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") - case KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case KafkaConfig.NumQuotaSamplesProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case KafkaConfig.QuotaWindowSizeSecondsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") diff --git a/docs/upgrade.html b/docs/upgrade.html index a0bc5af2c1ae6..04505b4bed38e 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -53,6 +53,8 @@
    Notable changes in 3
  • The MessageFormatter.init(Properties) method was removed. Please use configure(Map) instead.
  • Kafka Streams no longer has a compile time dependency on "connect:json" module (KAFKA-5146). Projects that were relying on this transitive dependency will have to explicitly declare it.
  • +
  • The deprecated quota.producer.default and quota.consumer.default configurations were removed (KAFKA-12591). + Dynamic quota defaults must be used instead.
  • From c7ab9b0ef8fe335c9376669d6ec9efd90291c955 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 9 Apr 2021 11:36:43 -0700 Subject: [PATCH 033/155] MINOR: add missing TimeoutException to Producer.send JavaDocs (#10495) Reviewers: Guozhang Wang , Boyang Chen , Chia-Ping Tsai --- .../java/org/apache/kafka/clients/producer/KafkaProducer.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 5a7b939d5cdda..0b01a866ecbfd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -874,6 +874,8 @@ public Future send(ProducerRecord record) { * when send is invoked after producer has been closed. * @throws InterruptException If the thread is interrupted while blocked * @throws SerializationException If the key or value are not valid objects given the configured serializers + * @throws TimeoutException If the record could not be appended to the send buffer due to memory unavailable + * or missing metadata within {@code max.block.ms}. * @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions. */ @Override From e4b6ffae5d932af6efb0764a12c7faa812758353 Mon Sep 17 00:00:00 2001 From: Jim Galasyn Date: Fri, 9 Apr 2021 11:41:52 -0700 Subject: [PATCH 034/155] MINOR: Fix left/outer join descriptions in Streams DSL reference topic (#10453) Reviewer: Matthias J. Sax --- docs/streams/developer-guide/dsl-api.html | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index 3c885d6579ca5..84fe4e7ffe7b6 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -2232,8 +2232,8 @@

  • Input records with a null key are ignored and do not trigger the join.
  • -
  • Input records with a null value are interpreted as tombstones for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not - trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding +
  • Input records with a null value are interpreted as tombstones for the corresponding key, which indicate the deletion of the key from the table. Right-tombstones trigger the join, + but left-tombstones don't: when an input tombstone is received, an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable).
  • @@ -2280,8 +2280,8 @@

  • Input records with a null key are ignored and do not trigger the join.
  • -
  • Input records with a null value are interpreted as tombstones for the corresponding key, which indicate the deletion of the key from the table. Tombstones do not - trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding +
  • Input records with a null value are interpreted as tombstones for the corresponding key, which indicate the deletion of the key from the table. Tombstones may trigger joins, + depending on the content in the left and right tables. When an input tombstone is received, an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in the join result KTable).
  • @@ -2515,7 +2515,8 @@
    KTable-KTable Foreign-Key literal">null value are interpreted as tombstones for the corresponding key, which indicate the - deletion of the key from the table. When an input + deletion of the key from the table. Tombstones do not + trigger the join. When an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the @@ -2574,8 +2575,9 @@
    KTable-KTable Foreign-Key literal">null value are interpreted as tombstones for the corresponding key, which indicate the - deletion of the key from the table. When an input - tombstone is received, then an output + deletion of the key from the table. Right-tombstones + trigger the join, but left-tombstones don't: + when an input tombstone is received, then an output tombstone is forwarded directly to the join result KTable if required (i.e. only if the corresponding key actually exists already in From db0323e9ba3f767614415d833a2081a33825a284 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 19:49:37 +0100 Subject: [PATCH 035/155] KAFKA-12449: Remove deprecated WindowStore#put (#10293) Removes `WindowStore#put(K,V)` that was deprecated via KIP-474. Reviewers: Matthias J. Sax --- docs/streams/upgrade-guide.html | 1 + .../internals/AbstractReadOnlyDecorator.java | 7 -- .../internals/AbstractReadWriteDecorator.java | 7 -- .../kafka/streams/state/WindowStore.java | 18 ---- .../state/internals/CachingWindowStore.java | 6 -- .../ChangeLoggingWindowBytesStore.java | 10 -- .../state/internals/InMemoryWindowStore.java | 6 -- .../state/internals/MeteredWindowStore.java | 7 -- .../state/internals/RocksDBWindowStore.java | 24 ----- .../TimestampedWindowStoreBuilder.java | 7 -- ...owToTimestampedWindowByteStoreAdapter.java | 7 -- .../internals/ProcessorContextImplTest.java | 10 +- .../AbstractWindowBytesStoreTest.java | 93 ++++++------------- .../CachingPersistentWindowStoreTest.java | 88 +++++++++--------- ...oggingTimestampedWindowBytesStoreTest.java | 6 +- .../ChangeLoggingWindowBytesStoreTest.java | 8 +- .../internals/InMemoryWindowStoreTest.java | 19 ++-- .../MeteredTimestampedWindowStoreTest.java | 6 +- .../internals/MeteredWindowStoreTest.java | 3 +- .../internals/RocksDBWindowStoreTest.java | 84 ++++++----------- .../kafka/streams/TopologyTestDriver.java | 7 -- .../kafka/streams/WindowStoreFacadeTest.java | 11 --- 22 files changed, 119 insertions(+), 316 deletions(-) diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 073badd95d35e..b0820357eb398 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -111,6 +111,7 @@

    Streams API
  • Overloaded KStream#join, leftJoin, outerJoin with KStream and Joined parameters: deprecated in Kafka 2.4.0 (KIP-479).
  • Overloaded KafkaStreams#metadataForKey: deprecated in Kafka 2.5.0 (KIP-535).
  • Overloaded KafkaStreams#store: deprecated in Kafka 2.5.0 (KIP-562).
  • +
  • WindowStore#put(K key, V value): deprecated in Kafka 2.4.0 (KIP-474).
  • The following dependencies were removed from Kafka Streams: diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index 4424cdbc2c763..6c0c97599df65 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -160,13 +160,6 @@ private WindowStoreReadOnlyDecorator(final WindowStore inner) { super(inner); } - @Deprecated - @Override - public void put(final K key, - final V value) { - throw new UnsupportedOperationException(ERROR_MESSAGE); - } - @Override public void put(final K key, final V value, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java index d07708919abd5..86dd33fb8566c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java @@ -154,13 +154,6 @@ static class WindowStoreReadWriteDecorator super(inner); } - @Deprecated - @Override - public void put(final K key, - final V value) { - wrapped().put(key, value); - } - @Override public void put(final K key, final V value, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index ef1f799b22317..3ac11b8c94651 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -35,24 +35,6 @@ */ public interface WindowStore extends StateStore, ReadOnlyWindowStore { - /** - * Use the current record timestamp as the {@code windowStartTimestamp} and - * delegate to {@link WindowStore#put(Object, Object, long)}. - *

    - * It's highly recommended to use {@link WindowStore#put(Object, Object, long)} instead, as the record timestamp - * is unlikely to be the correct windowStartTimestamp in general. - * - * @param key The key to associate the value to - * @param value The value to update, it can be null; - * if the serialized bytes are also null it is interpreted as delete - * @throws NullPointerException if the given key is {@code null} - * @deprecated as timestamp is not provided for the key-value pair, this causes inconsistency - * to identify the window frame to which the key belongs. - * Use {@link #put(Object, Object, long)} instead. - */ - @Deprecated - void put(K key, V value); - /** * Put a key-value pair into the window with given window start timestamp *

    diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index c750f3a6052c6..cc7e5c6828386 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -142,12 +142,6 @@ public boolean setFlushListener(final CacheFlushListener flushLi return true; } - @Deprecated - @Override - public synchronized void put(final Bytes key, - final byte[] value) { - put(key, value, context.timestamp()); - } @Override public synchronized void put(final Bytes key, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 6a367b1c3cfc1..70c89495af0fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -131,16 +131,6 @@ public KeyValueIterator, byte[]> backwardFetchAll(final long tim return wrapped().backwardFetchAll(timeFrom, timeTo); } - @Deprecated - @Override - public void put(final Bytes key, final byte[] value) { - // Note: It's incorrect to bypass the wrapped store here by delegating to another method, - // but we have no alternative. We must send a timestamped key to the changelog, which means - // we need to know what timestamp gets used for the record. Hopefully, we can deprecate this - // method in the future to resolve the situation. - put(key, value, context.timestamp()); - } - @Override public void put(final Bytes key, final byte[] value, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java index 0a1dc8632b53d..5dbfc0dfe02bd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryWindowStore.java @@ -108,12 +108,6 @@ public void init(final ProcessorContext context, final StateStore root) { open = true; } - @Deprecated - @Override - public void put(final Bytes key, final byte[] value) { - put(key, value, context.timestamp()); - } - @Override public void put(final Bytes key, final byte[] value, final long windowStartTimestamp) { removeExpiredSegments(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 1b419ec742633..8c77cf2b26c90 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -157,13 +157,6 @@ public boolean setFlushListener(final CacheFlushListener, V> listene return false; } - @Deprecated - @Override - public void put(final K key, - final V value) { - put(key, value, context != null ? context.timestamp() : 0L); - } - @Override public void put(final K key, final V value, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index b3ba65243fc84..394950525a322 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -18,10 +18,6 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; @@ -33,7 +29,6 @@ public class RocksDBWindowStore private final boolean retainDuplicates; private final long windowSize; - private InternalProcessorContext context; private int seqnum = 0; RocksDBWindowStore(final SegmentedBytesStore bytesStore, @@ -44,25 +39,6 @@ public class RocksDBWindowStore this.windowSize = windowSize; } - @Deprecated - @Override - public void init(final ProcessorContext context, final StateStore root) { - this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; - super.init(context, root); - } - - @Override - public void init(final StateStoreContext context, final StateStore root) { - this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; - super.init(context, root); - } - - @Deprecated - @Override - public void put(final Bytes key, final byte[] value) { - put(key, value, context != null ? context.timestamp() : 0L); - } - @Override public void put(final Bytes key, final byte[] value, final long windowStartTimestamp) { // Skip if value is null and duplicates are allowed since this delete is a no-op diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java index 075545f9ea79c..417b45b46cc7e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedWindowStoreBuilder.java @@ -122,13 +122,6 @@ public void init(final StateStoreContext context, final StateStore root) { wrapped.init(context, root); } - @Deprecated - @Override - public void put(final Bytes key, - final byte[] value) { - wrapped.put(key, value); - } - @Override public void put(final Bytes key, final byte[] value, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java index 586e87dbcaaa9..8d895fc7f88e7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowToTimestampedWindowByteStoreAdapter.java @@ -40,13 +40,6 @@ class WindowToTimestampedWindowByteStoreAdapter implements WindowStore store.put("1", 1L, 1L), "put()"); - checkThrowsUnsupportedOperation(() -> store.put("1", 1L), "put()"); assertEquals(iters.get(0), store.fetchAll(0L, 0L)); assertEquals(windowStoreIter, store.fetch(KEY, 0L, 1L)); @@ -246,7 +245,6 @@ public void globalTimestampedWindowStoreShouldBeReadOnly() { checkThrowsUnsupportedOperation(store::flush, "flush()"); checkThrowsUnsupportedOperation(() -> store.put("1", ValueAndTimestamp.make(1L, 1L), 1L), "put() [with timestamp]"); - checkThrowsUnsupportedOperation(() -> store.put("1", ValueAndTimestamp.make(1L, 1L)), "put() [no timestamp]"); assertEquals(timestampedIters.get(0), store.fetchAll(0L, 0L)); assertEquals(windowStoreIter, store.fetch(KEY, 0L, 1L)); @@ -335,7 +333,7 @@ public void localWindowStoreShouldNotAllowInitOrClose() { store.flush(); assertTrue(flushExecuted); - store.put("1", 1L); + store.put("1", 1L, 1L); assertTrue(putExecuted); assertEquals(iters.get(0), store.fetchAll(0L, 0L)); @@ -355,7 +353,7 @@ public void localTimestampedWindowStoreShouldNotAllowInitOrClose() { store.flush(); assertTrue(flushExecuted); - store.put("1", ValueAndTimestamp.make(1L, 1L)); + store.put("1", ValueAndTimestamp.make(1L, 1L), 1L); assertTrue(putExecuted); store.put("1", ValueAndTimestamp.make(1L, 1L), 1L); @@ -639,7 +637,7 @@ private WindowStore windowStoreMock() { expect(windowStore.fetch(anyString(), anyLong())).andReturn(VALUE); expect(windowStore.all()).andReturn(iters.get(2)); - windowStore.put(anyString(), anyLong()); + windowStore.put(anyString(), anyLong(), anyLong()); expectLastCall().andAnswer(() -> { putExecuted = true; return null; @@ -662,7 +660,7 @@ private TimestampedWindowStore timestampedWindowStoreMock() { expect(windowStore.fetch(anyString(), anyLong())).andReturn(VALUE_AND_TIMESTAMP); expect(windowStore.all()).andReturn(timestampedIters.get(2)); - windowStore.put(anyString(), anyObject(ValueAndTimestamp.class)); + windowStore.put(anyString(), anyObject(ValueAndTimestamp.class), anyLong()); expectLastCall().andAnswer(() -> { putExecuted = true; return null; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index aa4a412747245..8d73a2d24e36d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -782,24 +782,22 @@ public void testPutAndFetchAfter() { } @Test - @SuppressWarnings("deprecation") public void testPutSameKeyTimestamp() { windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); windowStore.init((StateStoreContext) context, windowStore); final long startTime = SEGMENT_INTERVAL - 4L; - setCurrentTime(startTime); - windowStore.put(0, "zero"); + windowStore.put(0, "zero", startTime); assertEquals( new HashSet<>(Collections.singletonList("zero")), valuesToSet(windowStore.fetch(0, ofEpochMilli(startTime - WINDOW_SIZE), ofEpochMilli(startTime + WINDOW_SIZE)))); - windowStore.put(0, "zero"); - windowStore.put(0, "zero+"); - windowStore.put(0, "zero++"); + windowStore.put(0, "zero", startTime); + windowStore.put(0, "zero+", startTime); + windowStore.put(0, "zero++", startTime); assertEquals( new HashSet<>(asList("zero", "zero", "zero+", "zero++")), @@ -847,7 +845,6 @@ public void testPutSameKeyTimestamp() { @Test public void shouldCloseOpenIteratorsWhenStoreIsClosedAndNotThrowInvalidStateStoreExceptionOnHasNext() { - setCurrentTime(0); windowStore.put(1, "one", 1L); windowStore.put(1, "two", 2L); windowStore.put(1, "three", 3L); @@ -905,18 +902,16 @@ public void shouldFetchAndIterateOverExactKeys() { } @Test - @SuppressWarnings("deprecation") public void testDeleteAndUpdate() { final long currentTime = 0; - setCurrentTime(currentTime); - windowStore.put(1, "one"); - windowStore.put(1, "one v2"); + windowStore.put(1, "one", currentTime); + windowStore.put(1, "one v2", currentTime); WindowStoreIterator iterator = windowStore.fetch(1, 0, currentTime); assertEquals(new KeyValue<>(currentTime, "one v2"), iterator.next()); - windowStore.put(1, null); + windowStore.put(1, null, currentTime); iterator = windowStore.fetch(1, 0, currentTime); assertFalse(iterator.hasNext()); } @@ -927,9 +922,8 @@ public void shouldReturnNullOnWindowNotFound() { } @Test - @SuppressWarnings("deprecation") public void shouldThrowNullPointerExceptionOnPutNullKey() { - assertThrows(NullPointerException.class, () -> windowStore.put(null, "anyValue")); + assertThrows(NullPointerException.class, () -> windowStore.put(null, "anyValue", 0L)); } @Test @@ -1120,11 +1114,9 @@ public void shouldNotThrowExceptionWhenFetchRangeIsExpired() { } @Test - @SuppressWarnings("deprecation") public void testWindowIteratorPeek() { final long currentTime = 0; - setCurrentTime(currentTime); - windowStore.put(1, "one"); + windowStore.put(1, "one", currentTime); final KeyValueIterator, String> iterator = windowStore.fetchAll(0L, currentTime); @@ -1151,25 +1143,20 @@ public void testValueIteratorPeek() { } @Test - @SuppressWarnings("deprecation") public void shouldNotThrowConcurrentModificationException() { long currentTime = 0; - setCurrentTime(currentTime); - windowStore.put(1, "one"); + windowStore.put(1, "one", currentTime); currentTime += WINDOW_SIZE * 10; - setCurrentTime(currentTime); - windowStore.put(1, "two"); + windowStore.put(1, "two", currentTime); final KeyValueIterator, String> iterator = windowStore.all(); currentTime += WINDOW_SIZE * 10; - setCurrentTime(currentTime); - windowStore.put(1, "three"); + windowStore.put(1, "three", currentTime); currentTime += WINDOW_SIZE * 10; - setCurrentTime(currentTime); - windowStore.put(2, "four"); + windowStore.put(2, "four", currentTime); // Iterator should return all records in store and not throw exception b/c some were added after fetch assertEquals(windowedPair(1, "one", 0), iterator.next()); @@ -1180,25 +1167,21 @@ public void shouldNotThrowConcurrentModificationException() { } @Test - @SuppressWarnings("deprecation") public void testFetchDuplicates() { windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); windowStore.init((StateStoreContext) context, windowStore); long currentTime = 0; - setCurrentTime(currentTime); - windowStore.put(1, "one"); - windowStore.put(1, "one-2"); + windowStore.put(1, "one", currentTime); + windowStore.put(1, "one-2", currentTime); currentTime += WINDOW_SIZE * 10; - setCurrentTime(currentTime); - windowStore.put(1, "two"); - windowStore.put(1, "two-2"); + windowStore.put(1, "two", currentTime); + windowStore.put(1, "two-2", currentTime); currentTime += WINDOW_SIZE * 10; - setCurrentTime(currentTime); - windowStore.put(1, "three"); - windowStore.put(1, "three-2"); + windowStore.put(1, "three", currentTime); + windowStore.put(1, "three-2", currentTime); final WindowStoreIterator iterator = windowStore.fetch(1, 0, WINDOW_SIZE * 10); @@ -1210,38 +1193,26 @@ public void testFetchDuplicates() { } - @SuppressWarnings("deprecation") private void putFirstBatch(final WindowStore store, @SuppressWarnings("SameParameterValue") final long startTime, final InternalMockProcessorContext context) { context.setRecordContext(createRecordContext(startTime)); - store.put(0, "zero"); - context.setRecordContext(createRecordContext(startTime + 1L)); - store.put(1, "one"); - context.setRecordContext(createRecordContext(startTime + 2L)); - store.put(2, "two"); - context.setRecordContext(createRecordContext(startTime + 4L)); - store.put(4, "four"); - context.setRecordContext(createRecordContext(startTime + 5L)); - store.put(5, "five"); + store.put(0, "zero", startTime); + store.put(1, "one", startTime + 1L); + store.put(2, "two", startTime + 2L); + store.put(4, "four", startTime + 4L); + store.put(5, "five", startTime + 5L); } - @SuppressWarnings("deprecation") private void putSecondBatch(final WindowStore store, @SuppressWarnings("SameParameterValue") final long startTime, final InternalMockProcessorContext context) { - context.setRecordContext(createRecordContext(startTime + 3L)); - store.put(2, "two+1"); - context.setRecordContext(createRecordContext(startTime + 4L)); - store.put(2, "two+2"); - context.setRecordContext(createRecordContext(startTime + 5L)); - store.put(2, "two+3"); - context.setRecordContext(createRecordContext(startTime + 6L)); - store.put(2, "two+4"); - context.setRecordContext(createRecordContext(startTime + 7L)); - store.put(2, "two+5"); - context.setRecordContext(createRecordContext(startTime + 8L)); - store.put(2, "two+6"); + store.put(2, "two+1", startTime + 3L); + store.put(2, "two+2", startTime + 4L); + store.put(2, "two+3", startTime + 5L); + store.put(2, "two+4", startTime + 6L); + store.put(2, "two+5", startTime + 7L); + store.put(2, "two+6", startTime + 8L); } long extractStoreTimestamp(final byte[] binaryKey) { @@ -1278,10 +1249,6 @@ private static KeyValue, V> windowedPair(final K key, final V return KeyValue.pair(new Windowed<>(key, WindowKeySchema.timeWindowForSize(timestamp, windowSize)), value); } - protected void setCurrentTime(final long currentTime) { - context.setRecordContext(createRecordContext(currentTime)); - } - private ProcessorRecordContext createRecordContext(final long time) { return new ProcessorRecordContext(time, 0, 0, "topic", null); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java index 66e49c8edfccf..735101f8158d1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java @@ -236,8 +236,8 @@ public void close() { @Test @SuppressWarnings("deprecation") public void shouldPutFetchFromCache() { - cachingStore.put(bytesKey("a"), bytesValue("a")); - cachingStore.put(bytesKey("b"), bytesValue("b")); + cachingStore.put(bytesKey("a"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("b"), bytesValue("b"), DEFAULT_TIMESTAMP); assertThat(cachingStore.fetch(bytesKey("a"), 10), equalTo(bytesValue("a"))); assertThat(cachingStore.fetch(bytesKey("b"), 10), equalTo(bytesValue("b"))); @@ -275,8 +275,8 @@ private String stringFrom(final byte[] from) { @Test @SuppressWarnings("deprecation") public void shouldPutFetchRangeFromCache() { - cachingStore.put(bytesKey("a"), bytesValue("a")); - cachingStore.put(bytesKey("b"), bytesValue("b")); + cachingStore.put(bytesKey("a"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("b"), bytesValue("b"), DEFAULT_TIMESTAMP); final KeyValueIterator, byte[]> iterator = cachingStore.fetch(bytesKey("a"), bytesKey("b"), ofEpochMilli(10), ofEpochMilli(10)); @@ -293,16 +293,15 @@ public void shouldPutFetchRangeFromCache() { } @Test - @SuppressWarnings("deprecation") public void shouldGetAllFromCache() { - cachingStore.put(bytesKey("a"), bytesValue("a")); - cachingStore.put(bytesKey("b"), bytesValue("b")); - cachingStore.put(bytesKey("c"), bytesValue("c")); - cachingStore.put(bytesKey("d"), bytesValue("d")); - cachingStore.put(bytesKey("e"), bytesValue("e")); - cachingStore.put(bytesKey("f"), bytesValue("f")); - cachingStore.put(bytesKey("g"), bytesValue("g")); - cachingStore.put(bytesKey("h"), bytesValue("h")); + cachingStore.put(bytesKey("a"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("b"), bytesValue("b"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("c"), bytesValue("c"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("d"), bytesValue("d"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("e"), bytesValue("e"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("f"), bytesValue("f"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("g"), bytesValue("g"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("h"), bytesValue("h"), DEFAULT_TIMESTAMP); final KeyValueIterator, byte[]> iterator = cachingStore.all(); final String[] array = {"a", "b", "c", "d", "e", "f", "g", "h"}; @@ -318,14 +317,14 @@ public void shouldGetAllFromCache() { @Test @SuppressWarnings("deprecation") public void shouldGetAllBackwardFromCache() { - cachingStore.put(bytesKey("a"), bytesValue("a")); - cachingStore.put(bytesKey("b"), bytesValue("b")); - cachingStore.put(bytesKey("c"), bytesValue("c")); - cachingStore.put(bytesKey("d"), bytesValue("d")); - cachingStore.put(bytesKey("e"), bytesValue("e")); - cachingStore.put(bytesKey("f"), bytesValue("f")); - cachingStore.put(bytesKey("g"), bytesValue("g")); - cachingStore.put(bytesKey("h"), bytesValue("h")); + cachingStore.put(bytesKey("a"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("b"), bytesValue("b"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("c"), bytesValue("c"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("d"), bytesValue("d"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("e"), bytesValue("e"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("f"), bytesValue("f"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("g"), bytesValue("g"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("h"), bytesValue("h"), DEFAULT_TIMESTAMP); final KeyValueIterator, byte[]> iterator = cachingStore.backwardAll(); final String[] array = {"h", "g", "f", "e", "d", "c", "b", "a"}; @@ -343,8 +342,7 @@ public void shouldGetAllBackwardFromCache() { public void shouldFetchAllWithinTimestampRange() { final String[] array = {"a", "b", "c", "d", "e", "f", "g", "h"}; for (int i = 0; i < array.length; i++) { - context.setTime(i); - cachingStore.put(bytesKey(array[i]), bytesValue(array[i])); + cachingStore.put(bytesKey(array[i]), bytesValue(array[i]), i); } final KeyValueIterator, byte[]> iterator = @@ -386,8 +384,7 @@ public void shouldFetchAllWithinTimestampRange() { public void shouldFetchAllBackwardWithinTimestampRange() { final String[] array = {"a", "b", "c", "d", "e", "f", "g", "h"}; for (int i = 0; i < array.length; i++) { - context.setTime(i); - cachingStore.put(bytesKey(array[i]), bytesValue(array[i])); + cachingStore.put(bytesKey(array[i]), bytesValue(array[i]), i); } final KeyValueIterator, byte[]> iterator = @@ -444,7 +441,7 @@ public void shouldFlushEvictedItemsIntoUnderlyingStore() { public void shouldForwardDirtyItemsWhenFlushCalled() { final Windowed windowedKey = new Windowed<>("1", new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)); - cachingStore.put(bytesKey("1"), bytesValue("a")); + cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); cachingStore.flush(); assertEquals("a", cacheListener.forwarded.get(windowedKey).newValue); assertNull(cacheListener.forwarded.get(windowedKey).oldValue); @@ -462,24 +459,24 @@ public void shouldForwardOldValuesWhenEnabled() { cachingStore.setFlushListener(cacheListener, true); final Windowed windowedKey = new Windowed<>("1", new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)); - cachingStore.put(bytesKey("1"), bytesValue("a")); - cachingStore.put(bytesKey("1"), bytesValue("b")); + cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP); cachingStore.flush(); assertEquals("b", cacheListener.forwarded.get(windowedKey).newValue); assertNull(cacheListener.forwarded.get(windowedKey).oldValue); cacheListener.forwarded.clear(); - cachingStore.put(bytesKey("1"), bytesValue("c")); + cachingStore.put(bytesKey("1"), bytesValue("c"), DEFAULT_TIMESTAMP); cachingStore.flush(); assertEquals("c", cacheListener.forwarded.get(windowedKey).newValue); assertEquals("b", cacheListener.forwarded.get(windowedKey).oldValue); - cachingStore.put(bytesKey("1"), null); + cachingStore.put(bytesKey("1"), null, DEFAULT_TIMESTAMP); cachingStore.flush(); assertNull(cacheListener.forwarded.get(windowedKey).newValue); assertEquals("c", cacheListener.forwarded.get(windowedKey).oldValue); cacheListener.forwarded.clear(); - cachingStore.put(bytesKey("1"), bytesValue("a")); - cachingStore.put(bytesKey("1"), bytesValue("b")); - cachingStore.put(bytesKey("1"), null); + cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("1"), null, DEFAULT_TIMESTAMP); cachingStore.flush(); assertNull(cacheListener.forwarded.get(windowedKey)); cacheListener.forwarded.clear(); @@ -490,23 +487,23 @@ public void shouldForwardOldValuesWhenEnabled() { public void shouldForwardOldValuesWhenDisabled() { final Windowed windowedKey = new Windowed<>("1", new TimeWindow(DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE)); - cachingStore.put(bytesKey("1"), bytesValue("a")); - cachingStore.put(bytesKey("1"), bytesValue("b")); + cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP); cachingStore.flush(); assertEquals("b", cacheListener.forwarded.get(windowedKey).newValue); assertNull(cacheListener.forwarded.get(windowedKey).oldValue); - cachingStore.put(bytesKey("1"), bytesValue("c")); + cachingStore.put(bytesKey("1"), bytesValue("c"), DEFAULT_TIMESTAMP); cachingStore.flush(); assertEquals("c", cacheListener.forwarded.get(windowedKey).newValue); assertNull(cacheListener.forwarded.get(windowedKey).oldValue); - cachingStore.put(bytesKey("1"), null); + cachingStore.put(bytesKey("1"), null, DEFAULT_TIMESTAMP); cachingStore.flush(); assertNull(cacheListener.forwarded.get(windowedKey).newValue); assertNull(cacheListener.forwarded.get(windowedKey).oldValue); cacheListener.forwarded.clear(); - cachingStore.put(bytesKey("1"), bytesValue("a")); - cachingStore.put(bytesKey("1"), bytesValue("b")); - cachingStore.put(bytesKey("1"), null); + cachingStore.put(bytesKey("1"), bytesValue("a"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("1"), bytesValue("b"), DEFAULT_TIMESTAMP); + cachingStore.put(bytesKey("1"), null, DEFAULT_TIMESTAMP); cachingStore.flush(); assertNull(cacheListener.forwarded.get(windowedKey)); cacheListener.forwarded.clear(); @@ -619,7 +616,7 @@ public void shouldIterateBackwardCacheAndStoreKeyRange() { @Test @SuppressWarnings("deprecation") public void shouldClearNamespaceCacheOnClose() { - cachingStore.put(bytesKey("a"), bytesValue("a")); + cachingStore.put(bytesKey("a"), bytesValue("a"), 0L); assertEquals(1, cache.size()); cachingStore.close(); assertEquals(0, cache.size()); @@ -641,7 +638,7 @@ public void shouldThrowIfTryingToFetchRangeFromClosedCachingStore() { @SuppressWarnings("deprecation") public void shouldThrowIfTryingToWriteToClosedCachingStore() { cachingStore.close(); - assertThrows(InvalidStateStoreException.class, () -> cachingStore.put(bytesKey("a"), bytesValue("a"))); + assertThrows(InvalidStateStoreException.class, () -> cachingStore.put(bytesKey("a"), bytesValue("a"), 0L)); } @Test @@ -789,13 +786,13 @@ public void shouldReturnSameResultsForSingleKeyFetchAndEqualKeyRangeBackwardFetc @Test @SuppressWarnings("deprecation") public void shouldThrowNullPointerExceptionOnPutNullKey() { - assertThrows(NullPointerException.class, () -> cachingStore.put(null, bytesValue("anyValue"))); + assertThrows(NullPointerException.class, () -> cachingStore.put(null, bytesValue("anyValue"), 0L)); } @Test @SuppressWarnings("deprecation") public void shouldNotThrowNullPointerExceptionOnPutNullValue() { - cachingStore.put(bytesKey("a"), null); + cachingStore.put(bytesKey("a"), null, 0L); } @Test @@ -919,13 +916,12 @@ private static KeyValue, byte[]> windowedPair(final String key, bytesValue(value)); } - @SuppressWarnings("deprecation") private int addItemsToCache() { int cachedSize = 0; int i = 0; while (cachedSize < MAX_CACHE_SIZE_BYTES) { final String kv = String.valueOf(i++); - cachingStore.put(bytesKey(kv), bytesValue(kv)); + cachingStore.put(bytesKey(kv), bytesValue(kv), DEFAULT_TIMESTAMP); cachedSize += memoryCacheEntrySize(kv.getBytes(), kv.getBytes(), TOPIC) + 8 + // timestamp 4; // sequenceNumber diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java index 6608739a8f412..50c18fe0573e5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingTimestampedWindowBytesStoreTest.java @@ -99,7 +99,7 @@ public void shouldLogPuts() { context.logChange(store.name(), key, value, 42); EasyMock.replay(context); - store.put(bytesKey, valueAndTimestamp); + store.put(bytesKey, valueAndTimestamp, context.timestamp()); EasyMock.verify(inner, context); } @@ -146,8 +146,8 @@ public void shouldRetainDuplicatesWhenSet() { EasyMock.replay(context); - store.put(bytesKey, valueAndTimestamp); - store.put(bytesKey, valueAndTimestamp); + store.put(bytesKey, valueAndTimestamp, context.timestamp()); + store.put(bytesKey, valueAndTimestamp, context.timestamp()); EasyMock.verify(inner, context); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java index b19b99ab98947..36e3297b560c1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStoreTest.java @@ -84,7 +84,6 @@ public void shouldDelegateInit() { } @Test - @SuppressWarnings("deprecation") public void shouldLogPuts() { inner.put(bytesKey, value, 0); EasyMock.expectLastCall(); @@ -98,7 +97,7 @@ public void shouldLogPuts() { context.logChange(store.name(), key, value, 0L); EasyMock.replay(context); - store.put(bytesKey, value); + store.put(bytesKey, value, context.timestamp()); EasyMock.verify(inner, context); } @@ -152,7 +151,6 @@ public void shouldDelegateToUnderlyingStoreWhenBackwardFetchingRange() { } @Test - @SuppressWarnings("deprecation") public void shouldRetainDuplicatesWhenSet() { store = new ChangeLoggingWindowBytesStore(inner, true, WindowKeySchema::toStoreKeyBinary); @@ -171,8 +169,8 @@ public void shouldRetainDuplicatesWhenSet() { EasyMock.replay(context); - store.put(bytesKey, value); - store.put(bytesKey, value); + store.put(bytesKey, value, context.timestamp()); + store.put(bytesKey, value, context.timestamp()); EasyMock.verify(inner, context); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java index e5f214292de68..2ef9bad4aa866 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryWindowStoreTest.java @@ -126,37 +126,30 @@ public void shouldNotExpireFromOpenIterator() { } @Test - @SuppressWarnings("deprecation") public void testExpiration() { long currentTime = 0; - setCurrentTime(currentTime); - windowStore.put(1, "one"); + windowStore.put(1, "one", currentTime); currentTime += RETENTION_PERIOD / 4; - setCurrentTime(currentTime); - windowStore.put(1, "two"); + windowStore.put(1, "two", currentTime); currentTime += RETENTION_PERIOD / 4; - setCurrentTime(currentTime); - windowStore.put(1, "three"); + windowStore.put(1, "three", currentTime); currentTime += RETENTION_PERIOD / 4; - setCurrentTime(currentTime); - windowStore.put(1, "four"); + windowStore.put(1, "four", currentTime); // increase current time to the full RETENTION_PERIOD to expire first record currentTime = currentTime + RETENTION_PERIOD / 4; - setCurrentTime(currentTime); - windowStore.put(1, "five"); + windowStore.put(1, "five", currentTime); KeyValueIterator, String> iterator = windowStore .fetchAll(0L, currentTime); // effect of this put (expires next oldest record, adds new one) should not be reflected in the already fetched results currentTime = currentTime + RETENTION_PERIOD / 4; - setCurrentTime(currentTime); - windowStore.put(1, "six"); + windowStore.put(1, "six", currentTime); // should only have middle 4 values, as (only) the first record was expired at the time of the fetch // and the last was inserted after the fetch diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java index 595cb0161eaac..c05c1ba5e67c3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredTimestampedWindowStoreTest.java @@ -202,7 +202,6 @@ public void shouldNotExceptionIfFetchReturnsNull() { } @Test - @SuppressWarnings("deprecation") public void shouldNotThrowExceptionIfSerdesCorrectlySetFromProcessorContext() { EasyMock.expect(innerStoreMock.name()).andStubReturn("mocked-store"); EasyMock.replay(innerStoreMock); @@ -217,7 +216,7 @@ public void shouldNotThrowExceptionIfSerdesCorrectlySetFromProcessorContext() { store.init((StateStoreContext) context, innerStoreMock); try { - store.put("key", ValueAndTimestamp.make(42L, 60000)); + store.put("key", ValueAndTimestamp.make(42L, 60000), 60000L); } catch (final StreamsException exception) { if (exception.getCause() instanceof ClassCastException) { fail("Serdes are not correctly set from processor context."); @@ -227,7 +226,6 @@ public void shouldNotThrowExceptionIfSerdesCorrectlySetFromProcessorContext() { } @Test - @SuppressWarnings("deprecation") public void shouldNotThrowExceptionIfSerdesCorrectlySetFromConstructorParameters() { EasyMock.expect(innerStoreMock.name()).andStubReturn("mocked-store"); EasyMock.replay(innerStoreMock); @@ -242,7 +240,7 @@ public void shouldNotThrowExceptionIfSerdesCorrectlySetFromConstructorParameters store.init((StateStoreContext) context, innerStoreMock); try { - store.put("key", ValueAndTimestamp.make(42L, 60000)); + store.put("key", ValueAndTimestamp.make(42L, 60000), 60000L); } catch (final StreamsException exception) { if (exception.getCause() instanceof ClassCastException) { fail("Serdes are not correctly set from constructor parameters."); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index a74aaea6f139d..cb3a268cace15 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -290,7 +290,6 @@ public void shouldRecordRestoreLatencyOnInit() { } @Test - @SuppressWarnings("deprecation") public void shouldRecordPutLatency() { final byte[] bytes = "a".getBytes(); innerStoreMock.put(eq(Bytes.wrap(bytes)), anyObject(), eq(context.timestamp())); @@ -298,7 +297,7 @@ public void shouldRecordPutLatency() { replay(innerStoreMock); store.init((StateStoreContext) context, store); - store.put("a", "a"); + store.put("a", "a", context.timestamp()); final Map metrics = context.metrics().metrics(); if (StreamsConfig.METRICS_0100_TO_24.equals(builtInMetricsVersion)) { assertEquals(1.0, getMetricByNameFilterByTags( diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index 9e9b0b4c312c6..ce85893970c10 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -81,26 +81,21 @@ void setClassLoggerToDebug() { } @Test - @SuppressWarnings("deprecation") public void shouldOnlyIterateOpenSegments() { long currentTime = 0; - setCurrentTime(currentTime); - windowStore.put(1, "one"); + windowStore.put(1, "one", currentTime); currentTime = currentTime + SEGMENT_INTERVAL; - setCurrentTime(currentTime); - windowStore.put(1, "two"); + windowStore.put(1, "two", currentTime); currentTime = currentTime + SEGMENT_INTERVAL; - setCurrentTime(currentTime); - windowStore.put(1, "three"); + windowStore.put(1, "three", currentTime); final WindowStoreIterator iterator = windowStore.fetch(1, 0L, currentTime); // roll to the next segment that will close the first currentTime = currentTime + SEGMENT_INTERVAL; - setCurrentTime(currentTime); - windowStore.put(1, "four"); + windowStore.put(1, "four", currentTime); // should only have 2 values as the first segment is no longer open assertEquals(new KeyValue<>(SEGMENT_INTERVAL, "two"), iterator.next()); @@ -109,22 +104,18 @@ public void shouldOnlyIterateOpenSegments() { } @Test - @SuppressWarnings("deprecation") public void testRolling() { // to validate segments final long startTime = SEGMENT_INTERVAL * 2; final long increment = SEGMENT_INTERVAL / 2; - setCurrentTime(startTime); - windowStore.put(0, "zero"); + windowStore.put(0, "zero", startTime); assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir)); - setCurrentTime(startTime + increment); - windowStore.put(1, "one"); + windowStore.put(1, "one", startTime + increment); assertEquals(Utils.mkSet(segments.segmentName(2)), segmentDirs(baseDir)); - setCurrentTime(startTime + increment * 2); - windowStore.put(2, "two"); + windowStore.put(2, "two", startTime + increment * 2); assertEquals( Utils.mkSet( segments.segmentName(2), @@ -133,8 +124,7 @@ public void testRolling() { segmentDirs(baseDir) ); - setCurrentTime(startTime + increment * 4); - windowStore.put(4, "four"); + windowStore.put(4, "four", startTime + increment * 4); assertEquals( Utils.mkSet( segments.segmentName(2), @@ -144,8 +134,7 @@ public void testRolling() { segmentDirs(baseDir) ); - setCurrentTime(startTime + increment * 5); - windowStore.put(5, "five"); + windowStore.put(5, "five", startTime + increment * 5); assertEquals( Utils.mkSet( segments.segmentName(2), @@ -192,8 +181,7 @@ public void testRolling() { ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE)))); - setCurrentTime(startTime + increment * 6); - windowStore.put(6, "six"); + windowStore.put(6, "six", startTime + increment * 6); assertEquals( Utils.mkSet( segments.segmentName(3), @@ -246,8 +234,7 @@ public void testRolling() { ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE)))); - setCurrentTime(startTime + increment * 7); - windowStore.put(7, "seven"); + windowStore.put(7, "seven", startTime + increment * 7); assertEquals( Utils.mkSet( segments.segmentName(3), @@ -306,8 +293,7 @@ public void testRolling() { ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE), ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE)))); - setCurrentTime(startTime + increment * 8); - windowStore.put(8, "eight"); + windowStore.put(8, "eight", startTime + increment * 8); assertEquals( Utils.mkSet( segments.segmentName(4), @@ -385,7 +371,6 @@ public void testRolling() { } @Test - @SuppressWarnings("deprecation") public void testSegmentMaintenance() { windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), @@ -393,23 +378,20 @@ public void testSegmentMaintenance() { windowStore.init((StateStoreContext) context, windowStore); context.setTime(0L); - setCurrentTime(0); - windowStore.put(0, "v"); + windowStore.put(0, "v", 0); assertEquals( Utils.mkSet(segments.segmentName(0L)), segmentDirs(baseDir) ); - setCurrentTime(SEGMENT_INTERVAL - 1); - windowStore.put(0, "v"); - windowStore.put(0, "v"); + windowStore.put(0, "v", SEGMENT_INTERVAL - 1); + windowStore.put(0, "v", SEGMENT_INTERVAL - 1); assertEquals( Utils.mkSet(segments.segmentName(0L)), segmentDirs(baseDir) ); - setCurrentTime(SEGMENT_INTERVAL); - windowStore.put(0, "v"); + windowStore.put(0, "v", SEGMENT_INTERVAL); assertEquals( Utils.mkSet(segments.segmentName(0L), segments.segmentName(1L)), segmentDirs(baseDir) @@ -431,8 +413,7 @@ public void testSegmentMaintenance() { segmentDirs(baseDir) ); - setCurrentTime(SEGMENT_INTERVAL * 3); - windowStore.put(0, "v"); + windowStore.put(0, "v", SEGMENT_INTERVAL * 3); iter = windowStore.fetch(0, ofEpochMilli(0L), ofEpochMilli(SEGMENT_INTERVAL * 4)); fetchedCount = 0; @@ -447,8 +428,7 @@ public void testSegmentMaintenance() { segmentDirs(baseDir) ); - setCurrentTime(SEGMENT_INTERVAL * 5); - windowStore.put(0, "v"); + windowStore.put(0, "v", SEGMENT_INTERVAL * 5); iter = windowStore.fetch(0, ofEpochMilli(SEGMENT_INTERVAL * 4), ofEpochMilli(SEGMENT_INTERVAL * 10)); fetchedCount = 0; @@ -512,29 +492,19 @@ public void testInitialLoading() { } @Test - @SuppressWarnings("deprecation") public void testRestore() throws Exception { final long startTime = SEGMENT_INTERVAL * 2; final long increment = SEGMENT_INTERVAL / 2; - setCurrentTime(startTime); - windowStore.put(0, "zero"); - setCurrentTime(startTime + increment); - windowStore.put(1, "one"); - setCurrentTime(startTime + increment * 2); - windowStore.put(2, "two"); - setCurrentTime(startTime + increment * 3); - windowStore.put(3, "three"); - setCurrentTime(startTime + increment * 4); - windowStore.put(4, "four"); - setCurrentTime(startTime + increment * 5); - windowStore.put(5, "five"); - setCurrentTime(startTime + increment * 6); - windowStore.put(6, "six"); - setCurrentTime(startTime + increment * 7); - windowStore.put(7, "seven"); - setCurrentTime(startTime + increment * 8); - windowStore.put(8, "eight"); + windowStore.put(0, "zero", startTime); + windowStore.put(1, "one", startTime + increment); + windowStore.put(2, "two", startTime + increment * 2); + windowStore.put(3, "three", startTime + increment * 3); + windowStore.put(4, "four", startTime + increment * 4); + windowStore.put(5, "five", startTime + increment * 5); + windowStore.put(6, "six", startTime + increment * 6); + windowStore.put(7, "seven", startTime + increment * 7); + windowStore.put(8, "eight", startTime + increment * 8); windowStore.flush(); windowStore.close(); diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index 659fb4775d4c1..f657209c18cd0 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -1360,13 +1360,6 @@ public void init(final StateStoreContext context, final StateStore root) { inner.init(context, root); } - @Deprecated - @Override - public void put(final K key, - final V value) { - inner.put(key, ValueAndTimestamp.make(value, ConsumerRecord.NO_TIMESTAMP)); - } - @Override public void put(final K key, final V value, diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/WindowStoreFacadeTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/WindowStoreFacadeTest.java index d184f980c9f39..434782c40b9b8 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/WindowStoreFacadeTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/WindowStoreFacadeTest.java @@ -70,17 +70,6 @@ public void shouldForwardInit() { verify(mockedWindowTimestampStore); } - @Test - @SuppressWarnings("deprecation") - public void shouldPutWithUnknownTimestamp() { - mockedWindowTimestampStore.put("key", ValueAndTimestamp.make("value", ConsumerRecord.NO_TIMESTAMP)); - expectLastCall(); - replay(mockedWindowTimestampStore); - - windowStoreFacade.put("key", "value"); - verify(mockedWindowTimestampStore); - } - @Test public void shouldPutWindowStartTimestampWithUnknownTimestamp() { mockedWindowTimestampStore.put("key", ValueAndTimestamp.make("value", ConsumerRecord.NO_TIMESTAMP), 21L); From c9edb8ad84ef1a275b1874f0617c07d6e33cdd3d Mon Sep 17 00:00:00 2001 From: Andrew Lee Date: Fri, 9 Apr 2021 12:12:52 -0700 Subject: [PATCH 036/155] MINOR: Fix typo in docs (#10423) Reviewers: Matthias J. Sax --- docs/quickstart.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/quickstart.html b/docs/quickstart.html index e70c63a5b641a..eda3ede8a8432 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -166,7 +166,7 @@

    Take a look at the Kafka Connect section - learn more about how to continuously import/export your data into and out of Kafka.

    + to learn more about how to continuously import/export your data into and out of Kafka.

    From f76b8e493832fe7374e58dff460b81eee6245ce5 Mon Sep 17 00:00:00 2001 From: Luke Chen <43372967+showuon@users.noreply.github.com> Date: Sat, 10 Apr 2021 03:19:14 +0800 Subject: [PATCH 037/155] KAFKA-9831: increase max.poll.interval.ms to avoid unexpected rebalance (#10301) Reviewers: Matthias J. Sax --- checkstyle/suppressions.xml | 2 +- .../integration/EosIntegrationTest.java | 220 ++++++++++++++---- 2 files changed, 171 insertions(+), 51 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 2ed1fac5df95d..7a2cd22eaa166 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -193,7 +193,7 @@ files="(StreamsPartitionAssignorTest|StreamThreadTest|StreamTaskTest|TopologyTestDriverTest).java"/> + files="(EosIntegrationTest|EosBetaUpgradeIntegrationTest|KStreamKStreamJoinTest|RocksDBWindowStoreTest).java"/> diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index 9179971ae0695..bb04267963644 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -84,10 +84,10 @@ import static org.apache.kafka.test.StreamsTestUtils.startKafkaStreamsAndWaitForRunningState; import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; @RunWith(Parameterized.class) @Category({IntegrationTest.class}) @@ -133,6 +133,8 @@ public static void closeCluster() { private static final AtomicInteger TEST_NUMBER = new AtomicInteger(0); + private volatile boolean hasUnexpectedError = false; + @Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new String[][] { @@ -279,18 +281,20 @@ private void runSimpleCopyTest(final int numberOfRestarts, inputData.size() ); - checkResultPerKey(committedRecords, inputData); + checkResultPerKey(committedRecords, inputData, "The committed records do not match what expected"); } } } - private void checkResultPerKey(final List> result, final List> expectedResult) { + private void checkResultPerKey(final List> result, + final List> expectedResult, + final String reason) { final Set allKeys = new HashSet<>(); addAllKeys(allKeys, result); addAllKeys(allKeys, expectedResult); for (final Long key : allKeys) { - assertThat(getAllRecordPerKey(key, result), equalTo(getAllRecordPerKey(key, expectedResult))); + assertThat(reason, getAllRecordPerKey(key, result), equalTo(getAllRecordPerKey(key, expectedResult))); } } @@ -391,19 +395,21 @@ public void shouldBeAbleToPerformMultipleTransactions() throws Exception { public void shouldNotViolateEosIfOneTaskFails() throws Exception { // this test writes 10 + 5 + 5 records per partition (running with 2 partitions) // the app is supposed to copy all 40 records into the output topic - // the app commits after each 10 records per partition, and thus will have 2*5 uncommitted writes + // + // the app first commits after each 10 records per partition(total 20 records), and thus will have 2 * 5 uncommitted writes // // the failure gets inject after 20 committed and 30 uncommitted records got received // -> the failure only kills one thread // after fail over, we should read 40 committed records (even if 50 record got written) - try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, eosConfig)) { + try (final KafkaStreams streams = getKafkaStreams("dummy", false, "appDir", 2, eosConfig, MAX_POLL_INTERVAL_MS)) { startKafkaStreamsAndWaitForRunningState(streams, MAX_WAIT_TIME_MS); final List> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L); final List> uncommittedDataBeforeFailure = prepareData(10L, 15L, 0L, 1L); - final List> dataBeforeFailure = new ArrayList<>(); + final List> dataBeforeFailure = new ArrayList<>( + committedDataBeforeFailure.size() + uncommittedDataBeforeFailure.size()); dataBeforeFailure.addAll(committedDataBeforeFailure); dataBeforeFailure.addAll(uncommittedDataBeforeFailure); @@ -415,13 +421,29 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { () -> commitRequested.get() == 2, MAX_WAIT_TIME_MS, "StreamsTasks did not request commit."); - writeInputData(uncommittedDataBeforeFailure); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + // p-1: ---> 10 rec + C - final List> uncommittedRecords = readResult(dataBeforeFailure.size(), null); final List> committedRecords = readResult(committedDataBeforeFailure.size(), CONSUMER_GROUP_ID); + checkResultPerKey( + committedRecords, + committedDataBeforeFailure, + "The committed records before failure do not match what expected"); - checkResultPerKey(committedRecords, committedDataBeforeFailure); - checkResultPerKey(uncommittedRecords, dataBeforeFailure); + writeInputData(uncommittedDataBeforeFailure); + + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + 5 rec (pending) + // p-1: ---> 10 rec + C + 5 rec (pending) + + final List> uncommittedRecords = readResult(dataBeforeFailure.size(), null); + checkResultPerKey( + uncommittedRecords, + dataBeforeFailure, + "The uncommitted records before failure do not match what expected"); errorInjected.set(true); writeInputData(dataAfterFailure); @@ -430,6 +452,11 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { () -> uncaughtException != null, MAX_WAIT_TIME_MS, "Should receive uncaught exception from one StreamThread."); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + 5 rec + C + 5 rec + C + // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C + final List> allCommittedRecords = readResult( committedDataBeforeFailure.size() + uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID + "_ALL"); @@ -438,17 +465,28 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID); - final List> allExpectedCommittedRecordsAfterRecovery = new ArrayList<>(); + final int allCommittedRecordsAfterRecoverySize = committedDataBeforeFailure.size() + + uncommittedDataBeforeFailure.size() + dataAfterFailure.size(); + final List> allExpectedCommittedRecordsAfterRecovery = new ArrayList<>(allCommittedRecordsAfterRecoverySize); allExpectedCommittedRecordsAfterRecovery.addAll(committedDataBeforeFailure); allExpectedCommittedRecordsAfterRecovery.addAll(uncommittedDataBeforeFailure); allExpectedCommittedRecordsAfterRecovery.addAll(dataAfterFailure); - final List> expectedCommittedRecordsAfterRecovery = new ArrayList<>(); + final int committedRecordsAfterRecoverySize = uncommittedDataBeforeFailure.size() + dataAfterFailure.size(); + final List> expectedCommittedRecordsAfterRecovery = new ArrayList<>(committedRecordsAfterRecoverySize); expectedCommittedRecordsAfterRecovery.addAll(uncommittedDataBeforeFailure); expectedCommittedRecordsAfterRecovery.addAll(dataAfterFailure); - checkResultPerKey(allCommittedRecords, allExpectedCommittedRecordsAfterRecovery); - checkResultPerKey(committedRecordsAfterFailure, expectedCommittedRecordsAfterRecovery); + checkResultPerKey( + allCommittedRecords, + allExpectedCommittedRecordsAfterRecovery, + "The all committed records after recovery do not match what expected"); + checkResultPerKey( + committedRecordsAfterFailure, + expectedCommittedRecordsAfterRecovery, + "The committed records after recovery do not match what expected"); + + assertThat("Should only get one uncaught exception from Streams.", hasUnexpectedError, is(false)); } } @@ -456,22 +494,26 @@ public void shouldNotViolateEosIfOneTaskFails() throws Exception { public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { // this test updates a store with 10 + 5 + 5 records per partition (running with 2 partitions) // the app is supposed to emit all 40 update records into the output topic - // the app commits after each 10 records per partition, and thus will have 2*5 uncommitted writes + // + // the app first commits after each 10 records per partition (total 20 records), and thus will have 2 * 5 uncommitted writes // and store updates (ie, another 5 uncommitted writes to a changelog topic per partition) - // in the uncommitted batch sending some data for the new key to validate that upon resuming they will not be shown up in the store + // in the uncommitted batch, sending some data for the new key to validate that upon resuming they will not be shown up in the store // - // the failure gets inject after 20 committed and 10 uncommitted records got received + // the failure gets inject after 20 committed and 30 uncommitted records got received // -> the failure only kills one thread // after fail over, we should read 40 committed records and the state stores should contain the correct sums // per key (even if some records got processed twice) - try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, eosConfig)) { + // We need more processing time under "with state" situation, so increasing the max.poll.interval.ms + // to avoid unexpected rebalance during test, which will cause unexpected fail over triggered + try (final KafkaStreams streams = getKafkaStreams("dummy", true, "appDir", 2, eosConfig, 3 * MAX_POLL_INTERVAL_MS)) { startKafkaStreamsAndWaitForRunningState(streams, MAX_WAIT_TIME_MS); final List> committedDataBeforeFailure = prepareData(0L, 10L, 0L, 1L); final List> uncommittedDataBeforeFailure = prepareData(10L, 15L, 0L, 1L, 2L, 3L); - final List> dataBeforeFailure = new ArrayList<>(); + final List> dataBeforeFailure = new ArrayList<>( + committedDataBeforeFailure.size() + uncommittedDataBeforeFailure.size()); dataBeforeFailure.addAll(committedDataBeforeFailure); dataBeforeFailure.addAll(uncommittedDataBeforeFailure); @@ -483,15 +525,36 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { () -> commitRequested.get() == 2, MAX_WAIT_TIME_MS, "SteamsTasks did not request commit."); - writeInputData(uncommittedDataBeforeFailure); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + // p-1: ---> 10 rec + C - final List> uncommittedRecords = readResult(dataBeforeFailure.size(), null); final List> committedRecords = readResult(committedDataBeforeFailure.size(), CONSUMER_GROUP_ID); + checkResultPerKey( + committedRecords, + computeExpectedResult(committedDataBeforeFailure), + "The committed records before failure do not match what expected"); + + writeInputData(uncommittedDataBeforeFailure); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + 5 rec (pending) + // p-1: ---> 10 rec + C + 5 rec (pending) + + final List> uncommittedRecords = readResult(dataBeforeFailure.size(), null); final List> expectedResultBeforeFailure = computeExpectedResult(dataBeforeFailure); - checkResultPerKey(committedRecords, computeExpectedResult(committedDataBeforeFailure)); - checkResultPerKey(uncommittedRecords, expectedResultBeforeFailure); - verifyStateStore(streams, getMaxPerKey(expectedResultBeforeFailure)); + + + checkResultPerKey( + uncommittedRecords, + expectedResultBeforeFailure, + "The uncommitted records before failure do not match what expected"); + verifyStateStore( + streams, + getMaxPerKey(expectedResultBeforeFailure), + "The state store content before failure do not match what expected"); errorInjected.set(true); writeInputData(dataAfterFailure); @@ -500,6 +563,11 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { () -> uncaughtException != null, MAX_WAIT_TIME_MS, "Should receive uncaught exception from one StreamThread."); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + 5 rec + C + 5 rec + C + // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C + final List> allCommittedRecords = readResult( committedDataBeforeFailure.size() + uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID + "_ALL"); @@ -508,19 +576,31 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { uncommittedDataBeforeFailure.size() + dataAfterFailure.size(), CONSUMER_GROUP_ID); - final List> allExpectedCommittedRecordsAfterRecovery = new ArrayList<>(); + final int allCommittedRecordsAfterRecoverySize = committedDataBeforeFailure.size() + + uncommittedDataBeforeFailure.size() + dataAfterFailure.size(); + final List> allExpectedCommittedRecordsAfterRecovery = new ArrayList<>(allCommittedRecordsAfterRecoverySize); allExpectedCommittedRecordsAfterRecovery.addAll(committedDataBeforeFailure); allExpectedCommittedRecordsAfterRecovery.addAll(uncommittedDataBeforeFailure); allExpectedCommittedRecordsAfterRecovery.addAll(dataAfterFailure); final List> expectedResult = computeExpectedResult(allExpectedCommittedRecordsAfterRecovery); - checkResultPerKey(allCommittedRecords, expectedResult); + checkResultPerKey( + allCommittedRecords, + expectedResult, + "The all committed records after recovery do not match what expected"); + checkResultPerKey( committedRecordsAfterFailure, - expectedResult.subList(committedDataBeforeFailure.size(), expectedResult.size())); + expectedResult.subList(committedDataBeforeFailure.size(), expectedResult.size()), + "The committed records after recovery do not match what expected"); + + verifyStateStore( + streams, + getMaxPerKey(expectedResult), + "The state store content after recovery do not match what expected"); - verifyStateStore(streams, getMaxPerKey(expectedResult)); + assertThat("Should only get one uncaught exception from Streams.", hasUnexpectedError, is(false)); } } @@ -528,9 +608,10 @@ public void shouldNotViolateEosIfOneTaskFailsWithState() throws Exception { public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() throws Exception { // this test writes 10 + 5 + 5 + 10 records per partition (running with 2 partitions) // the app is supposed to copy all 60 records into the output topic - // the app commits after each 10 records per partition, and thus will have 2*5 uncommitted writes // - // a stall gets injected after 20 committed and 30 uncommitted records got received + // the app first commits after each 10 records per partition, and thus will have 2 * 5 uncommitted writes + // + // Then, a stall gets injected after 20 committed and 30 uncommitted records got received // -> the stall only affects one thread and should trigger a rebalance // after rebalancing, we should read 40 committed records (even if 50 record got written) // @@ -538,8 +619,8 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th // we write the remaining 20 records and verify to read 60 result records try ( - final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, eosConfig); - final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, eosConfig) + final KafkaStreams streams1 = getKafkaStreams("streams1", false, "appDir1", 1, eosConfig, MAX_POLL_INTERVAL_MS); + final KafkaStreams streams2 = getKafkaStreams("streams2", false, "appDir2", 1, eosConfig, MAX_POLL_INTERVAL_MS) ) { startKafkaStreamsAndWaitForRunningState(streams1, MAX_WAIT_TIME_MS); startKafkaStreamsAndWaitForRunningState(streams2, MAX_WAIT_TIME_MS); @@ -547,7 +628,8 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th final List> committedDataBeforeStall = prepareData(0L, 10L, 0L, 1L); final List> uncommittedDataBeforeStall = prepareData(10L, 15L, 0L, 1L); - final List> dataBeforeStall = new ArrayList<>(); + final List> dataBeforeStall = new ArrayList<>( + committedDataBeforeStall.size() + uncommittedDataBeforeStall.size()); dataBeforeStall.addAll(committedDataBeforeStall); dataBeforeStall.addAll(uncommittedDataBeforeStall); @@ -561,13 +643,29 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th () -> commitRequested.get() == 2, MAX_WAIT_TIME_MS, "SteamsTasks did not request commit."); - writeInputData(uncommittedDataBeforeStall); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + // p-1: ---> 10 rec + C - final List> uncommittedRecords = readResult(dataBeforeStall.size(), null); final List> committedRecords = readResult(committedDataBeforeStall.size(), CONSUMER_GROUP_ID); + checkResultPerKey( + committedRecords, + committedDataBeforeStall, + "The committed records before stall do not match what expected"); + + writeInputData(uncommittedDataBeforeStall); - checkResultPerKey(committedRecords, committedDataBeforeStall); - checkResultPerKey(uncommittedRecords, dataBeforeStall); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + 5 rec (pending) + // p-1: ---> 10 rec + C + 5 rec (pending) + + final List> uncommittedRecords = readResult(dataBeforeStall.size(), null); + checkResultPerKey( + uncommittedRecords, + dataBeforeStall, + "The uncommitted records before stall do not match what expected"); LOG.info("Injecting Stall"); stallInjected.set(true); @@ -603,15 +701,24 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th "Streams1[" + streams1.allMetadata() + "]\n" + "Streams2[" + streams2.allMetadata() + "]"); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + 5 rec + C + 5 rec + C + // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C + final List> committedRecordsAfterRebalance = readResult( uncommittedDataBeforeStall.size() + dataToTriggerFirstRebalance.size(), CONSUMER_GROUP_ID); - final List> expectedCommittedRecordsAfterRebalance = new ArrayList<>(); + final List> expectedCommittedRecordsAfterRebalance = new ArrayList<>( + uncommittedDataBeforeStall.size() + dataToTriggerFirstRebalance.size()); expectedCommittedRecordsAfterRebalance.addAll(uncommittedDataBeforeStall); expectedCommittedRecordsAfterRebalance.addAll(dataToTriggerFirstRebalance); - checkResultPerKey(committedRecordsAfterRebalance, expectedCommittedRecordsAfterRebalance); + checkResultPerKey( + committedRecordsAfterRebalance, + expectedCommittedRecordsAfterRebalance, + "The all committed records after rebalance do not match what expected"); LOG.info("Releasing Stall"); doStall = false; @@ -630,25 +737,36 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th writeInputData(dataAfterSecondRebalance); + // expected end state per output partition (C == COMMIT; A == ABORT; ---> indicate the changes): + // + // p-0: ---> 10 rec + C + 5 rec + C + 5 rec + C + 10 rec + C + // p-1: ---> 10 rec + C + 5 rec + C + 5 rec + C + 10 rec + C + final List> allCommittedRecords = readResult( committedDataBeforeStall.size() + uncommittedDataBeforeStall.size() + dataToTriggerFirstRebalance.size() + dataAfterSecondRebalance.size(), CONSUMER_GROUP_ID + "_ALL"); - final List> allExpectedCommittedRecordsAfterRecovery = new ArrayList<>(); + final int allCommittedRecordsAfterRecoverySize = committedDataBeforeStall.size() + + uncommittedDataBeforeStall.size() + dataToTriggerFirstRebalance.size() + dataAfterSecondRebalance.size(); + final List> allExpectedCommittedRecordsAfterRecovery = new ArrayList<>(allCommittedRecordsAfterRecoverySize); allExpectedCommittedRecordsAfterRecovery.addAll(committedDataBeforeStall); allExpectedCommittedRecordsAfterRecovery.addAll(uncommittedDataBeforeStall); allExpectedCommittedRecordsAfterRecovery.addAll(dataToTriggerFirstRebalance); allExpectedCommittedRecordsAfterRecovery.addAll(dataAfterSecondRebalance); - checkResultPerKey(allCommittedRecords, allExpectedCommittedRecordsAfterRecovery); + checkResultPerKey( + allCommittedRecords, + allExpectedCommittedRecordsAfterRecovery, + "The all committed records after recovery do not match what expected"); } } private List> prepareData(final long fromInclusive, final long toExclusive, final Long... keys) { - final List> data = new ArrayList<>(); + final Long dataSize = keys.length * (toExclusive - fromInclusive); + final List> data = new ArrayList<>(dataSize.intValue()); for (final Long k : keys) { for (long v = fromInclusive; v < toExclusive; ++v) { @@ -664,7 +782,8 @@ private KafkaStreams getKafkaStreams(final String dummyHostName, final boolean withState, final String appDir, final int numberOfStreamsThreads, - final String eosConfig) { + final String eosConfig, + final int maxPollIntervalMs) { commitRequested = new AtomicInteger(0); errorInjected = new AtomicBoolean(false); stallInjected = new AtomicBoolean(false); @@ -761,7 +880,7 @@ public void close() { } properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "earliest"); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), 5 * 1000); properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), 5 * 1000 - 1); - properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), MAX_POLL_INTERVAL_MS); + properties.put(StreamsConfig.consumerPrefix(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG), maxPollIntervalMs); properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); properties.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath() + File.separator + appDir); properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, dummyHostName + ":2142"); @@ -776,9 +895,9 @@ public void close() { } final KafkaStreams streams = new KafkaStreams(builder.build(), config); streams.setUncaughtExceptionHandler((t, e) -> { - if (uncaughtException != null) { + if (uncaughtException != null || !e.getMessage().contains("Injected test exception")) { e.printStackTrace(System.err); - fail("Should only get one uncaught exception from Streams."); + hasUnexpectedError = true; } uncaughtException = e; }); @@ -860,16 +979,17 @@ private Set> getMaxPerKey(final List> } private void verifyStateStore(final KafkaStreams streams, - final Set> expectedStoreContent) throws Exception { + final Set> expectedStoreContent, + final String reason) throws Exception { final ReadOnlyKeyValueStore store = IntegrationTestUtils .getStore(300_000L, storeName, streams, QueryableStoreTypes.keyValueStore()); assertNotNull(store); final KeyValueIterator it = store.all(); while (it.hasNext()) { - assertTrue(expectedStoreContent.remove(it.next())); + assertTrue(reason, expectedStoreContent.remove(it.next())); } - assertTrue(expectedStoreContent.isEmpty()); + assertTrue(reason, expectedStoreContent.isEmpty()); } } From c9cab2beb8159ca83e6e2c930514ba97282e1ae9 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 9 Apr 2021 20:49:54 +0100 Subject: [PATCH 038/155] KAFKA-8410: Migrate KStream Stateless operators to new Processor API (#10381) Migrate KStream stateless operators to new Processor API. Following PRs will complete migration of KStream stateful operators and KTable. No expected functionality changes. Reviewers: John Roesler --- .../streams/kstream/ForeachProcessor.java | 34 ++++++++++++++ .../kstream/internals/KStreamBranch.java | 28 +++++------ .../kstream/internals/KStreamFilter.java | 19 ++++---- .../kstream/internals/KStreamFlatMap.java | 25 +++++----- .../internals/KStreamFlatMapValues.java | 25 +++++----- .../kstream/internals/KStreamImpl.java | 5 +- .../streams/kstream/internals/KStreamMap.java | 25 +++++----- .../kstream/internals/KStreamMapValues.java | 23 ++++----- .../kstream/internals/KStreamPeek.java | 25 +++++----- .../kstream/internals/KStreamPrint.java | 18 +++---- .../streams/kstream/internals/KTableImpl.java | 2 +- .../kstream/internals/PassThrough.java | 17 +++---- .../kstream/internals/PrintedInternal.java | 4 +- .../processor/api/ContextualProcessor.java | 47 +++++++++++++++++++ .../kafka/streams/kstream/PrintedTest.java | 34 ++++++-------- .../kstream/internals/KStreamBranchTest.java | 4 +- .../kstream/internals/KStreamPrintTest.java | 13 ++--- 17 files changed, 217 insertions(+), 131 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/ForeachProcessor.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/api/ContextualProcessor.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachProcessor.java new file mode 100644 index 0000000000000..cccd298cd5dca --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachProcessor.java @@ -0,0 +1,34 @@ +/* + * 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.streams.kstream; + +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.Record; + +public class ForeachProcessor implements Processor { + + private final ForeachAction action; + + public ForeachProcessor(final ForeachAction action) { + this.action = action; + } + + @Override + public void process(final Record record) { + action.apply(record.key(), record.value()); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java index 21b69f2322492..2d3fc76ab49d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java @@ -16,44 +16,44 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; -import org.apache.kafka.streams.processor.To; - import java.util.List; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class KStreamBranch implements ProcessorSupplier { +class KStreamBranch implements ProcessorSupplier { private final List> predicates; private final List childNodes; KStreamBranch(final List> predicates, - final List childNodes) { + final List childNodes) { this.predicates = predicates; this.childNodes = childNodes; } @Override - public Processor get() { + public Processor get() { return new KStreamBranchProcessor(); } - private class KStreamBranchProcessor extends AbstractProcessor { + private class KStreamBranchProcessor extends ContextualProcessor { + @Override - public void process(final K key, final V value) { + public void process(final Record record) { for (int i = 0; i < predicates.size(); i++) { - if (predicates.get(i).test(key, value)) { + if (predicates.get(i).test(record.key(), record.value())) { // use forward with child here and then break the loop // so that no record is going to be piped to multiple streams - context().forward(key, value, To.child(childNodes.get(i))); + context().forward(record, childNodes.get(i)); return; } } // using default child node if supplied if (childNodes.size() > predicates.size()) { - context().forward(key, value, To.child(childNodes.get(predicates.size()))); + context().forward(record, childNodes.get(predicates.size())); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java index ac03c18351bbf..ffafd10b460c4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java @@ -16,12 +16,13 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class KStreamFilter implements ProcessorSupplier { +class KStreamFilter implements ProcessorSupplier { private final Predicate predicate; private final boolean filterNot; @@ -32,15 +33,15 @@ public KStreamFilter(final Predicate predicate, final boolean filterNot) { } @Override - public Processor get() { + public Processor get() { return new KStreamFilterProcessor(); } - private class KStreamFilterProcessor extends AbstractProcessor { + private class KStreamFilterProcessor extends ContextualProcessor { @Override - public void process(final K key, final V value) { - if (filterNot ^ predicate.test(key, value)) { - context().forward(key, value); + public void process(final Record record) { + if (filterNot ^ predicate.test(record.key(), record.value())) { + context().forward(record); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java index e20ec90aa7a5a..b731a5c1c4f48 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java @@ -18,28 +18,31 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class KStreamFlatMap implements ProcessorSupplier { +class KStreamFlatMap implements ProcessorSupplier { - private final KeyValueMapper>> mapper; + private final KeyValueMapper>> mapper; - KStreamFlatMap(final KeyValueMapper>> mapper) { + KStreamFlatMap(final KeyValueMapper>> mapper) { this.mapper = mapper; } @Override - public Processor get() { + public Processor get() { return new KStreamFlatMapProcessor(); } - private class KStreamFlatMapProcessor extends AbstractProcessor { + private class KStreamFlatMapProcessor extends ContextualProcessor { @Override - public void process(final K key, final V value) { - for (final KeyValue newPair : mapper.apply(key, value)) { - context().forward(newPair.key, newPair.value); + public void process(final Record record) { + final Iterable> newKeyValues = + mapper.apply(record.key(), record.value()); + for (final KeyValue newPair : newKeyValues) { + context().forward(record.withKey(newPair.key).withValue(newPair.value)); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java index fedfe393c63b2..1008b297b3d3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -17,29 +17,30 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class KStreamFlatMapValues implements ProcessorSupplier { +class KStreamFlatMapValues implements ProcessorSupplier { - private final ValueMapperWithKey> mapper; + private final ValueMapperWithKey> mapper; - KStreamFlatMapValues(final ValueMapperWithKey> mapper) { + KStreamFlatMapValues(final ValueMapperWithKey> mapper) { this.mapper = mapper; } @Override - public Processor get() { + public Processor get() { return new KStreamFlatMapValuesProcessor(); } - private class KStreamFlatMapValuesProcessor extends AbstractProcessor { + private class KStreamFlatMapValuesProcessor extends ContextualProcessor { @Override - public void process(final K key, final V value) { - final Iterable newValues = mapper.apply(key, value); - for (final V1 v : newValues) { - context().forward(key, v); + public void process(final Record record) { + final Iterable newValues = mapper.apply(record.key(), record.value()); + for (final VOut v : newValues) { + context().forward(record.withValue(v)); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index bacddc5e70cfc..c013bd63a916f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -61,6 +61,7 @@ import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopicNameExtractor; +import org.apache.kafka.streams.kstream.ForeachProcessor; import org.apache.kafka.streams.processor.internals.InternalTopicProperties; import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor; import org.apache.kafka.streams.state.KeyValueStore; @@ -406,7 +407,7 @@ public void foreach(final ForeachAction action, final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, FOREACH_NAME); final ProcessorParameters processorParameters = - new ProcessorParameters<>(new KStreamPeek<>(action, false), name); + new ProcessorParameters<>(() -> new ForeachProcessor<>(action), name); final ProcessorGraphNode foreachNode = new ProcessorGraphNode<>(name, processorParameters); @@ -426,7 +427,7 @@ public KStream peek(final ForeachAction action, final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, PEEK_NAME); final ProcessorParameters processorParameters = - new ProcessorParameters<>(new KStreamPeek<>(action, true), name); + new ProcessorParameters<>(new KStreamPeek<>(action), name); final ProcessorGraphNode peekNode = new ProcessorGraphNode<>(name, processorParameters); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java index 8179ca8fe64f5..0fec71630383e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java @@ -16,30 +16,33 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class KStreamMap implements ProcessorSupplier { +class KStreamMap implements ProcessorSupplier { - private final KeyValueMapper> mapper; + private final KeyValueMapper> mapper; - public KStreamMap(final KeyValueMapper> mapper) { + public KStreamMap(final KeyValueMapper> mapper) { this.mapper = mapper; } @Override - public Processor get() { + public Processor get() { return new KStreamMapProcessor(); } - private class KStreamMapProcessor extends AbstractProcessor { + private class KStreamMapProcessor extends ContextualProcessor { + @Override - public void process(final K key, final V value) { - final KeyValue newPair = mapper.apply(key, value); - context().forward(newPair.key, newPair.value); + public void process(final Record record) { + final KeyValue newPair = + mapper.apply(record.key(), record.value()); + context().forward(record.withKey(newPair.key).withValue(newPair.value)); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java index 28c120e3d6221..f73bfdd53bc33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java @@ -17,28 +17,29 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ValueMapperWithKey; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class KStreamMapValues implements ProcessorSupplier { +class KStreamMapValues implements ProcessorSupplier { - private final ValueMapperWithKey mapper; + private final ValueMapperWithKey mapper; - public KStreamMapValues(final ValueMapperWithKey mapper) { + public KStreamMapValues(final ValueMapperWithKey mapper) { this.mapper = mapper; } @Override - public Processor get() { + public Processor get() { return new KStreamMapProcessor(); } - private class KStreamMapProcessor extends AbstractProcessor { + private class KStreamMapProcessor extends ContextualProcessor { @Override - public void process(final K readOnlyKey, final V value) { - final V1 newValue = mapper.apply(readOnlyKey, value); - context().forward(readOnlyKey, newValue); + public void process(final Record record) { + final VOut newValue = mapper.apply(record.key(), record.value()); + context().forward(record.withValue(newValue)); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java index 44d1d60ead821..69b5e7fc3316f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPeek.java @@ -17,32 +17,29 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ForeachAction; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class KStreamPeek implements ProcessorSupplier { +class KStreamPeek implements ProcessorSupplier { - private final boolean forwardDownStream; private final ForeachAction action; - public KStreamPeek(final ForeachAction action, final boolean forwardDownStream) { + public KStreamPeek(final ForeachAction action) { this.action = action; - this.forwardDownStream = forwardDownStream; } @Override - public Processor get() { + public Processor get() { return new KStreamPeekProcessor(); } - private class KStreamPeekProcessor extends AbstractProcessor { + private class KStreamPeekProcessor extends ContextualProcessor { @Override - public void process(final K key, final V value) { - action.apply(key, value); - if (forwardDownStream) { - context().forward(key, value); - } + public void process(final Record record) { + action.apply(record.key(), record.value()); + context().forward(record); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPrint.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPrint.java index 8ce698ab54631..a04662cbb8706 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPrint.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPrint.java @@ -17,28 +17,28 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.ForeachAction; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -public class KStreamPrint implements ProcessorSupplier { +public class KStreamPrint implements ProcessorSupplier { private final ForeachAction action; - + public KStreamPrint(final ForeachAction action) { this.action = action; } @Override - public Processor get() { + public Processor get() { return new KStreamPrintProcessor(); } - private class KStreamPrintProcessor extends AbstractProcessor { + private class KStreamPrintProcessor implements Processor { @Override - public void process(final K key, final V value) { - action.apply(key, value); + public void process(final Record record) { + action.apply(record.key(), record.value()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 98aee1304be8b..52f7b5f02f2b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -484,7 +484,7 @@ public KStream toStream(final Named named) { Objects.requireNonNull(named, "named can't be null"); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, TOSTREAM_NAME); - final ProcessorSupplier> kStreamMapValues = new KStreamMapValues<>((key, change) -> change.newValue); + final KStreamMapValues, V> kStreamMapValues = new KStreamMapValues<>((key, change) -> change.newValue); final ProcessorParameters processorParameters = unsafeCastProcessorParametersToCompletelyDifferentType( new ProcessorParameters<>(kStreamMapValues, name) ); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PassThrough.java index b83b3a405f43d..f357a464c2b23 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PassThrough.java @@ -16,21 +16,22 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ContextualProcessor; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; -class PassThrough implements ProcessorSupplier { +class PassThrough implements ProcessorSupplier { @Override - public Processor get() { + public Processor get() { return new PassThroughProcessor<>(); } - private static final class PassThroughProcessor extends AbstractProcessor { + private static final class PassThroughProcessor extends ContextualProcessor { @Override - public void process(final K key, final V value) { - context().forward(key, value); + public void process(final Record record) { + context().forward(record); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PrintedInternal.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PrintedInternal.java index 546e35321b2ca..0cd1760e69af8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PrintedInternal.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/PrintedInternal.java @@ -17,14 +17,14 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.kstream.Printed; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; public class PrintedInternal extends Printed { public PrintedInternal(final Printed printed) { super(printed); } - public ProcessorSupplier build(final String processorName) { + public ProcessorSupplier build(final String processorName) { return new KStreamPrint<>(new PrintForeachAction<>(outputStream, mapper, label != null ? label : processorName)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/api/ContextualProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/api/ContextualProcessor.java new file mode 100644 index 0000000000000..d2522e3e075df --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/api/ContextualProcessor.java @@ -0,0 +1,47 @@ +/* + * 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.streams.processor.api; + +/** + * An abstract implementation of {@link Processor} that manages the {@link ProcessorContext} instance and provides default no-op + * implementation of {@link #close()}. + * + * @param the type of input keys + * @param the type of input values + * @param the type of output keys + * @param the type of output values + */ +public abstract class ContextualProcessor implements Processor { + + protected ProcessorContext context; + + protected ContextualProcessor() {} + + @Override + public void init(final ProcessorContext context) { + this.context = context; + } + + /** + * Get the processor's context set during {@link #init(ProcessorContext) initialization}. + * + * @return the processor context; null only when called prior to {@link #init(ProcessorContext) initialization}. + */ + protected final ProcessorContext context() { + return context; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/PrintedTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/PrintedTest.java index a56dbbabc387f..212074fd23614 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/PrintedTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/PrintedTest.java @@ -19,8 +19,9 @@ import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.kstream.internals.PrintedInternal; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; @@ -59,11 +60,11 @@ public void after() { @Test public void shouldCreateProcessorThatPrintsToFile() throws IOException { final File file = TestUtils.tempFile(); - final ProcessorSupplier processorSupplier = new PrintedInternal<>( + final ProcessorSupplier processorSupplier = new PrintedInternal<>( Printed.toFile(file.getPath())) .build("processor"); - final Processor processor = processorSupplier.get(); - processor.process("hi", 1); + final Processor processor = processorSupplier.get(); + processor.process(new Record<>("hi", 1, 0L)); processor.close(); try (final InputStream stream = Files.newInputStream(file.toPath())) { final byte[] data = new byte[stream.available()]; @@ -74,36 +75,31 @@ public void shouldCreateProcessorThatPrintsToFile() throws IOException { @Test public void shouldCreateProcessorThatPrintsToStdOut() throws UnsupportedEncodingException { - final ProcessorSupplier supplier = new PrintedInternal<>(sysOutPrinter).build("processor"); - final Processor processor = supplier.get(); + final ProcessorSupplier supplier = new PrintedInternal<>(sysOutPrinter).build("processor"); + final Processor processor = supplier.get(); - processor.process("good", 2); + processor.process(new Record<>("good", 2, 0L)); processor.close(); assertThat(sysOut.toString(StandardCharsets.UTF_8.name()), equalTo("[processor]: good, 2\n")); } @Test public void shouldPrintWithLabel() throws UnsupportedEncodingException { - final Processor processor = new PrintedInternal<>(sysOutPrinter.withLabel("label")) + final Processor processor = new PrintedInternal<>(sysOutPrinter.withLabel("label")) .build("processor") .get(); - processor.process("hello", 3); + processor.process(new Record<>("hello", 3, 0L)); processor.close(); assertThat(sysOut.toString(StandardCharsets.UTF_8.name()), equalTo("[label]: hello, 3\n")); } @Test public void shouldPrintWithKeyValueMapper() throws UnsupportedEncodingException { - final Processor processor = new PrintedInternal<>(sysOutPrinter.withKeyValueMapper( - new KeyValueMapper() { - @Override - public String apply(final String key, final Integer value) { - return String.format("%s -> %d", key, value); - } - })).build("processor") - .get(); - processor.process("hello", 1); + final Processor processor = new PrintedInternal<>( + sysOutPrinter.withKeyValueMapper((key, value) -> String.format("%s -> %d", key, value)) + ).build("processor").get(); + processor.process(new Record<>("hello", 1, 0L)); processor.close(); assertThat(sysOut.toString(StandardCharsets.UTF_8.name()), equalTo("[processor]: hello -> 1\n")); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index e33186e805b53..1813522bcce2c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -60,8 +60,8 @@ public void testKStreamBranch() { assertEquals(3, branches.length); final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); - for (int i = 0; i < branches.length; i++) { - branches[i].process(supplier); + for (final KStream branch : branches) { + branch.process(supplier); } try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java index 990655612dd37..2915a118792a0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamPrintTest.java @@ -17,8 +17,9 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.api.Record; import org.easymock.EasyMock; import org.junit.Before; import org.junit.Test; @@ -33,7 +34,7 @@ public class KStreamPrintTest { private ByteArrayOutputStream byteOutStream; - private Processor printProcessor; + private Processor printProcessor; @Before public void setUp() { @@ -45,14 +46,13 @@ public void setUp() { "test-stream")); printProcessor = kStreamPrint.get(); - final ProcessorContext processorContext = EasyMock.createNiceMock(ProcessorContext.class); + final ProcessorContext processorContext = EasyMock.createNiceMock(ProcessorContext.class); EasyMock.replay(processorContext); printProcessor.init(processorContext); } @Test - @SuppressWarnings("unchecked") public void testPrintStreamWithProvidedKeyValueMapper() { final List> inputRecords = Arrays.asList( new KeyValue<>(0, "zero"), @@ -67,7 +67,8 @@ public void testPrintStreamWithProvidedKeyValueMapper() { "[test-stream]: 3, three"}; for (final KeyValue record: inputRecords) { - printProcessor.process(record.key, record.value); + final Record r = new Record<>(record.key, record.value, 0L); + printProcessor.process(r); } printProcessor.close(); From 872b44455c76affc80837b2ef8971753803edc86 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 9 Apr 2021 14:58:44 -0700 Subject: [PATCH 039/155] HOTFIX: delete removed WindowedStore.put() method (#10517) Reviewers: Boyang Chen --- .../RocksDBTimeOrderedWindowStore.java | 24 ------------------- 1 file changed, 24 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java index 9316d0cbbf6bc..3efeb321ee31a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java @@ -19,10 +19,6 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; @@ -44,7 +40,6 @@ public class RocksDBTimeOrderedWindowStore private final boolean retainDuplicates; private final long windowSize; - private InternalProcessorContext context; private int seqnum = 0; RocksDBTimeOrderedWindowStore(final SegmentedBytesStore bytesStore, @@ -55,25 +50,6 @@ public class RocksDBTimeOrderedWindowStore this.windowSize = windowSize; } - @Deprecated - @Override - public void init(final ProcessorContext context, final StateStore root) { - this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; - super.init(context, root); - } - - @Override - public void init(final StateStoreContext context, final StateStore root) { - this.context = context instanceof InternalProcessorContext ? (InternalProcessorContext) context : null; - super.init(context, root); - } - - @Deprecated - @Override - public void put(final Bytes key, final byte[] value) { - put(key, value, context != null ? context.timestamp() : 0L); - } - @Override public void put(final Bytes key, final byte[] value, final long timestamp) { // Skip if value is null and duplicates are allowed since this delete is a no-op From db688b1a5e5904eed5849fbab4940585e5a3d646 Mon Sep 17 00:00:00 2001 From: dengziming Date: Sat, 10 Apr 2021 10:15:11 +0800 Subject: [PATCH 040/155] KAFKA-12607; Test case for resigned state vote granting (#10510) This patch adds unit tests to verify vote behavior when in the "resigned" state. Reviewers: Jason Gustafson --- .../org/apache/kafka/raft/CandidateState.java | 2 +- .../kafka/raft/KafkaRaftClientTest.java | 112 ++++++++++++++++++ 2 files changed, 113 insertions(+), 1 deletion(-) diff --git a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java index 001fc19c11358..e9e1e0e0a69d7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/CandidateState.java +++ b/raft/src/main/java/org/apache/kafka/raft/CandidateState.java @@ -251,7 +251,7 @@ public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { @Override public String toString() { - return "Candidate(" + + return "CandidateState(" + "localId=" + localId + ", epoch=" + epoch + ", retries=" + retries + diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 54c38ddd3ef12..55d4e16dc86e7 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -137,6 +137,118 @@ public void testRejectVotesFromSameEpochAfterResigningCandidacy() throws Excepti context.assertSentVoteResponse(Errors.NONE, epoch, OptionalInt.empty(), false); } + @Test + public void testGrantVotesFromHigherEpochAfterResigningLeadership() throws Exception { + int localId = 0; + int remoteId = 1; + Set voters = Utils.mkSet(localId, remoteId); + int epoch = 2; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .updateRandom(random -> { + Mockito.doReturn(0).when(random).nextInt(DEFAULT_ELECTION_TIMEOUT_MS); + }) + .withElectedLeader(epoch, localId) + .build(); + + // Resign from leader, will restart in resigned state + assertTrue(context.client.quorum().isResigned()); + assertEquals(0L, context.log.endOffset().offset); + context.assertElectedLeader(epoch, localId); + + // Send vote request with higher epoch + context.deliverRequest(context.voteRequest(epoch + 1, remoteId, + context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.client.poll(); + + // We will first transition to unattached and then grant vote and then transition to voted + assertTrue(context.client.quorum().isVoted()); + context.assertVotedCandidate(epoch + 1, remoteId); + context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); + } + + @Test + public void testGrantVotesFromHigherEpochAfterResigningCandidacy() throws Exception { + int localId = 0; + int remoteId = 1; + Set voters = Utils.mkSet(localId, remoteId); + int epoch = 2; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .updateRandom(random -> { + Mockito.doReturn(0).when(random).nextInt(DEFAULT_ELECTION_TIMEOUT_MS); + }) + .withVotedCandidate(epoch, localId) + .build(); + + // Resign from candidate, will restart in candidate state + assertTrue(context.client.quorum().isCandidate()); + assertEquals(0L, context.log.endOffset().offset); + context.assertVotedCandidate(epoch, localId); + + // Send vote request with higher epoch + context.deliverRequest(context.voteRequest(epoch + 1, remoteId, + context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.client.poll(); + + // We will first transition to unattached and then grant vote and then transition to voted + assertTrue(context.client.quorum().isVoted()); + context.assertVotedCandidate(epoch + 1, remoteId); + context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); + } + + @Test + public void testGrantVotesWhenShuttingDown() throws Exception { + int localId = 0; + int remoteId = 1; + Set voters = Utils.mkSet(localId, remoteId); + int epoch = 2; + + RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch); + + // Beginning shutdown + context.client.shutdown(1000); + assertTrue(context.client.isShuttingDown()); + + // Send vote request with higher epoch + context.deliverRequest(context.voteRequest(epoch + 1, remoteId, + context.log.lastFetchedEpoch(), context.log.endOffset().offset)); + context.client.poll(); + + // We will first transition to unattached and then grant vote and then transition to voted + assertTrue(context.client.quorum().isVoted()); + context.assertVotedCandidate(epoch + 1, remoteId); + context.assertSentVoteResponse(Errors.NONE, epoch + 1, OptionalInt.empty(), true); + } + + @Test + public void testInitializeAsResignedAndBecomeCandidate() throws Exception { + int localId = 0; + int remoteId = 1; + Set voters = Utils.mkSet(localId, remoteId); + int epoch = 2; + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .updateRandom(random -> { + Mockito.doReturn(0).when(random).nextInt(DEFAULT_ELECTION_TIMEOUT_MS); + }) + .withElectedLeader(epoch, localId) + .build(); + + // Resign from leader, will restart in resigned state + assertTrue(context.client.quorum().isResigned()); + assertEquals(0L, context.log.endOffset().offset); + context.assertElectedLeader(epoch, localId); + + // Election timeout + context.time.sleep(context.electionTimeoutMs()); + context.client.poll(); + + // Become candidate in a new epoch + assertTrue(context.client.quorum().isCandidate()); + context.assertVotedCandidate(epoch + 1, localId); + } + @Test public void testInitializeAsResignedLeaderFromStateStore() throws Exception { int localId = 0; From 2004f5b2cac733b33e776311ad0144b03aca40e4 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 11 Apr 2021 22:48:42 -0700 Subject: [PATCH 041/155] MINOR: Use `testRuntimeOnly` instead of `testRuntime` in storage modules (#10524) `testRuntime` is deprecated in Gradle 6.x and has been removed in Gradle 7.0. Reviewers: Chia-Ping Tsai , Satish Duggana --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index ac8f9e3d47154..0ae4094e38e10 100644 --- a/build.gradle +++ b/build.gradle @@ -1347,7 +1347,7 @@ project(':storage:api') { testImplementation libs.junitJupiter testImplementation libs.mockitoCore - testRuntime libs.slf4jlog4j + testRuntimeOnly libs.slf4jlog4j } task createVersionFile(dependsOn: determineCommitId) { @@ -1409,7 +1409,7 @@ project(':storage') { testImplementation libs.junitJupiter testImplementation libs.mockitoCore - testRuntime libs.slf4jlog4j + testRuntimeOnly libs.slf4jlog4j } task createVersionFile(dependsOn: determineCommitId) { From 2899200c9741a647f89808f212267a689b17ebaf Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 11 Apr 2021 22:49:33 -0700 Subject: [PATCH 042/155] MINOR: Remove gradleSetup from Jenkinsfile (#10522) We no longer need this since: 1. The PR and branch jobs are configured to `clean before checkout`. 2. The Gradle build outputs the gradle version on start-up. The description of `clean before checkout` is: > Clean up the workspace before every checkout by deleting all untracked files and directories, including those which are specified in .gitignore. It also resets all tracked files to their versioned state. This ensures that the workspace is in the same state as if you cloned and checked out in a brand-new empty directory, and ensures that your build is not affected by the files generated by the previous build. Reviewers: Chia-Ping Tsai --- Jenkinsfile | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/Jenkinsfile b/Jenkinsfile index c9ea085b8f3c5..a966bf26d94b2 100644 --- a/Jenkinsfile +++ b/Jenkinsfile @@ -17,14 +17,6 @@ * */ -def setupGradle() { - // Delete gradle cache to workaround cache corruption bugs, see KAFKA-3167 - dir('.gradle') { - deleteDir() - } - sh './gradlew -version' -} - def doValidation() { sh """ ./gradlew -PscalaVersion=$SCALA_VERSION clean compileJava compileScala compileTestJava compileTestScala \ @@ -125,7 +117,6 @@ pipeline { SCALA_VERSION=2.12 } steps { - setupGradle() doValidation() doTest(env) tryStreamsArchetype() @@ -145,7 +136,6 @@ pipeline { SCALA_VERSION=2.13 } steps { - setupGradle() doValidation() doTest(env) echo 'Skipping Kafka Streams archetype test for Java 11' @@ -165,7 +155,6 @@ pipeline { SCALA_VERSION=2.13 } steps { - setupGradle() doValidation() doTest(env) echo 'Skipping Kafka Streams archetype test for Java 15' @@ -182,7 +171,6 @@ pipeline { SCALA_VERSION=2.12 } steps { - setupGradle() doValidation() catchError(buildResult: 'SUCCESS', stageResult: 'FAILURE') { doTest(env, 'unitTest') @@ -214,7 +202,6 @@ pipeline { SCALA_VERSION=2.13 } steps { - setupGradle() doValidation() doTest(env) tryStreamsArchetype() @@ -238,7 +225,6 @@ pipeline { SCALA_VERSION=2.12 } steps { - setupGradle() doValidation() doTest(env) echo 'Skipping Kafka Streams archetype test for Java 11' @@ -262,7 +248,6 @@ pipeline { SCALA_VERSION=2.12 } steps { - setupGradle() doValidation() doTest(env) echo 'Skipping Kafka Streams archetype test for Java 15' From 73df36d2412b980dd92b4dfc2a38a0ee4fd0fe57 Mon Sep 17 00:00:00 2001 From: dengziming Date: Mon, 12 Apr 2021 17:49:24 +0800 Subject: [PATCH 043/155] MINOR: Remove some unnecessary cyclomatic complexity suppressions (#10488) Reviewers: Chia-Ping Tsai --- checkstyle/suppressions.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 7a2cd22eaa166..fa1f5cc103f47 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -68,7 +68,7 @@ files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/> + files="(ConsumerCoordinator|Fetcher|KafkaProducer|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer).java"/> @@ -127,7 +127,7 @@ + files="(JsonConverter|Values|ConnectHeaders).java"/> From 0b0eaefc60a39aac2f6e90a0eff87deff0f77bf2 Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Mon, 12 Apr 2021 23:35:15 +0900 Subject: [PATCH 044/155] KAFKA-12408: Document omitted ReplicaManager metrics (#10258) Reviewers: Tom Bentley --- docs/ops.html | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/ops.html b/docs/ops.html index 6420652211487..8c157b89d60d7 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -1578,6 +1578,11 @@

    taskFuture = executor.submit(workerTask); + assertTrue(awaitLatch(pollLatch)); + //Failure in poll should trigger automatic stop of the worker + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPollAfterCancel() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + final CountDownLatch pollLatch = new CountDownLatch(1); + final CountDownLatch workerCancelLatch = new CountDownLatch(1); + final RuntimeException exception = new RuntimeException(); + EasyMock.expect(sourceTask.poll()).andAnswer(() -> { + pollLatch.countDown(); + assertTrue(awaitLatch(workerCancelLatch)); + throw exception; + }); + + offsetReader.close(); + PowerMock.expectLastCall(); + + producer.close(Duration.ZERO); + PowerMock.expectLastCall(); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.cancel(); + workerCancelLatch.countDown(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPollAfterStop() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + final CountDownLatch pollLatch = new CountDownLatch(1); + final CountDownLatch workerStopLatch = new CountDownLatch(1); + final RuntimeException exception = new RuntimeException(); + EasyMock.expect(sourceTask.poll()).andAnswer(() -> { + pollLatch.countDown(); + assertTrue(awaitLatch(workerStopLatch)); + throw exception; + }); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + assertTrue(awaitLatch(pollLatch)); workerTask.stop(); + workerStopLatch.countDown(); assertTrue(workerTask.awaitStop(1000)); taskFuture.get(); From e677782112743d16eca411dd1e1f905f67956da1 Mon Sep 17 00:00:00 2001 From: Marco Aurelio Lotz Date: Mon, 12 Apr 2021 18:28:23 +0200 Subject: [PATCH 046/155] KAFKA-9527: fix NPE when using time-based argument for Stream Resetter Tool (#10042) Reviewers: Jorge Esteban Quilcate Otoya , Matthias J. Sax --- .../scala/kafka/tools/StreamsResetter.java | 20 ++++-- .../streams/tools/StreamsResetterTest.java | 61 +++++++++++++++++++ 2 files changed, 77 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/tools/StreamsResetter.java b/core/src/main/scala/kafka/tools/StreamsResetter.java index 993fd4c7d960b..063607f6e63fa 100644 --- a/core/src/main/scala/kafka/tools/StreamsResetter.java +++ b/core/src/main/scala/kafka/tools/StreamsResetter.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.requests.ListOffsetsResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; @@ -52,6 +53,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Properties; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -492,9 +494,10 @@ private void resetByDuration(final Consumer client, resetToDatetime(client, inputTopicPartitions, Instant.now().minus(duration).toEpochMilli()); } - private void resetToDatetime(final Consumer client, - final Set inputTopicPartitions, - final Long timestamp) { + // visible for testing + public void resetToDatetime(final Consumer client, + final Set inputTopicPartitions, + final Long timestamp) { final Map topicPartitionsAndTimes = new HashMap<>(inputTopicPartitions.size()); for (final TopicPartition topicPartition : inputTopicPartitions) { topicPartitionsAndTimes.put(topicPartition, timestamp); @@ -503,7 +506,16 @@ private void resetToDatetime(final Consumer client, final Map topicPartitionsAndOffset = client.offsetsForTimes(topicPartitionsAndTimes); for (final TopicPartition topicPartition : inputTopicPartitions) { - client.seek(topicPartition, topicPartitionsAndOffset.get(topicPartition).offset()); + final Optional partitionOffset = Optional.ofNullable(topicPartitionsAndOffset.get(topicPartition)) + .map(OffsetAndTimestamp::offset) + .filter(offset -> offset != ListOffsetsResponse.UNKNOWN_OFFSET); + if (partitionOffset.isPresent()) { + client.seek(topicPartition, partitionOffset.get()); + } else { + client.seekToEnd(Collections.singletonList(topicPartition)); + System.out.println("Partition " + topicPartition.partition() + " from topic " + topicPartition.topic() + + " is empty, without a committed record. Falling back to latest known offset."); + } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java index 1e2215dafb1c5..0383c3c379727 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; @@ -31,6 +32,7 @@ import org.junit.Test; import java.time.Duration; +import java.time.Instant; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -76,6 +78,28 @@ public void testResetToSpecificOffsetWhenBetweenBeginningAndEndOffset() { assertEquals(3, records.count()); } + @Test + public void testResetOffsetToSpecificOffsetWhenAfterEndOffset() { + final long beginningOffset = 5L; + final long endOffset = 10L; + final MockConsumer emptyConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + emptyConsumer.assign(Collections.singletonList(topicPartition)); + + final Map beginningOffsetsMap = new HashMap<>(); + beginningOffsetsMap.put(topicPartition, beginningOffset); + emptyConsumer.updateBeginningOffsets(beginningOffsetsMap); + + final Map endOffsetsMap = new HashMap<>(); + endOffsetsMap.put(topicPartition, endOffset); + emptyConsumer.updateEndOffsets(endOffsetsMap); + // resetOffsetsTo only seeks the offset, but does not commit. + streamsResetter.resetOffsetsTo(emptyConsumer, inputTopicPartitions, endOffset + 2L); + + final long position = emptyConsumer.position(topicPartition); + + assertEquals(endOffset, position); + } + @Test public void testResetToSpecificOffsetWhenBeforeBeginningOffset() { final Map endOffsets = new HashMap<>(); @@ -247,6 +271,29 @@ public void shouldDetermineInternalTopicBasedOnTopicName1() { assertTrue(streamsResetter.matchesInternalTopicFormat("appId-KTABLE-FK-JOIN-SUBSCRIPTION-REGISTRATION-12323232-topic")); } + @Test + public void testResetToDatetimeWhenPartitionIsEmptyResetsToLatestOffset() { + final long beginningAndEndOffset = 5L; // Empty partition implies beginning offset == end offset + final MockConsumer emptyConsumer = new EmptyPartitionConsumer<>(OffsetResetStrategy.EARLIEST); + emptyConsumer.assign(Collections.singletonList(topicPartition)); + + final Map beginningOffsetsMap = new HashMap<>(); + beginningOffsetsMap.put(topicPartition, beginningAndEndOffset); + emptyConsumer.updateBeginningOffsets(beginningOffsetsMap); + + final Map endOffsetsMap = new HashMap<>(); + endOffsetsMap.put(topicPartition, beginningAndEndOffset); + emptyConsumer.updateEndOffsets(endOffsetsMap); + + final long yesterdayTimestamp = Instant.now().minus(Duration.ofDays(1)).toEpochMilli(); + // resetToDatetime only seeks the offset, but does not commit. + streamsResetter.resetToDatetime(emptyConsumer, inputTopicPartitions, yesterdayTimestamp); + + final long position = emptyConsumer.position(topicPartition); + + assertEquals(beginningAndEndOffset, position); + } + private Cluster createCluster(final int numNodes) { final HashMap nodes = new HashMap<>(); for (int i = 0; i < numNodes; ++i) { @@ -257,4 +304,18 @@ private Cluster createCluster(final int numNodes) { Collections.emptySet(), nodes.get(0)); } + private static class EmptyPartitionConsumer extends MockConsumer { + + public EmptyPartitionConsumer(final OffsetResetStrategy offsetResetStrategy) { + super(offsetResetStrategy); + } + + @Override + public synchronized Map offsetsForTimes(final Map timestampsToSearch) { + final Map topicPartitionToOffsetAndTimestamp = new HashMap<>(); + timestampsToSearch.keySet().forEach(k -> topicPartitionToOffsetAndTimestamp.put(k, null)); + return topicPartitionToOffsetAndTimestamp; + } + } + } \ No newline at end of file From c972c4ea64d9b939e3ef6e8b8d6d8b4017e32c6e Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Mon, 12 Apr 2021 14:02:24 -0700 Subject: [PATCH 047/155] MINOR: Improve description of `max.poll.records` config (#10506) Reviewers: Jason Gustafson --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 93ff024031e51..72c3aee1408f6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -66,7 +66,9 @@ public class ConsumerConfig extends AbstractConfig { /** max.poll.records */ public static final String MAX_POLL_RECORDS_CONFIG = "max.poll.records"; - private static final String MAX_POLL_RECORDS_DOC = "The maximum number of records returned in a single call to poll()."; + private static final String MAX_POLL_RECORDS_DOC = "The maximum number of records returned in a single call to poll()." + + " Note, that " + MAX_POLL_RECORDS_CONFIG + " does not impact the underlying fetching behavior." + + " The consumer will cache the records from each fetch request and returns them incrementally from each poll."; /** max.poll.interval.ms */ public static final String MAX_POLL_INTERVAL_MS_CONFIG = CommonClientConfigs.MAX_POLL_INTERVAL_MS_CONFIG; From c608d8480edad982bd121fc8b70cd8347eff2e60 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Mon, 12 Apr 2021 18:24:18 -0700 Subject: [PATCH 048/155] KAFKA-7606: Remove deprecated options from StreamsResetter (#10411) Remove deprecated --zookeeper and --execute flags Reviewers: Matthias J. Sax --- .../src/main/scala/kafka/tools/StreamsResetter.java | 10 +--------- docs/streams/upgrade-guide.html | 4 +++- .../integration/AbstractResetIntegrationTest.java | 4 ++-- .../streams/integration/ResetIntegrationTest.java | 13 +++++-------- 4 files changed, 11 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/tools/StreamsResetter.java b/core/src/main/scala/kafka/tools/StreamsResetter.java index 063607f6e63fa..485bcdf838f50 100644 --- a/core/src/main/scala/kafka/tools/StreamsResetter.java +++ b/core/src/main/scala/kafka/tools/StreamsResetter.java @@ -105,7 +105,6 @@ public class StreamsResetter { private static OptionSpecBuilder dryRunOption; private static OptionSpec helpOption; private static OptionSpec versionOption; - private static OptionSpecBuilder executeOption; private static OptionSpec commandConfigOption; private static OptionSpecBuilder forceOption; @@ -249,14 +248,11 @@ private void parseArguments(final String[] args) { .describedAs("file name"); forceOption = optionParser.accepts("force", "Force the removal of members of the consumer group (intended to remove stopped members if a long session timeout was used). " + "Make sure to shut down all stream applications when this option is specified to avoid unexpected rebalances."); - executeOption = optionParser.accepts("execute", "Execute the command."); + dryRunOption = optionParser.accepts("dry-run", "Display the actions that would be performed without executing the reset commands."); helpOption = optionParser.accepts("help", "Print usage information.").forHelp(); versionOption = optionParser.accepts("version", "Print version information and exit.").forHelp(); - // TODO: deprecated in 1.0; can be removed eventually: https://issues.apache.org/jira/browse/KAFKA-7606 - optionParser.accepts("zookeeper", "Zookeeper option is deprecated by bootstrap.servers, as the reset tool would no longer access Zookeeper directly."); - try { options = optionParser.parse(args); if (args.length == 0 || options.has(helpOption)) { @@ -269,10 +265,6 @@ private void parseArguments(final String[] args) { CommandLineUtils.printUsageAndDie(optionParser, e.getMessage()); } - if (options.has(executeOption) && options.has(dryRunOption)) { - CommandLineUtils.printUsageAndDie(optionParser, "Only one of --dry-run and --execute can be specified"); - } - final Set> allScenarioOptions = new HashSet<>(); allScenarioOptions.add(toOffsetOption); allScenarioOptions.add(toDatetimeOption); diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index b0820357eb398..0269be9ebf95d 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -111,6 +111,8 @@

    Streams API
  • Overloaded KStream#join, leftJoin, outerJoin with KStream and Joined parameters: deprecated in Kafka 2.4.0 (KIP-479).
  • Overloaded KafkaStreams#metadataForKey: deprecated in Kafka 2.5.0 (KIP-535).
  • Overloaded KafkaStreams#store: deprecated in Kafka 2.5.0 (KIP-562).
  • +
  • --zookeeper flag of the application reset tool: deprecated in Kafka 1.0 (KIP-198).
  • +
  • --execute flag of the application reset tool: deprecated in Kafka 1.1 (KIP-171).
  • WindowStore#put(K key, V value): deprecated in Kafka 2.4.0 (KIP-474).
  • @@ -120,7 +122,7 @@

    Streams API
  • Connect-json: As of Kafka Streams no longer has a compile time dependency on "connect:json" module (KAFKA-5146). Projects that were relying on this transitive dependency will have to explicitly declare it.
  • - +

    Streams API changes in 2.8.0

    We extended StreamJoined to include the options withLoggingEnabled() and withLoggingDisabled() in diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java index 2aa1369405831..76c53b15443de 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java @@ -362,8 +362,8 @@ protected boolean tryCleanGlobal(final boolean withIntermediateTopics, final List parameterList = new ArrayList<>( Arrays.asList("--application-id", appID, "--bootstrap-servers", cluster.bootstrapServers(), - "--input-topics", INPUT_TOPIC, - "--execute")); + "--input-topics", INPUT_TOPIC + )); if (withIntermediateTopics) { parameterList.add("--intermediate-topics"); parameterList.add(INTERMEDIATE_USER_TOPIC); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java index bb041e68a10ce..f355afaa7286d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java @@ -101,8 +101,7 @@ public void shouldNotAllowToResetWhileStreamsIsRunning() { final String[] parameters = new String[] { "--application-id", appID, "--bootstrap-servers", cluster.bootstrapServers(), - "--input-topics", NON_EXISTING_TOPIC, - "--execute" + "--input-topics", NON_EXISTING_TOPIC }; final Properties cleanUpConfig = new Properties(); cleanUpConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); @@ -121,13 +120,12 @@ public void shouldNotAllowToResetWhileStreamsIsRunning() { } @Test - public void shouldNotAllowToResetWhenInputTopicAbsent() throws Exception { + public void shouldNotAllowToResetWhenInputTopicAbsent() { final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); final String[] parameters = new String[] { "--application-id", appID, "--bootstrap-servers", cluster.bootstrapServers(), - "--input-topics", NON_EXISTING_TOPIC, - "--execute" + "--input-topics", NON_EXISTING_TOPIC }; final Properties cleanUpConfig = new Properties(); cleanUpConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); @@ -138,13 +136,12 @@ public void shouldNotAllowToResetWhenInputTopicAbsent() throws Exception { } @Test - public void shouldNotAllowToResetWhenIntermediateTopicAbsent() throws Exception { + public void shouldNotAllowToResetWhenIntermediateTopicAbsent() { final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); final String[] parameters = new String[] { "--application-id", appID, "--bootstrap-servers", cluster.bootstrapServers(), - "--intermediate-topics", NON_EXISTING_TOPIC, - "--execute" + "--intermediate-topics", NON_EXISTING_TOPIC }; final Properties cleanUpConfig = new Properties(); cleanUpConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); From 88eb24db40cc47786b6c72fa9f77032281f6e1b0 Mon Sep 17 00:00:00 2001 From: dengziming Date: Tue, 13 Apr 2021 09:37:01 +0800 Subject: [PATCH 049/155] KAFKA-12637: Remove deprecated PartitionAssignor interface (#10512) Remove PartitionAssignor and related classes, update docs and move unit test Reviewers: Anna Sophie Blee-Goldman --- .../consumer/ConsumerPartitionAssignor.java | 41 +++++ .../kafka/clients/consumer/KafkaConsumer.java | 8 +- .../internals/AbstractCoordinator.java | 2 +- .../consumer/internals/PartitionAssignor.java | 147 --------------- .../internals/PartitionAssignorAdapter.java | 140 -------------- .../ConsumerPartitionAssignorTest.java | 109 +++++++++++ .../PartitionAssignorAdapterTest.java | 171 ------------------ docs/upgrade.html | 4 +- .../internals/StreamsPartitionAssignor.java | 2 +- 9 files changed, 159 insertions(+), 465 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapter.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java delete mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapterTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java index 9fd7673a025df..0704e33740373 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java @@ -17,13 +17,17 @@ package org.apache.kafka.clients.consumer; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Optional; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Utils; /** * This interface is used to define custom partition assignment for use in @@ -254,4 +258,41 @@ public static RebalanceProtocol forId(byte id) { } } + /** + * Get a list of configured instances of {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor} + * based on the class names/types specified by {@link org.apache.kafka.clients.consumer.ConsumerConfig#PARTITION_ASSIGNMENT_STRATEGY_CONFIG} + */ + static List getAssignorInstances(List assignorClasses, Map configs) { + List assignors = new ArrayList<>(); + + if (assignorClasses == null) + return assignors; + + for (Object klass : assignorClasses) { + // first try to get the class if passed in as a string + if (klass instanceof String) { + try { + klass = Class.forName((String) klass, true, Utils.getContextOrKafkaClassLoader()); + } catch (ClassNotFoundException classNotFound) { + throw new KafkaException(klass + " ClassNotFoundException exception occurred", classNotFound); + } + } + + if (klass instanceof Class) { + Object assignor = Utils.newInstance((Class) klass); + if (assignor instanceof Configurable) + ((Configurable) assignor).configure(configs); + + if (assignor instanceof ConsumerPartitionAssignor) { + assignors.add((ConsumerPartitionAssignor) assignor); + } else { + throw new KafkaException(klass + " is not an instance of " + ConsumerPartitionAssignor.class.getName()); + } + } else { + throw new KafkaException("List contains element of type " + klass.getClass().getName() + ", expected String or Class"); + } + } + return assignors; + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 460383c563e5c..1f4bc7ce97751 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -81,8 +81,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; -import static org.apache.kafka.clients.consumer.internals.PartitionAssignorAdapter.getAssignorInstances; - /** * A client that consumes records from a Kafka cluster. *

    @@ -768,8 +766,10 @@ public KafkaConsumer(Map configs, config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), heartbeatIntervalMs); //Will avoid blocking an extended period of time to prevent heartbeat thread starvation - this.assignors = getAssignorInstances(config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), - config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId))); + this.assignors = ConsumerPartitionAssignor.getAssignorInstances( + config.getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG), + config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)) + ); // no coordinator will be constructed for the default (null) group id this.coordinator = !groupId.isPresent() ? null : diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 82daa7bb1b37d..ba26427eb9633 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -396,7 +396,7 @@ private void closeHeartbeatThread() { * If this function returns false, the state can be in one of the following: * * UNJOINED: got error response but times out before being able to re-join, heartbeat disabled * * PREPARING_REBALANCE: not yet received join-group response before timeout, heartbeat disabled - * * COMPLETING_REBALANCE: not yet received sync-group response before timeout, hearbeat enabled + * * COMPLETING_REBALANCE: not yet received sync-group response before timeout, heartbeat enabled * * Visible for testing. * diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java deleted file mode 100644 index 43bd7519169d4..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * 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.clients.consumer.internals; - -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.TopicPartition; - -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Map; -import java.util.Set; - -/** - * This interface is used to define custom partition assignment for use in - * {@link org.apache.kafka.clients.consumer.KafkaConsumer}. Members of the consumer group subscribe - * to the topics they are interested in and forward their subscriptions to a Kafka broker serving - * as the group coordinator. The coordinator selects one member to perform the group assignment and - * propagates the subscriptions of all members to it. Then {@link #assign(Cluster, Map)} is called - * to perform the assignment and the results are forwarded back to each respective members - * - * In some cases, it is useful to forward additional metadata to the assignor in order to make - * assignment decisions. For this, you can override {@link #subscription(Set)} and provide custom - * userData in the returned Subscription. For example, to have a rack-aware assignor, an implementation - * can use this user data to forward the rackId belonging to each member. - * - * This interface has been deprecated in 2.4, custom assignors should now implement - * {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor}. Note that maintaining compatibility - * for an internal interface here is a special case, as {@code PartitionAssignor} was meant to be a public API - * although it was placed in the internals package. Users should not expect internal interfaces or classes to - * not be removed or maintain compatibility in any way. - */ -@Deprecated -public interface PartitionAssignor { - - /** - * Return a serializable object representing the local member's subscription. This can include - * additional information as well (e.g. local host/rack information) which can be leveraged in - * {@link #assign(Cluster, Map)}. - * @param topics Topics subscribed to through {@link org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(java.util.Collection)} - * and variants - * @return Non-null subscription with optional user data - */ - Subscription subscription(Set topics); - - /** - * Perform the group assignment given the member subscriptions and current cluster metadata. - * @param metadata Current topic/broker metadata known by consumer - * @param subscriptions Subscriptions from all members provided through {@link #subscription(Set)} - * @return A map from the members to their respective assignment. This should have one entry - * for all members who in the input subscription map. - */ - Map assign(Cluster metadata, Map subscriptions); - - /** - * Callback which is invoked when a group member receives its assignment from the leader. - * @param assignment The local member's assignment as provided by the leader in {@link #assign(Cluster, Map)} - */ - void onAssignment(Assignment assignment); - - /** - * Callback which is invoked when a group member receives its assignment from the leader. - * @param assignment The local member's assignment as provided by the leader in {@link #assign(Cluster, Map)} - * @param generation The consumer group generation associated with this partition assignment (optional) - */ - default void onAssignment(Assignment assignment, int generation) { - onAssignment(assignment); - } - - /** - * Unique name for this assignor (e.g. "range" or "roundrobin" or "sticky") - * @return non-null unique name - */ - String name(); - - class Subscription { - private final List topics; - private final ByteBuffer userData; - - public Subscription(List topics, ByteBuffer userData) { - this.topics = topics; - this.userData = userData; - } - - public Subscription(List topics) { - this(topics, ByteBuffer.wrap(new byte[0])); - } - - public List topics() { - return topics; - } - - public ByteBuffer userData() { - return userData; - } - - @Override - public String toString() { - return "Subscription(" + - "topics=" + topics + - ')'; - } - } - - class Assignment { - private final List partitions; - private final ByteBuffer userData; - - public Assignment(List partitions, ByteBuffer userData) { - this.partitions = partitions; - this.userData = userData; - } - - public Assignment(List partitions) { - this(partitions, ByteBuffer.wrap(new byte[0])); - } - - public List partitions() { - return partitions; - } - - public ByteBuffer userData() { - return userData; - } - - @Override - public String toString() { - return "Assignment(" + - "partitions=" + partitions + - ')'; - } - } - -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapter.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapter.java deleted file mode 100644 index 8fb791ab4faa8..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapter.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.clients.consumer.internals; - -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.Configurable; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.utils.Utils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This adapter class is used to ensure backwards compatibility for those who have implemented the {@link PartitionAssignor} - * interface, which has been deprecated in favor of the new {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor}. - *

    - * Note that maintaining compatibility for an internal interface here is a special case, as {@code PartitionAssignor} - * was meant to be a public API although it was placed in the internals package. Users should not expect internal - * interfaces or classes to not be removed or maintain compatibility in any way. - */ -@SuppressWarnings("deprecation") -public class PartitionAssignorAdapter implements ConsumerPartitionAssignor { - - private static final Logger LOG = LoggerFactory.getLogger(PartitionAssignorAdapter.class); - private final PartitionAssignor oldAssignor; - - PartitionAssignorAdapter(PartitionAssignor oldAssignor) { - this.oldAssignor = oldAssignor; - } - - @Override - public ByteBuffer subscriptionUserData(Set topics) { - return oldAssignor.subscription(topics).userData(); - } - - @Override - public GroupAssignment assign(Cluster metadata, GroupSubscription groupSubscription) { - return toNewGroupAssignment(oldAssignor.assign(metadata, toOldGroupSubscription(groupSubscription))); - } - - @Override - public void onAssignment(Assignment assignment, ConsumerGroupMetadata metadata) { - oldAssignor.onAssignment(toOldAssignment(assignment), metadata.generationId()); - } - - @Override - public String name() { - return oldAssignor.name(); - } - - private static PartitionAssignor.Assignment toOldAssignment(Assignment newAssignment) { - return new PartitionAssignor.Assignment(newAssignment.partitions(), newAssignment.userData()); - } - - private static Map toOldGroupSubscription(GroupSubscription newSubscriptions) { - Map oldSubscriptions = new HashMap<>(); - for (Map.Entry entry : newSubscriptions.groupSubscription().entrySet()) { - String member = entry.getKey(); - Subscription newSubscription = entry.getValue(); - oldSubscriptions.put(member, new PartitionAssignor.Subscription( - newSubscription.topics(), newSubscription.userData())); - } - return oldSubscriptions; - } - - private static GroupAssignment toNewGroupAssignment(Map oldAssignments) { - Map newAssignments = new HashMap<>(); - for (Map.Entry entry : oldAssignments.entrySet()) { - String member = entry.getKey(); - PartitionAssignor.Assignment oldAssignment = entry.getValue(); - newAssignments.put(member, new Assignment(oldAssignment.partitions(), oldAssignment.userData())); - } - return new GroupAssignment(newAssignments); - } - - /** - * Get a list of configured instances of {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor} - * based on the class names/types specified by {@link org.apache.kafka.clients.consumer.ConsumerConfig#PARTITION_ASSIGNMENT_STRATEGY_CONFIG} - * where any instances of the old {@link PartitionAssignor} interface are wrapped in an adapter to the new - * {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor} interface - */ - public static List getAssignorInstances(List assignorClasses, Map configs) { - List assignors = new ArrayList<>(); - - if (assignorClasses == null) - return assignors; - - for (Object klass : assignorClasses) { - // first try to get the class if passed in as a string - if (klass instanceof String) { - try { - klass = Class.forName((String) klass, true, Utils.getContextOrKafkaClassLoader()); - } catch (ClassNotFoundException classNotFound) { - throw new KafkaException(klass + " ClassNotFoundException exception occurred", classNotFound); - } - } - - if (klass instanceof Class) { - Object assignor = Utils.newInstance((Class) klass); - if (assignor instanceof Configurable) - ((Configurable) assignor).configure(configs); - - if (assignor instanceof ConsumerPartitionAssignor) { - assignors.add((ConsumerPartitionAssignor) assignor); - } else if (assignor instanceof PartitionAssignor) { - assignors.add(new PartitionAssignorAdapter((PartitionAssignor) assignor)); - LOG.warn("The PartitionAssignor interface has been deprecated, " - + "please implement the ConsumerPartitionAssignor interface instead."); - } else { - throw new KafkaException(klass + " is not an instance of " + PartitionAssignor.class.getName() - + " or an instance of " + ConsumerPartitionAssignor.class.getName()); - } - } else { - throw new KafkaException("List contains element of type " + klass.getClass().getName() + ", expected String or Class"); - } - } - return assignors; - } -} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java new file mode 100644 index 0000000000000..03070008c6b01 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java @@ -0,0 +1,109 @@ +/* + * 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.clients.consumer; + + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.getAssignorInstances; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ConsumerPartitionAssignorTest { + + @Test + public void shouldInstantiateAssignor() { + List assignors = getAssignorInstances( + Collections.singletonList(StickyAssignor.class.getName()), + Collections.emptyMap() + ); + assertTrue(assignors.get(0) instanceof StickyAssignor); + } + + @Test + public void shouldInstantiateListOfAssignors() { + List assignors = getAssignorInstances( + Arrays.asList(StickyAssignor.class.getName(), CooperativeStickyAssignor.class.getName()), + Collections.emptyMap() + ); + assertTrue(assignors.get(0) instanceof StickyAssignor); + assertTrue(assignors.get(1) instanceof CooperativeStickyAssignor); + } + + @Test + public void shouldThrowKafkaExceptionOnNonAssignor() { + assertThrows(KafkaException.class, () -> getAssignorInstances( + Collections.singletonList(String.class.getName()), + Collections.emptyMap()) + ); + } + + @Test + public void shouldThrowKafkaExceptionOnAssignorNotFound() { + assertThrows(KafkaException.class, () -> getAssignorInstances( + Collections.singletonList("Non-existent assignor"), + Collections.emptyMap()) + ); + } + + @Test + public void shouldInstantiateFromClassType() { + List classTypes = + initConsumerConfigWithClassTypes(Collections.singletonList(StickyAssignor.class)) + .getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG); + List assignors = getAssignorInstances(classTypes, Collections.emptyMap()); + assertTrue(assignors.get(0) instanceof StickyAssignor); + } + + @Test + public void shouldInstantiateFromListOfClassTypes() { + List classTypes = initConsumerConfigWithClassTypes( + Arrays.asList(StickyAssignor.class, CooperativeStickyAssignor.class) + ).getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG); + + List assignors = getAssignorInstances(classTypes, Collections.emptyMap()); + + assertTrue(assignors.get(0) instanceof StickyAssignor); + assertTrue(assignors.get(1) instanceof CooperativeStickyAssignor); + } + + @Test + public void shouldThrowKafkaExceptionOnListWithNonAssignorClassType() { + List classTypes = + initConsumerConfigWithClassTypes(Arrays.asList(StickyAssignor.class, String.class)) + .getList(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG); + + assertThrows(KafkaException.class, () -> getAssignorInstances(classTypes, Collections.emptyMap())); + } + + private ConsumerConfig initConsumerConfigWithClassTypes(List classTypes) { + Properties props = new Properties(); + props.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classTypes); + return new ConsumerConfig(props); + } +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapterTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapterTest.java deleted file mode 100644 index 7ce0e02dc823a..0000000000000 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/PartitionAssignorAdapterTest.java +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.clients.consumer.internals; - -import static org.apache.kafka.clients.consumer.internals.PartitionAssignorAdapter.getAssignorInstances; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Set; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.StickyAssignor; -import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.jupiter.api.Test; - -public class PartitionAssignorAdapterTest { - - private List classNames; - private List classTypes; - - @Test - public void shouldInstantiateNewAssignors() { - classNames = Arrays.asList(StickyAssignor.class.getName()); - List assignors = getAssignorInstances(classNames, Collections.emptyMap()); - assertTrue(StickyAssignor.class.isInstance(assignors.get(0))); - } - - @Test - public void shouldAdaptOldAssignors() { - classNames = Arrays.asList(OldPartitionAssignor.class.getName()); - List assignors = getAssignorInstances(classNames, Collections.emptyMap()); - assertTrue(PartitionAssignorAdapter.class.isInstance(assignors.get(0))); - } - - @Test - public void shouldThrowKafkaExceptionOnNonAssignor() { - classNames = Arrays.asList(String.class.getName()); - assertThrows(KafkaException.class, () -> getAssignorInstances(classNames, Collections.emptyMap())); - } - - @Test - public void shouldThrowKafkaExceptionOnAssignorNotFound() { - classNames = Arrays.asList("Non-existent assignor"); - assertThrows(KafkaException.class, () -> getAssignorInstances(classNames, Collections.emptyMap())); - } - - @Test - public void shouldInstantiateFromListOfOldAndNewClassTypes() { - Properties props = new Properties(); - props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - - classTypes = Arrays.asList(StickyAssignor.class, OldPartitionAssignor.class); - - props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classTypes); - KafkaConsumer consumer = new KafkaConsumer<>( - props, new StringDeserializer(), new StringDeserializer()); - - consumer.close(); - } - - @Test - public void shouldThrowKafkaExceptionOnListWithNonAssignorClassType() { - Properties props = new Properties(); - props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - - classTypes = Arrays.asList(StickyAssignor.class, OldPartitionAssignor.class, String.class); - - props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classTypes); - assertThrows(KafkaException.class, () -> new KafkaConsumer<>( - props, new StringDeserializer(), new StringDeserializer())); - } - - @Test - public void testOnAssignment() { - OldPartitionAssignor oldAssignor = new OldPartitionAssignor(); - ConsumerPartitionAssignor adaptedAssignor = new PartitionAssignorAdapter(oldAssignor); - - TopicPartition tp1 = new TopicPartition("tp1", 1); - TopicPartition tp2 = new TopicPartition("tp2", 2); - List partitions = Arrays.asList(tp1, tp2); - - adaptedAssignor.onAssignment(new Assignment(partitions), new ConsumerGroupMetadata("")); - - assertEquals(oldAssignor.partitions, partitions); - } - - @Test - public void testAssign() { - ConsumerPartitionAssignor adaptedAssignor = new PartitionAssignorAdapter(new OldPartitionAssignor()); - - Map subscriptions = new HashMap<>(); - subscriptions.put("C1", new Subscription(Arrays.asList("topic1"))); - subscriptions.put("C2", new Subscription(Arrays.asList("topic1", "topic2"))); - subscriptions.put("C3", new Subscription(Arrays.asList("topic2", "topic3"))); - GroupSubscription groupSubscription = new GroupSubscription(subscriptions); - - Map assignments = adaptedAssignor.assign(null, groupSubscription).groupAssignment(); - - assertEquals(assignments.get("C1").partitions(), Arrays.asList(new TopicPartition("topic1", 1))); - assertEquals(assignments.get("C2").partitions(), Arrays.asList(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1))); - assertEquals(assignments.get("C3").partitions(), Arrays.asList(new TopicPartition("topic2", 1), new TopicPartition("topic3", 1))); - } - - /* - * Dummy assignor just gives each consumer partition 1 of each topic it's subscribed to - */ - @SuppressWarnings("deprecation") - public static class OldPartitionAssignor implements PartitionAssignor { - - List partitions = null; - - @Override - public Subscription subscription(Set topics) { - return new Subscription(new ArrayList<>(topics), null); - } - - @Override - public Map assign(Cluster metadata, Map subscriptions) { - Map assignments = new HashMap<>(); - for (Map.Entry entry : subscriptions.entrySet()) { - List partitions = new ArrayList<>(); - for (String topic : entry.getValue().topics()) { - partitions.add(new TopicPartition(topic, 1)); - } - assignments.put(entry.getKey(), new Assignment(partitions, null)); - } - return assignments; - } - - @Override - public void onAssignment(Assignment assignment) { - partitions = assignment.partitions(); - } - - @Override - public String name() { - return "old-assignor"; - } - } - -} diff --git a/docs/upgrade.html b/docs/upgrade.html index 04505b4bed38e..0112bfc744d46 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -49,8 +49,10 @@
    Notable changes in 3 were removed. These methods were not intended to be public API and there is no replacement.
  • The NoOffsetForPartitionException.partition() method was removed. Please use partitions() instead.
  • -
  • The Scala kafka.common.MessageFormatter was removed. Plese use the Java org.apache.kafka.common.MessageFormatter.
  • +
  • The Scala kafka.common.MessageFormatter was removed. Please use the Java org.apache.kafka.common.MessageFormatter.
  • The MessageFormatter.init(Properties) method was removed. Please use configure(Map) instead.
  • +
  • The deprecated org.apache.kafka.clients.consumer.internals.PartitionAssignor class has been removed. Please use + org.apache.kafka.clients.consumer.ConsumerPartitionAssignor instead.
  • Kafka Streams no longer has a compile time dependency on "connect:json" module (KAFKA-5146). Projects that were relying on this transitive dependency will have to explicitly declare it.
  • The deprecated quota.producer.default and quota.consumer.default configurations were removed (KAFKA-12591). diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index ea6a7834a4d3d..2e517f1bcbb69 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -190,7 +190,7 @@ public String toString() { /** * We need to have the PartitionAssignor and its StreamThread to be mutually accessible since the former needs - * later's cached metadata while sending subscriptions, and the latter needs former's returned assignment when + * latter's cached metadata while sending subscriptions, and the latter needs former's returned assignment when * adding tasks. * * @throws KafkaException if the stream thread is not specified From 1e8a7c45efe77e25f39b6db775ad37c2f39b80ef Mon Sep 17 00:00:00 2001 From: Jim Hurne Date: Tue, 13 Apr 2021 00:05:49 -0400 Subject: [PATCH 050/155] MINOR: Switch to using the Gradle RAT plugin (#10491) The Gradle RAT plugin properly declares inputs and outputs and is also cachable. This also relieves the Kafka developers from maintaining the build integration with RAT. The generated RAT report is identical to the one generated previously. The only difference is the RAT report name: the RAT plugin sets the HTML report name to `index.html` (still under `build/rat`). Verified that the rat task fails if unlicensed files are present (and not excluded). Also `./gradlew rat` succeeds when there is no .git folder. --- build.gradle | 12 +++-- gradle/rat.gradle | 120 ---------------------------------------------- 2 files changed, 9 insertions(+), 123 deletions(-) delete mode 100644 gradle/rat.gradle diff --git a/build.gradle b/build.gradle index 0ae4094e38e10..db94e4996b973 100644 --- a/build.gradle +++ b/build.gradle @@ -35,6 +35,7 @@ plugins { id 'idea' id 'java-library' id 'org.owasp.dependencycheck' version '6.1.1' + id 'org.nosphere.apache.rat' version "0.7.0" id "com.github.spotbugs" version '4.6.0' apply false id 'org.gradle.test-retry' version '1.2.0' apply false @@ -135,8 +136,11 @@ ext { apply from: file('wrapper.gradle') if (file('.git').exists()) { - apply from: file('gradle/rat.gradle') rat { + verbose = true + reportDir.set(project.file('build/rat')) + stylesheet.set(file('gradle/resources/rat-output-to-html.xsl')) + // Exclude everything under the directory that git should be ignoring via .gitignore or that isn't checked in. These // restrict us only to files that are checked in or are staged. def repo = Grgit.open(currentDir: project.getRootDir()) @@ -159,11 +163,13 @@ if (file('.git').exists()) { 'streams/quickstart/java/src/test/resources/projects/basic/goal.txt', 'streams/streams-scala/logs/*', 'licenses/*', - '**/generated/**' + '**/generated/**', + 'clients/src/test/resources/serializedData/*' ]) } +} else { + rat.enabled = false } - println("Starting build with version $version using Gradle $gradleVersion, Java ${JavaVersion.current()} and Scala ${versions.scala}") subprojects { diff --git a/gradle/rat.gradle b/gradle/rat.gradle deleted file mode 100644 index 22a02f4ffe798..0000000000000 --- a/gradle/rat.gradle +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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. - */ - -import org.gradle.api.internal.project.IsolatedAntBuilder - -import java.nio.charset.StandardCharsets - -apply plugin: RatPlugin - -class RatTask extends DefaultTask { - @Input - List excludes - - def reportDir = project.file('build/rat') - def stylesheet = project.file('gradle/resources/rat-output-to-html.xsl').getAbsolutePath() - def xmlReport = new File(reportDir, 'rat-report.xml') - def htmlReport = new File(reportDir, 'rat-report.html') - - def generateXmlReport(File reportDir) { - def antBuilder = services.get(IsolatedAntBuilder) - def ratClasspath = project.configurations.rat - def projectPath = project.getRootDir().getAbsolutePath() - antBuilder.withClasspath(ratClasspath).execute { - ant.taskdef(resource: 'org/apache/rat/anttasks/antlib.xml') - ant.report(format: 'xml', reportFile: xmlReport) { - fileset(dir: projectPath) { - patternset { - excludes.each { - exclude(name: it) - } - } - } - } - } - } - - def printUnknownFiles() { - def ratXml = new XmlParser().parse(xmlReport) - def unknownLicenses = 0 - ratXml.resource.each { resource -> - if (resource.'license-approval'.@name[0] == "false") { - println('Unknown license: ' + resource.@name) - unknownLicenses++ - } - } - if (unknownLicenses > 0) { - throw new GradleException("Found " + unknownLicenses + " files with " + - "unknown licenses.") - } - } - - def generateHtmlReport() { - def antBuilder = services.get(IsolatedAntBuilder) - def ratClasspath = project.configurations.rat - antBuilder.withClasspath(ratClasspath).execute { - ant.xslt( - in: xmlReport, - style: stylesheet, - out: htmlReport, - classpath: ratClasspath) - } - println('Rat report: ' + htmlReport) - } - - @TaskAction - def rat() { - if (!reportDir.exists()) { - reportDir.mkdirs() - } - def origEncoding = System.getProperty("file.encoding") - try { - System.setProperty("file.encoding", StandardCharsets.UTF_8.name()) //affects the output of the ant rat task - generateXmlReport(reportDir) - printUnknownFiles() - generateHtmlReport() - } finally { - System.setProperty("file.encoding", origEncoding) - } - } -} - -class RatPlugin implements Plugin { - void apply(Project project) { - configureDependencies(project) - project.plugins.apply(JavaPlugin); - Task ratTask = project.task("rat", - type: RatTask, - group: 'Build', - description: 'Runs Apache Rat checks.') - project.tasks[JavaPlugin.TEST_TASK_NAME].dependsOn ratTask - } - - void configureDependencies(final Project project) { - project.configurations { - rat - } - project.repositories { - mavenCentral() - } - project.dependencies { - rat 'org.apache.rat:apache-rat-tasks:0.13' - } - } -} From a776020688ad1e050b7f129d28f7481ae6d55ce6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 12 Apr 2021 22:26:54 -0700 Subject: [PATCH 051/155] KAFKA-12658: Include kafka-shell jar and dependencies in release tar (#10531) Verified that `./bin/kafka-metadata-shell.sh --help` on the release tarball works as expected. It failed with a `ClassNotFoundException` before this change. Reviewers: Chia-Ping Tsai , Gwen (Chen) Shapira --- build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/build.gradle b/build.gradle index db94e4996b973..39a13ab845005 100644 --- a/build.gradle +++ b/build.gradle @@ -983,6 +983,8 @@ project(':core') { from(project.siteDocsTar) { into("site-docs/") } from(project(':tools').jar) { into("libs/") } from(project(':tools').configurations.runtimeClasspath) { into("libs/") } + from(project(':shell').jar) { into("libs/") } + from(project(':shell').configurations.runtimeClasspath) { into("libs/") } from(project(':connect:api').jar) { into("libs/") } from(project(':connect:api').configurations.runtimeClasspath) { into("libs/") } from(project(':connect:runtime').jar) { into("libs/") } From 80de188727a98d282bac29fda8751c5be0189f52 Mon Sep 17 00:00:00 2001 From: lamberken <2217232293@qq.com> Date: Tue, 13 Apr 2021 14:28:31 +0800 Subject: [PATCH 052/155] KAFKA-12611: Fix using random payload in ProducerPerformance incorrectly (#10469) Reviewers: Chia-Ping Tsai --- .../kafka/tools/ProducerPerformance.java | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index 1386c164f585d..0ddff32baf4d8 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -112,12 +112,10 @@ public static void main(String[] args) throws Exception { /* setup perf test */ byte[] payload = null; - Random random = new Random(0); if (recordSize != null) { payload = new byte[recordSize]; - for (int i = 0; i < payload.length; ++i) - payload[i] = (byte) (random.nextInt(26) + 65); } + Random random = new Random(0); ProducerRecord record; Stats stats = new Stats(numRecords, 5000); long startMs = System.currentTimeMillis(); @@ -127,15 +125,21 @@ public static void main(String[] args) throws Exception { int currentTransactionSize = 0; long transactionStartTime = 0; for (long i = 0; i < numRecords; i++) { + + if (payloadFilePath != null) { + payload = payloadByteList.get(random.nextInt(payloadByteList.size())); + } else if (recordSize != null) { + for (int j = 0; j < payload.length; ++j) + payload[j] = (byte) (random.nextInt(26) + 65); + } else { + throw new IllegalArgumentException("no payload File Path or record Size provided"); + } + if (transactionsEnabled && currentTransactionSize == 0) { producer.beginTransaction(); transactionStartTime = System.currentTimeMillis(); } - - if (payloadFilePath != null) { - payload = payloadByteList.get(random.nextInt(payloadByteList.size())); - } record = new ProducerRecord<>(topicName, payload); long sendStartMs = System.currentTimeMillis(); From 235d13ff2a1a4f97e4a38c3a6a2478579ae134c7 Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Tue, 13 Apr 2021 15:40:55 +0900 Subject: [PATCH 053/155] =?UTF-8?q?KAFKA-12655=20Update=20Jetty:=209.4.38.?= =?UTF-8?q?v20210224=20=E2=86=92=209.4.39.v20210325=20(#10526)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Edwin , Manikumar Reddy , Luke Chen, xjin-Confluent, Chia-Ping Tsai --- gradle/dependencies.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index b7795a48daaa9..91df77adcb2a7 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -69,7 +69,7 @@ versions += [ jacksonDatabind: "2.10.5.1", jacoco: "0.8.5", javassist: "3.27.0-GA", - jetty: "9.4.38.v20210224", + jetty: "9.4.39.v20210325", jersey: "2.31", jline: "3.12.1", jmh: "1.27", From 23633cd1ec3c8a044af9bb9da362f74100c20326 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Tue, 13 Apr 2021 11:48:17 +0200 Subject: [PATCH 054/155] KAFKA-12584; Remove deprecated `Sum` and `Total` classes (#10511) `Sum` and `Total` classes were deprecated and replaced by `WindowedSum` and `CumulativeSum` in 2.4. This patch removes them for 3.0. Reviewers: Chia-Ping Tsai , Ismael Juma --- .../kafka/common/metrics/stats/Sum.java | 29 ------------------- .../kafka/common/metrics/stats/Total.java | 29 ------------------- docs/upgrade.html | 2 ++ 3 files changed, 2 insertions(+), 58 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/metrics/stats/Sum.java delete mode 100644 clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Sum.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Sum.java deleted file mode 100644 index 17188b835c434..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Sum.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.common.metrics.stats; - -/** - * A {@link SampledStat} that maintains the sum of what it has seen. - * This is a sampled version of {@link CumulativeSum}. - * - * See also {@link WindowedCount} if you want to increment the value by 1 on each recording. - * - * @deprecated since 2.4 . Use {@link WindowedSum} instead - */ -@Deprecated -public class Sum extends WindowedSum { -} diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java deleted file mode 100644 index 23f7d040f8f90..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.common.metrics.stats; - -/** - * An non-sampled cumulative total maintained over all time. - * This is a non-sampled version of {@link WindowedSum}. - * - * See also {@link CumulativeCount} if you just want to increment the value by 1 on each recording. - * - * @deprecated since 2.4 . Use {@link CumulativeSum} instead. - */ -@Deprecated -public class Total extends CumulativeSum { -} \ No newline at end of file diff --git a/docs/upgrade.html b/docs/upgrade.html index 0112bfc744d46..dd839089743be 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -32,6 +32,8 @@
    Notable changes in 3
  • The Scala Authorizer, SimpleAclAuthorizer and related classes have been removed. Please use the Java Authorizer and AclAuthorizer instead.
  • The Metric#value() method was removed (KAFKA-12573).
  • +
  • The Sum and Total classes were removed (KAFKA-12584). + Please use WindowedSum and CumulativeSum instead.
  • The PrincipalBuilder, DefaultPrincipalBuilder and ResourceFilter classes were removed.
  • Various constants and constructors were removed from SslConfigs, SaslConfigs, AclBinding and AclBindingFilter.
  • From 327809024fab07c44cd6d87985db5b9d14fd43bb Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Tue, 13 Apr 2021 22:44:03 +0530 Subject: [PATCH 055/155] KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager. (#10218) KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager. Added inmemory implementation for RemoteStorageManager and RemoteLogMetadataManager. A major part of inmemory RLMM will be used in the default RLMM implementation which will be based on topic storage. These will be used in unit tests for tiered storage. Added tests for both the implementations and their supported classes. This is part of tiered storage implementation, KIP-405. Reivewers: Kowshik Prakasam , Jun Rao --- checkstyle/import-control.xml | 5 + .../java/org/apache/kafka/test/TestUtils.java | 41 ++ .../storage/RemoteLogMetadataManager.java | 14 +- .../remote/storage/RemoteLogSegmentState.java | 31 +- .../storage/RemotePartitionDeleteState.java | 35 +- .../storage/RemoteStorageException.java | 9 +- .../storage/RemoteLogLeaderEpochState.java | 172 ++++++++ .../storage/RemoteLogMetadataCache.java | 316 ++++++++++++++ .../storage/RemoteLogMetadataCacheTest.java | 387 ++++++++++++++++++ .../InmemoryRemoteLogMetadataManager.java | 161 ++++++++ .../InmemoryRemoteLogMetadataManagerTest.java | 130 ++++++ .../storage/InmemoryRemoteStorageManager.java | 177 ++++++++ .../InmemoryRemoteStorageManagerTest.java | 246 +++++++++++ storage/src/test/resources/log4j.properties | 21 + 14 files changed, 1725 insertions(+), 20 deletions(-) create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogLeaderEpochState.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java create mode 100644 storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java create mode 100644 storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java create mode 100644 storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java create mode 100644 storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java create mode 100644 storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java create mode 100644 storage/src/test/resources/log4j.properties diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index a5235039d913f..e7e2e4dca414f 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -267,6 +267,11 @@ + + + + + diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 712bbd173e803..5e34ae8c5b369 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -49,6 +49,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Properties; import java.util.Random; @@ -535,4 +536,44 @@ public static void setFieldValue(Object obj, String fieldName, Object value) thr field.setAccessible(true); field.set(obj, value); } + + /** + * Returns true if both iterators have same elements in the same order. + * + * @param iterator1 first iterator. + * @param iterator2 second iterator. + * @param type of element in the iterators. + */ + public static boolean sameElementsWithOrder(Iterator iterator1, + Iterator iterator2) { + while (iterator1.hasNext()) { + if (!iterator2.hasNext()) { + return false; + } + + if (!Objects.equals(iterator1.next(), iterator2.next())) { + return false; + } + } + + return !iterator2.hasNext(); + } + + /** + * Returns true if both the iterators have same set of elements irrespective of order and duplicates. + * + * @param iterator1 first iterator. + * @param iterator2 second iterator. + * @param type of element in the iterators. + */ + public static boolean sameElementsWithoutOrder(Iterator iterator1, + Iterator iterator2) { + // Check both the iterators have the same set of elements irrespective of order and duplicates. + Set allSegmentsSet = new HashSet<>(); + iterator1.forEachRemaining(allSegmentsSet::add); + Set expectedSegmentsSet = new HashSet<>(); + iterator2.forEachRemaining(expectedSegmentsSet::add); + + return allSegmentsSet.equals(expectedSegmentsSet); + } } diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java index 59f5dce2220b6..98f6bca369ca9 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java @@ -113,14 +113,14 @@ void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate remoteLogSegm * the given leader-epoch for the offset, else returns {@link Optional#empty()}. * * @param topicIdPartition topic partition - * @param offset offset * @param epochForOffset leader epoch for the given offset + * @param offset offset * @return the requested remote log segment metadata if it exists. * @throws RemoteStorageException if there are any storage related errors occurred. */ Optional remoteLogSegmentMetadata(TopicIdPartition topicIdPartition, - long offset, - int epochForOffset) + int epochForOffset, + long offset) throws RemoteStorageException; /** @@ -133,8 +133,8 @@ Optional remoteLogSegmentMetadata(TopicIdPartition top * @return the requested highest log offset if exists. * @throws RemoteStorageException if there are any storage related errors occurred. */ - Optional highestLogOffset(TopicIdPartition topicIdPartition, - int leaderEpoch) throws RemoteStorageException; + Optional highestOffsetForEpoch(TopicIdPartition topicIdPartition, + int leaderEpoch) throws RemoteStorageException; /** * This method is used to update the metadata about remote partition delete event. Currently, it allows updating the @@ -158,12 +158,12 @@ void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartit throws RemoteStorageException; /** - * List all the remote log segment metadata of the given topicIdPartition. + * Returns all the remote log segment metadata of the given topicIdPartition. *

    * Remote Partition Removers uses this method to fetch all the segments for a given topic partition, so that they * can delete them. * - * @return Iterator of remote log segment metadata for the given topic partition. + * @return Iterator of all the remote log segment metadata for the given topic partition. */ Iterator listRemoteLogSegments(TopicIdPartition topicIdPartition) throws RemoteStorageException; diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java index 12540af128120..bf0befe310c31 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentState.java @@ -21,18 +21,20 @@ import java.util.Arrays; import java.util.Collections; import java.util.Map; +import java.util.Objects; import java.util.function.Function; import java.util.stream.Collectors; /** - * It indicates the state of the remote log segment. This will be based on the action executed on this + * This enum indicates the state of the remote log segment. This will be based on the action executed on this * segment by the remote log service implementation. *

    - * It goes through the below state transitions. + * It goes through the below state transitions. Self transition is treated as valid. This allows updating with the + * same state in case of retries and failover. *

    *

      * +---------------------+            +----------------------+
    - * |COPY_SEGMENT_STARTED |----------->|COPY_SEGMENT_FINISHED |
    + * |COPY_SEGMENT_STARTED |----------> |COPY_SEGMENT_FINISHED |
      * +-------------------+-+            +--+-------------------+
      *                     |                 |
      *                     |                 |
    @@ -87,4 +89,27 @@ public byte id() {
         public static RemoteLogSegmentState forId(byte id) {
             return STATE_TYPES.get(id);
         }
    +
    +    public static boolean isValidTransition(RemoteLogSegmentState srcState, RemoteLogSegmentState targetState) {
    +        Objects.requireNonNull(targetState, "targetState can not be null");
    +
    +        if (srcState == null) {
    +            // If the source state is null, check the target state as the initial state viz COPY_SEGMENT_STARTED
    +            // This ensures simplicity here as we don't have to define one more type to represent the state 'null' like
    +            // COPY_SEGMENT_NOT_STARTED, have the null check by the caller and pass that state.
    +            return targetState == COPY_SEGMENT_STARTED;
    +        } else if (srcState == targetState) {
    +            // Self transition is treated as valid. This is to maintain the idempotency for the state in case of retries
    +            // or failover.
    +            return true;
    +        } else if (srcState == COPY_SEGMENT_STARTED) {
    +            return targetState == COPY_SEGMENT_FINISHED || targetState == DELETE_SEGMENT_STARTED;
    +        } else if (srcState == COPY_SEGMENT_FINISHED) {
    +            return targetState == DELETE_SEGMENT_STARTED;
    +        } else if (srcState == DELETE_SEGMENT_STARTED) {
    +            return targetState == DELETE_SEGMENT_FINISHED;
    +        } else {
    +            return false;
    +        }
    +    }
     }
    diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java
    index d5172fc1f942e..e2fad1ace083b 100644
    --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java
    +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteState.java
    @@ -21,25 +21,27 @@
     import java.util.Arrays;
     import java.util.Collections;
     import java.util.Map;
    +import java.util.Objects;
     import java.util.function.Function;
     import java.util.stream.Collectors;
     
     /**
    - * It indicates the deletion state of the remote topic partition. This will be based on the action executed on this
    + * This enum indicates the deletion state of the remote topic partition. This will be based on the action executed on this
      * partition by the remote log service implementation.
    - * State transitions are mentioned below.
    + * State transitions are mentioned below. Self transition is treated as valid. This allows updating with the
    + * same state in case of retries and failover.
      * 

    *

      * +-------------------------+
      * |DELETE_PARTITION_MARKED  |
      * +-----------+-------------+
    - * |
    - * |
    + *             |
    + *             |
      * +-----------v--------------+
      * |DELETE_PARTITION_STARTED  |
      * +-----------+--------------+
    - * |
    - * |
    + *             |
    + *             |
      * +-----------v--------------+
      * |DELETE_PARTITION_FINISHED |
      * +--------------------------+
    @@ -83,4 +85,25 @@ public static RemotePartitionDeleteState forId(byte id) {
             return STATE_TYPES.get(id);
         }
     
    +    public static boolean isValidTransition(RemotePartitionDeleteState srcState,
    +                                            RemotePartitionDeleteState targetState) {
    +        Objects.requireNonNull(targetState, "targetState can not be null");
    +
    +        if (srcState == null) {
    +            // If the source state is null, check the target state as the initial state viz DELETE_PARTITION_MARKED.
    +            // This ensures simplicity here as we don't have to define one more type to represent the state 'null' like
    +            // DELETE_PARTITION_NOT_MARKED, have the null check by the caller and pass that state.
    +            return targetState == DELETE_PARTITION_MARKED;
    +        } else if (srcState == targetState) {
    +            // Self transition is treated as valid. This is to maintain the idempotency for the state in case of retries
    +            // or failover.
    +            return true;
    +        } else if (srcState == DELETE_PARTITION_MARKED) {
    +            return targetState == DELETE_PARTITION_STARTED;
    +        } else if (srcState == DELETE_PARTITION_STARTED) {
    +            return targetState == DELETE_PARTITION_FINISHED;
    +        } else {
    +            return false;
    +        }
    +    }
     }
    \ No newline at end of file
    diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java
    index f2488cb2957fb..bd392fc674c90 100644
    --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java
    +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageException.java
    @@ -17,8 +17,7 @@
     package org.apache.kafka.server.log.remote.storage;
     
     /**
    - * Exception thrown when there is a remote storage error.
    - * This can be used as the base exception by implementors of
    + * Exception thrown when there is a remote storage error. This can be used as the base exception by implementors of
      * {@link RemoteStorageManager} or {@link RemoteLogMetadataManager} to create extended exceptions.
      */
     public class RemoteStorageException extends Exception {
    @@ -28,9 +27,11 @@ public RemoteStorageException(final String message) {
             super(message);
         }
     
    -    public RemoteStorageException(final String message,
    -                                  final Throwable cause) {
    +    public RemoteStorageException(final String message, final Throwable cause) {
             super(message, cause);
         }
     
    +    public RemoteStorageException(Throwable cause) {
    +        super(cause);
    +    }
     }
    diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogLeaderEpochState.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogLeaderEpochState.java
    new file mode 100644
    index 0000000000000..7535aff8cd709
    --- /dev/null
    +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogLeaderEpochState.java
    @@ -0,0 +1,172 @@
    +/*
    + * 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.server.log.remote.metadata.storage;
    +
    +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId;
    +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata;
    +import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.Iterator;
    +import java.util.Map;
    +import java.util.NavigableMap;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentSkipListMap;
    +
    +/**
    + * This class represents the in-memory state of segments associated with a leader epoch. This includes the mapping of offset to
    + * segment ids and unreferenced segments which are not mapped to any offset but they exist in remote storage.
    + * 

    + * This is used by {@link RemoteLogMetadataCache} to track the segments for each leader epoch. + */ +class RemoteLogLeaderEpochState { + + // It contains offset to segment ids mapping with the segment state as COPY_SEGMENT_FINISHED. + private final NavigableMap offsetToId = new ConcurrentSkipListMap<>(); + + /** + * It represents unreferenced segments for this leader epoch. It contains the segments still in COPY_SEGMENT_STARTED + * and DELETE_SEGMENT_STARTED state or these have been replaced by callers with other segments having the same + * start offset for the leader epoch. These will be returned by {@link RemoteLogMetadataCache#listAllRemoteLogSegments()} + * and {@link RemoteLogMetadataCache#listRemoteLogSegments(int leaderEpoch)} so that callers can clean them up if + * they still exist. These will be cleaned from the cache once they reach DELETE_SEGMENT_FINISHED state. + */ + private final Set unreferencedSegmentIds = ConcurrentHashMap.newKeySet(); + + // It represents the highest log offset of the segments that reached the COPY_SEGMENT_FINISHED state. + private volatile Long highestLogOffset; + + /** + * Returns all the segments associated with this leader epoch sorted by start offset in ascending order. + * + * @param idToSegmentMetadata mapping of id to segment metadata. This will be used to get RemoteLogSegmentMetadata + * for an id to be used for sorting. + */ + Iterator listAllRemoteLogSegments(Map idToSegmentMetadata) + throws RemoteResourceNotFoundException { + // Return all the segments including unreferenced metadata. + int size = offsetToId.size() + unreferencedSegmentIds.size(); + if (size == 0) { + return Collections.emptyIterator(); + } + + ArrayList metadataList = new ArrayList<>(size); + collectConvertedIdToMetadata(offsetToId.values(), idToSegmentMetadata, metadataList); + + if (!unreferencedSegmentIds.isEmpty()) { + collectConvertedIdToMetadata(unreferencedSegmentIds, idToSegmentMetadata, metadataList); + + // Sort only when unreferenced entries exist as they are already sorted in offsetToId. + metadataList.sort(Comparator.comparingLong(RemoteLogSegmentMetadata::startOffset)); + } + + return metadataList.iterator(); + } + + private void collectConvertedIdToMetadata(Collection segmentIds, + Map idToSegmentMetadata, + Collection result) throws RemoteResourceNotFoundException { + for (RemoteLogSegmentId id : segmentIds) { + RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(id); + if (metadata == null) { + throw new RemoteResourceNotFoundException("No remote log segment metadata found for :" + id); + } + result.add(metadata); + } + } + + void handleSegmentWithCopySegmentStartedState(RemoteLogSegmentId remoteLogSegmentId) { + // Add this to unreferenced set of segments for the respective leader epoch. + unreferencedSegmentIds.add(remoteLogSegmentId); + } + + void handleSegmentWithCopySegmentFinishedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId, + Long leaderEpochEndOffset) { + // Add the segment epochs mapping as the segment is copied successfully. + RemoteLogSegmentId oldEntry = offsetToId.put(startOffset, remoteLogSegmentId); + + // Remove the metadata from unreferenced entries as it is successfully copied and added to the offset mapping. + unreferencedSegmentIds.remove(remoteLogSegmentId); + + // Add the old entry to unreferenced entries as the mapping is removed for the old entry. + if (oldEntry != null) { + unreferencedSegmentIds.add(oldEntry); + } + + // Update the highest offset entry for this leader epoch as we added a new mapping. + if (highestLogOffset == null || leaderEpochEndOffset > highestLogOffset) { + highestLogOffset = leaderEpochEndOffset; + } + } + + void handleSegmentWithDeleteSegmentStartedState(Long startOffset, RemoteLogSegmentId remoteLogSegmentId, + Long leaderEpochEndOffset) { + // Remove the offset mappings as this segment is getting deleted. + offsetToId.remove(startOffset, remoteLogSegmentId); + + // Add this entry to unreferenced set for the leader epoch as it is being deleted. + // This allows any retries of deletion as these are returned from listAllSegments and listSegments(leaderEpoch). + unreferencedSegmentIds.add(remoteLogSegmentId); + } + + void handleSegmentWithDeleteSegmentFinishedState(long startOffset, RemoteLogSegmentId remoteLogSegmentId, + Long leaderEpochEndOffset) { + // It completely removes the tracking of this segment as it is considered as deleted. + unreferencedSegmentIds.remove(remoteLogSegmentId); + } + + Long highestLogOffset() { + return highestLogOffset; + } + + /** + * Returns the RemoteLogSegmentId of a segment for the given offset, if there exists a mapping associated with + * the greatest offset less than or equal to the given offset, or null if there is no such mapping. + * + * @param offset offset + */ + RemoteLogSegmentId floorEntry(long offset) { + Map.Entry entry = offsetToId.floorEntry(offset); + + return entry == null ? null : entry.getValue(); + } + + /** + * Action interface to act on remote log segment transition for the given {@link RemoteLogLeaderEpochState}. + */ + @FunctionalInterface + interface Action { + + /** + * Performs this operation with the given {@code remoteLogLeaderEpochState}. + * + * @param remoteLogLeaderEpochState In-memory state of the segments for a leader epoch. + * @param startOffset start offset of the segment. + * @param segmentId segment id. + * @param leaderEpochEndOffset end offset for the given leader epoch. + */ + void accept(RemoteLogLeaderEpochState remoteLogLeaderEpochState, + Long startOffset, + RemoteLogSegmentId segmentId, + Long leaderEpochEndOffset); + } + +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java new file mode 100644 index 0000000000000..5853d7fb81678 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCache.java @@ -0,0 +1,316 @@ +/* + * 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.server.log.remote.metadata.storage; + +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +/** + * This class provides an in-memory cache of remote log segment metadata. This maintains the lineage of segments + * with respect to leader epochs. + *

    + * Remote log segment can go through the state transitions as mentioned in {@link RemoteLogSegmentState}. + *

    + * This class will have all the segments which did not reach terminal state viz DELETE_SEGMENT_FINISHED. That means,any + * segment reaching the terminal state will get cleared from this instance. + * This class provides different methods to fetch segment metadata like {@link #remoteLogSegmentMetadata(int, long)}, + * {@link #highestOffsetForEpoch(int)}, {@link #listRemoteLogSegments(int)}, {@link #listAllRemoteLogSegments()}. Those + * methods have different semantics to fetch the segment based on its state. + *

    + *

      + *
    • + * {@link RemoteLogSegmentState#COPY_SEGMENT_STARTED}: + *
      + * Segment in this state indicates it is not yet copied successfully. So, these segments will not be + * accessible for reads but these are considered for cleanups when a partition is deleted. + *
    • + *
    • + * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED}: + *
      + * Segment in this state indicates it is successfully copied and it is available for reads. So, these segments + * will be accessible for reads. But this should be available for any cleanup activity like deleting segments by the + * caller of this class. + *
    • + *
    • + * {@link RemoteLogSegmentState#DELETE_SEGMENT_STARTED}: + * Segment in this state indicates it is getting deleted. That means, it is not available for reads. But it should be + * available for any cleanup activity like deleting segments by the caller of this class. + *
    • + *
    • + * {@link RemoteLogSegmentState#DELETE_SEGMENT_FINISHED}: + * Segment in this state indicate it is already deleted. That means, it is not available for any activity including + * reads or cleanup activity. This cache will clear entries containing this state. + *
    • + *
    + * + *

    + * The below table summarizes whether the segment with the respective state are available for the given methods. + *

    + * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
    + * |  Method / SegmentState          | COPY_SEGMENT_STARTED | COPY_SEGMENT_FINISHED  | DELETE_SEGMENT_STARTED  | DELETE_SEGMENT_FINISHED |
    + * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
    + * | remoteLogSegmentMetadata        |        No            |           Yes          |          No             |           No            |
    + * | (int leaderEpoch, long offset)  |                      |                        |                         |                         |
    + * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
    + * | listRemoteLogSegments           |        Yes           |           Yes          |          Yes            |           No            |
    + * | (int leaderEpoch)               |                      |                        |                         |                         |
    + * |---------------------------------+----------------------+------------------------+-------------------------+-------------------------|
    + * | listAllRemoteLogSegments()      |        Yes           |           Yes          |          Yes            |           No            |
    + * |                                 |                      |                        |                         |                         |
    + * +---------------------------------+----------------------+------------------------+-------------------------+-------------------------+
    + * 
    + *

    + *

    + */ +public class RemoteLogMetadataCache { + + private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCache.class); + + // It contains all the segment-id to metadata mappings which did not reach the terminal state viz DELETE_SEGMENT_FINISHED. + private final ConcurrentMap idToSegmentMetadata + = new ConcurrentHashMap<>(); + + // It contains leader epoch to the respective entry containing the state. + // TODO We are not clearing the entry for epoch when RemoteLogLeaderEpochState becomes empty. This will be addressed + // later. We will look into it when we integrate these APIs along with RemoteLogManager changes. + // https://issues.apache.org/jira/browse/KAFKA-12641 + private final ConcurrentMap leaderEpochEntries = new ConcurrentHashMap<>(); + + /** + * Returns {@link RemoteLogSegmentMetadata} if it exists for the given leader-epoch containing the offset and with + * {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} state, else returns {@link Optional#empty()}. + * + * @param leaderEpoch leader epoch for the given offset + * @param offset offset + * @return the requested remote log segment metadata if it exists. + */ + public Optional remoteLogSegmentMetadata(int leaderEpoch, long offset) { + RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch); + + if (remoteLogLeaderEpochState == null) { + return Optional.empty(); + } + + // Look for floor entry as the given offset may exist in this entry. + RemoteLogSegmentId remoteLogSegmentId = remoteLogLeaderEpochState.floorEntry(offset); + if (remoteLogSegmentId == null) { + // If the offset is lower than the minimum offset available in metadata then return empty. + return Optional.empty(); + } + + RemoteLogSegmentMetadata metadata = idToSegmentMetadata.get(remoteLogSegmentId); + // Check whether the given offset with leaderEpoch exists in this segment. + // Check for epoch's offset boundaries with in this segment. + // 1. Get the next epoch's start offset -1 if exists + // 2. If no next epoch exists, then segment end offset can be considered as epoch's relative end offset. + Map.Entry nextEntry = metadata.segmentLeaderEpochs().higherEntry(leaderEpoch); + long epochEndOffset = (nextEntry != null) ? nextEntry.getValue() - 1 : metadata.endOffset(); + + // Return empty when target offset > epoch's end offset. + return offset > epochEndOffset ? Optional.empty() : Optional.of(metadata); + } + + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate) + throws RemoteResourceNotFoundException { + log.debug("Updating remote log segment metadata: [{}]", metadataUpdate); + Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null"); + + RemoteLogSegmentState targetState = metadataUpdate.state(); + RemoteLogSegmentId remoteLogSegmentId = metadataUpdate.remoteLogSegmentId(); + RemoteLogSegmentMetadata existingMetadata = idToSegmentMetadata.get(remoteLogSegmentId); + if (existingMetadata == null) { + throw new RemoteResourceNotFoundException("No remote log segment metadata found for :" + + remoteLogSegmentId); + } + + // Check the state transition. + checkStateTransition(existingMetadata.state(), targetState); + + switch (targetState) { + case COPY_SEGMENT_STARTED: + // Callers should use addCopyInProgressSegment to add RemoteLogSegmentMetadata with state as + // RemoteLogSegmentState.COPY_SEGMENT_STARTED. + throw new IllegalArgumentException("metadataUpdate: " + metadataUpdate + " with state " + RemoteLogSegmentState.COPY_SEGMENT_STARTED + + " can not be updated"); + case COPY_SEGMENT_FINISHED: + handleSegmentWithCopySegmentFinishedState(metadataUpdate, existingMetadata); + break; + case DELETE_SEGMENT_STARTED: + handleSegmentWithDeleteSegmentStartedState(metadataUpdate, existingMetadata); + break; + case DELETE_SEGMENT_FINISHED: + handleSegmentWithDeleteSegmentFinishedState(metadataUpdate, existingMetadata); + break; + default: + throw new IllegalArgumentException("Metadata with the state " + targetState + " is not supported"); + } + } + + private void handleSegmentWithCopySegmentFinishedState(RemoteLogSegmentMetadataUpdate metadataUpdate, + RemoteLogSegmentMetadata existingMetadata) { + log.debug("Adding remote log segment metadata to leader epoch mappings with update: [{}]", metadataUpdate); + + doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata, + RemoteLogLeaderEpochState::handleSegmentWithCopySegmentFinishedState); + + // Put the entry with the updated metadata. + idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(), + existingMetadata.createWithUpdates(metadataUpdate)); + } + + private void handleSegmentWithDeleteSegmentStartedState(RemoteLogSegmentMetadataUpdate metadataUpdate, + RemoteLogSegmentMetadata existingMetadata) { + log.debug("Cleaning up the state for : [{}]", metadataUpdate); + + doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata, + RemoteLogLeaderEpochState::handleSegmentWithDeleteSegmentStartedState); + + // Put the entry with the updated metadata. + idToSegmentMetadata.put(existingMetadata.remoteLogSegmentId(), + existingMetadata.createWithUpdates(metadataUpdate)); + } + + private void handleSegmentWithDeleteSegmentFinishedState(RemoteLogSegmentMetadataUpdate metadataUpdate, + RemoteLogSegmentMetadata existingMetadata) { + log.debug("Removing the entry as it reached the terminal state: [{}]", metadataUpdate); + + doHandleSegmentStateTransitionForLeaderEpochs(existingMetadata, + RemoteLogLeaderEpochState::handleSegmentWithDeleteSegmentFinishedState); + + // Remove the segment's id to metadata mapping because this segment is considered as deleted and it cleared all + // the state of this segment in the cache. + idToSegmentMetadata.remove(existingMetadata.remoteLogSegmentId()); + } + + private void doHandleSegmentStateTransitionForLeaderEpochs(RemoteLogSegmentMetadata existingMetadata, + RemoteLogLeaderEpochState.Action action) { + RemoteLogSegmentId remoteLogSegmentId = existingMetadata.remoteLogSegmentId(); + Map leaderEpochToOffset = existingMetadata.segmentLeaderEpochs(); + + // Go through all the leader epochs and apply the given action. + for (Map.Entry entry : leaderEpochToOffset.entrySet()) { + Integer leaderEpoch = entry.getKey(); + Long startOffset = entry.getValue(); + RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch); + + if (remoteLogLeaderEpochState == null) { + throw new IllegalStateException("RemoteLogLeaderEpochState does not exist for the leader epoch: " + + leaderEpoch); + } else { + long leaderEpochEndOffset = highestOffsetForEpoch(leaderEpoch, existingMetadata); + action.accept(remoteLogLeaderEpochState, startOffset, remoteLogSegmentId, leaderEpochEndOffset); + } + } + } + + private long highestOffsetForEpoch(Integer leaderEpoch, RemoteLogSegmentMetadata segmentMetadata) { + // Compute the highest offset for the leader epoch with in the segment + NavigableMap epochToOffset = segmentMetadata.segmentLeaderEpochs(); + Map.Entry nextEntry = epochToOffset.higherEntry(leaderEpoch); + + return nextEntry != null ? nextEntry.getValue() - 1 : segmentMetadata.endOffset(); + } + + /** + * Returns all the segments stored in this cache. + * + * @return + */ + public Iterator listAllRemoteLogSegments() { + // Return all the segments including unreferenced metadata. + return Collections.unmodifiableCollection(idToSegmentMetadata.values()).iterator(); + } + + /** + * Returns all the segments mapped to the leader epoch that exist in this cache sorted by {@link RemoteLogSegmentMetadata#startOffset()}. + * + * @param leaderEpoch leader epoch. + */ + public Iterator listRemoteLogSegments(int leaderEpoch) + throws RemoteResourceNotFoundException { + RemoteLogLeaderEpochState remoteLogLeaderEpochState = leaderEpochEntries.get(leaderEpoch); + if (remoteLogLeaderEpochState == null) { + return Collections.emptyIterator(); + } + + return remoteLogLeaderEpochState.listAllRemoteLogSegments(idToSegmentMetadata); + } + + /** + * Returns the highest offset of a segment for the given leader epoch if exists, else it returns empty. The segments + * that have reached the {@link RemoteLogSegmentState#COPY_SEGMENT_FINISHED} or later states are considered here. + * + * @param leaderEpoch leader epoch + */ + public Optional highestOffsetForEpoch(int leaderEpoch) { + RemoteLogLeaderEpochState entry = leaderEpochEntries.get(leaderEpoch); + return entry != null ? Optional.ofNullable(entry.highestLogOffset()) : Optional.empty(); + } + + /** + * This method tracks the given remote segment as not yet available for reads. It does not add the segment + * leader epoch offset mapping until this segment reaches COPY_SEGMENT_FINISHED state. + * + * @param remoteLogSegmentMetadata RemoteLogSegmentMetadata instance + */ + public void addCopyInProgressSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) { + log.debug("Adding to in-progress state: [{}]", remoteLogSegmentMetadata); + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + // This method is allowed only to add remote log segment with the initial state(which is RemoteLogSegmentState.COPY_SEGMENT_STARTED) + // but not to update the existing remote log segment metadata. + if (remoteLogSegmentMetadata.state() != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + throw new IllegalArgumentException( + "Given remoteLogSegmentMetadata:" + remoteLogSegmentMetadata + " should have state as " + RemoteLogSegmentState.COPY_SEGMENT_STARTED + + " but it contains state as: " + remoteLogSegmentMetadata.state()); + } + + RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId(); + RemoteLogSegmentMetadata existingMetadata = idToSegmentMetadata.get(remoteLogSegmentId); + checkStateTransition(existingMetadata != null ? existingMetadata.state() : null, + remoteLogSegmentMetadata.state()); + + for (Integer epoch : remoteLogSegmentMetadata.segmentLeaderEpochs().keySet()) { + leaderEpochEntries.computeIfAbsent(epoch, leaderEpoch -> new RemoteLogLeaderEpochState()) + .handleSegmentWithCopySegmentStartedState(remoteLogSegmentId); + } + + idToSegmentMetadata.put(remoteLogSegmentId, remoteLogSegmentMetadata); + } + + private void checkStateTransition(RemoteLogSegmentState existingState, RemoteLogSegmentState targetState) { + if (!RemoteLogSegmentState.isValidTransition(existingState, targetState)) { + throw new IllegalStateException( + "Current state: " + existingState + " can not be transitioned to target state: " + targetState); + } + } + +} diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java new file mode 100644 index 0000000000000..a01c4fb4c379f --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataCacheTest.java @@ -0,0 +1,387 @@ +/* + * 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.server.log.remote.metadata.storage; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Stream; + +public class RemoteLogMetadataCacheTest { + private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataCacheTest.class); + + private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(), + new TopicPartition("foo", 0)); + private static final int SEG_SIZE = 1024 * 1024; + private static final int BROKER_ID_0 = 0; + private static final int BROKER_ID_1 = 1; + + private final Time time = new MockTime(1); + + @Test + public void testSegmentsLifeCycleInCache() throws Exception { + RemoteLogMetadataCache cache = new RemoteLogMetadataCache(); + // Create remote log segment metadata and add them to RemoteLogMetadataCache. + + // segment 0 + // offsets: [0-100] + // leader epochs (0,0), (1,20), (2,80) + Map segment0LeaderEpochs = new HashMap<>(); + segment0LeaderEpochs.put(0, 0L); + segment0LeaderEpochs.put(1, 20L); + segment0LeaderEpochs.put(2, 80L); + RemoteLogSegmentId segment0Id = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + RemoteLogSegmentMetadata segment0Metadata = new RemoteLogSegmentMetadata(segment0Id, 0L, 100L, + -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segment0LeaderEpochs); + cache.addCopyInProgressSegment(segment0Metadata); + + // We should not get this as the segment is still getting copied and it is not yet considered successful until + // it reaches RemoteLogSegmentState.COPY_SEGMENT_FINISHED. + Assertions.assertFalse(cache.remoteLogSegmentMetadata(40, 1).isPresent()); + + // Check that these leader epochs are to considered for highest offsets as they are still getting copied and + // they did nto reach COPY_SEGMENT_FINISHED state. + Stream.of(0, 1, 2).forEach(epoch -> Assertions.assertFalse(cache.highestOffsetForEpoch(epoch).isPresent())); + + RemoteLogSegmentMetadataUpdate segment0Update = new RemoteLogSegmentMetadataUpdate( + segment0Id, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1); + cache.updateRemoteLogSegmentMetadata(segment0Update); + RemoteLogSegmentMetadata expectedSegment0Metadata = segment0Metadata.createWithUpdates(segment0Update); + + // segment 1 + // offsets: [101 - 200] + // no changes in leadership with in this segment + // leader epochs (2, 101) + Map segment1LeaderEpochs = Collections.singletonMap(2, 101L); + RemoteLogSegmentMetadata segment1Metadata = createSegmentUpdateWithState(cache, segment1LeaderEpochs, 101L, 200L, + RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + + // segment 2 + // offsets: [201 - 300] + // moved to epoch 3 in between + // leader epochs (2, 201), (3, 240) + Map segment2LeaderEpochs = new HashMap<>(); + segment2LeaderEpochs.put(2, 201L); + segment2LeaderEpochs.put(3, 240L); + RemoteLogSegmentMetadata segment2Metadata = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201L, 300L, + RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + + // segment 3 + // offsets: [250 - 400] + // leader epochs (3, 250), (4, 370) + Map segment3LeaderEpochs = new HashMap<>(); + segment3LeaderEpochs.put(3, 250L); + segment3LeaderEpochs.put(4, 370L); + RemoteLogSegmentMetadata segment3Metadata = createSegmentUpdateWithState(cache, segment3LeaderEpochs, 250L, 400L, + RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + + ////////////////////////////////////////////////////////////////////////////////////////// + // Four segments are added with different boundaries and leader epochs. + // Search for cache.remoteLogSegmentMetadata(leaderEpoch, offset) for different + // epochs and offsets + ////////////////////////////////////////////////////////////////////////////////////////// + + HashMap expectedEpochOffsetToSegmentMetadata = new HashMap<>(); + // Existing metadata entries. + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 40), expectedSegment0Metadata); + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(2, 110), segment1Metadata); + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 240), segment2Metadata); + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(3, 250), segment3Metadata); + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 375), segment3Metadata); + + // Non existing metadata entries. + // Search for offset 110, epoch 1, and it should not exist. + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(1, 110), null); + // Search for non existing offset 401, epoch 4. + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(4, 401), null); + // Search for non existing epoch 5. + expectedEpochOffsetToSegmentMetadata.put(new EpochOffset(5, 301), null); + + for (Map.Entry entry : expectedEpochOffsetToSegmentMetadata.entrySet()) { + EpochOffset epochOffset = entry.getKey(); + Optional segmentMetadata = cache.remoteLogSegmentMetadata(epochOffset.epoch, epochOffset.offset); + RemoteLogSegmentMetadata expectedSegmentMetadata = entry.getValue(); + log.debug("Searching for {} , result: {}, expected: {} ", epochOffset, segmentMetadata, + expectedSegmentMetadata); + if (expectedSegmentMetadata != null) { + Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadata); + } else { + Assertions.assertFalse(segmentMetadata.isPresent()); + } + } + + // Update segment with state as DELETE_SEGMENT_STARTED. + // It should not be available when we search for that segment. + cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(), + time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID_1)); + Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent()); + + // Update segment with state as DELETE_SEGMENT_FINISHED. + // It should not be available when we search for that segment. + cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(expectedSegment0Metadata.remoteLogSegmentId(), + time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1)); + Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 10).isPresent()); + + ////////////////////////////////////////////////////////////////////////////////////////// + // Search for cache.highestLogOffset(leaderEpoch) for all the leader epochs + ////////////////////////////////////////////////////////////////////////////////////////// + + Map expectedEpochToHighestOffset = new HashMap<>(); + expectedEpochToHighestOffset.put(0, 19L); + expectedEpochToHighestOffset.put(1, 79L); + expectedEpochToHighestOffset.put(2, 239L); + expectedEpochToHighestOffset.put(3, 369L); + expectedEpochToHighestOffset.put(4, 400L); + + for (Map.Entry entry : expectedEpochToHighestOffset.entrySet()) { + Integer epoch = entry.getKey(); + Long expectedOffset = entry.getValue(); + Optional offset = cache.highestOffsetForEpoch(epoch); + log.debug("Fetching highest offset for epoch: {} , returned: {} , expected: {}", epoch, offset, expectedOffset); + Assertions.assertEquals(Optional.of(expectedOffset), offset); + } + + // Search for non existing leader epoch + Optional highestOffsetForEpoch5 = cache.highestOffsetForEpoch(5); + Assertions.assertFalse(highestOffsetForEpoch5.isPresent()); + } + + private RemoteLogSegmentMetadata createSegmentUpdateWithState(RemoteLogMetadataCache cache, + Map segmentLeaderEpochs, + long startOffset, + long endOffset, + RemoteLogSegmentState state) + throws RemoteResourceNotFoundException { + RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, startOffset, endOffset, -1L, + BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs); + cache.addCopyInProgressSegment(segmentMetadata); + + RemoteLogSegmentMetadataUpdate segMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId, + time.milliseconds(), state, BROKER_ID_1); + cache.updateRemoteLogSegmentMetadata(segMetadataUpdate); + + return segmentMetadata.createWithUpdates(segMetadataUpdate); + } + + @Test + public void testCacheSegmentWithCopySegmentStartedState() throws Exception { + RemoteLogMetadataCache cache = new RemoteLogMetadataCache(); + + // Create a segment with state COPY_SEGMENT_STARTED, and check for searching that segment and listing the + // segments. + RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0L, 50L, -1L, BROKER_ID_0, + time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); + cache.addCopyInProgressSegment(segmentMetadata); + + // This segment should not be available as the state is not reached to COPY_SEGMENT_FINISHED. + Optional segMetadataForOffset0Epoch0 = cache.remoteLogSegmentMetadata(0, 0); + Assertions.assertFalse(segMetadataForOffset0Epoch0.isPresent()); + + // cache.listRemoteLogSegments APIs should contain the above segment. + checkListSegments(cache, 0, segmentMetadata); + } + + @Test + public void testCacheSegmentWithCopySegmentFinishedState() throws Exception { + RemoteLogMetadataCache cache = new RemoteLogMetadataCache(); + + // Create a segment and move it to state COPY_SEGMENT_FINISHED. and check for searching that segment and + // listing the segments. + RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 101L), + 101L, 200L, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + + // Search should return the above segment. + Optional segMetadataForOffset150 = cache.remoteLogSegmentMetadata(0, 150); + Assertions.assertEquals(Optional.of(segmentMetadata), segMetadataForOffset150); + + // cache.listRemoteLogSegments should contain the above segments. + checkListSegments(cache, 0, segmentMetadata); + } + + @Test + public void testCacheSegmentWithDeleteSegmentStartedState() throws Exception { + RemoteLogMetadataCache cache = new RemoteLogMetadataCache(); + + // Create a segment and move it to state DELETE_SEGMENT_STARTED, and check for searching that segment and + // listing the segments. + RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 201L), + 201L, 300L, RemoteLogSegmentState.DELETE_SEGMENT_STARTED); + + // Search should not return the above segment as their leader epoch state is cleared. + Optional segmentMetadataForOffset250Epoch0 = cache.remoteLogSegmentMetadata(0, 250); + Assertions.assertFalse(segmentMetadataForOffset250Epoch0.isPresent()); + + checkListSegments(cache, 0, segmentMetadata); + } + + @Test + public void testCacheSegmentsWithDeleteSegmentFinishedState() throws Exception { + RemoteLogMetadataCache cache = new RemoteLogMetadataCache(); + + // Create a segment and move it to state DELETE_SEGMENT_FINISHED, and check for searching that segment and + // listing the segments. + RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 301L), + 301L, 400L, RemoteLogSegmentState.DELETE_SEGMENT_STARTED); + + // Search should not return the above segment as their leader epoch state is cleared. + Assertions.assertFalse(cache.remoteLogSegmentMetadata(0, 350).isPresent()); + + RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), + time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1); + cache.updateRemoteLogSegmentMetadata(segmentMetadataUpdate); + + // listRemoteLogSegments(0) and listRemoteLogSegments() should not contain the above segment. + Assertions.assertFalse(cache.listRemoteLogSegments(0).hasNext()); + Assertions.assertFalse(cache.listAllRemoteLogSegments().hasNext()); + } + + @Test + public void testCacheListSegments() throws Exception { + RemoteLogMetadataCache cache = new RemoteLogMetadataCache(); + + // Create a few segments and add them to the cache. + RemoteLogSegmentMetadata segment0 = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 0L), 0, 100, + RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + RemoteLogSegmentMetadata segment1 = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 101L), 101, 200, + RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + Map segment2LeaderEpochs = new HashMap<>(); + segment2LeaderEpochs.put(0, 201L); + segment2LeaderEpochs.put(1, 301L); + RemoteLogSegmentMetadata segment2 = createSegmentUpdateWithState(cache, segment2LeaderEpochs, 201, 400, + RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + + // listRemoteLogSegments(0) and listAllRemoteLogSegments() should contain all the above segments. + List expectedSegmentsForEpoch0 = Arrays.asList(segment0, segment1, segment2); + Assertions.assertTrue(TestUtils.sameElementsWithOrder(cache.listRemoteLogSegments(0), + expectedSegmentsForEpoch0.iterator())); + Assertions.assertTrue(TestUtils.sameElementsWithoutOrder(cache.listAllRemoteLogSegments(), + expectedSegmentsForEpoch0.iterator())); + + // listRemoteLogSegments(1) should contain only segment2. + List expectedSegmentsForEpoch1 = Collections.singletonList(segment2); + Assertions.assertTrue(TestUtils.sameElementsWithOrder(cache.listRemoteLogSegments(1), + expectedSegmentsForEpoch1.iterator())); + } + + @Test + public void testAPIsWithInvalidArgs() { + RemoteLogMetadataCache cache = new RemoteLogMetadataCache(); + + Assertions.assertThrows(NullPointerException.class, () -> cache.addCopyInProgressSegment(null)); + Assertions.assertThrows(NullPointerException.class, () -> cache.updateRemoteLogSegmentMetadata(null)); + + // Check for invalid state updates to addCopyInProgressSegment method. + for (RemoteLogSegmentState state : RemoteLogSegmentState.values()) { + if (state != RemoteLogSegmentState.COPY_SEGMENT_STARTED) { + RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata( + new RemoteLogSegmentId(TP0, Uuid.randomUuid()), 0, 100L, + -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, Collections.singletonMap(0, 0L)); + RemoteLogSegmentMetadata updatedMetadata = segmentMetadata + .createWithUpdates(new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), + time.milliseconds(), state, BROKER_ID_1)); + Assertions.assertThrows(IllegalArgumentException.class, () -> + cache.addCopyInProgressSegment(updatedMetadata)); + } + } + + // Check for updating non existing segment-id. + Assertions.assertThrows(RemoteResourceNotFoundException.class, () -> { + RemoteLogSegmentId nonExistingId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(nonExistingId, + time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_STARTED, BROKER_ID_1)); + }); + + // Check for invalid state transition. + Assertions.assertThrows(IllegalStateException.class, () -> { + RemoteLogSegmentMetadata segmentMetadata = createSegmentUpdateWithState(cache, Collections.singletonMap(0, 0L), 0, + 100, RemoteLogSegmentState.COPY_SEGMENT_FINISHED); + cache.updateRemoteLogSegmentMetadata(new RemoteLogSegmentMetadataUpdate(segmentMetadata.remoteLogSegmentId(), + time.milliseconds(), RemoteLogSegmentState.DELETE_SEGMENT_FINISHED, BROKER_ID_1)); + }); + } + + private void checkListSegments(RemoteLogMetadataCache cache, + int leaderEpoch, + RemoteLogSegmentMetadata expectedSegment) + throws RemoteResourceNotFoundException { + // cache.listRemoteLogSegments(leaderEpoch) should contain the above segment. + Iterator segmentsIter = cache.listRemoteLogSegments(leaderEpoch); + Assertions.assertTrue(segmentsIter.hasNext() && Objects.equals(segmentsIter.next(), expectedSegment)); + + // cache.listAllRemoteLogSegments() should contain the above segment. + Iterator allSegmentsIter = cache.listAllRemoteLogSegments(); + Assertions.assertTrue(allSegmentsIter.hasNext() && Objects.equals(allSegmentsIter.next(), expectedSegment)); + } + + private static class EpochOffset { + final int epoch; + final long offset; + + private EpochOffset(int epoch, long offset) { + this.epoch = epoch; + this.offset = offset; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EpochOffset that = (EpochOffset) o; + return epoch == that.epoch && offset == that.offset; + } + + @Override + public int hashCode() { + return Objects.hash(epoch, offset); + } + + @Override + public String toString() { + return "EpochOffset{" + + "epoch=" + epoch + + ", offset=" + offset + + '}'; + } + } +} \ No newline at end of file diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java new file mode 100644 index 0000000000000..9d3f47dbb6fab --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManager.java @@ -0,0 +1,161 @@ +/* + * 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.server.log.remote.storage; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * This class is an implementation of {@link RemoteLogMetadataManager} backed by in-memory store. + * This class is not completely thread safe. + */ +public class InmemoryRemoteLogMetadataManager implements RemoteLogMetadataManager { + private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteLogMetadataManager.class); + + private Map idToPartitionDeleteMetadata = + new ConcurrentHashMap<>(); + + private Map idToRemoteLogMetadataCache = new ConcurrentHashMap<>(); + + @Override + public void addRemoteLogSegmentMetadata(RemoteLogSegmentMetadata remoteLogSegmentMetadata) + throws RemoteStorageException { + log.debug("Adding remote log segment : [{}]", remoteLogSegmentMetadata); + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + RemoteLogSegmentId remoteLogSegmentId = remoteLogSegmentMetadata.remoteLogSegmentId(); + + idToRemoteLogMetadataCache + .computeIfAbsent(remoteLogSegmentId.topicIdPartition(), id -> new RemoteLogMetadataCache()) + .addCopyInProgressSegment(remoteLogSegmentMetadata); + } + + @Override + public void updateRemoteLogSegmentMetadata(RemoteLogSegmentMetadataUpdate metadataUpdate) + throws RemoteStorageException { + log.debug("Updating remote log segment: [{}]", metadataUpdate); + Objects.requireNonNull(metadataUpdate, "metadataUpdate can not be null"); + + getRemoteLogMetadataCache(metadataUpdate.remoteLogSegmentId().topicIdPartition()) + .updateRemoteLogSegmentMetadata(metadataUpdate); + } + + private RemoteLogMetadataCache getRemoteLogMetadataCache(TopicIdPartition topicIdPartition) + throws RemoteResourceNotFoundException { + RemoteLogMetadataCache remoteLogMetadataCache = idToRemoteLogMetadataCache.get(topicIdPartition); + if (remoteLogMetadataCache == null) { + throw new RemoteResourceNotFoundException("No existing metadata found for partition: " + topicIdPartition); + } + + return remoteLogMetadataCache; + } + + @Override + public Optional remoteLogSegmentMetadata(TopicIdPartition topicIdPartition, + int epochForOffset, + long offset) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + return getRemoteLogMetadataCache(topicIdPartition).remoteLogSegmentMetadata(epochForOffset, offset); + } + + @Override + public Optional highestOffsetForEpoch(TopicIdPartition topicIdPartition, + int leaderEpoch) throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + return getRemoteLogMetadataCache(topicIdPartition).highestOffsetForEpoch(leaderEpoch); + } + + @Override + public void putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata remotePartitionDeleteMetadata) + throws RemoteStorageException { + log.debug("Adding delete state with: [{}]", remotePartitionDeleteMetadata); + Objects.requireNonNull(remotePartitionDeleteMetadata, "remotePartitionDeleteMetadata can not be null"); + + TopicIdPartition topicIdPartition = remotePartitionDeleteMetadata.topicIdPartition(); + + RemotePartitionDeleteState targetState = remotePartitionDeleteMetadata.state(); + RemotePartitionDeleteMetadata existingMetadata = idToPartitionDeleteMetadata.get(topicIdPartition); + RemotePartitionDeleteState existingState = existingMetadata != null ? existingMetadata.state() : null; + if (!RemotePartitionDeleteState.isValidTransition(existingState, targetState)) { + throw new IllegalStateException("Current state: " + existingState + ", target state: " + targetState); + } + + idToPartitionDeleteMetadata.put(topicIdPartition, remotePartitionDeleteMetadata); + + if (targetState == RemotePartitionDeleteState.DELETE_PARTITION_FINISHED) { + // Remove the association for the partition. + idToRemoteLogMetadataCache.remove(topicIdPartition); + idToPartitionDeleteMetadata.remove(topicIdPartition); + } + } + + @Override + public Iterator listRemoteLogSegments(TopicIdPartition topicIdPartition) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + return getRemoteLogMetadataCache(topicIdPartition).listAllRemoteLogSegments(); + } + + @Override + public Iterator listRemoteLogSegments(TopicIdPartition topicIdPartition, int leaderEpoch) + throws RemoteStorageException { + Objects.requireNonNull(topicIdPartition, "topicIdPartition can not be null"); + + return getRemoteLogMetadataCache(topicIdPartition).listRemoteLogSegments(leaderEpoch); + } + + @Override + public void onPartitionLeadershipChanges(Set leaderPartitions, + Set followerPartitions) { + // It is not applicable for this implementation. This will track the segments that are added/updated as part of + // this instance. It does not depend upon any leader or follower transitions. + } + + @Override + public void onStopPartitions(Set partitions) { + // It is not applicable for this implementation. This will track the segments that are added/updated as part of + // this instance. It does not depend upon stopped partitions. + } + + @Override + public void close() throws IOException { + // Clearing the references to the map and assigning empty immutable maps. + // Practically, this instance will not be used once it is closed. + idToPartitionDeleteMetadata = Collections.emptyMap(); + idToRemoteLogMetadataCache = Collections.emptyMap(); + } + + @Override + public void configure(Map configs) { + // Intentionally left blank here as nothing to be initialized here. + } +} diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java new file mode 100644 index 0000000000000..51b99756892fc --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteLogMetadataManagerTest.java @@ -0,0 +1,130 @@ +/* + * 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.server.log.remote.storage; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCache; +import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataCacheTest; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * This class covers basic unit tests for {@link InmemoryRemoteLogMetadataManager}. InmemoryRemoteLogMetadataManager is + * used only in integration tests but not in production code. It mostly uses {@link RemoteLogMetadataCache} and it has + * broad test coverage with {@link RemoteLogMetadataCacheTest}. + */ +public class InmemoryRemoteLogMetadataManagerTest { + + private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(), + new TopicPartition("foo", 0)); + private static final int SEG_SIZE = 1024 * 1024; + private static final int BROKER_ID_0 = 0; + private static final int BROKER_ID_1 = 1; + + private final Time time = new MockTime(1); + + @Test + public void testFetchSegments() throws Exception { + InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager(); + + // 1.Create a segment with state COPY_SEGMENT_STARTED, and this segment should not be available. + Map segmentLeaderEpochs = Collections.singletonMap(0, 101L); + RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 101L, 200L, -1L, BROKER_ID_0, + time.milliseconds(), SEG_SIZE, segmentLeaderEpochs); + rlmm.addRemoteLogSegmentMetadata(segmentMetadata); + + // Search should not return the above segment. + Assertions.assertFalse(rlmm.remoteLogSegmentMetadata(TP0, 0, 150).isPresent()); + + // 2.Move that segment to COPY_SEGMENT_FINISHED state and this segment should be available. + RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate(segmentId, time.milliseconds(), + RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1); + rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate); + RemoteLogSegmentMetadata expectedSegmentMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate); + + // Search should return the above segment. + Optional segmentMetadataForOffset150 = rlmm.remoteLogSegmentMetadata(TP0, 0, 150); + Assertions.assertEquals(Optional.of(expectedSegmentMetadata), segmentMetadataForOffset150); + } + + @Test + public void testRemotePartitionDeletion() throws Exception { + InmemoryRemoteLogMetadataManager rlmm = new InmemoryRemoteLogMetadataManager(); + + // Create remote log segment metadata and add them to RLMM. + + // segment 0 + // offsets: [0-100] + // leader epochs (0,0), (1,20), (2,80) + Map segmentLeaderEpochs = new HashMap<>(); + segmentLeaderEpochs.put(0, 0L); + segmentLeaderEpochs.put(1, 20L); + segmentLeaderEpochs.put(2, 50L); + segmentLeaderEpochs.put(3, 80L); + RemoteLogSegmentId segmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + RemoteLogSegmentMetadata segmentMetadata = new RemoteLogSegmentMetadata(segmentId, 0L, 100L, + -1L, BROKER_ID_0, time.milliseconds(), SEG_SIZE, segmentLeaderEpochs); + rlmm.addRemoteLogSegmentMetadata(segmentMetadata); + RemoteLogSegmentMetadataUpdate segmentMetadataUpdate = new RemoteLogSegmentMetadataUpdate( + segmentId, time.milliseconds(), RemoteLogSegmentState.COPY_SEGMENT_FINISHED, BROKER_ID_1); + rlmm.updateRemoteLogSegmentMetadata(segmentMetadataUpdate); + + RemoteLogSegmentMetadata expectedSegMetadata = segmentMetadata.createWithUpdates(segmentMetadataUpdate); + + // Check that the segment exists in RLMM. + Optional segMetadataForOffset30Epoch1 = rlmm.remoteLogSegmentMetadata(TP0, 1, 30L); + Assertions.assertEquals(Optional.of(expectedSegMetadata), segMetadataForOffset30Epoch1); + + // Mark the partition for deletion. + rlmm.putRemotePartitionDeleteMetadata( + createRemotePartitionDeleteMetadata(RemotePartitionDeleteState.DELETE_PARTITION_MARKED)); + + Optional segmentMetadataAfterDelMark = rlmm.remoteLogSegmentMetadata(TP0, + 1, 30L); + Assertions.assertEquals(Optional.of(expectedSegMetadata), segmentMetadataAfterDelMark); + + // Set the partition deletion state as started. Partition and segments should still be accessible as they are not + // yet deleted. + rlmm.putRemotePartitionDeleteMetadata( + createRemotePartitionDeleteMetadata(RemotePartitionDeleteState.DELETE_PARTITION_STARTED)); + + Optional segmentMetadataAfterDelStart = rlmm.remoteLogSegmentMetadata(TP0, + 1, 30L); + Assertions.assertEquals(Optional.of(expectedSegMetadata), segmentMetadataAfterDelStart); + + // Set the partition deletion state as finished. RLMM should clear all its internal state for that partition. + rlmm.putRemotePartitionDeleteMetadata( + createRemotePartitionDeleteMetadata(RemotePartitionDeleteState.DELETE_PARTITION_FINISHED)); + + Assertions.assertThrows(RemoteResourceNotFoundException.class, + () -> rlmm.remoteLogSegmentMetadata(TP0, 1, 30L)); + } + + private RemotePartitionDeleteMetadata createRemotePartitionDeleteMetadata(RemotePartitionDeleteState state) { + return new RemotePartitionDeleteMetadata(TP0, state, time.milliseconds(), BROKER_ID_0); + } +} \ No newline at end of file diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java new file mode 100644 index 0000000000000..cbc994e35b6dd --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManager.java @@ -0,0 +1,177 @@ +/* + * 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.server.log.remote.storage; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.util.Collections; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +/** + * This class is an implementation of {@link RemoteStorageManager} backed by in-memory store. + */ +public class InmemoryRemoteStorageManager implements RemoteStorageManager { + private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManager.class); + + // Map of key to log data, which can be segment or any of its indexes. + private Map keyToLogData = new ConcurrentHashMap<>(); + + static String generateKeyForSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata) { + return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + ".segment"; + } + + static String generateKeyForIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata, + IndexType indexType) { + return remoteLogSegmentMetadata.remoteLogSegmentId().id().toString() + "." + indexType.toString(); + } + + // visible for testing. + boolean containsKey(String key) { + return keyToLogData.containsKey(key); + } + + @Override + public void copyLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata, + LogSegmentData logSegmentData) + throws RemoteStorageException { + log.debug("copying log segment and indexes for : {}", remoteLogSegmentMetadata); + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + Objects.requireNonNull(logSegmentData, "logSegmentData can not be null"); + + if (keyToLogData.containsKey(generateKeyForSegment(remoteLogSegmentMetadata))) { + throw new RemoteStorageException("It already contains the segment for the given id: " + + remoteLogSegmentMetadata.remoteLogSegmentId()); + } + + try { + keyToLogData.put(generateKeyForSegment(remoteLogSegmentMetadata), + Files.readAllBytes(logSegmentData.logSegment())); + keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.TRANSACTION), + Files.readAllBytes(logSegmentData.txnIndex())); + keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.LEADER_EPOCH), + logSegmentData.leaderEpochIndex().array()); + keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.PRODUCER_SNAPSHOT), + Files.readAllBytes(logSegmentData.producerSnapshotIndex())); + keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.OFFSET), + Files.readAllBytes(logSegmentData.offsetIndex())); + keyToLogData.put(generateKeyForIndex(remoteLogSegmentMetadata, IndexType.TIMESTAMP), + Files.readAllBytes(logSegmentData.timeIndex())); + } catch (Exception e) { + throw new RemoteStorageException(e); + } + log.debug("copied log segment and indexes for : {} successfully.", remoteLogSegmentMetadata); + } + + @Override + public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata, + int startPosition) + throws RemoteStorageException { + log.debug("Received fetch segment request at start position: [{}] for [{}]", startPosition, remoteLogSegmentMetadata); + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + return fetchLogSegment(remoteLogSegmentMetadata, startPosition, Integer.MAX_VALUE); + } + + @Override + public InputStream fetchLogSegment(RemoteLogSegmentMetadata remoteLogSegmentMetadata, + int startPosition, + int endPosition) throws RemoteStorageException { + log.debug("Received fetch segment request at start position: [{}] and end position: [{}] for segment [{}]", + startPosition, endPosition, remoteLogSegmentMetadata); + + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + + if (startPosition < 0 || endPosition < 0) { + throw new IllegalArgumentException("Given start position or end position must not be negative."); + } + + if (endPosition < startPosition) { + throw new IllegalArgumentException("end position must be greater than or equal to start position"); + } + + String key = generateKeyForSegment(remoteLogSegmentMetadata); + byte[] segment = keyToLogData.get(key); + + if (segment == null) { + throw new RemoteResourceNotFoundException("No remote log segment found with start offset:" + + remoteLogSegmentMetadata.startOffset() + " and id: " + + remoteLogSegmentMetadata.remoteLogSegmentId()); + } + + if (startPosition >= segment.length) { + throw new IllegalArgumentException("start position: " + startPosition + + " must be less than the length of the segment: " + segment.length); + } + + // If the given (endPosition + 1) is more than the segment length then the segment length is taken into account. + // Computed length should never be more than the existing segment size. + int length = Math.min(segment.length - 1, endPosition) - startPosition + 1; + log.debug("Length of the segment to be sent: [{}], for segment: [{}]", length, remoteLogSegmentMetadata); + + return new ByteArrayInputStream(segment, startPosition, length); + } + + @Override + public InputStream fetchIndex(RemoteLogSegmentMetadata remoteLogSegmentMetadata, + IndexType indexType) throws RemoteStorageException { + log.debug("Received fetch request for index type: [{}], segment [{}]", indexType, remoteLogSegmentMetadata); + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + Objects.requireNonNull(indexType, "indexType can not be null"); + + String key = generateKeyForIndex(remoteLogSegmentMetadata, indexType); + byte[] index = keyToLogData.get(key); + if (index == null) { + throw new RemoteResourceNotFoundException("No remote log segment index found with start offset:" + + remoteLogSegmentMetadata.startOffset() + " and id: " + + remoteLogSegmentMetadata.remoteLogSegmentId()); + } + + return new ByteArrayInputStream(index); + } + + @Override + public void deleteLogSegmentData(RemoteLogSegmentMetadata remoteLogSegmentMetadata) throws RemoteStorageException { + log.info("Deleting log segment for: [{}]", remoteLogSegmentMetadata); + Objects.requireNonNull(remoteLogSegmentMetadata, "remoteLogSegmentMetadata can not be null"); + String segmentKey = generateKeyForSegment(remoteLogSegmentMetadata); + keyToLogData.remove(segmentKey); + for (IndexType indexType : IndexType.values()) { + String key = generateKeyForIndex(remoteLogSegmentMetadata, indexType); + keyToLogData.remove(key); + } + log.info("Deleted log segment successfully for: [{}]", remoteLogSegmentMetadata); + } + + @Override + public void close() throws IOException { + // Clearing the references to the map and assigning empty immutable map. + // Practically, this instance will not be used once it is closed. + keyToLogData = Collections.emptyMap(); + } + + @Override + public void configure(Map configs) { + // Intentionally left blank here as nothing to be initialized here. + } +} diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java new file mode 100644 index 0000000000000..f60692fd99105 --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/InmemoryRemoteStorageManagerTest.java @@ -0,0 +1,246 @@ +/* + * 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.server.log.remote.storage; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.SeekableByteChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; + +public class InmemoryRemoteStorageManagerTest { + private static final Logger log = LoggerFactory.getLogger(InmemoryRemoteStorageManagerTest.class); + + private static final TopicPartition TP = new TopicPartition("foo", 1); + private static final File DIR = TestUtils.tempDirectory("inmem-rsm-"); + private static final Random RANDOM = new Random(); + + @Test + public void testCopyLogSegment() throws Exception { + InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager(); + RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata(); + LogSegmentData logSegmentData = createLogSegmentData(); + // Copy all the segment data. + rsm.copyLogSegmentData(segmentMetadata, logSegmentData); + + // Check that the segment data exists in in-memory RSM. + boolean containsSegment = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForSegment(segmentMetadata)); + Assertions.assertTrue(containsSegment); + + // Check that the indexes exist in in-memory RSM. + for (RemoteStorageManager.IndexType indexType : RemoteStorageManager.IndexType.values()) { + boolean containsIndex = rsm.containsKey(InmemoryRemoteStorageManager.generateKeyForIndex(segmentMetadata, indexType)); + Assertions.assertTrue(containsIndex); + } + } + + private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() { + TopicIdPartition topicPartition = new TopicIdPartition(Uuid.randomUuid(), TP); + RemoteLogSegmentId id = new RemoteLogSegmentId(topicPartition, Uuid.randomUuid()); + return new RemoteLogSegmentMetadata(id, 100L, 200L, System.currentTimeMillis(), 0, + System.currentTimeMillis(), 100, Collections.singletonMap(1, 100L)); + } + + @Test + public void testFetchLogSegmentIndexes() throws Exception { + InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager(); + RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata(); + int segSize = 100; + LogSegmentData logSegmentData = createLogSegmentData(segSize); + + // Copy the segment + rsm.copyLogSegmentData(segmentMetadata, logSegmentData); + + // Check segment data exists for the copied segment. + try (InputStream segmentStream = rsm.fetchLogSegment(segmentMetadata, 0)) { + checkContentSame(segmentStream, logSegmentData.logSegment()); + } + + HashMap expectedIndexToPaths = new HashMap<>(); + expectedIndexToPaths.put(RemoteStorageManager.IndexType.OFFSET, logSegmentData.offsetIndex()); + expectedIndexToPaths.put(RemoteStorageManager.IndexType.TIMESTAMP, logSegmentData.timeIndex()); + expectedIndexToPaths.put(RemoteStorageManager.IndexType.TRANSACTION, logSegmentData.txnIndex()); + expectedIndexToPaths.put(RemoteStorageManager.IndexType.PRODUCER_SNAPSHOT, logSegmentData.producerSnapshotIndex()); + + // Check all segment indexes exist for the copied segment. + for (Map.Entry entry : expectedIndexToPaths.entrySet()) { + RemoteStorageManager.IndexType indexType = entry.getKey(); + Path indexPath = entry.getValue(); + log.debug("Fetching index type: {}, indexPath: {}", indexType, indexPath); + + try (InputStream offsetIndexStream = rsm.fetchIndex(segmentMetadata, indexType)) { + checkContentSame(offsetIndexStream, indexPath); + } + } + + try (InputStream leaderEpochIndexStream = rsm.fetchIndex(segmentMetadata, RemoteStorageManager.IndexType.LEADER_EPOCH)) { + ByteBuffer leaderEpochIndex = logSegmentData.leaderEpochIndex(); + Assertions.assertEquals(leaderEpochIndex, + readAsByteBuffer(leaderEpochIndexStream, leaderEpochIndex.array().length)); + } + } + + @Test + public void testFetchSegmentsForRange() throws Exception { + InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager(); + RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata(); + int segSize = 100; + LogSegmentData logSegmentData = createLogSegmentData(segSize); + Path path = logSegmentData.logSegment(); + + // Copy the segment + rsm.copyLogSegmentData(segmentMetadata, logSegmentData); + + // 1. Fetch segment for startPos at 0 + doTestFetchForRange(rsm, segmentMetadata, path, 0, 40); + + // 2. Fetch segment for start and end positions as start and end of the segment. + doTestFetchForRange(rsm, segmentMetadata, path, 0, segSize); + + // 3. Fetch segment for endPos at the end of segment. + doTestFetchForRange(rsm, segmentMetadata, path, 90, segSize - 90); + + // 4. Fetch segment only for the start position. + doTestFetchForRange(rsm, segmentMetadata, path, 0, 1); + + // 5. Fetch segment only for the end position. + doTestFetchForRange(rsm, segmentMetadata, path, segSize - 1, 1); + + // 6. Fetch for any range other than boundaries. + doTestFetchForRange(rsm, segmentMetadata, path, 3, 90); + } + + private void doTestFetchForRange(InmemoryRemoteStorageManager rsm, RemoteLogSegmentMetadata rlsm, Path path, + int startPos, int len) throws Exception { + // Read from the segment for the expected range. + ByteBuffer expectedSegRangeBytes = ByteBuffer.allocate(len); + try (SeekableByteChannel seekableByteChannel = Files.newByteChannel(path)) { + seekableByteChannel.position(startPos).read(expectedSegRangeBytes); + } + expectedSegRangeBytes.rewind(); + + // Fetch from in-memory RSM for the same range + ByteBuffer fetchedSegRangeBytes = ByteBuffer.allocate(len); + try (InputStream segmentRangeStream = rsm.fetchLogSegment(rlsm, startPos, startPos + len - 1)) { + Utils.readFully(segmentRangeStream, fetchedSegRangeBytes); + } + fetchedSegRangeBytes.rewind(); + Assertions.assertEquals(expectedSegRangeBytes, fetchedSegRangeBytes); + } + + @Test + public void testFetchInvalidRange() throws Exception { + InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager(); + RemoteLogSegmentMetadata remoteLogSegmentMetadata = createRemoteLogSegmentMetadata(); + int segSize = 100; + LogSegmentData logSegmentData = createLogSegmentData(segSize); + + // Copy the segment + rsm.copyLogSegmentData(remoteLogSegmentMetadata, logSegmentData); + + // Check fetch segments with invalid ranges like startPos < endPos + Assertions.assertThrows(Exception.class, () -> rsm.fetchLogSegment(remoteLogSegmentMetadata, 2, 1)); + + // Check fetch segments with invalid ranges like startPos or endPos as negative. + Assertions.assertThrows(Exception.class, () -> rsm.fetchLogSegment(remoteLogSegmentMetadata, -1, 0)); + Assertions.assertThrows(Exception.class, () -> rsm.fetchLogSegment(remoteLogSegmentMetadata, -2, -1)); + } + + @Test + public void testDeleteSegment() throws Exception { + InmemoryRemoteStorageManager rsm = new InmemoryRemoteStorageManager(); + RemoteLogSegmentMetadata segmentMetadata = createRemoteLogSegmentMetadata(); + LogSegmentData logSegmentData = createLogSegmentData(); + + // Copy a log segment. + rsm.copyLogSegmentData(segmentMetadata, logSegmentData); + + // Check that the copied segment exists in rsm and it is same. + try (InputStream segmentStream = rsm.fetchLogSegment(segmentMetadata, 0)) { + checkContentSame(segmentStream, logSegmentData.logSegment()); + } + + // Delete segment and check that it does not exist in RSM. + rsm.deleteLogSegmentData(segmentMetadata); + + // Check that the segment data does not exist. + Assertions.assertThrows(RemoteResourceNotFoundException.class, () -> rsm.fetchLogSegment(segmentMetadata, 0)); + + // Check that the segment data does not exist for range. + Assertions.assertThrows(RemoteResourceNotFoundException.class, () -> rsm.fetchLogSegment(segmentMetadata, 0, 1)); + + // Check that all the indexes are not found. + for (RemoteStorageManager.IndexType indexType : RemoteStorageManager.IndexType.values()) { + Assertions.assertThrows(RemoteResourceNotFoundException.class, () -> rsm.fetchIndex(segmentMetadata, indexType)); + } + } + + private void checkContentSame(InputStream segmentStream, Path path) throws IOException { + byte[] segmentBytes = Files.readAllBytes(path); + ByteBuffer byteBuffer = readAsByteBuffer(segmentStream, segmentBytes.length); + Assertions.assertEquals(ByteBuffer.wrap(segmentBytes), byteBuffer); + } + + private ByteBuffer readAsByteBuffer(InputStream segmentStream, + int len) throws IOException { + ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[len]); + Utils.readFully(segmentStream, byteBuffer); + byteBuffer.rewind(); + return byteBuffer; + } + + private LogSegmentData createLogSegmentData() throws Exception { + return createLogSegmentData(100); + } + + private LogSegmentData createLogSegmentData(int segSize) throws Exception { + int prefix = Math.abs(RANDOM.nextInt()); + Path segment = new File(DIR, prefix + ".seg").toPath(); + Files.write(segment, TestUtils.randomBytes(segSize)); + + Path offsetIndex = new File(DIR, prefix + ".oi").toPath(); + Files.write(offsetIndex, TestUtils.randomBytes(10)); + + Path timeIndex = new File(DIR, prefix + ".ti").toPath(); + Files.write(timeIndex, TestUtils.randomBytes(10)); + + Path txnIndex = new File(DIR, prefix + ".txni").toPath(); + Files.write(txnIndex, TestUtils.randomBytes(10)); + + Path producerSnapshotIndex = new File(DIR, prefix + ".psi").toPath(); + Files.write(producerSnapshotIndex, TestUtils.randomBytes(10)); + + ByteBuffer leaderEpochIndex = ByteBuffer.wrap(TestUtils.randomBytes(10)); + return new LogSegmentData(segment, offsetIndex, timeIndex, txnIndex, producerSnapshotIndex, leaderEpochIndex); + } +} diff --git a/storage/src/test/resources/log4j.properties b/storage/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..d128d8605c26b --- /dev/null +++ b/storage/src/test/resources/log4j.properties @@ -0,0 +1,21 @@ +# 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. +log4j.rootLogger=OFF, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n + +log4j.logger.org.apache.kafka.server.log.remote.storage=INFO From a8a6952e4a0b6f161dee4caa3eb0cb0fbf23091c Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Tue, 13 Apr 2021 11:00:22 -0700 Subject: [PATCH 056/155] KAFKA-12471: Implement createPartitions in KIP-500 mode (#10343) Implement the createPartitions RPC which adds more partitions to a topic in the KIP-500 controller. Factor out some of the logic for validating manual partition assignments, so that it can be shared between createTopics and createPartitions. Add a startPartition argument to the replica placer. Reviewers: Jason Gustafson --- checkstyle/suppressions.xml | 2 +- .../scala/kafka/server/ControllerApis.scala | 71 +++++++- .../test/java/kafka/test/MockController.java | 27 +++ .../kafka/server/ControllerApisTest.scala | 36 +++- .../controller/BrokerHeartbeatManager.java | 7 +- .../controller/ClusterControlManager.java | 6 +- .../apache/kafka/controller/Controller.java | 12 ++ .../kafka/controller/QuorumController.java | 12 ++ .../controller/ReplicaPlacementPolicy.java | 5 +- .../controller/ReplicationControlManager.java | 160 +++++++++++++++--- .../SimpleReplicaPlacementPolicy.java | 3 +- .../controller/ClusterControlManagerTest.java | 2 +- .../ReplicationControlManagerTest.java | 134 +++++++++++++++ 13 files changed, 438 insertions(+), 39 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index fa1f5cc103f47..197702d7a67bf 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -273,7 +273,7 @@ + files="(KafkaEventQueue|ReplicationControlManager).java"/> diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index cc47ae9ca69e2..13812fd408f75 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -19,7 +19,7 @@ package kafka.server import java.util import java.util.Collections -import java.util.concurrent.ExecutionException +import java.util.concurrent.{CompletableFuture, ExecutionException} import kafka.network.RequestChannel import kafka.raft.RaftManager @@ -32,11 +32,14 @@ import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_A import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{ApiException, ClusterAuthorizationException, InvalidRequestException, TopicDeletionDisabledException} import org.apache.kafka.common.internals.FatalExitError +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult +import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection} import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseBroker -import org.apache.kafka.common.message.{BeginQuorumEpochResponseData, BrokerHeartbeatResponseData, BrokerRegistrationResponseData, CreateTopicsRequestData, CreateTopicsResponseData, DeleteTopicsRequestData, DeleteTopicsResponseData, DescribeQuorumResponseData, EndQuorumEpochResponseData, FetchResponseData, MetadataResponseData, SaslAuthenticateResponseData, SaslHandshakeResponseData, UnregisterBrokerResponseData, VoteResponseData} -import org.apache.kafka.common.protocol.Errors.{INVALID_REQUEST, TOPIC_AUTHORIZATION_FAILED} +import org.apache.kafka.common.message._ +import org.apache.kafka.common.protocol.Errors._ import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors} import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.Resource @@ -89,6 +92,7 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.ENVELOPE => handleEnvelopeRequest(request) case ApiKeys.SASL_HANDSHAKE => handleSaslHandshakeRequest(request) case ApiKeys.SASL_AUTHENTICATE => handleSaslAuthenticateRequest(request) + case ApiKeys.CREATE_PARTITIONS => handleCreatePartitions(request) case _ => throw new ApiException(s"Unsupported ApiKey ${request.context.header.apiKey}") } } catch { @@ -538,4 +542,65 @@ class ControllerApis(val requestChannel: RequestChannel, } }) } + + def handleCreatePartitions(request: RequestChannel.Request): Unit = { + val future = createPartitions(request.body[CreatePartitionsRequest].data, + authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME), + names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(n => n)) + future.whenComplete((responses, exception) => { + if (exception != null) { + requestHelper.handleError(request, exception) + } else { + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => { + val responseData = new CreatePartitionsResponseData(). + setResults(responses). + setThrottleTimeMs(requestThrottleMs) + new CreatePartitionsResponse(responseData) + }) + } + }) + } + + def createPartitions(request: CreatePartitionsRequestData, + hasClusterAuth: Boolean, + getCreatableTopics: Iterable[String] => Set[String]) + : CompletableFuture[util.List[CreatePartitionsTopicResult]] = { + val responses = new util.ArrayList[CreatePartitionsTopicResult]() + val duplicateTopicNames = new util.HashSet[String]() + val topicNames = new util.HashSet[String]() + request.topics().forEach { + topic => + if (!topicNames.add(topic.name())) { + duplicateTopicNames.add(topic.name()) + } + } + duplicateTopicNames.forEach { topicName => + responses.add(new CreatePartitionsTopicResult(). + setName(topicName). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Duplicate topic name.")) + topicNames.remove(topicName) + } + val authorizedTopicNames = { + if (hasClusterAuth) { + topicNames.asScala + } else { + getCreatableTopics(topicNames.asScala) + } + } + val topics = new util.ArrayList[CreatePartitionsTopic] + topicNames.forEach { topicName => + if (authorizedTopicNames.contains(topicName)) { + topics.add(request.topics().find(topicName)) + } else { + responses.add(new CreatePartitionsTopicResult(). + setName(topicName). + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code())) + } + } + controller.createPartitions(topics).thenApply { results => + results.forEach(response => responses.add(response)) + responses + } + } } diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java index 5bfddd49df4f5..e28380516b35c 100644 --- a/core/src/test/java/kafka/test/MockController.java +++ b/core/src/test/java/kafka/test/MockController.java @@ -25,6 +25,8 @@ import org.apache.kafka.common.message.AlterIsrResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; import org.apache.kafka.common.message.CreateTopicsResponseData; @@ -41,8 +43,10 @@ import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metadata.FeatureMapAndEpoch; +import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; @@ -227,6 +231,29 @@ public CompletableFuture waitForReadyBrokers(int minBrokers) { throw new UnsupportedOperationException(); } + @Override + synchronized public CompletableFuture> + createPartitions(List topicList) { + if (!active) { + CompletableFuture> future = new CompletableFuture<>(); + future.completeExceptionally(NOT_CONTROLLER_EXCEPTION); + return future; + } + List results = new ArrayList<>(); + for (CreatePartitionsTopic topic : topicList) { + if (topicNameToId.containsKey(topic.name())) { + results.add(new CreatePartitionsTopicResult().setName(topic.name()). + setErrorCode(Errors.NONE.code()). + setErrorMessage(null)); + } else { + results.add(new CreatePartitionsTopicResult().setName(topic.name()). + setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()). + setErrorMessage("No such topic as " + topic.name())); + } + } + return CompletableFuture.completedFuture(results); + } + @Override public CompletableFuture beginWritingSnapshot() { throw new UnsupportedOperationException(); diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 71b107c8973c9..a0fb556a3f6dc 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -32,15 +32,19 @@ import org.apache.kafka.common.Uuid.ZERO_UUID import org.apache.kafka.common.errors.{InvalidRequestException, NotControllerException, TopicDeletionDisabledException} import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.ApiMessageType.ListenerType -import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicCollection} +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult +import org.apache.kafka.common.message.CreateTopicsRequestData +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.DeleteTopicsRequestData.DeleteTopicState import org.apache.kafka.common.message.DeleteTopicsResponseData.DeletableTopicResult -import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreateTopicsRequestData, DeleteTopicsRequestData} +import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreatePartitionsRequestData, DeleteTopicsRequestData} import org.apache.kafka.common.network.{ClientInformation, ListenerName} -import org.apache.kafka.common.protocol.Errors.{CLUSTER_AUTHORIZATION_FAILED, INVALID_REQUEST, NONE, TOPIC_AUTHORIZATION_FAILED, UNKNOWN_TOPIC_ID, UNKNOWN_TOPIC_OR_PARTITION} +import org.apache.kafka.common.protocol.Errors._ import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerRegistrationRequest, BrokerRegistrationResponse, RequestContext, RequestHeader, RequestTestUtils} +import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.controller.Controller import org.apache.kafka.metadata.ApiMessageAndVersion @@ -371,6 +375,30 @@ class ControllerApisTest { _ => Set("foo", "bar"))) } + @Test + def testCreatePartitionsRequest(): Unit = { + val controller = new MockController.Builder(). + newInitialTopic("foo", Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q")). + newInitialTopic("bar", Uuid.fromString("VlFu5c51ToiNx64wtwkhQw")).build() + val controllerApis = createControllerApis(None, controller) + val request = new CreatePartitionsRequestData() + request.topics().add(new CreatePartitionsTopic().setName("foo").setAssignments(null).setCount(5)) + request.topics().add(new CreatePartitionsTopic().setName("bar").setAssignments(null).setCount(5)) + request.topics().add(new CreatePartitionsTopic().setName("bar").setAssignments(null).setCount(5)) + request.topics().add(new CreatePartitionsTopic().setName("bar").setAssignments(null).setCount(5)) + request.topics().add(new CreatePartitionsTopic().setName("baz").setAssignments(null).setCount(5)) + assertEquals(Set(new CreatePartitionsTopicResult().setName("foo"). + setErrorCode(NONE.code()). + setErrorMessage(null), + new CreatePartitionsTopicResult().setName("bar"). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Duplicate topic name."), + new CreatePartitionsTopicResult().setName("baz"). + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code()). + setErrorMessage(null)), + controllerApis.createPartitions(request, false, _ => Set("foo", "bar")).get().asScala.toSet) + } + @AfterEach def tearDown(): Unit = { quotas.shutdown() diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java index 4a41fb8c7303d..d78ac8833cad8 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokerHeartbeatManager.java @@ -435,6 +435,7 @@ List findStaleBrokers() { /** * Place replicas on unfenced brokers. * + * @param startPartition The partition ID to start with. * @param numPartitions The number of partitions to place. * @param numReplicas The number of replicas for each partition. * @param idToRack A function mapping broker id to broker rack. @@ -444,14 +445,16 @@ List findStaleBrokers() { * * @throws InvalidReplicationFactorException If too many replicas were requested. */ - List> placeReplicas(int numPartitions, short numReplicas, + List> placeReplicas(int startPartition, + int numPartitions, + short numReplicas, Function> idToRack, ReplicaPlacementPolicy policy) { // TODO: support using fenced brokers here if necessary to get to the desired // number of replicas. We probably need to add a fenced boolean in UsableBroker. Iterator iterator = new UsableBrokerIterator( unfenced.iterator(), idToRack); - return policy.createPlacement(numPartitions, numReplicas, iterator); + return policy.createPlacement(startPartition, numPartitions, numReplicas, iterator); } static class UsableBrokerIterator implements Iterator { diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 05380fc49e421..312d788ef930c 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -310,11 +310,13 @@ public void replay(UnfenceBrokerRecord record) { } } - public List> placeReplicas(int numPartitions, short numReplicas) { + public List> placeReplicas(int startPartition, + int numPartitions, + short numReplicas) { if (heartbeatManager == null) { throw new RuntimeException("ClusterControlManager is not active."); } - return heartbeatManager.placeReplicas(numPartitions, numReplicas, + return heartbeatManager.placeReplicas(startPartition, numPartitions, numReplicas, id -> brokerRegistrations.get(id).rack(), placementPolicy); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java index 7892c1991ee9a..f39af148caa27 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java @@ -24,6 +24,8 @@ import org.apache.kafka.common.message.AlterIsrResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.ElectLeadersRequestData; @@ -36,6 +38,7 @@ import org.apache.kafka.metadata.FeatureMapAndEpoch; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -195,6 +198,15 @@ CompletableFuture> alterClientQuotas( */ CompletableFuture beginWritingSnapshot(); + /** + * Create partitions on certain topics. + * + * @param topics The list of topics to create partitions for. + * @return A future yielding per-topic results. + */ + CompletableFuture> + createPartitions(List topics); + /** * Begin shutting down, but don't block. You must still call close to clean up all * resources. diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index db141c60f1375..f3f9ddd653205 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -43,6 +43,8 @@ import org.apache.kafka.common.message.AlterIsrResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.ElectLeadersRequestData; @@ -1094,6 +1096,16 @@ public CompletableFuture> alterClientQuotas( }); } + @Override + public CompletableFuture> + createPartitions(List topics) { + if (topics.isEmpty()) { + return CompletableFuture.completedFuture(Collections.emptyList()); + } + return appendWriteEvent("createPartitions", () -> + replicationControl.createPartitions(topics)); + } + @Override public CompletableFuture beginWritingSnapshot() { CompletableFuture future = new CompletableFuture<>(); diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicaPlacementPolicy.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicaPlacementPolicy.java index 44de85db34ef2..097463f7e76da 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicaPlacementPolicy.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicaPlacementPolicy.java @@ -32,6 +32,7 @@ interface ReplicaPlacementPolicy { /** * Create a new replica placement. * + * @param startPartition The partition ID to start with. * @param numPartitions The number of partitions to create placements for. * @param numReplicas The number of replicas to create for each partitions. * Must be positive. @@ -41,7 +42,9 @@ interface ReplicaPlacementPolicy { * * @throws InvalidReplicationFactorException If too many replicas were requested. */ - List> createPlacement(int numPartitions, short numReplicas, + List> createPlacement(int startPartition, + int numPartitions, + short numReplicas, Iterator iterator) throws InvalidReplicationFactorException; } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index 10e849091931f..f169d1fb4aafc 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -23,14 +23,21 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.errors.BrokerIdNotRegisteredException; +import org.apache.kafka.common.errors.InvalidPartitionsException; +import org.apache.kafka.common.errors.InvalidReplicaAssignmentException; import org.apache.kafka.common.errors.InvalidReplicationFactorException; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnknownTopicIdException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.message.AlterIsrRequestData; import org.apache.kafka.common.message.AlterIsrResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsAssignment; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; @@ -66,13 +73,13 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.OptionalInt; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; @@ -451,34 +458,18 @@ private ApiError createTopic(CreatableTopic topic, "A manual partition assignment was specified, but numPartitions " + "was not set to -1."); } + OptionalInt replicationFactor = OptionalInt.empty(); for (CreatableReplicaAssignment assignment : topic.assignments()) { if (newParts.containsKey(assignment.partitionIndex())) { return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, "Found multiple manual partition assignments for partition " + assignment.partitionIndex()); } - HashSet brokerIds = new HashSet<>(); - for (int brokerId : assignment.brokerIds()) { - if (!brokerIds.add(brokerId)) { - return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, - "The manual partition assignment specifies the same node " + - "id more than once."); - } else if (!clusterControl.unfenced(brokerId)) { - return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, - "The manual partition assignment contains node " + brokerId + - ", but that node is not usable."); - } - } - int[] replicas = new int[assignment.brokerIds().size()]; - for (int i = 0; i < replicas.length; i++) { - replicas[i] = assignment.brokerIds().get(i); - } - int[] isr = new int[assignment.brokerIds().size()]; - for (int i = 0; i < replicas.length; i++) { - isr[i] = assignment.brokerIds().get(i); - } - newParts.put(assignment.partitionIndex(), - new PartitionControlInfo(replicas, isr, null, null, isr[0], 0, 0)); + validateManualPartitionAssignment(assignment.brokerIds(), replicationFactor); + replicationFactor = OptionalInt.of(assignment.brokerIds().size()); + int[] replicas = Replicas.toArray(assignment.brokerIds()); + newParts.put(assignment.partitionIndex(), new PartitionControlInfo( + replicas, replicas, null, null, replicas[0], 0, 0)); } } else if (topic.replicationFactor() < -1 || topic.replicationFactor() == 0) { return new ApiError(Errors.INVALID_REPLICATION_FACTOR, @@ -497,7 +488,7 @@ private ApiError createTopic(CreatableTopic topic, defaultReplicationFactor : topic.replicationFactor(); try { List> replicas = clusterControl. - placeReplicas(numPartitions, replicationFactor); + placeReplicas(0, numPartitions, replicationFactor); for (int partitionId = 0; partitionId < replicas.size(); partitionId++) { int[] r = Replicas.toArray(replicas.get(partitionId)); newParts.put(partitionId, @@ -1007,6 +998,127 @@ ControllerResult maybeFenceStaleBrokers() { return ControllerResult.of(records, null); } + ControllerResult> + createPartitions(List topics) { + List records = new ArrayList<>(); + List results = new ArrayList<>(); + for (CreatePartitionsTopic topic : topics) { + ApiError apiError = ApiError.NONE; + try { + createPartitions(topic, records); + } catch (ApiException e) { + apiError = ApiError.fromThrowable(e); + } catch (Exception e) { + log.error("Unexpected createPartitions error for {}", topic, e); + apiError = ApiError.fromThrowable(e); + } + results.add(new CreatePartitionsTopicResult(). + setName(topic.name()). + setErrorCode(apiError.error().code()). + setErrorMessage(apiError.message())); + } + return new ControllerResult<>(records, results, true); + } + + void createPartitions(CreatePartitionsTopic topic, + List records) { + Uuid topicId = topicsByName.get(topic.name()); + if (topicId == null) { + throw new UnknownTopicOrPartitionException(); + } + TopicControlInfo topicInfo = topics.get(topicId); + if (topicInfo == null) { + throw new UnknownTopicOrPartitionException(); + } + if (topic.count() == topicInfo.parts.size()) { + throw new InvalidPartitionsException("Topic already has " + + topicInfo.parts.size() + " partition(s)."); + } else if (topic.count() < topicInfo.parts.size()) { + throw new InvalidPartitionsException("The topic " + topic.name() + " currently " + + "has " + topicInfo.parts.size() + " partition(s); " + topic.count() + + " would not be an increase."); + } + int additional = topic.count() - topicInfo.parts.size(); + if (topic.assignments() != null) { + if (topic.assignments().size() != additional) { + throw new InvalidReplicaAssignmentException("Attempted to add " + additional + + " additional partition(s), but only " + topic.assignments().size() + + " assignment(s) were specified."); + } + } + Iterator iterator = topicInfo.parts.values().iterator(); + if (!iterator.hasNext()) { + throw new UnknownServerException("Invalid state: topic " + topic.name() + + " appears to have no partitions."); + } + PartitionControlInfo partitionInfo = iterator.next(); + if (partitionInfo.replicas.length > Short.MAX_VALUE) { + throw new UnknownServerException("Invalid replication factor " + + partitionInfo.replicas.length + ": expected a number equal to less than " + + Short.MAX_VALUE); + } + short replicationFactor = (short) partitionInfo.replicas.length; + int startPartitionId = topicInfo.parts.size(); + + List> placements; + if (topic.assignments() != null) { + placements = new ArrayList<>(); + for (CreatePartitionsAssignment assignment : topic.assignments()) { + validateManualPartitionAssignment(assignment.brokerIds(), + OptionalInt.of(replicationFactor)); + placements.add(assignment.brokerIds()); + } + } else { + placements = clusterControl.placeReplicas(startPartitionId, additional, + replicationFactor); + } + int partitionId = startPartitionId; + for (List placement : placements) { + records.add(new ApiMessageAndVersion(new PartitionRecord(). + setPartitionId(partitionId). + setTopicId(topicId). + setReplicas(placement). + setIsr(placement). + setRemovingReplicas(null). + setAddingReplicas(null). + setLeader(placement.get(0)). + setLeaderEpoch(0). + setPartitionEpoch(0), (short) 0)); + partitionId++; + } + } + + void validateManualPartitionAssignment(List assignment, + OptionalInt replicationFactor) { + if (assignment.isEmpty()) { + throw new InvalidReplicaAssignmentException("The manual partition " + + "assignment includes an empty replica list."); + } + List sortedBrokerIds = new ArrayList<>(assignment); + sortedBrokerIds.sort(Integer::compare); + Integer prevBrokerId = null; + for (Integer brokerId : sortedBrokerIds) { + if (!clusterControl.brokerRegistrations().containsKey(brokerId)) { + throw new InvalidReplicaAssignmentException("The manual partition " + + "assignment includes broker " + brokerId + ", but no such broker is " + + "registered."); + } + if (brokerId.equals(prevBrokerId)) { + throw new InvalidReplicaAssignmentException("The manual partition " + + "assignment includes the broker " + prevBrokerId + " more than " + + "once."); + } + prevBrokerId = brokerId; + } + if (replicationFactor.isPresent() && + sortedBrokerIds.size() != replicationFactor.getAsInt()) { + throw new InvalidReplicaAssignmentException("The manual partition " + + "assignment includes a partition with " + sortedBrokerIds.size() + + " replica(s), but this is not consistent with previous " + + "partitions, which have " + replicationFactor.getAsInt() + " replica(s)."); + } + } + class ReplicationControlIterator implements Iterator> { private final long epoch; private final Iterator iterator; diff --git a/metadata/src/main/java/org/apache/kafka/controller/SimpleReplicaPlacementPolicy.java b/metadata/src/main/java/org/apache/kafka/controller/SimpleReplicaPlacementPolicy.java index 95e96cd62f4af..a2f7c892e45dc 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/SimpleReplicaPlacementPolicy.java +++ b/metadata/src/main/java/org/apache/kafka/controller/SimpleReplicaPlacementPolicy.java @@ -41,7 +41,8 @@ public SimpleReplicaPlacementPolicy(Random random) { } @Override - public List> createPlacement(int numPartitions, + public List> createPlacement(int startPartition, + int numPartitions, short numReplicas, Iterator iterator) { List usable = new ArrayList<>(); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 009e8811e1514..9b9ddb0cc1a78 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -142,7 +142,7 @@ public void testPlaceReplicas(int numUsableBrokers) throws Exception { String.format("broker %d was not unfenced.", i)); } for (int i = 0; i < 100; i++) { - List> results = clusterControl.placeReplicas(1, (short) 3); + List> results = clusterControl.placeReplicas(0, 1, (short) 3); HashSet seen = new HashSet<>(); for (Integer result : results.get(0)) { assertTrue(result >= 0); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 6739944a02988..e524581f32e36 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -20,10 +20,14 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.InvalidReplicaAssignmentException; import org.apache.kafka.common.errors.StaleBrokerEpochException; import org.apache.kafka.common.message.AlterIsrRequestData; import org.apache.kafka.common.message.AlterIsrResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsAssignment; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; @@ -57,6 +61,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.kafka.common.protocol.Errors.INVALID_PARTITIONS; +import static org.apache.kafka.common.protocol.Errors.INVALID_REPLICA_ASSIGNMENT; import static org.apache.kafka.common.protocol.Errors.INVALID_TOPIC_EXCEPTION; import static org.apache.kafka.common.protocol.Errors.NONE; import static org.apache.kafka.common.protocol.Errors.UNKNOWN_TOPIC_ID; @@ -518,4 +524,132 @@ public void testDeleteTopics() throws Exception { Long.MAX_VALUE, Collections.singleton("foo"))); assertEmptyTopicConfigs(ctx, "foo"); } + + + @Test + public void testCreatePartitions() throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + ReplicationControlManager replicationControl = ctx.replicationControl; + CreateTopicsRequestData request = new CreateTopicsRequestData(); + request.topics().add(new CreatableTopic().setName("foo"). + setNumPartitions(3).setReplicationFactor((short) 2)); + request.topics().add(new CreatableTopic().setName("bar"). + setNumPartitions(4).setReplicationFactor((short) 2)); + request.topics().add(new CreatableTopic().setName("quux"). + setNumPartitions(2).setReplicationFactor((short) 2)); + request.topics().add(new CreatableTopic().setName("foo2"). + setNumPartitions(2).setReplicationFactor((short) 2)); + registerBroker(0, ctx); + unfenceBroker(0, ctx); + registerBroker(1, ctx); + unfenceBroker(1, ctx); + ControllerResult createTopicResult = + replicationControl.createTopics(request); + ctx.replay(createTopicResult.records()); + List topics = new ArrayList<>(); + topics.add(new CreatePartitionsTopic(). + setName("foo").setCount(5).setAssignments(null)); + topics.add(new CreatePartitionsTopic(). + setName("bar").setCount(3).setAssignments(null)); + topics.add(new CreatePartitionsTopic(). + setName("baz").setCount(3).setAssignments(null)); + topics.add(new CreatePartitionsTopic(). + setName("quux").setCount(2).setAssignments(null)); + ControllerResult> createPartitionsResult = + replicationControl.createPartitions(topics); + assertEquals(Arrays.asList(new CreatePartitionsTopicResult(). + setName("foo"). + setErrorCode(NONE.code()). + setErrorMessage(null), + new CreatePartitionsTopicResult(). + setName("bar"). + setErrorCode(INVALID_PARTITIONS.code()). + setErrorMessage("The topic bar currently has 4 partition(s); 3 would not be an increase."), + new CreatePartitionsTopicResult(). + setName("baz"). + setErrorCode(UNKNOWN_TOPIC_OR_PARTITION.code()). + setErrorMessage(null), + new CreatePartitionsTopicResult(). + setName("quux"). + setErrorCode(INVALID_PARTITIONS.code()). + setErrorMessage("Topic already has 2 partition(s).")), + createPartitionsResult.response()); + ctx.replay(createPartitionsResult.records()); + List topics2 = new ArrayList<>(); + topics2.add(new CreatePartitionsTopic(). + setName("foo").setCount(6).setAssignments(Arrays.asList( + new CreatePartitionsAssignment().setBrokerIds(Arrays.asList(1, 0))))); + topics2.add(new CreatePartitionsTopic(). + setName("bar").setCount(5).setAssignments(Arrays.asList( + new CreatePartitionsAssignment().setBrokerIds(Arrays.asList(1))))); + topics2.add(new CreatePartitionsTopic(). + setName("quux").setCount(4).setAssignments(Arrays.asList( + new CreatePartitionsAssignment().setBrokerIds(Arrays.asList(1, 0))))); + topics2.add(new CreatePartitionsTopic(). + setName("foo2").setCount(3).setAssignments(Arrays.asList( + new CreatePartitionsAssignment().setBrokerIds(Arrays.asList(2, 0))))); + ControllerResult> createPartitionsResult2 = + replicationControl.createPartitions(topics2); + assertEquals(Arrays.asList(new CreatePartitionsTopicResult(). + setName("foo"). + setErrorCode(NONE.code()). + setErrorMessage(null), + new CreatePartitionsTopicResult(). + setName("bar"). + setErrorCode(INVALID_REPLICA_ASSIGNMENT.code()). + setErrorMessage("The manual partition assignment includes a partition " + + "with 1 replica(s), but this is not consistent with previous " + + "partitions, which have 2 replica(s)."), + new CreatePartitionsTopicResult(). + setName("quux"). + setErrorCode(INVALID_REPLICA_ASSIGNMENT.code()). + setErrorMessage("Attempted to add 2 additional partition(s), but only 1 assignment(s) were specified."), + new CreatePartitionsTopicResult(). + setName("foo2"). + setErrorCode(INVALID_REPLICA_ASSIGNMENT.code()). + setErrorMessage("The manual partition assignment includes broker 2, but " + + "no such broker is registered.")), + createPartitionsResult2.response()); + ctx.replay(createPartitionsResult2.records()); + } + + @Test + public void testValidateGoodManualPartitionAssignments() throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + registerBroker(1, ctx); + registerBroker(2, ctx); + registerBroker(3, ctx); + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1), + OptionalInt.of(1)); + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1), + OptionalInt.empty()); + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1, 2, 3), + OptionalInt.of(3)); + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1, 2, 3), + OptionalInt.empty()); + } + + @Test + public void testValidateBadManualPartitionAssignments() throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + registerBroker(1, ctx); + registerBroker(2, ctx); + assertEquals("The manual partition assignment includes an empty replica list.", + assertThrows(InvalidReplicaAssignmentException.class, () -> + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(), + OptionalInt.empty())).getMessage()); + assertEquals("The manual partition assignment includes broker 3, but no such " + + "broker is registered.", assertThrows(InvalidReplicaAssignmentException.class, () -> + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1, 2, 3), + OptionalInt.empty())).getMessage()); + assertEquals("The manual partition assignment includes the broker 2 more than " + + "once.", assertThrows(InvalidReplicaAssignmentException.class, () -> + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1, 2, 2), + OptionalInt.empty())).getMessage()); + assertEquals("The manual partition assignment includes a partition with 2 " + + "replica(s), but this is not consistent with previous partitions, which have " + + "3 replica(s).", assertThrows(InvalidReplicaAssignmentException.class, () -> + ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1, 2), + OptionalInt.of(3))).getMessage()); + } } From 232ffc358a415897634261371827b460f1b161c6 Mon Sep 17 00:00:00 2001 From: Luke Chen <43372967+showuon@users.noreply.github.com> Date: Wed, 14 Apr 2021 02:41:12 +0800 Subject: [PATCH 057/155] KAFKA-9295: improve KTableKTableForeignKeyInnerJoinMultiIntegrationTest (#10409) Wait for Streams to get to RUNNING before proceeding with the test, some general cleanup Reviewers: Anna Sophie Blee-Goldman --- ...reignKeyInnerJoinMultiIntegrationTest.java | 34 ++++++++----------- 1 file changed, 15 insertions(+), 19 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java index 45e249e689c3c..6156c484b290a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java @@ -57,7 +57,9 @@ import java.util.Set; import java.util.function.Function; +import static java.time.Duration.ofSeconds; import static java.util.Arrays.asList; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; import static org.junit.Assert.assertEquals; @Category({IntegrationTest.class}) @@ -154,8 +156,6 @@ public void before() throws IOException { streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, stateDirBasePath + "-1"); streamsConfigTwo.put(StreamsConfig.STATE_DIR_CONFIG, stateDirBasePath + "-2"); streamsConfigThree.put(StreamsConfig.STATE_DIR_CONFIG, stateDirBasePath + "-3"); - - IntegrationTestUtils.purgeLocalStreamsState(asList(streamsConfig, streamsConfigTwo, streamsConfigThree)); } @After @@ -172,38 +172,34 @@ public void after() throws IOException { streamsThree.close(); streamsThree = null; } - IntegrationTestUtils.purgeLocalStreamsState(streamsConfig); + IntegrationTestUtils.purgeLocalStreamsState(asList(streamsConfig, streamsConfigTwo, streamsConfigThree)); } - private enum JoinType { - INNER - } + private String innerJoinType = "INNER"; @Test public void shouldInnerJoinMultiPartitionQueryable() throws Exception { final Set> expectedOne = new HashSet<>(); expectedOne.add(new KeyValue<>(1, "value1=1.33,value2=10,value3=waffle")); - verifyKTableKTableJoin(JoinType.INNER, expectedOne, true); + verifyKTableKTableJoin(expectedOne); } - private void verifyKTableKTableJoin(final JoinType joinType, - final Set> expectedResult, - final boolean verifyQueryableState) throws Exception { - final String queryableName = verifyQueryableState ? joinType + "-store1" : null; - final String queryableNameTwo = verifyQueryableState ? joinType + "-store2" : null; + private void verifyKTableKTableJoin(final Set> expectedResult) throws Exception { + final String queryableName = innerJoinType + "-store1"; + final String queryableNameTwo = innerJoinType + "-store2"; streams = prepareTopology(queryableName, queryableNameTwo, streamsConfig); streamsTwo = prepareTopology(queryableName, queryableNameTwo, streamsConfigTwo); streamsThree = prepareTopology(queryableName, queryableNameTwo, streamsConfigThree); - streams.start(); - streamsTwo.start(); - streamsThree.start(); + + final List kafkaStreamsList = asList(streams, streamsTwo, streamsThree); + startApplicationAndWaitUntilRunning(kafkaStreamsList, ofSeconds(60)); final Set> result = new HashSet<>(IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( - CONSUMER_CONFIG, - OUTPUT, - expectedResult.size())); + CONSUMER_CONFIG, + OUTPUT, + expectedResult.size())); assertEquals(expectedResult, result); } @@ -274,7 +270,7 @@ private static KafkaStreams prepareTopology(final String queryableName, final ValueJoiner joinerTwo = (value1, value2) -> value1 + ",value3=" + value2; table1.join(table2, tableOneKeyExtractor, joiner, materialized) - .join(table3, joinedTableKeyExtractor, joinerTwo, materializedTwo) + .join(table3, joinedTableKeyExtractor, joinerTwo, materializedTwo) .toStream() .to(OUTPUT, Produced.with(serdeScope.decorateSerde(Serdes.Integer(), streamsConfig, true), From b163706dc92d15476a9c0749e7b1176594067203 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Wed, 14 Apr 2021 11:45:45 -0700 Subject: [PATCH 058/155] KAFKA-12650: fix NPE in InternalTopicManagerTest (#10529) Reviewers: A. Sophie Blee-Goldman , Luke Chen (@showuon), Bruno Cadonna , Chia-Ping Tsai --- .../streams/processor/internals/InternalTopicManagerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index ead5a28c89f1e..653250c9d8a7a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -153,9 +153,9 @@ public void shouldNotCreateTopicsWithEmptyInput() throws Exception { @Test public void shouldOnlyRetryNotSuccessfulFuturesDuringSetup() { - final AdminClient admin = EasyMock.createNiceMock(AdminClient.class); + final AdminClient admin = EasyMock.createMock(AdminClient.class); final StreamsConfig streamsConfig = new StreamsConfig(config); - final InternalTopicManager topicManager = new InternalTopicManager(Time.SYSTEM, admin, streamsConfig); + final InternalTopicManager topicManager = new InternalTopicManager(new MockTime(1L), admin, streamsConfig); final KafkaFutureImpl createTopicFailFuture = new KafkaFutureImpl<>(); createTopicFailFuture.completeExceptionally(new TopicExistsException("exists")); final KafkaFutureImpl createTopicSuccessfulFuture = new KafkaFutureImpl<>(); From 89933f21f204abf75336464d3ac24a4fdd254628 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 14 Apr 2021 14:38:37 -0700 Subject: [PATCH 059/155] KAFKA-12612: Remove `checksum` from ConsumerRecord/RecordMetadata for 3.0 (#10470) The methods have been deprecated since 0.11 without replacement since message format 2 moved the checksum to the record batch (instead of the record). Unfortunately, we did not deprecate the constructors that take a checksum (even though we intended to) so we cannot remove them. I have deprecated them for removal in 4.0 and added a single non deprecated constructor to `ConsumerRecord` and `RecordMetadata` that take all remaining parameters. `ConsumerRecord` could do with one additional convenience constructor, but that requires a KIP and hence should be done separately. Also: * Removed `ChecksumMessageFormatter`, which is technically not public API, but may have been used with the console consumer. * Updated all usages of `ConsumerRecord`/`RecordMetadata` constructors to use the non deprecated ones. * Added tests for deprecated `ConsumerRecord/`RecordMetadata` constructors. Reviewers: Chia-Ping Tsai , David Jacot --- .../clients/consumer/ConsumerRecord.java | 115 +++++++++------ .../clients/consumer/internals/Fetcher.java | 2 +- .../kafka/clients/producer/KafkaProducer.java | 2 +- .../kafka/clients/producer/MockProducer.java | 10 +- .../clients/producer/RecordMetadata.java | 53 ++++--- .../internals/FutureRecordMetadata.java | 12 +- .../producer/internals/ProducerBatch.java | 6 +- .../internals/ProducerInterceptors.java | 2 +- .../kafka/common/record/DefaultRecord.java | 12 -- .../common/record/MemoryRecordsBuilder.java | 79 ++++------ .../clients/consumer/ConsumerRecordTest.java | 68 +++++++-- .../clients/consumer/ConsumerRecordsTest.java | 11 +- .../clients/consumer/MockConsumerTest.java | 13 +- .../internals/ConsumerInterceptorsTest.java | 14 +- .../clients/producer/RecordMetadataTest.java | 47 +++--- .../clients/producer/RecordSendTest.java | 6 +- .../internals/FutureRecordMetadataTest.java | 1 - .../producer/internals/ProducerBatchTest.java | 26 ---- .../internals/ProducerInterceptorsTest.java | 2 +- .../record/MemoryRecordsBuilderTest.java | 3 +- .../kafka/test/MockConsumerInterceptor.java | 10 +- .../connect/mirror/MirrorSourceTaskTest.java | 7 +- .../errors/WorkerErrantRecordReporter.java | 5 +- .../connect/runtime/WorkerSinkTaskTest.java | 14 +- .../runtime/WorkerSinkTaskThreadedTest.java | 15 +- .../connect/runtime/WorkerSourceTaskTest.java | 2 +- .../runtime/errors/ProcessingContextTest.java | 2 +- .../storage/KafkaConfigBackingStoreTest.java | 138 ++++++++++++------ .../storage/KafkaOffsetBackingStoreTest.java | 38 +++-- .../storage/KafkaStatusBackingStoreTest.java | 4 +- .../kafka/connect/util/KafkaBasedLogTest.java | 32 ++-- .../scala/kafka/tools/ConsoleConsumer.scala | 22 +-- .../kafka/api/BaseProducerSendTest.scala | 3 - .../tools/DefaultMessageFormatterTest.scala | 5 +- .../kafka/tools/ConsoleConsumerTest.scala | 25 +--- docs/upgrade.html | 14 +- .../internals/RecordDeserializer.java | 6 +- .../state/internals/RecordConverters.java | 1 - .../LogAndSkipOnInvalidTimestampTest.java | 8 +- .../processor/TimestampExtractorTest.java | 8 +- .../internals/GlobalStateTaskTest.java | 4 +- .../internals/RecordDeserializerTest.java | 8 +- .../processor/internals/RecordQueueTest.java | 74 ++++++---- .../processor/internals/StreamTaskTest.java | 17 ++- .../processor/internals/StreamThreadTest.java | 26 ++-- .../testutil/ConsumerRecordUtil.java | 8 +- .../state/internals/RecordConvertersTest.java | 5 +- .../TimeOrderedKeyValueBufferTest.java | 114 ++++++++------- .../kafka/test/MockRestoreConsumer.java | 5 +- .../kafka/streams/TopologyTestDriver.java | 9 +- .../streams/test/ConsumerRecordFactory.java | 5 +- .../kafka/streams/test/TestRecordTest.java | 5 +- 52 files changed, 636 insertions(+), 487 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index a7dad7bc5e2a9..2ae93e8220037 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -18,7 +18,6 @@ import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.record.DefaultRecord; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.TimestampType; @@ -32,6 +31,12 @@ public class ConsumerRecord { public static final long NO_TIMESTAMP = RecordBatch.NO_TIMESTAMP; public static final int NULL_SIZE = -1; + + /** + * @deprecated checksums are no longer exposed by this class, this constant will be removed in Apache Kafka 4.0 + * (deprecated since 3.0). + */ + @Deprecated public static final int NULL_CHECKSUM = -1; private final String topic; @@ -46,8 +51,6 @@ public class ConsumerRecord { private final V value; private final Optional leaderEpoch; - private volatile Long checksum; - /** * Creates a record to be received from a specified topic and partition (provided for * compatibility with Kafka 0.9 before the message format supported timestamps and before @@ -64,8 +67,52 @@ public ConsumerRecord(String topic, long offset, K key, V value) { - this(topic, partition, offset, NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, - NULL_CHECKSUM, NULL_SIZE, NULL_SIZE, key, value); + this(topic, partition, offset, NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, NULL_SIZE, NULL_SIZE, key, value, + new RecordHeaders(), Optional.empty()); + } + + /** + * Creates a record to be received from a specified topic and partition + * + * @param topic The topic this record is received from + * @param partition The partition of the topic this record is received from + * @param offset The offset of this record in the corresponding Kafka partition + * @param timestamp The timestamp of the record. + * @param timestampType The timestamp type + * @param serializedKeySize The length of the serialized key + * @param serializedValueSize The length of the serialized value + * @param key The key of the record, if one exists (null is allowed) + * @param value The record contents + * @param headers The headers of the record + * @param leaderEpoch Optional leader epoch of the record (may be empty for legacy record formats) + */ + public ConsumerRecord(String topic, + int partition, + long offset, + long timestamp, + TimestampType timestampType, + int serializedKeySize, + int serializedValueSize, + K key, + V value, + Headers headers, + Optional leaderEpoch) { + if (topic == null) + throw new IllegalArgumentException("Topic cannot be null"); + if (headers == null) + throw new IllegalArgumentException("Headers cannot be null"); + + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.timestamp = timestamp; + this.timestampType = timestampType; + this.serializedKeySize = serializedKeySize; + this.serializedValueSize = serializedValueSize; + this.key = key; + this.value = value; + this.headers = headers; + this.leaderEpoch = leaderEpoch; } /** @@ -77,12 +124,15 @@ public ConsumerRecord(String topic, * @param offset The offset of this record in the corresponding Kafka partition * @param timestamp The timestamp of the record. * @param timestampType The timestamp type - * @param checksum The checksum (CRC32) of the full record * @param serializedKeySize The length of the serialized key * @param serializedValueSize The length of the serialized value * @param key The key of the record, if one exists (null is allowed) * @param value The record contents + * + * @deprecated use one of the constructors without a `checksum` parameter. This constructor will be removed in + * Apache Kafka 4.0 (deprecated since 3.0). */ + @Deprecated public ConsumerRecord(String topic, int partition, long offset, @@ -93,8 +143,8 @@ public ConsumerRecord(String topic, int serializedValueSize, K key, V value) { - this(topic, partition, offset, timestamp, timestampType, checksum, serializedKeySize, serializedValueSize, - key, value, new RecordHeaders()); + this(topic, partition, offset, timestamp, timestampType, serializedKeySize, serializedValueSize, + key, value, new RecordHeaders(), Optional.empty()); } /** @@ -105,13 +155,16 @@ public ConsumerRecord(String topic, * @param offset The offset of this record in the corresponding Kafka partition * @param timestamp The timestamp of the record. * @param timestampType The timestamp type - * @param checksum The checksum (CRC32) of the full record * @param serializedKeySize The length of the serialized key * @param serializedValueSize The length of the serialized value * @param key The key of the record, if one exists (null is allowed) * @param value The record contents * @param headers The headers of the record. + * + * @deprecated use one of the constructors without a `checksum` parameter. This constructor will be removed in + * Apache Kafka 4.0 (deprecated since 3.0). */ + @Deprecated public ConsumerRecord(String topic, int partition, long offset, @@ -123,7 +176,7 @@ public ConsumerRecord(String topic, K key, V value, Headers headers) { - this(topic, partition, offset, timestamp, timestampType, checksum, serializedKeySize, serializedValueSize, + this(topic, partition, offset, timestamp, timestampType, serializedKeySize, serializedValueSize, key, value, headers, Optional.empty()); } @@ -135,14 +188,17 @@ public ConsumerRecord(String topic, * @param offset The offset of this record in the corresponding Kafka partition * @param timestamp The timestamp of the record. * @param timestampType The timestamp type - * @param checksum The checksum (CRC32) of the full record * @param serializedKeySize The length of the serialized key * @param serializedValueSize The length of the serialized value * @param key The key of the record, if one exists (null is allowed) * @param value The record contents * @param headers The headers of the record * @param leaderEpoch Optional leader epoch of the record (may be empty for legacy record formats) + * + * @deprecated use one of the constructors without a `checksum` parameter. This constructor will be removed in + * Apache Kafka 4.0 (deprecated since 3.0). */ + @Deprecated public ConsumerRecord(String topic, int partition, long offset, @@ -155,23 +211,8 @@ public ConsumerRecord(String topic, V value, Headers headers, Optional leaderEpoch) { - if (topic == null) - throw new IllegalArgumentException("Topic cannot be null"); - if (headers == null) - throw new IllegalArgumentException("Headers cannot be null"); - - this.topic = topic; - this.partition = partition; - this.offset = offset; - this.timestamp = timestamp; - this.timestampType = timestampType; - this.checksum = checksum; - this.serializedKeySize = serializedKeySize; - this.serializedValueSize = serializedValueSize; - this.key = key; - this.value = value; - this.headers = headers; - this.leaderEpoch = leaderEpoch; + this(topic, partition, offset, timestamp, timestampType, serializedKeySize, serializedValueSize, key, value, headers, + leaderEpoch); } /** @@ -230,24 +271,6 @@ public TimestampType timestampType() { return timestampType; } - /** - * The checksum (CRC32) of the record. - * - * @deprecated As of Kafka 0.11.0. Because of the potential for message format conversion on the broker, the - * checksum returned by the broker may not match what was computed by the producer. - * It is therefore unsafe to depend on this checksum for end-to-end delivery guarantees. Additionally, - * message format v2 does not include a record-level checksum (for performance, the record checksum - * was replaced with a batch checksum). To maintain compatibility, a partial checksum computed from - * the record timestamp, serialized key size, and serialized value size is returned instead, but - * this should not be depended on for end-to-end reliability. - */ - @Deprecated - public long checksum() { - if (checksum == null) - this.checksum = DefaultRecord.computePartialChecksum(timestamp, serializedKeySize, serializedValueSize); - return this.checksum; - } - /** * The size of the serialized, uncompressed key in bytes. If key is null, the returned size * is -1. diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 8f9b7a2e5bc3b..fa214ff9005d3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -1385,7 +1385,7 @@ private ConsumerRecord parseRecord(TopicPartition partition, byte[] valueByteArray = valueBytes == null ? null : Utils.toArray(valueBytes); V value = valueBytes == null ? null : this.valueDeserializer.deserialize(partition.topic(), headers, valueByteArray); return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, - timestamp, timestampType, record.checksumOrNull(), + timestamp, timestampType, keyByteArray == null ? ConsumerRecord.NULL_SIZE : keyByteArray.length, valueByteArray == null ? ConsumerRecord.NULL_SIZE : valueByteArray.length, key, value, headers, leaderEpoch); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 0b01a866ecbfd..7f573d9203424 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1359,7 +1359,7 @@ private InterceptorCallback(Callback userCallback, ProducerInterceptors in } public void onCompletion(RecordMetadata metadata, Exception exception) { - metadata = metadata != null ? metadata : new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1L, -1, -1); + metadata = metadata != null ? metadata : new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1); this.interceptors.onAcknowledgement(metadata, exception); if (this.userCallback != null) this.userCallback.onCompletion(metadata, exception); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 650b69795d882..f733869d7fd0d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -311,10 +311,12 @@ public synchronized Future send(ProducerRecord record, Cal TopicPartition topicPartition = new TopicPartition(record.topic(), partition); ProduceRequestResult result = new ProduceRequestResult(topicPartition); FutureRecordMetadata future = new FutureRecordMetadata(result, 0, RecordBatch.NO_TIMESTAMP, - 0L, 0, 0, Time.SYSTEM); + 0, 0, Time.SYSTEM); long offset = nextOffset(topicPartition); - Completion completion = new Completion(offset, new RecordMetadata(topicPartition, 0, offset, - RecordBatch.NO_TIMESTAMP, 0L, 0, 0), result, callback, topicPartition); + long baseOffset = Math.max(0, offset - Integer.MAX_VALUE); + int batchIndex = (int) Math.min(Integer.MAX_VALUE, offset); + Completion completion = new Completion(offset, new RecordMetadata(topicPartition, baseOffset, batchIndex, + RecordBatch.NO_TIMESTAMP, 0, 0), result, callback, topicPartition); if (!this.transactionInFlight) this.sent.add(record); @@ -537,7 +539,7 @@ public void complete(RuntimeException e) { if (e == null) callback.onCompletion(metadata, null); else - callback.onCompletion(new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1L, -1, -1), e); + callback.onCompletion(new RecordMetadata(tp, -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1), e); } result.done(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java index 1ae5cf654b8de..8efc4c4eeba60 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.producer; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.DefaultRecord; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.ProduceResponse; @@ -42,20 +41,37 @@ public final class RecordMetadata { private final int serializedValueSize; private final TopicPartition topicPartition; - private volatile Long checksum; - - public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp, - Long checksum, int serializedKeySize, int serializedValueSize) { - // ignore the relativeOffset if the base offset is -1, - // since this indicates the offset is unknown - this.offset = baseOffset == -1 ? baseOffset : baseOffset + relativeOffset; + /** + * Creates a new instance with the provided parameters. + */ + public RecordMetadata(TopicPartition topicPartition, long baseOffset, int batchIndex, long timestamp, + int serializedKeySize, int serializedValueSize) { + // ignore the batchIndex if the base offset is -1, since this indicates the offset is unknown + this.offset = baseOffset == -1 ? baseOffset : baseOffset + batchIndex; this.timestamp = timestamp; - this.checksum = checksum; this.serializedKeySize = serializedKeySize; this.serializedValueSize = serializedValueSize; this.topicPartition = topicPartition; } + /** + * Creates a new instance with the provided parameters. + * + * @deprecated use constructor without `checksum` parameter. This constructor will be removed in + * Apache Kafka 4.0 (deprecated since 3.0). + */ + @Deprecated + public RecordMetadata(TopicPartition topicPartition, long baseOffset, long batchIndex, long timestamp, + Long checksum, int serializedKeySize, int serializedValueSize) { + this(topicPartition, baseOffset, batchIndexToInt(batchIndex), timestamp, serializedKeySize, serializedValueSize); + } + + private static int batchIndexToInt(long batchIndex) { + if (batchIndex > Integer.MAX_VALUE) + throw new IllegalArgumentException("batchIndex is larger than Integer.MAX_VALUE: " + batchIndex); + return (int) batchIndex; + } + /** * Indicates whether the record metadata includes the offset. * @return true if the offset is included in the metadata, false otherwise. @@ -89,25 +105,6 @@ public long timestamp() { return this.timestamp; } - /** - * The checksum (CRC32) of the record. - * - * @deprecated As of Kafka 0.11.0. Because of the potential for message format conversion on the broker, the - * computed checksum may not match what was stored on the broker, or what will be returned to the consumer. - * It is therefore unsafe to depend on this checksum for end-to-end delivery guarantees. Additionally, - * message format v2 does not include a record-level checksum (for performance, the record checksum - * was replaced with a batch checksum). To maintain compatibility, a partial checksum computed from - * the record timestamp, serialized key size, and serialized value size is returned instead, but - * this should not be depended on for end-to-end reliability. - */ - @Deprecated - public long checksum() { - if (checksum == null) - // The checksum is null only for message format v2 and above, which do not have a record-level checksum. - this.checksum = DefaultRecord.computePartialChecksum(timestamp, serializedKeySize, serializedValueSize); - return this.checksum; - } - /** * The size of the serialized, uncompressed key in bytes. If key is null, the returned size * is -1. diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java index a9665a991bd3a..0026237a36258 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java @@ -32,18 +32,16 @@ public final class FutureRecordMetadata implements Future { private final ProduceRequestResult result; private final int batchIndex; private final long createTimestamp; - private final Long checksum; private final int serializedKeySize; private final int serializedValueSize; private final Time time; private volatile FutureRecordMetadata nextRecordMetadata = null; - public FutureRecordMetadata(ProduceRequestResult result, int batchIndex, long createTimestamp, - Long checksum, int serializedKeySize, int serializedValueSize, Time time) { + public FutureRecordMetadata(ProduceRequestResult result, int batchIndex, long createTimestamp, int serializedKeySize, + int serializedValueSize, Time time) { this.result = result; this.batchIndex = batchIndex; this.createTimestamp = createTimestamp; - this.checksum = checksum; this.serializedKeySize = serializedKeySize; this.serializedValueSize = serializedValueSize; this.time = time; @@ -101,15 +99,11 @@ RecordMetadata valueOrError() throws ExecutionException { return value(); } - Long checksumOrNull() { - return this.checksum; - } - RecordMetadata value() { if (nextRecordMetadata != null) return nextRecordMetadata.value(); return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.batchIndex, - timestamp(), this.checksum, this.serializedKeySize, this.serializedValueSize); + timestamp(), this.serializedKeySize, this.serializedValueSize); } private long timestamp() { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java index 6f76c799b281c..4da03627be145 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java @@ -106,12 +106,12 @@ public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, if (!recordsBuilder.hasRoomFor(timestamp, key, value, headers)) { return null; } else { - Long checksum = this.recordsBuilder.append(timestamp, key, value, headers); + this.recordsBuilder.append(timestamp, key, value, headers); this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(), recordsBuilder.compressionType(), key, value, headers)); this.lastAppendTime = now; FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount, - timestamp, checksum, + timestamp, key == null ? -1 : key.length, value == null ? -1 : value.length, Time.SYSTEM); @@ -136,7 +136,7 @@ private boolean tryAppendForSplit(long timestamp, ByteBuffer key, ByteBuffer val this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(), recordsBuilder.compressionType(), key, value, headers)); FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount, - timestamp, thunk.future.checksumOrNull(), + timestamp, key == null ? -1 : key.remaining(), value == null ? -1 : value.remaining(), Time.SYSTEM); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java index 61a8b7a170f88..ceec552e26d80 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java @@ -114,7 +114,7 @@ public void onSendError(ProducerRecord record, TopicPartition interceptTop record.partition() == null ? RecordMetadata.UNKNOWN_PARTITION : record.partition()); } interceptor.onAcknowledgement(new RecordMetadata(interceptTopicPartition, -1, -1, - RecordBatch.NO_TIMESTAMP, Long.valueOf(-1L), -1, -1), exception); + RecordBatch.NO_TIMESTAMP, -1, -1), exception); } } catch (Exception e) { // do not propagate interceptor exceptions, just log diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java index d85f1000bc1c9..bbaed94a31ae6 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java @@ -20,8 +20,6 @@ import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.utils.ByteUtils; -import org.apache.kafka.common.utils.Checksums; -import org.apache.kafka.common.utils.Crc32C; import org.apache.kafka.common.utils.PrimitiveRef; import org.apache.kafka.common.utils.PrimitiveRef.IntRef; import org.apache.kafka.common.utils.Utils; @@ -33,7 +31,6 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Objects; -import java.util.zip.Checksum; import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V2; @@ -632,13 +629,4 @@ static int recordSizeUpperBound(ByteBuffer key, ByteBuffer value, Header[] heade int valueSize = value == null ? -1 : value.remaining(); return MAX_RECORD_OVERHEAD + sizeOf(keySize, valueSize, headers); } - - - public static long computePartialChecksum(long timestamp, int serializedKeySize, int serializedValueSize) { - Checksum checksum = Crc32C.create(); - Checksums.updateLong(checksum, timestamp); - Checksums.updateInt(checksum, serializedKeySize); - Checksums.updateInt(checksum, serializedValueSize); - return checksum.getValue(); - } } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 91edf2cd58449..59ad2100a1140 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -396,9 +396,9 @@ private int writeLegacyCompressedWrapperHeader() { } /** - * Append a record and return its checksum for message format v0 and v1, or null for v2 and above. + * Append a new record at the given offset. */ - private Long appendWithOffset(long offset, boolean isControlRecord, long timestamp, ByteBuffer key, + private void appendWithOffset(long offset, boolean isControlRecord, long timestamp, ByteBuffer key, ByteBuffer value, Header[] headers) { try { if (isControlRecord != isControlBatch) @@ -419,9 +419,8 @@ private Long appendWithOffset(long offset, boolean isControlRecord, long timesta if (magic > RecordBatch.MAGIC_VALUE_V1) { appendDefaultRecord(offset, timestamp, key, value, headers); - return null; } else { - return appendLegacyRecord(offset, timestamp, key, value, magic); + appendLegacyRecord(offset, timestamp, key, value, magic); } } catch (IOException e) { throw new KafkaException("I/O exception when writing to the append stream, closing", e); @@ -435,10 +434,9 @@ private Long appendWithOffset(long offset, boolean isControlRecord, long timesta * @param key The record key * @param value The record value * @param headers The record headers if there are any - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long appendWithOffset(long offset, long timestamp, byte[] key, byte[] value, Header[] headers) { - return appendWithOffset(offset, false, timestamp, wrapNullable(key), wrapNullable(value), headers); + public void appendWithOffset(long offset, long timestamp, byte[] key, byte[] value, Header[] headers) { + appendWithOffset(offset, false, timestamp, wrapNullable(key), wrapNullable(value), headers); } /** @@ -448,10 +446,9 @@ public Long appendWithOffset(long offset, long timestamp, byte[] key, byte[] val * @param key The record key * @param value The record value * @param headers The record headers if there are any - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long appendWithOffset(long offset, long timestamp, ByteBuffer key, ByteBuffer value, Header[] headers) { - return appendWithOffset(offset, false, timestamp, key, value, headers); + public void appendWithOffset(long offset, long timestamp, ByteBuffer key, ByteBuffer value, Header[] headers) { + appendWithOffset(offset, false, timestamp, key, value, headers); } /** @@ -460,10 +457,9 @@ public Long appendWithOffset(long offset, long timestamp, ByteBuffer key, ByteBu * @param timestamp The record timestamp * @param key The record key * @param value The record value - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long appendWithOffset(long offset, long timestamp, byte[] key, byte[] value) { - return appendWithOffset(offset, timestamp, wrapNullable(key), wrapNullable(value), Record.EMPTY_HEADERS); + public void appendWithOffset(long offset, long timestamp, byte[] key, byte[] value) { + appendWithOffset(offset, timestamp, wrapNullable(key), wrapNullable(value), Record.EMPTY_HEADERS); } /** @@ -472,20 +468,18 @@ public Long appendWithOffset(long offset, long timestamp, byte[] key, byte[] val * @param timestamp The record timestamp * @param key The record key * @param value The record value - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long appendWithOffset(long offset, long timestamp, ByteBuffer key, ByteBuffer value) { - return appendWithOffset(offset, timestamp, key, value, Record.EMPTY_HEADERS); + public void appendWithOffset(long offset, long timestamp, ByteBuffer key, ByteBuffer value) { + appendWithOffset(offset, timestamp, key, value, Record.EMPTY_HEADERS); } /** * Append a new record at the given offset. * @param offset The absolute offset of the record in the log buffer * @param record The record to append - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long appendWithOffset(long offset, SimpleRecord record) { - return appendWithOffset(offset, record.timestamp(), record.key(), record.value(), record.headers()); + public void appendWithOffset(long offset, SimpleRecord record) { + appendWithOffset(offset, record.timestamp(), record.key(), record.value(), record.headers()); } /** @@ -494,15 +488,14 @@ public Long appendWithOffset(long offset, SimpleRecord record) { * * @param offset The absolute offset of the record in the log buffer * @param record The record to append - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long appendControlRecordWithOffset(long offset, SimpleRecord record) { + public void appendControlRecordWithOffset(long offset, SimpleRecord record) { short typeId = ControlRecordType.parseTypeId(record.key()); ControlRecordType type = ControlRecordType.fromTypeId(typeId); if (type == ControlRecordType.UNKNOWN) throw new IllegalArgumentException("Cannot append record with unknown control record type " + typeId); - return appendWithOffset(offset, true, record.timestamp(), + appendWithOffset(offset, true, record.timestamp(), record.key(), record.value(), record.headers()); } @@ -511,10 +504,9 @@ public Long appendControlRecordWithOffset(long offset, SimpleRecord record) { * @param timestamp The record timestamp * @param key The record key * @param value The record value - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long append(long timestamp, ByteBuffer key, ByteBuffer value) { - return append(timestamp, key, value, Record.EMPTY_HEADERS); + public void append(long timestamp, ByteBuffer key, ByteBuffer value) { + append(timestamp, key, value, Record.EMPTY_HEADERS); } /** @@ -525,8 +517,8 @@ public Long append(long timestamp, ByteBuffer key, ByteBuffer value) { * @param headers The record headers if there are any * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long append(long timestamp, ByteBuffer key, ByteBuffer value, Header[] headers) { - return appendWithOffset(nextSequentialOffset(), timestamp, key, value, headers); + public void append(long timestamp, ByteBuffer key, ByteBuffer value, Header[] headers) { + appendWithOffset(nextSequentialOffset(), timestamp, key, value, headers); } /** @@ -536,8 +528,8 @@ public Long append(long timestamp, ByteBuffer key, ByteBuffer value, Header[] he * @param value The record value * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long append(long timestamp, byte[] key, byte[] value) { - return append(timestamp, wrapNullable(key), wrapNullable(value), Record.EMPTY_HEADERS); + public void append(long timestamp, byte[] key, byte[] value) { + append(timestamp, wrapNullable(key), wrapNullable(value), Record.EMPTY_HEADERS); } /** @@ -546,19 +538,17 @@ public Long append(long timestamp, byte[] key, byte[] value) { * @param key The record key * @param value The record value * @param headers The record headers if there are any - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long append(long timestamp, byte[] key, byte[] value, Header[] headers) { - return append(timestamp, wrapNullable(key), wrapNullable(value), headers); + public void append(long timestamp, byte[] key, byte[] value, Header[] headers) { + append(timestamp, wrapNullable(key), wrapNullable(value), headers); } /** * Append a new record at the next sequential offset. * @param record The record to append - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - public Long append(SimpleRecord record) { - return appendWithOffset(nextSequentialOffset(), record); + public void append(SimpleRecord record) { + appendWithOffset(nextSequentialOffset(), record); } /** @@ -566,36 +556,29 @@ public Long append(SimpleRecord record) { * @param timestamp The record timestamp * @param type The control record type (cannot be UNKNOWN) * @param value The control record value - * @return CRC of the record or null if record-level CRC is not supported for the message format */ - private Long appendControlRecord(long timestamp, ControlRecordType type, ByteBuffer value) { + private void appendControlRecord(long timestamp, ControlRecordType type, ByteBuffer value) { Struct keyStruct = type.recordKey(); ByteBuffer key = ByteBuffer.allocate(keyStruct.sizeOf()); keyStruct.writeTo(key); key.flip(); - return appendWithOffset(nextSequentialOffset(), true, timestamp, key, value, Record.EMPTY_HEADERS); + appendWithOffset(nextSequentialOffset(), true, timestamp, key, value, Record.EMPTY_HEADERS); } - /** - * Return CRC of the record or null if record-level CRC is not supported for the message format - */ - public Long appendEndTxnMarker(long timestamp, EndTransactionMarker marker) { + public void appendEndTxnMarker(long timestamp, EndTransactionMarker marker) { if (producerId == RecordBatch.NO_PRODUCER_ID) throw new IllegalArgumentException("End transaction marker requires a valid producerId"); if (!isTransactional) throw new IllegalArgumentException("End transaction marker depends on batch transactional flag being enabled"); ByteBuffer value = marker.serializeValue(); - return appendControlRecord(timestamp, marker.controlType(), value); + appendControlRecord(timestamp, marker.controlType(), value); } - /** - * Return CRC of the record or null if record-level CRC is not supported for the message format - */ - public Long appendLeaderChangeMessage(long timestamp, LeaderChangeMessage leaderChangeMessage) { + public void appendLeaderChangeMessage(long timestamp, LeaderChangeMessage leaderChangeMessage) { if (partitionLeaderEpoch == RecordBatch.NO_PARTITION_LEADER_EPOCH) { throw new IllegalArgumentException("Partition leader epoch must be valid, but get " + partitionLeaderEpoch); } - return appendControlRecord(timestamp, ControlRecordType.LEADER_CHANGE, + appendControlRecord(timestamp, ControlRecordType.LEADER_CHANGE, MessageUtil.toByteBuffer(leaderChangeMessage, ControlRecordUtils.LEADER_CHANGE_SCHEMA_VERSION)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java index 2dc1dde24e5c3..848f75c678d19 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java @@ -16,11 +16,12 @@ */ package org.apache.kafka.clients.consumer; +import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.record.DefaultRecord; import org.apache.kafka.common.record.TimestampType; import org.junit.jupiter.api.Test; +import java.nio.charset.StandardCharsets; import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -28,8 +29,7 @@ public class ConsumerRecordTest { @Test - @SuppressWarnings("deprecation") - public void testOldConstructor() { + public void testShortConstructor() { String topic = "topic"; int partition = 0; long offset = 23; @@ -44,7 +44,6 @@ public void testOldConstructor() { assertEquals(value, record.value()); assertEquals(TimestampType.NO_TIMESTAMP_TYPE, record.timestampType()); assertEquals(ConsumerRecord.NO_TIMESTAMP, record.timestamp()); - assertEquals(ConsumerRecord.NULL_CHECKSUM, record.checksum()); assertEquals(ConsumerRecord.NULL_SIZE, record.serializedKeySize()); assertEquals(ConsumerRecord.NULL_SIZE, record.serializedValueSize()); assertEquals(Optional.empty(), record.leaderEpoch()); @@ -52,14 +51,63 @@ public void testOldConstructor() { } @Test - @SuppressWarnings("deprecation") - public void testNullChecksumInConstructor() { + @Deprecated + public void testConstructorsWithChecksum() { + String topic = "topic"; + int partition = 0; + long offset = 23; + long timestamp = 23434217432432L; + TimestampType timestampType = TimestampType.CREATE_TIME; String key = "key"; String value = "value"; - long timestamp = 242341324L; - ConsumerRecord record = new ConsumerRecord<>("topic", 0, 23L, timestamp, - TimestampType.CREATE_TIME, null, key.length(), value.length(), key, value, new RecordHeaders()); - assertEquals(DefaultRecord.computePartialChecksum(timestamp, key.length(), value.length()), record.checksum()); + long checksum = 50L; + int serializedKeySize = 100; + int serializedValueSize = 1142; + + ConsumerRecord record = new ConsumerRecord<>(topic, partition, offset, timestamp, timestampType, + checksum, serializedKeySize, serializedValueSize, key, value); + assertEquals(topic, record.topic()); + assertEquals(partition, record.partition()); + assertEquals(offset, record.offset()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + assertEquals(timestampType, record.timestampType()); + assertEquals(timestamp, record.timestamp()); + assertEquals(serializedKeySize, record.serializedKeySize()); + assertEquals(serializedValueSize, record.serializedValueSize()); + assertEquals(Optional.empty(), record.leaderEpoch()); + assertEquals(new RecordHeaders(), record.headers()); + + RecordHeaders headers = new RecordHeaders(); + headers.add(new RecordHeader("header key", "header value".getBytes(StandardCharsets.UTF_8))); + record = new ConsumerRecord<>(topic, partition, offset, timestamp, timestampType, + checksum, serializedKeySize, serializedValueSize, key, value, headers); + assertEquals(topic, record.topic()); + assertEquals(partition, record.partition()); + assertEquals(offset, record.offset()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + assertEquals(timestampType, record.timestampType()); + assertEquals(timestamp, record.timestamp()); + assertEquals(serializedKeySize, record.serializedKeySize()); + assertEquals(serializedValueSize, record.serializedValueSize()); + assertEquals(Optional.empty(), record.leaderEpoch()); + assertEquals(headers, record.headers()); + + Optional leaderEpoch = Optional.of(10); + record = new ConsumerRecord<>(topic, partition, offset, timestamp, timestampType, + checksum, serializedKeySize, serializedValueSize, key, value, headers, leaderEpoch); + assertEquals(topic, record.topic()); + assertEquals(partition, record.partition()); + assertEquals(offset, record.offset()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + assertEquals(timestampType, record.timestampType()); + assertEquals(timestamp, record.timestamp()); + assertEquals(serializedKeySize, record.serializedKeySize()); + assertEquals(serializedValueSize, record.serializedValueSize()); + assertEquals(leaderEpoch, record.leaderEpoch()); + assertEquals(headers, record.headers()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java index e34b6b1f576a8..d414450ac0e06 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java @@ -24,7 +24,10 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; + import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.junit.jupiter.api.Test; @@ -37,10 +40,12 @@ public void iterator() throws Exception { String topic = "topic"; records.put(new TopicPartition(topic, 0), new ArrayList>()); - ConsumerRecord record1 = new ConsumerRecord<>(topic, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, "value1"); - ConsumerRecord record2 = new ConsumerRecord<>(topic, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 2, "value2"); + ConsumerRecord record1 = new ConsumerRecord<>(topic, 1, 0, 0L, TimestampType.CREATE_TIME, + 0, 0, 1, "value1", new RecordHeaders(), Optional.empty()); + ConsumerRecord record2 = new ConsumerRecord<>(topic, 1, 1, 0L, TimestampType.CREATE_TIME, + 0, 0, 2, "value2", new RecordHeaders(), Optional.empty()); records.put(new TopicPartition(topic, 1), Arrays.asList(record1, record2)); - records.put(new TopicPartition(topic, 2), new ArrayList>()); + records.put(new TopicPartition(topic, 2), new ArrayList<>()); ConsumerRecords consumerRecords = new ConsumerRecords<>(records); Iterator> iter = consumerRecords.iterator(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java index 9a74b0667cd1b..be2ea0edf1649 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.junit.jupiter.api.Test; @@ -47,8 +48,10 @@ public void testSimpleMock() { beginningOffsets.put(new TopicPartition("test", 1), 0L); consumer.updateBeginningOffsets(beginningOffsets); consumer.seek(new TopicPartition("test", 0), 0); - ConsumerRecord rec1 = new ConsumerRecord<>("test", 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, "key1", "value1"); - ConsumerRecord rec2 = new ConsumerRecord<>("test", 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, "key2", "value2"); + ConsumerRecord rec1 = new ConsumerRecord<>("test", 0, 0, 0L, TimestampType.CREATE_TIME, + 0, 0, "key1", "value1", new RecordHeaders(), Optional.empty()); + ConsumerRecord rec2 = new ConsumerRecord<>("test", 0, 1, 0L, TimestampType.CREATE_TIME, + 0, 0, "key2", "value2", new RecordHeaders(), Optional.empty()); consumer.addRecord(rec1); consumer.addRecord(rec2); ConsumerRecords recs = consumer.poll(Duration.ofMillis(1)); @@ -74,8 +77,10 @@ public void testSimpleMockDeprecated() { beginningOffsets.put(new TopicPartition("test", 1), 0L); consumer.updateBeginningOffsets(beginningOffsets); consumer.seek(new TopicPartition("test", 0), 0); - ConsumerRecord rec1 = new ConsumerRecord<>("test", 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, "key1", "value1"); - ConsumerRecord rec2 = new ConsumerRecord<>("test", 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, "key2", "value2"); + ConsumerRecord rec1 = new ConsumerRecord<>("test", 0, 0, 0L, TimestampType.CREATE_TIME, + 0, 0, "key1", "value1", new RecordHeaders(), Optional.empty()); + ConsumerRecord rec2 = new ConsumerRecord<>("test", 0, 1, 0L, TimestampType.CREATE_TIME, + 0, 0, "key2", "value2", new RecordHeaders(), Optional.empty()); consumer.addRecord(rec1); consumer.addRecord(rec2); ConsumerRecords recs = consumer.poll(1); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java index 1fad0c5bf67d4..19ac2560929ce 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerInterceptorsTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.junit.jupiter.api.Test; @@ -30,6 +31,7 @@ import java.util.List; import java.util.Map; import java.util.HashMap; +import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -43,8 +45,8 @@ public class ConsumerInterceptorsTest { private final TopicPartition tp = new TopicPartition(topic, partition); private final TopicPartition filterTopicPart1 = new TopicPartition("test5", filterPartition1); private final TopicPartition filterTopicPart2 = new TopicPartition("test6", filterPartition2); - private final ConsumerRecord consumerRecord = - new ConsumerRecord<>(topic, partition, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, 1); + private final ConsumerRecord consumerRecord = new ConsumerRecord<>(topic, partition, 0, 0L, + TimestampType.CREATE_TIME, 0, 0, 1, 1, new RecordHeaders(), Optional.empty()); private int onCommitCount = 0; private int onConsumeCount = 0; @@ -117,11 +119,11 @@ public void testOnConsumeChain() { List> list1 = new ArrayList<>(); list1.add(consumerRecord); List> list2 = new ArrayList<>(); - list2.add(new ConsumerRecord<>(filterTopicPart1.topic(), filterTopicPart1.partition(), 0, - 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, 1)); + list2.add(new ConsumerRecord<>(filterTopicPart1.topic(), filterTopicPart1.partition(), 0, 0L, + TimestampType.CREATE_TIME, 0, 0, 1, 1, new RecordHeaders(), Optional.empty())); List> list3 = new ArrayList<>(); - list3.add(new ConsumerRecord<>(filterTopicPart2.topic(), filterTopicPart2.partition(), 0, - 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, 1)); + list3.add(new ConsumerRecord<>(filterTopicPart2.topic(), filterTopicPart2.partition(), 0, 0L, TimestampType.CREATE_TIME, + 0, 0, 1, 1, new RecordHeaders(), Optional.empty())); records.put(tp, list1); records.put(filterTopicPart1, list2); records.put(filterTopicPart2, list3); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordMetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordMetadataTest.java index 51764dc48684b..b6207fba8c305 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordMetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordMetadataTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.clients.producer; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.DefaultRecord; import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -26,56 +25,66 @@ public class RecordMetadataTest { @Test - @SuppressWarnings("deprecation") - public void testConstructionWithMissingRelativeOffset() { + public void testConstructionWithMissingBatchIndex() { TopicPartition tp = new TopicPartition("foo", 0); long timestamp = 2340234L; int keySize = 3; int valueSize = 5; - Long checksum = 908923L; - RecordMetadata metadata = new RecordMetadata(tp, -1L, -1L, timestamp, checksum, keySize, valueSize); + RecordMetadata metadata = new RecordMetadata(tp, -1L, -1, timestamp, keySize, valueSize); assertEquals(tp.topic(), metadata.topic()); assertEquals(tp.partition(), metadata.partition()); assertEquals(timestamp, metadata.timestamp()); assertFalse(metadata.hasOffset()); assertEquals(-1L, metadata.offset()); - assertEquals(checksum.longValue(), metadata.checksum()); assertEquals(keySize, metadata.serializedKeySize()); assertEquals(valueSize, metadata.serializedValueSize()); } @Test - @SuppressWarnings("deprecation") - public void testConstructionWithRelativeOffset() { + public void testConstructionWithBatchIndexOffset() { TopicPartition tp = new TopicPartition("foo", 0); long timestamp = 2340234L; int keySize = 3; int valueSize = 5; long baseOffset = 15L; - long relativeOffset = 3L; - Long checksum = 908923L; + int batchIndex = 3; - RecordMetadata metadata = new RecordMetadata(tp, baseOffset, relativeOffset, timestamp, checksum, - keySize, valueSize); + RecordMetadata metadata = new RecordMetadata(tp, baseOffset, batchIndex, timestamp, keySize, valueSize); assertEquals(tp.topic(), metadata.topic()); assertEquals(tp.partition(), metadata.partition()); assertEquals(timestamp, metadata.timestamp()); - assertEquals(baseOffset + relativeOffset, metadata.offset()); - assertEquals(checksum.longValue(), metadata.checksum()); + assertEquals(baseOffset + batchIndex, metadata.offset()); assertEquals(keySize, metadata.serializedKeySize()); assertEquals(valueSize, metadata.serializedValueSize()); } @Test - @SuppressWarnings("deprecation") - public void testNullChecksum() { + @Deprecated + public void testConstructionWithChecksum() { + TopicPartition tp = new TopicPartition("foo", 0); long timestamp = 2340234L; + long baseOffset = 15L; + long batchIndex = 3L; int keySize = 3; int valueSize = 5; - RecordMetadata metadata = new RecordMetadata(new TopicPartition("foo", 0), 15L, 3L, timestamp, null, - keySize, valueSize); - assertEquals(DefaultRecord.computePartialChecksum(timestamp, keySize, valueSize), metadata.checksum()); + + RecordMetadata metadata = new RecordMetadata(tp, baseOffset, batchIndex, timestamp, null, keySize, valueSize); + assertEquals(tp.topic(), metadata.topic()); + assertEquals(tp.partition(), metadata.partition()); + assertEquals(timestamp, metadata.timestamp()); + assertEquals(baseOffset + batchIndex, metadata.offset()); + assertEquals(keySize, metadata.serializedKeySize()); + assertEquals(valueSize, metadata.serializedValueSize()); + + long checksum = 133424L; + metadata = new RecordMetadata(tp, baseOffset, batchIndex, timestamp, checksum, keySize, valueSize); + assertEquals(tp.topic(), metadata.topic()); + assertEquals(tp.partition(), metadata.partition()); + assertEquals(timestamp, metadata.timestamp()); + assertEquals(baseOffset + batchIndex, metadata.offset()); + assertEquals(keySize, metadata.serializedKeySize()); + assertEquals(valueSize, metadata.serializedValueSize()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java index b7cfd4e583eda..86632eecdc889 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java @@ -48,7 +48,7 @@ public class RecordSendTest { public void testTimeout() throws Exception { ProduceRequestResult request = new ProduceRequestResult(topicPartition); FutureRecordMetadata future = new FutureRecordMetadata(request, relOffset, - RecordBatch.NO_TIMESTAMP, 0L, 0, 0, Time.SYSTEM); + RecordBatch.NO_TIMESTAMP, 0, 0, Time.SYSTEM); assertFalse(future.isDone(), "Request is not completed"); try { future.get(5, TimeUnit.MILLISECONDS); @@ -68,7 +68,7 @@ public void testTimeout() throws Exception { @Test public void testError() throws Exception { FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, new CorruptRecordException(), 50L), - relOffset, RecordBatch.NO_TIMESTAMP, 0L, 0, 0, Time.SYSTEM); + relOffset, RecordBatch.NO_TIMESTAMP, 0, 0, Time.SYSTEM); assertThrows(ExecutionException.class, future::get); } @@ -78,7 +78,7 @@ public void testError() throws Exception { @Test public void testBlocking() throws Exception { FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, null, 50L), - relOffset, RecordBatch.NO_TIMESTAMP, 0L, 0, 0, Time.SYSTEM); + relOffset, RecordBatch.NO_TIMESTAMP, 0, 0, Time.SYSTEM); assertEquals(baseOffset + relOffset, future.get().offset()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadataTest.java index 74649c80b9062..1fd7a438e3a43 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadataTest.java @@ -67,7 +67,6 @@ private FutureRecordMetadata futureRecordMetadata(ProduceRequestResult produceRe produceRequestResult, 0, RecordBatch.NO_TIMESTAMP, - 0L, 0, 0, time diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java index b6ee58245aedc..6af8289f9cede 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerBatchTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.record.CompressionType; -import org.apache.kafka.common.record.LegacyRecord; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.Record; @@ -60,14 +59,6 @@ public class ProducerBatchTest { private final MemoryRecordsBuilder memoryRecordsBuilder = MemoryRecords.builder(ByteBuffer.allocate(512), CompressionType.NONE, TimestampType.CREATE_TIME, 128); - @Test - public void testChecksumNullForMagicV2() { - ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); - FutureRecordMetadata future = batch.tryAppend(now, null, new byte[10], Record.EMPTY_HEADERS, null, now); - assertNotNull(future); - assertNull(future.checksumOrNull()); - } - @Test public void testBatchAbort() throws Exception { ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), memoryRecordsBuilder, now); @@ -138,23 +129,6 @@ public void testBatchCannotCompleteTwice() throws Exception { assertEquals(10L, recordMetadata.timestamp()); } - @Test - public void testAppendedChecksumMagicV0AndV1() { - for (byte magic : Arrays.asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1)) { - MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(128), magic, - CompressionType.NONE, TimestampType.CREATE_TIME, 0L); - ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now); - byte[] key = "hi".getBytes(); - byte[] value = "there".getBytes(); - - FutureRecordMetadata future = batch.tryAppend(now, key, value, Record.EMPTY_HEADERS, null, now); - assertNotNull(future); - byte attributes = LegacyRecord.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME); - long expectedChecksum = LegacyRecord.computeChecksum(magic, attributes, now, key, value); - assertEquals(expectedChecksum, future.checksumOrNull().longValue()); - } - } - @Test public void testSplitPreservesHeaders() { for (CompressionType compressionType : CompressionType.values()) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java index 3cc63a9d503d9..cd15a3ea81254 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java @@ -144,7 +144,7 @@ public void testOnAcknowledgementChain() { ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList); // verify onAck is called on all interceptors - RecordMetadata meta = new RecordMetadata(tp, 0, 0, 0, Long.valueOf(0L), 0, 0); + RecordMetadata meta = new RecordMetadata(tp, 0, 0, 0, 0, 0); interceptors.onAcknowledgement(meta, null); assertEquals(2, onAckCount); diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java index 13eaa9d21cba1..5dedf66f36d32 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java @@ -437,11 +437,10 @@ public void testAppendedChecksumConsistency(Args args) { TimestampType.CREATE_TIME, 0L, LegacyRecord.NO_TIMESTAMP, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); - Long checksumOrNull = builder.append(1L, "key".getBytes(), "value".getBytes()); + builder.append(1L, "key".getBytes(), "value".getBytes()); MemoryRecords memoryRecords = builder.build(); List records = TestUtils.toList(memoryRecords.records()); assertEquals(1, records.size()); - assertEquals(checksumOrNull, records.get(0).checksumOrNull()); } @ParameterizedTest diff --git a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java index 3fcc1575f5863..b01584b62a245 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java @@ -26,12 +26,14 @@ import org.apache.kafka.common.ClusterResource; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.header.internals.RecordHeaders; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -56,7 +58,6 @@ public void configure(Map configs) { } @Override - @SuppressWarnings("deprecation") public ConsumerRecords onConsume(ConsumerRecords records) { // This will ensure that we get the cluster metadata when onConsume is called for the first time @@ -69,13 +70,14 @@ public ConsumerRecords onConsume(ConsumerRecords for (ConsumerRecord record: records.records(tp)) { lst.add(new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), record.timestamp(), record.timestampType(), - record.checksum(), record.serializedKeySize(), + record.serializedKeySize(), record.serializedValueSize(), - record.key(), record.value().toUpperCase(Locale.ROOT))); + record.key(), record.value().toUpperCase(Locale.ROOT), + new RecordHeaders(), Optional.empty())); } recordMap.put(tp, lst); } - return new ConsumerRecords(recordMap); + return new ConsumerRecords<>(recordMap); } @Override diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java index 0b47cc91d2cc9..9cf09f8c4cfd0 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceTaskTest.java @@ -31,6 +31,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -49,7 +50,7 @@ public void testSerde() { headers.add("header1", new byte[]{'l', 'm', 'n', 'o'}); headers.add("header2", new byte[]{'p', 'q', 'r', 's', 't'}); ConsumerRecord consumerRecord = new ConsumerRecord<>("topic1", 2, 3L, 4L, - TimestampType.CREATE_TIME, 0L, 5, 6, key, value, headers); + TimestampType.CREATE_TIME, 5, 6, key, value, headers, Optional.empty()); MirrorSourceTask mirrorSourceTask = new MirrorSourceTask(null, null, "cluster7", new DefaultReplicationPolicy(), 50); SourceRecord sourceRecord = mirrorSourceTask.convertRecord(consumerRecord); @@ -111,9 +112,9 @@ public void testPoll() { new RecordHeader(headerKey, "value".getBytes()), }); consumerRecordsList.add(new ConsumerRecord<>(topicName, 0, 0, System.currentTimeMillis(), - TimestampType.CREATE_TIME, 0L, key1.length, value1.length, key1, value1, headers)); + TimestampType.CREATE_TIME, key1.length, value1.length, key1, value1, headers, Optional.empty())); consumerRecordsList.add(new ConsumerRecord<>(topicName, 1, 1, System.currentTimeMillis(), - TimestampType.CREATE_TIME, 0L, key2.length, value2.length, key2, value2, headers)); + TimestampType.CREATE_TIME, key2.length, value2.length, key2, value2, headers, Optional.empty())); ConsumerRecords consumerRecords = new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(topicName, 0), consumerRecordsList)); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/WorkerErrantRecordReporter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/WorkerErrantRecordReporter.java index b709e28c5edfe..aa1d0be36f493 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/WorkerErrantRecordReporter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/errors/WorkerErrantRecordReporter.java @@ -33,6 +33,7 @@ import java.util.LinkedList; import java.util.List; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -95,8 +96,8 @@ public Future report(SinkRecord record, Throwable error) { int valLength = value != null ? value.length : -1; consumerRecord = new ConsumerRecord<>(record.topic(), record.kafkaPartition(), - record.kafkaOffset(), record.timestamp(), record.timestampType(), -1L, keyLength, - valLength, key, value, headers); + record.kafkaOffset(), record.timestamp(), record.timestampType(), keyLength, + valLength, key, value, headers, Optional.empty()); } Future future = retryWithToleranceOperator.executeFailed(Stage.TASK_PUT, SinkTask.class, consumerRecord, error); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index a7d68a61520a2..b7492d31f502f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -75,6 +75,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -1111,8 +1112,10 @@ public void testCommitWithOutOfOrderCallback() throws Exception { long timestamp = RecordBatch.NO_TIMESTAMP; TimestampType timestampType = TimestampType.NO_TIMESTAMP_TYPE; List> records = new ArrayList<>(); - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE)); - records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE)); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, timestamp, timestampType, + 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); + records.add(new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET + recordsReturnedTp3 + 1, timestamp, timestampType, + 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())); recordsReturnedTp1 += 1; recordsReturnedTp3 += 1; return new ConsumerRecords<>(Collections.singletonMap(new TopicPartition(TOPIC, PARTITION), records)); @@ -1495,9 +1498,9 @@ public void testHeadersWithCustomConverter() throws Exception { expectConsumerPoll(Arrays.asList( new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, - 0L, 0, 0, keyA.getBytes(), valueA.getBytes(encodingA), headersA), + 0, 0, keyA.getBytes(), valueA.getBytes(encodingA), headersA, Optional.empty()), new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 2, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, - 0L, 0, 0, keyB.getBytes(), valueB.getBytes(encodingB), headersB) + 0, 0, keyB.getBytes(), valueB.getBytes(encodingB), headersB, Optional.empty()) )); expectTransformation(2, null); @@ -1614,7 +1617,8 @@ private void expectConsumerPoll(final int numMessages, final long timestamp, fin () -> { List> records = new ArrayList<>(); for (int i = 0; i < numMessages; i++) - records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, 0L, 0, 0, RAW_KEY, RAW_VALUE, headers)); + records.add(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + i, timestamp, timestampType, + 0, 0, RAW_KEY, RAW_VALUE, headers, Optional.empty())); recordsReturnedTp1 += numMessages; return new ConsumerRecords<>( numMessages > 0 ? diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 336f24b4958ca..7b4474bc65644 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -64,6 +64,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -555,9 +556,8 @@ private Capture> expectPolls(final long pollDelayMs) thro ConsumerRecords records = new ConsumerRecords<>( Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), - Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) - ))); + Arrays.asList(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, + 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())))); recordsReturned++; return records; }); @@ -586,9 +586,8 @@ private IExpectationSetters expectOnePoll() { ConsumerRecords records = new ConsumerRecords<>( Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), - Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) - ))); + Arrays.asList(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, + 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty())))); recordsReturned++; return records; }); @@ -617,8 +616,8 @@ private IExpectationSetters expectRebalanceDuringPoll() throws Exception ConsumerRecords records = new ConsumerRecords<>( Collections.singletonMap( new TopicPartition(TOPIC, PARTITION), - Arrays.asList( - new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, 0L, 0, 0, RAW_KEY, RAW_VALUE) + Arrays.asList(new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturned, TIMESTAMP, TIMESTAMP_TYPE, + 0, 0, RAW_KEY, RAW_VALUE, new RecordHeaders(), Optional.empty()) ))); recordsReturned++; return records; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index aefc3a94e8762..4d358ca46b1d6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -1435,7 +1435,7 @@ private Capture> expectSendRecord( for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { if (sendSuccess) { cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, - 0L, 0L, 0, 0), null); + 0L, 0, 0), null); } else { cb.onCompletion(null, new TopicAuthorizationException("foo")); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ProcessingContextTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ProcessingContextTest.java index 3e5e0820c021c..89f101382e018 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ProcessingContextTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/ProcessingContextTest.java @@ -48,7 +48,7 @@ private void testReport(int numberOfReports) { Future result = context.report(); fs.forEach(f -> { assertFalse(result.isDone()); - f.complete(new RecordMetadata(new TopicPartition("t", 0), 0, 0, 0, 0L, 0, 0)); + f.complete(new RecordMetadata(new TopicPartition("t", 0), 0, 0, 0, 0, 0)); }); assertTrue(result.isDone()); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index 4504d3997675c..d3960e5bf46cf 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.data.Field; @@ -55,6 +56,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; @@ -464,11 +466,16 @@ public void testPutTaskConfigsZeroTasks() throws Exception { public void testRestoreTargetState() throws Exception { expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4))); + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, TARGET_STATE_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(4), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); @@ -506,10 +513,14 @@ public void testBackgroundUpdateTargetState() throws Exception { expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3))); + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); @@ -549,10 +560,14 @@ public void testBackgroundConnectorDeletion() throws Exception { expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3))); + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); @@ -607,11 +622,16 @@ public void testBackgroundConnectorDeletion() throws Exception { public void testRestoreTargetStateUnexpectedDeletion() throws Exception { expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4))); + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, TARGET_STATE_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(4), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); @@ -651,14 +671,21 @@ public void testRestore() throws Exception { expectConfigure(); // Overwrite each type at least once to ensure we see the latest data after loading List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(4), new RecordHeaders(), Optional.empty()), // Connector after root update should make it through, task update shouldn't - new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), - new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6))); + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(5), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(6), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); @@ -707,12 +734,18 @@ public void testRestoreConnectorDeletion() throws Exception { expectConfigure(); // Overwrite each type at least once to ensure we see the latest data after loading List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TARGET_STATE_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), - new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5))); + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0, TARGET_STATE_KEYS.get(0), + CONFIGS_SERIALIZED.get(4), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(5), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); @@ -752,15 +785,23 @@ public void testRestoreZeroTasks() throws Exception { expectConfigure(); // Overwrite each type at least once to ensure we see the latest data after loading List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(1), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(3), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(4), new RecordHeaders(), Optional.empty()), // Connector after root update should make it through, task update shouldn't - new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), - new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6)), - new ConsumerRecord<>(TOPIC, 0, 7, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(7))); + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(5), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(6), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 7, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(7), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); @@ -807,12 +848,16 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio expectConfigure(); List> existingRecords = Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), // This is the record that has been compacted: //new ConsumerRecord<>(TOPIC, 0, 1, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), - new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), - new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), - new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5))); + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(1), + CONFIGS_SERIALIZED.get(2), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(4), new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), + CONFIGS_SERIALIZED.get(5), new RecordHeaders(), Optional.empty())); LinkedHashMap deserialized = new LinkedHashMap<>(); deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0)); @@ -966,8 +1011,11 @@ private void expectReadToEnd(final LinkedHashMap serializedConfi EasyMock.expect(storeLog.readToEnd()) .andAnswer(() -> { TestFuture future = new TestFuture<>(); - for (Map.Entry entry : serializedConfigs.entrySet()) - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, entry.getKey(), entry.getValue())); + for (Map.Entry entry : serializedConfigs.entrySet()) { + capturedConsumedCallback.getValue().onCompletion(null, + new ConsumerRecord<>(TOPIC, 0, logOffset++, 0L, TimestampType.CREATE_TIME, 0, 0, + entry.getKey(), entry.getValue(), new RecordHeaders(), Optional.empty())); + } future.resolveOnGet((Void) null); return future; }); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java index d21606809ccb1..07eadc8789313 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.util.Callback; @@ -46,6 +47,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -149,10 +151,14 @@ public void testStartStop() throws Exception { public void testReloadOnStart() throws Exception { expectConfigure(); expectStart(Arrays.asList( - new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE.array()), - new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE.array()), - new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array()), - new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array()) + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY.array(), TP0_VALUE.array(), + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY.array(), TP1_VALUE.array(), + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array(), + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array(), + new RecordHeaders(), Optional.empty()) )); expectStop(); expectClusterId(); @@ -196,8 +202,12 @@ public void testGetSet() throws Exception { final Capture> secondGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(secondGetReadToEndCallback)); PowerMock.expectLastCall().andAnswer(() -> { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE.array())); + capturedConsumedCallback.getValue().onCompletion(null, + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY.array(), TP0_VALUE.array(), + new RecordHeaders(), Optional.empty())); + capturedConsumedCallback.getValue().onCompletion(null, + new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY.array(), TP1_VALUE.array(), + new RecordHeaders(), Optional.empty())); secondGetReadToEndCallback.getValue().onCompletion(null, null); return null; }); @@ -206,8 +216,12 @@ public void testGetSet() throws Exception { final Capture> thirdGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(thirdGetReadToEndCallback)); PowerMock.expectLastCall().andAnswer(() -> { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array())); + capturedConsumedCallback.getValue().onCompletion(null, + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY.array(), TP0_VALUE_NEW.array(), + new RecordHeaders(), Optional.empty())); + capturedConsumedCallback.getValue().onCompletion(null, + new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY.array(), TP1_VALUE_NEW.array(), + new RecordHeaders(), Optional.empty())); thirdGetReadToEndCallback.getValue().onCompletion(null, null); return null; }); @@ -271,8 +285,12 @@ public void testGetSetNull() throws Exception { final Capture> secondGetReadToEndCallback = EasyMock.newCapture(); storeLog.readToEnd(EasyMock.capture(secondGetReadToEndCallback)); PowerMock.expectLastCall().andAnswer(() -> { - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, null, TP0_VALUE.array())); - capturedConsumedCallback.getValue().onCompletion(null, new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY.array(), null)); + capturedConsumedCallback.getValue().onCompletion(null, + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, null, TP0_VALUE.array(), + new RecordHeaders(), Optional.empty())); + capturedConsumedCallback.getValue().onCompletion(null, + new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY.array(), null, + new RecordHeaders(), Optional.empty())); secondGetReadToEndCallback.getValue().onCompletion(null, null); return null; }); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java index 4c0c7f90a650f..f19be75a3915a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.connect.data.Schema; @@ -45,6 +46,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Optional; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.capture; @@ -436,7 +438,7 @@ public void deleteTaskState() { private static ConsumerRecord consumerRecord(long offset, String key, byte[] value) { return new ConsumerRecord<>(STATUS_TOPIC, 0, offset, System.currentTimeMillis(), - TimestampType.CREATE_TIME, 0L, 0, 0, key, value); + TimestampType.CREATE_TIME, 0, 0, key, value, new RecordHeaders(), Optional.empty()); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java index e36f2a902facd..8fae57efaf988 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/KafkaBasedLogTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.Time; @@ -56,6 +57,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -187,12 +189,14 @@ public void testReloadOnStart() throws Exception { consumer.scheduleNopPollTask(); consumer.scheduleNopPollTask(); consumer.schedulePollTask(() -> - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)) + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY, TP0_VALUE, + new RecordHeaders(), Optional.empty())) ); consumer.scheduleNopPollTask(); consumer.scheduleNopPollTask(); consumer.schedulePollTask(() -> - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE)) + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY, TP1_VALUE, + new RecordHeaders(), Optional.empty())) ); consumer.schedulePollTask(finishedLatch::countDown); }); @@ -306,13 +310,17 @@ public void testSendAndReadToEnd() throws Exception { consumer.scheduleNopPollTask(); consumer.scheduleNopPollTask(); consumer.schedulePollTask(() -> { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY, TP0_VALUE, + new RecordHeaders(), Optional.empty())); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY, TP0_VALUE_NEW, + new RecordHeaders(), Optional.empty())); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY, TP1_VALUE, + new RecordHeaders(), Optional.empty())); }); consumer.schedulePollTask(() -> - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP1_KEY, TP1_VALUE_NEW))); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TP1_KEY, TP1_VALUE_NEW, + new RecordHeaders(), Optional.empty()))); // Already have FutureCallback that should be invoked/awaited, so no need for follow up finishedLatch }); @@ -357,8 +365,10 @@ public void testPollConsumerError() throws Exception { consumer.scheduleNopPollTask(); consumer.scheduleNopPollTask(); consumer.schedulePollTask(() -> { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY, TP0_VALUE_NEW, + new RecordHeaders(), Optional.empty())); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY, TP0_VALUE_NEW, + new RecordHeaders(), Optional.empty())); }); consumer.schedulePollTask(finishedLatch::countDown); @@ -411,8 +421,10 @@ public void testGetOffsetsConsumerErrorOnReadToEnd() throws Exception { consumer.scheduleNopPollTask(); consumer.scheduleNopPollTask(); consumer.schedulePollTask(() -> { - consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE)); - consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TP0_KEY, TP0_VALUE_NEW)); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY, TP0_VALUE, + new RecordHeaders(), Optional.empty())); + consumer.addRecord(new ConsumerRecord<>(TOPIC, 1, 0, 0L, TimestampType.CREATE_TIME, 0, 0, TP0_KEY, TP0_VALUE_NEW, + new RecordHeaders(), Optional.empty())); }); consumer.schedulePollTask(finishedLatch::countDown); diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index ce58214b79da6..1269ff5b51b75 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -22,7 +22,7 @@ import java.nio.charset.StandardCharsets import java.time.Duration import java.util.concurrent.CountDownLatch import java.util.regex.Pattern -import java.util.{Collections, Locale, Map, Properties, Random} +import java.util.{Collections, Locale, Map, Optional, Properties, Random} import com.typesafe.scalalogging.LazyLogging import joptsimple._ import kafka.utils.Implicits._ @@ -35,7 +35,6 @@ import org.apache.kafka.common.requests.ListOffsetsRequest import org.apache.kafka.common.serialization.{ByteArrayDeserializer, Deserializer} import org.apache.kafka.common.utils.Utils -import scala.annotation.nowarn import scala.jdk.CollectionConverters._ /** @@ -112,8 +111,8 @@ object ConsoleConsumer extends Logging { } messageCount += 1 try { - formatter.writeTo(new ConsumerRecord(msg.topic, msg.partition, msg.offset, msg.timestamp, - msg.timestampType, 0, 0, 0, msg.key, msg.value, msg.headers), output) + formatter.writeTo(new ConsumerRecord(msg.topic, msg.partition, msg.offset, msg.timestamp, msg.timestampType, + 0, 0, msg.key, msg.value, msg.headers, Optional.empty[Integer]), output) } catch { case e: Throwable => if (skipMessageOnError) { @@ -614,18 +613,3 @@ class NoOpMessageFormatter extends MessageFormatter { def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream): Unit = {} } -class ChecksumMessageFormatter extends MessageFormatter { - private var topicStr: String = _ - - override def configure(configs: Map[String, _]): Unit = { - topicStr = if (configs.containsKey("topic")) - configs.get("topic").toString + ":" - else - "" - } - - @nowarn("cat=deprecation") - def writeTo(consumerRecord: ConsumerRecord[Array[Byte], Array[Byte]], output: PrintStream): Unit = { - output.println(topicStr + "checksum:" + consumerRecord.checksum) - } -} diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 70188e4758ef4..1f7e9d16e758b 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -35,7 +35,6 @@ import org.apache.kafka.common.{KafkaException, TopicPartition} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} -import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.collection.mutable.Buffer import scala.concurrent.ExecutionException @@ -102,7 +101,6 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { * 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected. * 2. Last message of the non-blocking send should return the correct offset metadata */ - @nowarn("cat=deprecation") @Test def testSendOffset(): Unit = { val producer = createProducer(brokerList) @@ -123,7 +121,6 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { case 2 => assertEquals(metadata.serializedValueSize, "value".getBytes(StandardCharsets.UTF_8).length) case _ => assertTrue(metadata.serializedValueSize > 0) } - assertNotEquals(metadata.checksum(), 0) offset += 1 } else { fail(s"Send callback returns the following exception: $exception") diff --git a/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala b/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala index ea72f4f3bbed1..f2ca243a8495d 100644 --- a/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala +++ b/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala @@ -30,6 +30,7 @@ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} +import java.util.Optional import scala.jdk.CollectionConverters._ class DefaultMessageFormatterTest { @@ -210,12 +211,12 @@ object DefaultMessageFormatterTest { offset, timestamp, timestampType, - 0L, 0, 0, if (key == null) null else key.getBytes(StandardCharsets.UTF_8), if (value == null) null else value.getBytes(StandardCharsets.UTF_8), - new RecordHeaders(headers.asJava)) + new RecordHeaders(headers.asJava), + Optional.empty[Integer]) } private def withResource[Resource <: Closeable, Result](resource: Resource)(handler: Resource => Result): Result = { diff --git a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala index 17a74590d2185..91ccbcc707306 100644 --- a/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala +++ b/core/src/test/scala/unit/kafka/tools/ConsoleConsumerTest.scala @@ -19,8 +19,7 @@ package kafka.tools import java.io.{ByteArrayOutputStream, PrintStream} import java.nio.file.Files -import java.util.{HashMap, Map => JMap} - +import java.util.{HashMap, Optional, Map => JMap} import kafka.tools.ConsoleConsumer.ConsumerWrapper import kafka.utils.{Exit, TestUtils} import org.apache.kafka.clients.consumer.{ConsumerRecord, MockConsumer, OffsetResetStrategy} @@ -31,6 +30,7 @@ import org.apache.kafka.test.MockDeserializer import org.mockito.Mockito._ import org.mockito.ArgumentMatchers import ArgumentMatchers._ +import org.apache.kafka.common.header.internals.RecordHeaders import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test} @@ -498,7 +498,8 @@ class ConsoleConsumerTest { assertEquals("NO_TIMESTAMP\tPartition:0\tOffset:123\tkey\tvalue\n", out.toString) out = new ByteArrayOutputStream() - val record2 = new ConsumerRecord("topic", 0, 123, 123L, TimestampType.CREATE_TIME, 321L, -1, -1, "key".getBytes, "value".getBytes) + val record2 = new ConsumerRecord("topic", 0, 123, 123L, TimestampType.CREATE_TIME, -1, -1, "key".getBytes, "value".getBytes, + new RecordHeaders(), Optional.empty[Integer]) formatter.writeTo(record2, new PrintStream(out)) assertEquals("CreateTime:123\tPartition:0\tOffset:123\tkey\tvalue\n", out.toString) formatter.close() @@ -515,22 +516,4 @@ class ConsoleConsumerTest { assertEquals("", out.toString) } - @Test - def testChecksumMessageFormatter(): Unit = { - val record = new ConsumerRecord("topic", 0, 123, "key".getBytes, "value".getBytes) - val formatter = new ChecksumMessageFormatter() - val configs: JMap[String, String] = new HashMap() - - formatter.configure(configs) - var out = new ByteArrayOutputStream() - formatter.writeTo(record, new PrintStream(out)) - assertEquals("checksum:-1\n", out.toString) - - configs.put("topic", "topic1") - formatter.configure(configs) - out = new ByteArrayOutputStream() - formatter.writeTo(record, new PrintStream(out)) - assertEquals("topic1:checksum:-1\n", out.toString) - } - } diff --git a/docs/upgrade.html b/docs/upgrade.html index dd839089743be..6a632e0728cef 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -27,6 +27,8 @@
    Notable changes in 3 or updating the application not to use internal classes.
  • The Streams API removed all deprecated APIs that were deprecated in version 2.5.0 or earlier. For a complete list of removed APIs compare the detailed Kafka Streams upgrade notes.
  • +
  • Kafka Streams no longer has a compile time dependency on "connect:json" module (KAFKA-5146). + Projects that were relying on this transitive dependency will have to explicitly declare it.
  • A number of deprecated classes, methods and tools have been removed from the clients, core and tools modules:
  • diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java index 269dff2822290..20f144922ad2d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordDeserializer.java @@ -25,6 +25,8 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.slf4j.Logger; +import java.util.Optional; + import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DESERIALIZATION_EXCEPTION_HANDLER_CLASS_CONFIG; class RecordDeserializer { @@ -59,12 +61,12 @@ ConsumerRecord deserialize(final ProcessorContext processorConte rawRecord.offset(), rawRecord.timestamp(), TimestampType.CREATE_TIME, - rawRecord.checksum(), rawRecord.serializedKeySize(), rawRecord.serializedValueSize(), sourceNode.deserializeKey(rawRecord.topic(), rawRecord.headers(), rawRecord.key()), sourceNode.deserializeValue(rawRecord.topic(), rawRecord.headers(), rawRecord.value()), - rawRecord.headers() + rawRecord.headers(), + Optional.empty() ); } catch (final Exception deserializationException) { final DeserializationExceptionHandler.DeserializationHandlerResponse response; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java index 8305d5216afc3..1f2e5930a211b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RecordConverters.java @@ -38,7 +38,6 @@ public final class RecordConverters { record.offset(), timestamp, record.timestampType(), - record.checksum(), record.serializedKeySize(), record.serializedValueSize(), record.key(), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java index 93ea448fbdc43..5474f9ff15484 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/LogAndSkipOnInvalidTimestampTest.java @@ -17,9 +17,12 @@ package org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.junit.Test; +import java.util.Optional; + import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -44,9 +47,10 @@ public void logAndSkipOnInvalidTimestamp() { TimestampType.NO_TIMESTAMP_TYPE, 0, 0, - 0, null, - null), + null, + new RecordHeaders(), + Optional.empty()), 0 ); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java index bd235be76d454..dca9a70f617a8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TimestampExtractorTest.java @@ -17,8 +17,11 @@ package org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; +import java.util.Optional; + import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -36,9 +39,10 @@ void testExtractMetadataTimestamp(final TimestampExtractor extractor) { TimestampType.NO_TIMESTAMP_TYPE, 0, 0, - 0, null, - null), + null, + new RecordHeaders(), + Optional.empty()), 0 ); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java index 3cc06be7858f8..e4bc600133341 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; @@ -40,6 +41,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import java.util.Set; import static java.util.Arrays.asList; @@ -148,7 +150,7 @@ private void maybeDeserialize(final GlobalStateUpdateTask globalStateTask, final boolean failExpected) { final ConsumerRecord record = new ConsumerRecord<>( topic2, 1, 1, 0L, TimestampType.CREATE_TIME, - 0L, 0, 0, key, recordValue + 0, 0, key, recordValue, new RecordHeaders(), Optional.empty() ); globalStateTask.initialize(); try { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java index 4ae9d32bc44fd..18d17aeddadeb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordDeserializerTest.java @@ -26,6 +26,8 @@ import org.apache.kafka.common.utils.LogContext; import org.junit.Test; +import java.util.Optional; + import static org.junit.Assert.assertEquals; public class RecordDeserializerTest { @@ -36,14 +38,13 @@ public class RecordDeserializerTest { 1, 10, TimestampType.LOG_APPEND_TIME, - 5L, 3, 5, new byte[0], new byte[0], - headers); + headers, + Optional.empty()); - @SuppressWarnings("deprecation") @Test public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { final RecordDeserializer recordDeserializer = new RecordDeserializer( @@ -60,7 +61,6 @@ public void shouldReturnConsumerRecordWithDeserializedValueWhenNoExceptions() { assertEquals(rawRecord.topic(), record.topic()); assertEquals(rawRecord.partition(), record.partition()); assertEquals(rawRecord.offset(), record.offset()); - assertEquals(rawRecord.checksum(), record.checksum()); assertEquals("key", record.key()); assertEquals("value", record.value()); assertEquals(rawRecord.timestamp(), record.timestamp()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java index 895e0452e1c31..142e85a30589a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.IntegerDeserializer; @@ -45,6 +46,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Optional; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -103,9 +105,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 2, 1, 3 final List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty())); queue.addRawRecords(list1); @@ -128,9 +130,9 @@ public void testTimeTracking() { // add three 3 out-of-order records with timestamp 4, 1, 2 // now with 3, 4, 1, 2 final List> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty())); queue.addRawRecords(list2); @@ -161,9 +163,9 @@ public void testTimeTracking() { // add three more records with 4, 5, 6 final List> list3 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, new RecordHeaders(), Optional.empty())); queue.addRawRecords(list3); @@ -202,10 +204,14 @@ public void shouldTrackPartitionTimeAsMaxProcessedTimestamp() { // add three 3 out-of-order records with timestamp 2, 1, 3, 4 final List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty())); queue.addRawRecords(list1); assertThat(queue.partitionTime(), is(RecordQueue.UNKNOWN)); @@ -231,10 +237,14 @@ public void shouldSetTimestampAndRespectMaxTimestampPolicy() { assertThat(queue.partitionTime(), is(150L)); final List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 200, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 100, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 300, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 400, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 200, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 100, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 300, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 400, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty())); queue.addRawRecords(list1); assertThat(queue.partitionTime(), is(150L)); @@ -253,7 +263,8 @@ public void shouldSetTimestampAndRespectMaxTimestampPolicy() { public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() { final byte[] key = Serdes.Long().serializer().serialize("foo", 1L); final List> records = Collections.singletonList( - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, key, recordValue)); + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, key, recordValue, + new RecordHeaders(), Optional.empty())); final StreamsException exception = assertThrows( StreamsException.class, @@ -266,7 +277,8 @@ public void shouldThrowStreamsExceptionWhenKeyDeserializationFails() { public void shouldThrowStreamsExceptionWhenValueDeserializationFails() { final byte[] value = Serdes.Long().serializer().serialize("foo", 1L); final List> records = Collections.singletonList( - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, value)); + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, value, + new RecordHeaders(), Optional.empty())); final StreamsException exception = assertThrows( StreamsException.class, @@ -279,7 +291,8 @@ public void shouldThrowStreamsExceptionWhenValueDeserializationFails() { public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHandler() { final byte[] key = Serdes.Long().serializer().serialize("foo", 1L); final List> records = Collections.singletonList( - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, key, recordValue)); + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, key, recordValue, + new RecordHeaders(), Optional.empty())); queueThatSkipsDeserializeErrors.addRawRecords(records); assertEquals(0, queueThatSkipsDeserializeErrors.size()); @@ -289,7 +302,8 @@ public void shouldNotThrowStreamsExceptionWhenKeyDeserializationFailsWithSkipHan public void shouldNotThrowStreamsExceptionWhenValueDeserializationFailsWithSkipHandler() { final byte[] value = Serdes.Long().serializer().serialize("foo", 1L); final List> records = Collections.singletonList( - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, value)); + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, value, + new RecordHeaders(), Optional.empty())); queueThatSkipsDeserializeErrors.addRawRecords(records); assertEquals(0, queueThatSkipsDeserializeErrors.size()); @@ -298,7 +312,8 @@ public void shouldNotThrowStreamsExceptionWhenValueDeserializationFailsWithSkipH @Test public void shouldThrowOnNegativeTimestamp() { final List> records = Collections.singletonList( - new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty())); final RecordQueue queue = new RecordQueue( new TopicPartition("topic", 1), @@ -323,7 +338,8 @@ public void shouldThrowOnNegativeTimestamp() { @Test public void shouldDropOnNegativeTimestamp() { final List> records = Collections.singletonList( - new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 1, -1L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty())); final RecordQueue queue = new RecordQueue( new TopicPartition("topic", 1), @@ -355,10 +371,14 @@ public void shouldPassPartitionTimeToTimestampExtractor() { // add three 3 out-of-order records with timestamp 2, 1, 3, 4 final List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty()), + new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0, 0, recordKey, recordValue, + new RecordHeaders(), Optional.empty())); assertEquals(RecordQueue.UNKNOWN, timestampExtractor.partitionTime); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 3a6a7c55872b1..fa80e46e31330 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.KafkaMetricsContext; @@ -81,6 +82,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -2539,11 +2541,12 @@ private ConsumerRecord getConsumerRecordWithOffsetAsTimestamp(fi offset, offset, // use the offset as the timestamp TimestampType.CREATE_TIME, - 0L, 0, 0, recordKey, - intSerializer.serialize(null, value) + intSerializer.serialize(null, value), + new RecordHeaders(), + Optional.empty() ); } @@ -2555,11 +2558,12 @@ private ConsumerRecord getConsumerRecordWithOffsetAsTimestamp(fi offset, offset, // use the offset as the timestamp TimestampType.CREATE_TIME, - 0L, 0, 0, recordKey, - recordValue + recordValue, + new RecordHeaders(), + Optional.empty() ); } @@ -2570,11 +2574,12 @@ private ConsumerRecord getConsumerRecordWithOffsetAsTimestamp(fi offset, offset, // use the offset as the timestamp TimestampType.CREATE_TIME, - 0L, 0, 0, new IntegerSerializer().serialize(topic1, key), - recordValue + recordValue, + new RecordHeaders(), + Optional.empty() ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 82b1996046295..80fb2c90b996a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -37,6 +37,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.KafkaMetricsContext; @@ -1853,11 +1854,12 @@ public void process(final Object key, final Object value) {} 100L, 100L, TimestampType.CREATE_TIME, - ConsumerRecord.NULL_CHECKSUM, "K".getBytes().length, "V".getBytes().length, "K".getBytes(), - "V".getBytes())); + "V".getBytes(), + new RecordHeaders(), + Optional.empty())); thread.runOnce(); @@ -1940,11 +1942,12 @@ public void process(final Object key, final Object value) { 110L, 110L, TimestampType.CREATE_TIME, - ConsumerRecord.NULL_CHECKSUM, "K".getBytes().length, "V".getBytes().length, "K".getBytes(), - "V".getBytes())); + "V".getBytes(), + new RecordHeaders(), + Optional.empty())); thread.runOnce(); @@ -2148,22 +2151,24 @@ private void shouldLogAndRecordSkippedMetricForDeserializationException(final St ++offset, -1, TimestampType.CREATE_TIME, - ConsumerRecord.NULL_CHECKSUM, -1, -1, new byte[0], - "I am not an integer.".getBytes())); + "I am not an integer.".getBytes(), + new RecordHeaders(), + Optional.empty())); mockConsumer.addRecord(new ConsumerRecord<>( t1p1.topic(), t1p1.partition(), ++offset, -1, TimestampType.CREATE_TIME, - ConsumerRecord.NULL_CHECKSUM, -1, -1, new byte[0], - "I am not an integer.".getBytes())); + "I am not an integer.".getBytes(), + new RecordHeaders(), + Optional.empty())); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(RecordDeserializer.class)) { thread.runOnce(); @@ -2860,11 +2865,12 @@ private void addRecord(final MockConsumer mockConsumer, offset, timestamp, TimestampType.CREATE_TIME, - ConsumerRecord.NULL_CHECKSUM, -1, -1, new byte[0], - new byte[0])); + new byte[0], + new RecordHeaders(), + Optional.empty())); } StandbyTask standbyTask(final TaskManager taskManager, final TopicPartition partition) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/ConsumerRecordUtil.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/ConsumerRecordUtil.java index a702fdc6d64c4..4a6fbbd489b00 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/ConsumerRecordUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/testutil/ConsumerRecordUtil.java @@ -17,8 +17,11 @@ package org.apache.kafka.streams.processor.internals.testutil; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; +import java.util.Optional; + public final class ConsumerRecordUtil { private ConsumerRecordUtil() {} @@ -36,11 +39,12 @@ public static ConsumerRecord record(final String topic, offset, 0L, TimestampType.CREATE_TIME, - 0L, 0, 0, key, - value + value, + new RecordHeaders(), + Optional.empty() ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RecordConvertersTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RecordConvertersTest.java index bacbacd2fed01..e409ca1d48a77 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RecordConvertersTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RecordConvertersTest.java @@ -17,10 +17,12 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.record.TimestampType; import org.junit.Test; import java.nio.ByteBuffer; +import java.util.Optional; import static org.apache.kafka.streams.state.internals.RecordConverters.rawValueToTimestampedValue; import static org.junit.Assert.assertArrayEquals; @@ -41,7 +43,8 @@ public void shouldAddTimestampToValueOnConversionWhenValueIsNotNull() { final long timestamp = 10L; final byte[] value = new byte[1]; final ConsumerRecord inputRecord = new ConsumerRecord<>( - "topic", 1, 0, timestamp, TimestampType.CREATE_TIME, 0L, 0, 0, new byte[0], value); + "topic", 1, 0, timestamp, TimestampType.CREATE_TIME, 0, 0, new byte[0], value, + new RecordHeaders(), Optional.empty()); final byte[] expectedValue = ByteBuffer.allocate(9).putLong(timestamp).put(value).array(); final byte[] actualValue = timestampedValueConverter.convert(inputRecord).value(); assertArrayEquals(expectedValue, actualValue); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 0037fbb84dce3..7d420c7768f4c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -47,6 +47,7 @@ import java.util.Collection; import java.util.LinkedList; import java.util.List; +import java.util.Optional; import java.util.Properties; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; @@ -387,9 +388,10 @@ public void shouldRestoreOldUnversionedFormat() { TimestampType.CREATE_TIME, -1, -1, - -1, "todelete".getBytes(UTF_8), - hexStringToByteArray(toDeleteBinaryValue)), + hexStringToByteArray(toDeleteBinaryValue), + new RecordHeaders(), + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 1, @@ -397,9 +399,10 @@ public void shouldRestoreOldUnversionedFormat() { TimestampType.CREATE_TIME, -1, -1, - -1, "asdf".getBytes(UTF_8), - hexStringToByteArray(asdfBinaryValue)), + hexStringToByteArray(asdfBinaryValue), + new RecordHeaders(), + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, @@ -407,9 +410,10 @@ public void shouldRestoreOldUnversionedFormat() { TimestampType.CREATE_TIME, -1, -1, - -1, "zxcv".getBytes(UTF_8), - hexStringToByteArray(zxcvBinaryValue1)), + hexStringToByteArray(zxcvBinaryValue1), + new RecordHeaders(), + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 3, @@ -417,9 +421,10 @@ public void shouldRestoreOldUnversionedFormat() { TimestampType.CREATE_TIME, -1, -1, - -1, "zxcv".getBytes(UTF_8), - hexStringToByteArray(zxcvBinaryValue2)) + hexStringToByteArray(zxcvBinaryValue2), + new RecordHeaders(), + Optional.empty()) )); assertThat(buffer.numRecords(), is(3)); @@ -434,9 +439,10 @@ public void shouldRestoreOldUnversionedFormat() { TimestampType.CREATE_TIME, -1, -1, - -1, "todelete".getBytes(UTF_8), - null) + null, + new RecordHeaders(), + Optional.empty()) )); assertThat(buffer.numRecords(), is(2)); @@ -501,45 +507,45 @@ public void shouldRestoreV1Format() { 0, 999, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), hexStringToByteArray(toDeleteBinary), - v1FlagHeaders), + v1FlagHeaders, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 1, 9999, TimestampType.CREATE_TIME, - -1L, -1, -1, "asdf".getBytes(UTF_8), hexStringToByteArray(asdfBinary), - v1FlagHeaders), + v1FlagHeaders, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, 99, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary1), - v1FlagHeaders), + v1FlagHeaders, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 3, 100, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary2), - v1FlagHeaders) + v1FlagHeaders, + Optional.empty()) )); assertThat(buffer.numRecords(), is(3)); @@ -552,11 +558,12 @@ public void shouldRestoreV1Format() { 3, 3, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), - null) + null, + new RecordHeaders(), + Optional.empty()) )); assertThat(buffer.numRecords(), is(2)); @@ -622,45 +629,45 @@ public void shouldRestoreV2Format() { 0, 999, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), hexStringToByteArray(toDeleteBinary), - v2FlagHeaders), + v2FlagHeaders, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 1, 9999, TimestampType.CREATE_TIME, - -1L, -1, -1, "asdf".getBytes(UTF_8), hexStringToByteArray(asdfBinary), - v2FlagHeaders), + v2FlagHeaders, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, 99, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary1), - v2FlagHeaders), + v2FlagHeaders, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, 100, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary2), - v2FlagHeaders) + v2FlagHeaders, + Optional.empty()) )); assertThat(buffer.numRecords(), is(3)); @@ -673,11 +680,12 @@ public void shouldRestoreV2Format() { 3, 3, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), - null) + null, + new RecordHeaders(), + Optional.empty()) )); assertThat(buffer.numRecords(), is(2)); @@ -745,45 +753,45 @@ public void shouldRestoreV3FormatWithV2Header() { 0, 999, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), hexStringToByteArray(toDeleteBinary), - headers), + headers, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 1, 9999, TimestampType.CREATE_TIME, - -1L, -1, -1, "asdf".getBytes(UTF_8), hexStringToByteArray(asdfBinary), - headers), + headers, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, 99, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary1), - headers), + headers, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, 100, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary2), - headers) + headers, + Optional.empty()) )); assertThat(buffer.numRecords(), is(3)); @@ -796,11 +804,12 @@ public void shouldRestoreV3FormatWithV2Header() { 3, 3, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), - null) + null, + new RecordHeaders(), + Optional.empty()) )); assertThat(buffer.numRecords(), is(2)); @@ -865,45 +874,45 @@ public void shouldRestoreV3Format() { 0, 999, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), hexStringToByteArray(toDeleteBinary), - headers), + headers, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 1, 9999, TimestampType.CREATE_TIME, - -1L, -1, -1, "asdf".getBytes(UTF_8), hexStringToByteArray(asdfBinary), - headers), + headers, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, 99, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary1), - headers), + headers, + Optional.empty()), new ConsumerRecord<>("changelog-topic", 0, 2, 100, TimestampType.CREATE_TIME, - -1L, -1, -1, "zxcv".getBytes(UTF_8), hexStringToByteArray(zxcvBinary2), - headers) + headers, + Optional.empty()) )); assertThat(buffer.numRecords(), is(3)); @@ -916,11 +925,12 @@ public void shouldRestoreV3Format() { 3, 3, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), - null) + null, + new RecordHeaders(), + Optional.empty()) )); assertThat(buffer.numRecords(), is(2)); @@ -979,12 +989,12 @@ public void shouldNotRestoreUnrecognizedVersionRecord() { 0, 999, TimestampType.CREATE_TIME, - -1L, -1, -1, "todelete".getBytes(UTF_8), ByteBuffer.allocate(Long.BYTES + todeleteValue.length).putLong(0L).put(todeleteValue).array(), - unknownFlagHeaders) + unknownFlagHeaders, + Optional.empty()) )); fail("expected an exception"); } catch (final IllegalArgumentException expected) { diff --git a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java index 244b35f146532..239887252cb9d 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java +++ b/streams/src/test/java/org/apache/kafka/test/MockRestoreConsumer.java @@ -28,6 +28,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import java.util.Optional; public class MockRestoreConsumer extends MockConsumer { private final Serializer keySerializer; @@ -60,10 +61,10 @@ public void reset() { public void bufferRecord(final ConsumerRecord record) { recordBuffer.add( new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), record.timestamp(), - record.timestampType(), 0L, 0, 0, + record.timestampType(), 0, 0, keySerializer.serialize(record.topic(), record.headers(), record.key()), valueSerializer.serialize(record.topic(), record.headers(), record.value()), - record.headers())); + record.headers(), Optional.empty())); endOffset = record.offset(); super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index f657209c18cd0..bd93367c6b264 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -101,6 +101,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.Optional; import java.util.Properties; import java.util.Queue; import java.util.Set; @@ -606,12 +607,12 @@ private void enqueueTaskRecord(final String inputTopic, offset, timestamp, TimestampType.CREATE_TIME, - (long) ConsumerRecord.NULL_CHECKSUM, key == null ? ConsumerRecord.NULL_SIZE : key.length, value == null ? ConsumerRecord.NULL_SIZE : value.length, key, value, - headers)) + headers, + Optional.empty())) ); } @@ -663,12 +664,12 @@ private void processGlobalRecord(final TopicPartition globalInputTopicPartition, offsetsByTopicOrPatternPartition.get(globalInputTopicPartition).incrementAndGet() - 1, timestamp, TimestampType.CREATE_TIME, - (long) ConsumerRecord.NULL_CHECKSUM, key == null ? ConsumerRecord.NULL_SIZE : key.length, value == null ? ConsumerRecord.NULL_SIZE : value.length, key, value, - headers) + headers, + Optional.empty()) ); globalStateTask.flushState(); } diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java index ae71de85bb7b3..15164f68a4ebd 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Objects; +import java.util.Optional; /** * Factory to create {@link ConsumerRecord consumer records} for a single single-partitioned topic with given key and @@ -190,12 +191,12 @@ public ConsumerRecord create(final String topicName, -1L, timestampMs, TimestampType.CREATE_TIME, - (long) ConsumerRecord.NULL_CHECKSUM, serializedKey == null ? 0 : serializedKey.length, serializedValue == null ? 0 : serializedValue.length, serializedKey, serializedValue, - headers); + headers, + Optional.empty()); } /** diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/TestRecordTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/TestRecordTest.java index b16ea03c8a61f..ad3b1a2a20a43 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/TestRecordTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/TestRecordTest.java @@ -27,6 +27,7 @@ import org.junit.jupiter.api.Test; import java.time.Instant; +import java.util.Optional; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -152,8 +153,8 @@ public void testToString() { @Test public void testConsumerRecord() { final String topicName = "topic"; - final ConsumerRecord consumerRecord = - new ConsumerRecord<>(topicName, 1, 0, recordMs, TimestampType.CREATE_TIME, 0L, 0, 0, key, value, headers); + final ConsumerRecord consumerRecord = new ConsumerRecord<>(topicName, 1, 0, recordMs, + TimestampType.CREATE_TIME, 0, 0, key, value, headers, Optional.empty()); final TestRecord testRecord = new TestRecord<>(consumerRecord); final TestRecord expectedRecord = new TestRecord<>(key, value, headers, recordTime); assertEquals(expectedRecord, testRecord); From 07fb69c16cf2ce1aa2d45b5c7e0dd04a11e42d96 Mon Sep 17 00:00:00 2001 From: Justine Olshan Date: Thu, 15 Apr 2021 08:01:26 -0700 Subject: [PATCH 060/155] MINOR: Update default for field in DeleteTopicsRequest When using DeleteTopicsRequest with topic IDs, the topic name field should be null. Before, the code was programmatically assigning null, but it will be easier and less error prone to simply set that as the default. Tested using the previously created `DeleteTopicsRequestTest.java` file. Reviewers: David Jacot --- .../apache/kafka/common/requests/DeleteTopicsRequest.java | 6 ------ .../main/resources/common/message/DeleteTopicsRequest.json | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java index dbcc207105ffb..1322630361ded 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteTopicsRequest.java @@ -44,12 +44,6 @@ public Builder(DeleteTopicsRequestData data) { public DeleteTopicsRequest build(short version) { if (version >= 6 && !data.topicNames().isEmpty()) { data.setTopics(groupByTopic(data.topicNames())); - } else if (version >= 6) { - for (DeleteTopicState topic : data.topics()) { - if (topic.name() != null && topic.name().equals("")) { - topic.setName(null); - } - } } return new DeleteTopicsRequest(data, version); } diff --git a/clients/src/main/resources/common/message/DeleteTopicsRequest.json b/clients/src/main/resources/common/message/DeleteTopicsRequest.json index 2e020e9802a7a..19bdc8a29135e 100644 --- a/clients/src/main/resources/common/message/DeleteTopicsRequest.json +++ b/clients/src/main/resources/common/message/DeleteTopicsRequest.json @@ -31,7 +31,7 @@ "fields": [ { "name": "Topics", "type": "[]DeleteTopicState", "versions": "6+", "about": "The name or topic ID of the topic", "fields": [ - {"name": "Name", "type": "string", "versions": "6+", "nullableVersions": "6+", "entityType": "topicName", "about": "The topic name"}, + {"name": "Name", "type": "string", "versions": "6+", "nullableVersions": "6+", "default": "null", "entityType": "topicName", "about": "The topic name"}, {"name": "TopicId", "type": "uuid", "versions": "6+", "about": "The unique topic ID"} ]}, { "name": "TopicNames", "type": "[]string", "versions": "0-5", "entityType": "topicName", "ignorable": true, From 2e25a1d4d44b517e133b227cf4cb817a7c82bc0a Mon Sep 17 00:00:00 2001 From: Ron Dagostino Date: Thu, 15 Apr 2021 13:13:41 -0400 Subject: [PATCH 061/155] MINOR: shut down the KafkaScheduler later (#10538) Be sure to shut down the kafka scheduler thread after the LogManager, since the latter uses the former. Reviewers: Colin P. McCabe , Ismael Juma --- core/src/main/scala/kafka/server/BrokerServer.scala | 5 +++-- core/src/main/scala/kafka/server/KafkaServer.scala | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 4790b9f8b186f..59572bb90abc2 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -422,8 +422,6 @@ class BrokerServer( CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this) if (controlPlaneRequestHandlerPool != null) CoreUtils.swallow(controlPlaneRequestHandlerPool.shutdown(), this) - if (kafkaScheduler != null) - CoreUtils.swallow(kafkaScheduler.shutdown(), this) if (dataPlaneRequestProcessor != null) CoreUtils.swallow(dataPlaneRequestProcessor.close(), this) @@ -453,6 +451,9 @@ class BrokerServer( if (logManager != null) CoreUtils.swallow(logManager.shutdown(), this) + // be sure to shutdown scheduler after log manager + if (kafkaScheduler != null) + CoreUtils.swallow(kafkaScheduler.shutdown(), this) if (quotaManagers != null) CoreUtils.swallow(quotaManagers.shutdown(), this) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 456ad5ec080a1..02fb28c67dbb1 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -675,8 +675,6 @@ class KafkaServer( CoreUtils.swallow(dataPlaneRequestHandlerPool.shutdown(), this) if (controlPlaneRequestHandlerPool != null) CoreUtils.swallow(controlPlaneRequestHandlerPool.shutdown(), this) - if (kafkaScheduler != null) - CoreUtils.swallow(kafkaScheduler.shutdown(), this) if (dataPlaneRequestProcessor != null) CoreUtils.swallow(dataPlaneRequestProcessor.close(), this) @@ -704,6 +702,9 @@ class KafkaServer( if (logManager != null) CoreUtils.swallow(logManager.shutdown(), this) + // be sure to shutdown scheduler after log manager + if (kafkaScheduler != null) + CoreUtils.swallow(kafkaScheduler.shutdown(), this) if (kafkaController != null) CoreUtils.swallow(kafkaController.shutdown(), this) From 31b4ba8c5a77b7fc5b92fce5228a872f860b0f18 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Thu, 15 Apr 2021 10:45:24 -0700 Subject: [PATCH 062/155] MINOR: fix some bugs in ControllerApis.scala (#10505) Fix some cases where ControllerApis was blocking on the controller thread. This should not be necessary, since the controller thread can just interface directly with the network threads. Fix some cases where ControllerApis wasn't doing authorization correctly. Since the previous release of KRaft did not support authorizers, this bug is not as severe as it could have been, but it still needs to be fixed. Add authorization unit tests for each API. Add support for the deprecated ALTER_CONFIGS API in ControllerApis. It was already supported in QuorumController, but wasn't exposed previously. Fix how we validate duplicate config resources and unknown config resource types in ControllerApis. Duplicates should yield an INVALID_REQUEST error, and unknown resource types should give an error with the corresponding numerical resource type and UNSUPPORTED_VERSION. Fix some redaction code in RequestChannel that was throwing an exception when duplicate config resources were present in the request. Fix a comment in ControllerApis#deleteTopics that no longer reflects what the code is doing when we don't have "describe" permission. Add function stubs for the KIP-455 reassignment APIs in ControllerApis and QuorumController. Reviewers: Chia-Ping Tsai , David Arthur --- .../scala/kafka/network/RequestChannel.scala | 42 +- .../scala/kafka/server/ControllerApis.scala | 385 ++++++++++++------ .../test/java/kafka/test/MockController.java | 74 +++- .../kafka/server/ControllerApisTest.scala | 304 +++++++++++++- .../apache/kafka/controller/Controller.java | 24 ++ .../kafka/controller/QuorumController.java | 20 + 6 files changed, 687 insertions(+), 162 deletions(-) diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index a2a2144f0128e..2530abf8dbdb1 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -32,8 +32,6 @@ import kafka.utils.Implicits._ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.ApiMessageType.ListenerType -import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData -import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData._ import org.apache.kafka.common.network.Send import org.apache.kafka.common.protocol.{ApiKeys, Errors, ObjectSerializationCache} import org.apache.kafka.common.requests._ @@ -169,32 +167,24 @@ object RequestChannel extends Logging { bodyAndSize.request match { case alterConfigs: AlterConfigsRequest => - val loggableConfigs = alterConfigs.configs().asScala.map { case (resource, config) => - val loggableEntries = new AlterConfigsRequest.Config(config.entries.asScala.map { entry => - new AlterConfigsRequest.ConfigEntry(entry.name, KafkaConfig.loggableValue(resource.`type`, entry.name, entry.value)) - }.asJavaCollection) - (resource, loggableEntries) - }.asJava - new AlterConfigsRequest.Builder(loggableConfigs, alterConfigs.validateOnly).build(alterConfigs.version()) + val newData = alterConfigs.data().duplicate() + newData.resources().forEach(resource => { + val resourceType = ConfigResource.Type.forId(resource.resourceType()) + resource.configs().forEach(config => { + config.setValue(KafkaConfig.loggableValue(resourceType, config.name(), config.value())) + }) + }) + new AlterConfigsRequest(newData, alterConfigs.version()) case alterConfigs: IncrementalAlterConfigsRequest => - val resources = new AlterConfigsResourceCollection(alterConfigs.data.resources.size) - alterConfigs.data.resources.forEach { resource => - val newResource = new AlterConfigsResource() - .setResourceName(resource.resourceName) - .setResourceType(resource.resourceType) - resource.configs.forEach { config => - newResource.configs.add(new AlterableConfig() - .setName(config.name) - .setValue(KafkaConfig.loggableValue(ConfigResource.Type.forId(resource.resourceType), config.name, config.value)) - .setConfigOperation(config.configOperation)) - } - resources.add(newResource) - } - val data = new IncrementalAlterConfigsRequestData() - .setValidateOnly(alterConfigs.data.validateOnly()) - .setResources(resources) - new IncrementalAlterConfigsRequest.Builder(data).build(alterConfigs.version) + val newData = alterConfigs.data().duplicate() + newData.resources().forEach(resource => { + val resourceType = ConfigResource.Type.forId(resource.resourceType()) + resource.configs().forEach(config => { + config.setValue(KafkaConfig.loggableValue(resourceType, config.name(), config.value())) + }) + }) + new IncrementalAlterConfigsRequest.Builder(newData).build(alterConfigs.version()) case _ => bodyAndSize.request diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index 13812fd408f75..3f958f5b2dd33 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -19,6 +19,7 @@ package kafka.server import java.util import java.util.Collections +import java.util.Map.Entry import java.util.concurrent.{CompletableFuture, ExecutionException} import kafka.network.RequestChannel @@ -27,16 +28,17 @@ import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.Logging import org.apache.kafka.clients.admin.AlterConfigOp import org.apache.kafka.common.Uuid.ZERO_UUID -import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.common.acl.AclOperation.{ALTER, ALTER_CONFIGS, CLUSTER_ACTION, CREATE, DELETE, DESCRIBE} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{ApiException, ClusterAuthorizationException, InvalidRequestException, TopicDeletionDisabledException} import org.apache.kafka.common.internals.FatalExitError +import org.apache.kafka.common.message.AlterConfigsResponseData.{AlterConfigsResourceResponse => OldAlterConfigsResourceResponse} import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.DeleteTopicsResponseData.{DeletableTopicResult, DeletableTopicResultCollection} +import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseBroker import org.apache.kafka.common.message._ import org.apache.kafka.common.protocol.Errors._ @@ -46,6 +48,7 @@ import org.apache.kafka.common.resource.Resource import org.apache.kafka.common.resource.Resource.CLUSTER_NAME import org.apache.kafka.common.resource.ResourceType.{CLUSTER, TOPIC} import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.controller.Controller import org.apache.kafka.metadata.{ApiMessageAndVersion, BrokerHeartbeatReply, BrokerRegistrationReply, VersionRange} import org.apache.kafka.server.authorizer.Authorizer @@ -79,6 +82,7 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.CREATE_TOPICS => handleCreateTopics(request) case ApiKeys.DELETE_TOPICS => handleDeleteTopics(request) case ApiKeys.API_VERSIONS => handleApiVersionsRequest(request) + case ApiKeys.ALTER_CONFIGS => handleLegacyAlterConfigs(request) case ApiKeys.VOTE => handleVote(request) case ApiKeys.BEGIN_QUORUM_EPOCH => handleBeginQuorumEpoch(request) case ApiKeys.END_QUORUM_EPOCH => handleEndQuorumEpoch(request) @@ -89,6 +93,8 @@ class ControllerApis(val requestChannel: RequestChannel, case ApiKeys.UNREGISTER_BROKER => handleUnregisterBroker(request) case ApiKeys.ALTER_CLIENT_QUOTAS => handleAlterClientQuotas(request) case ApiKeys.INCREMENTAL_ALTER_CONFIGS => handleIncrementalAlterConfigs(request) + case ApiKeys.ALTER_PARTITION_REASSIGNMENTS => handleAlterPartitionReassignments(request) + case ApiKeys.LIST_PARTITION_REASSIGNMENTS => handleListPartitionReassignments(request) case ApiKeys.ENVELOPE => handleEnvelopeRequest(request) case ApiKeys.SASL_HANDSHAKE => handleSaslHandshakeRequest(request) case ApiKeys.SASL_AUTHENTICATE => handleSaslAuthenticateRequest(request) @@ -112,18 +118,18 @@ class ControllerApis(val requestChannel: RequestChannel, } def handleSaslHandshakeRequest(request: RequestChannel.Request): Unit = { - val responseData = new SaslHandshakeResponseData().setErrorCode(Errors.ILLEGAL_SASL_STATE.code) + val responseData = new SaslHandshakeResponseData().setErrorCode(ILLEGAL_SASL_STATE.code) requestHelper.sendResponseMaybeThrottle(request, _ => new SaslHandshakeResponse(responseData)) } def handleSaslAuthenticateRequest(request: RequestChannel.Request): Unit = { val responseData = new SaslAuthenticateResponseData() - .setErrorCode(Errors.ILLEGAL_SASL_STATE.code) + .setErrorCode(ILLEGAL_SASL_STATE.code) .setErrorMessage("SaslAuthenticate request received after successful authentication") requestHelper.sendResponseMaybeThrottle(request, _ => new SaslAuthenticateResponse(responseData)) } - private def handleFetch(request: RequestChannel.Request): Unit = { + def handleFetch(request: RequestChannel.Request): Unit = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new FetchResponse(response.asInstanceOf[FetchResponseData])) } @@ -140,7 +146,7 @@ class ControllerApis(val requestChannel: RequestChannel, .setPort(node.port) .setRack(node.rack)) } - metadataResponseData.setClusterId(metaProperties.clusterId.toString) + metadataResponseData.setClusterId(metaProperties.clusterId) if (controller.isActive) { metadataResponseData.setControllerId(config.nodeId) } else { @@ -164,16 +170,23 @@ class ControllerApis(val requestChannel: RequestChannel, } def handleDeleteTopics(request: RequestChannel.Request): Unit = { - val responses = deleteTopics(request.body[DeleteTopicsRequest].data, + val deleteTopicsRequest = request.body[DeleteTopicsRequest] + val future = deleteTopics(deleteTopicsRequest.data, request.context.apiVersion, authHelper.authorize(request.context, DELETE, CLUSTER, CLUSTER_NAME), names => authHelper.filterByAuthorized(request.context, DESCRIBE, TOPIC, names)(n => n), names => authHelper.filterByAuthorized(request.context, DELETE, TOPIC, names)(n => n)) - requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { - val responseData = new DeleteTopicsResponseData(). - setResponses(new DeletableTopicResultCollection(responses.iterator)). - setThrottleTimeMs(throttleTimeMs) - new DeleteTopicsResponse(responseData) + future.whenComplete((results, exception) => { + requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { + if (exception != null) { + deleteTopicsRequest.getErrorResponse(throttleTimeMs, exception) + } else { + val responseData = new DeleteTopicsResponseData(). + setResponses(new DeletableTopicResultCollection(results.iterator)). + setThrottleTimeMs(throttleTimeMs) + new DeleteTopicsResponse(responseData) + } + }) }) } @@ -181,7 +194,8 @@ class ControllerApis(val requestChannel: RequestChannel, apiVersion: Int, hasClusterAuth: Boolean, getDescribableTopics: Iterable[String] => Set[String], - getDeletableTopics: Iterable[String] => Set[String]): util.List[DeletableTopicResult] = { + getDeletableTopics: Iterable[String] => Set[String]) + : CompletableFuture[util.List[DeletableTopicResult]] = { // Check if topic deletion is enabled at all. if (!config.deleteTopicEnable) { if (apiVersion < 3) { @@ -242,87 +256,99 @@ class ControllerApis(val requestChannel: RequestChannel, val toAuthenticate = new util.HashSet[String] toAuthenticate.addAll(providedNames) val idToName = new util.HashMap[Uuid, String] - controller.findTopicNames(providedIds).get().forEach { (id, nameOrError) => - if (nameOrError.isError) { - appendResponse(null, id, nameOrError.error()) - } else { - toAuthenticate.add(nameOrError.result()) - idToName.put(id, nameOrError.result()) - } - } - // Get the list of deletable topics (those we can delete) and the list of describeable - // topics. If a topic can't be deleted or described, we have to act like it doesn't - // exist, even when it does. - val topicsToAuthenticate = toAuthenticate.asScala - val (describeable, deletable) = if (hasClusterAuth) { - (topicsToAuthenticate.toSet, topicsToAuthenticate.toSet) - } else { - (getDescribableTopics(topicsToAuthenticate), getDeletableTopics(topicsToAuthenticate)) - } - // For each topic that was provided by ID, check if authentication failed. - // If so, remove it from the idToName map and create an error response for it. - val iterator = idToName.entrySet().iterator() - while (iterator.hasNext) { - val entry = iterator.next() - val id = entry.getKey - val name = entry.getValue - if (!deletable.contains(name)) { - if (describeable.contains(name)) { - appendResponse(name, id, new ApiError(TOPIC_AUTHORIZATION_FAILED)) + controller.findTopicNames(providedIds).thenCompose(topicNames => { + topicNames.forEach { (id, nameOrError) => + if (nameOrError.isError) { + appendResponse(null, id, nameOrError.error()) } else { - appendResponse(null, id, new ApiError(TOPIC_AUTHORIZATION_FAILED)) + toAuthenticate.add(nameOrError.result()) + idToName.put(id, nameOrError.result()) } - iterator.remove() } - } - // For each topic that was provided by name, check if authentication failed. - // If so, create an error response for it. Otherwise, add it to the idToName map. - controller.findTopicIds(providedNames).get().forEach { (name, idOrError) => - if (!describeable.contains(name)) { - appendResponse(name, ZERO_UUID, new ApiError(TOPIC_AUTHORIZATION_FAILED)) - } else if (idOrError.isError) { - appendResponse(name, ZERO_UUID, idOrError.error) - } else if (deletable.contains(name)) { - val id = idOrError.result() - if (duplicateProvidedIds.contains(id) || idToName.put(id, name) != null) { - // This is kind of a weird case: what if we supply topic ID X and also a name - // that maps to ID X? In that case, _if authorization succeeds_, we end up - // here. If authorization doesn't succeed, we refrain from commenting on the - // situation since it would reveal topic ID mappings. - duplicateProvidedIds.add(id) - idToName.remove(id) - appendResponse(name, id, new ApiError(INVALID_REQUEST, - "The provided topic name maps to an ID that was already supplied.")) - } + // Get the list of deletable topics (those we can delete) and the list of describeable + // topics. + val topicsToAuthenticate = toAuthenticate.asScala + val (describeable, deletable) = if (hasClusterAuth) { + (topicsToAuthenticate.toSet, topicsToAuthenticate.toSet) } else { - appendResponse(name, ZERO_UUID, new ApiError(TOPIC_AUTHORIZATION_FAILED)) + (getDescribableTopics(topicsToAuthenticate), getDeletableTopics(topicsToAuthenticate)) } - } - // Finally, the idToName map contains all the topics that we are authorized to delete. - // Perform the deletion and create responses for each one. - val idToError = controller.deleteTopics(idToName.keySet).get() - idToError.forEach { (id, error) => - appendResponse(idToName.get(id), id, error) - } - // Shuffle the responses so that users can not use patterns in their positions to - // distinguish between absent topics and topics we are not permitted to see. - Collections.shuffle(responses) - responses + // For each topic that was provided by ID, check if authentication failed. + // If so, remove it from the idToName map and create an error response for it. + val iterator = idToName.entrySet().iterator() + while (iterator.hasNext) { + val entry = iterator.next() + val id = entry.getKey + val name = entry.getValue + if (!deletable.contains(name)) { + if (describeable.contains(name)) { + appendResponse(name, id, new ApiError(TOPIC_AUTHORIZATION_FAILED)) + } else { + appendResponse(null, id, new ApiError(TOPIC_AUTHORIZATION_FAILED)) + } + iterator.remove() + } + } + // For each topic that was provided by name, check if authentication failed. + // If so, create an error response for it. Otherwise, add it to the idToName map. + controller.findTopicIds(providedNames).thenCompose(topicIds => { + topicIds.forEach { (name, idOrError) => + if (!describeable.contains(name)) { + appendResponse(name, ZERO_UUID, new ApiError(TOPIC_AUTHORIZATION_FAILED)) + } else if (idOrError.isError) { + appendResponse(name, ZERO_UUID, idOrError.error) + } else if (deletable.contains(name)) { + val id = idOrError.result() + if (duplicateProvidedIds.contains(id) || idToName.put(id, name) != null) { + // This is kind of a weird case: what if we supply topic ID X and also a name + // that maps to ID X? In that case, _if authorization succeeds_, we end up + // here. If authorization doesn't succeed, we refrain from commenting on the + // situation since it would reveal topic ID mappings. + duplicateProvidedIds.add(id) + idToName.remove(id) + appendResponse(name, id, new ApiError(INVALID_REQUEST, + "The provided topic name maps to an ID that was already supplied.")) + } + } else { + appendResponse(name, ZERO_UUID, new ApiError(TOPIC_AUTHORIZATION_FAILED)) + } + } + // Finally, the idToName map contains all the topics that we are authorized to delete. + // Perform the deletion and create responses for each one. + controller.deleteTopics(idToName.keySet).thenApply(idToError => { + idToError.forEach { (id, error) => + appendResponse(idToName.get(id), id, error) + } + // Shuffle the responses so that users can not use patterns in their positions to + // distinguish between absent topics and topics we are not permitted to see. + Collections.shuffle(responses) + responses + }) + }) + }) } def handleCreateTopics(request: RequestChannel.Request): Unit = { - val responseData = createTopics(request.body[CreateTopicsRequest].data(), + val createTopicsRequest = request.body[CreateTopicsRequest] + val future = createTopics(createTopicsRequest.data(), authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME), names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity)) - requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { - responseData.setThrottleTimeMs(throttleTimeMs) - new CreateTopicsResponse(responseData) + future.whenComplete((result, exception) => { + requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { + if (exception != null) { + createTopicsRequest.getErrorResponse(throttleTimeMs, exception) + } else { + result.setThrottleTimeMs(throttleTimeMs) + new CreateTopicsResponse(result) + } + }) }) } def createTopics(request: CreateTopicsRequestData, hasClusterAuth: Boolean, - getCreatableTopics: Iterable[String] => Set[String]): CreateTopicsResponseData = { + getCreatableTopics: Iterable[String] => Set[String]) + : CompletableFuture[CreateTopicsResponseData] = { val topicNames = new util.HashSet[String]() val duplicateTopicNames = new util.HashSet[String]() request.topics().forEach { topicData => @@ -347,21 +373,22 @@ class ControllerApis(val requestChannel: RequestChannel, iterator.remove() } } - val response = controller.createTopics(effectiveRequest).get() - duplicateTopicNames.forEach { name => - response.topics().add(new CreatableTopicResult(). - setName(name). - setErrorCode(INVALID_REQUEST.code()). - setErrorMessage("Found multiple entries for this topic.")) - } - topicNames.forEach { name => - if (!authorizedTopicNames.contains(name)) { + controller.createTopics(effectiveRequest).thenApply(response => { + duplicateTopicNames.forEach { name => response.topics().add(new CreatableTopicResult(). setName(name). - setErrorCode(TOPIC_AUTHORIZATION_FAILED.code())) + setErrorCode(INVALID_REQUEST.code). + setErrorMessage("Duplicate topic name.")) } - } - response + topicNames.forEach { name => + if (!authorizedTopicNames.contains(name)) { + response.topics().add(new CreatableTopicResult(). + setName(name). + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code)) + } + } + response + }) } def handleApiVersionsRequest(request: RequestChannel.Request): Unit = { @@ -374,7 +401,7 @@ class ControllerApis(val requestChannel: RequestChannel, def createResponseCallback(requestThrottleMs: Int): ApiVersionsResponse = { val apiVersionRequest = request.body[ApiVersionsRequest] if (apiVersionRequest.hasUnsupportedRequestVersion) { - apiVersionRequest.getErrorResponse(requestThrottleMs, Errors.UNSUPPORTED_VERSION.exception) + apiVersionRequest.getErrorResponse(requestThrottleMs, UNSUPPORTED_VERSION.exception) } else if (!apiVersionRequest.isValid) { apiVersionRequest.getErrorResponse(requestThrottleMs, INVALID_REQUEST.exception) } else { @@ -384,29 +411,106 @@ class ControllerApis(val requestChannel: RequestChannel, requestHelper.sendResponseMaybeThrottle(request, createResponseCallback) } - private def handleVote(request: RequestChannel.Request): Unit = { + def authorizeAlterResource(requestContext: RequestContext, + resource: ConfigResource): ApiError = { + resource.`type` match { + case ConfigResource.Type.BROKER => + if (authHelper.authorize(requestContext, ALTER_CONFIGS, CLUSTER, CLUSTER_NAME)) { + new ApiError(NONE) + } else { + new ApiError(CLUSTER_AUTHORIZATION_FAILED) + } + case ConfigResource.Type.TOPIC => + if (authHelper.authorize(requestContext, ALTER_CONFIGS, TOPIC, resource.name)) { + new ApiError(NONE) + } else { + new ApiError(TOPIC_AUTHORIZATION_FAILED) + } + case rt => new ApiError(INVALID_REQUEST, s"Unexpected resource type $rt.") + } + } + + def handleLegacyAlterConfigs(request: RequestChannel.Request): Unit = { + val response = new AlterConfigsResponseData() + val alterConfigsRequest = request.body[AlterConfigsRequest] + val duplicateResources = new util.HashSet[ConfigResource] + val configChanges = new util.HashMap[ConfigResource, util.Map[String, String]]() + alterConfigsRequest.data.resources.forEach { resource => + val configResource = new ConfigResource( + ConfigResource.Type.forId(resource.resourceType), resource.resourceName()) + if (configResource.`type`().equals(ConfigResource.Type.UNKNOWN)) { + response.responses().add(new OldAlterConfigsResourceResponse(). + setErrorCode(UNSUPPORTED_VERSION.code()). + setErrorMessage("Unknown resource type " + resource.resourceType() + "."). + setResourceName(resource.resourceName()). + setResourceType(resource.resourceType())) + } else if (!duplicateResources.contains(configResource)) { + val configs = new util.HashMap[String, String]() + resource.configs().forEach(config => configs.put(config.name(), config.value())) + if (configChanges.put(configResource, configs) != null) { + duplicateResources.add(configResource) + configChanges.remove(configResource) + response.responses().add(new OldAlterConfigsResourceResponse(). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Duplicate resource."). + setResourceName(resource.resourceName()). + setResourceType(resource.resourceType())) + } + } + } + val iterator = configChanges.keySet().iterator() + while (iterator.hasNext) { + val resource = iterator.next() + val apiError = authorizeAlterResource(request.context, resource) + if (apiError.isFailure) { + response.responses().add(new OldAlterConfigsResourceResponse(). + setErrorCode(apiError.error().code()). + setErrorMessage(apiError.message()). + setResourceName(resource.name()). + setResourceType(resource.`type`().id())) + iterator.remove() + } + } + controller.legacyAlterConfigs(configChanges, alterConfigsRequest.data.validateOnly) + .whenComplete((controllerResults, exception) => { + if (exception != null) { + requestHelper.handleError(request, exception) + } else { + controllerResults.entrySet().forEach(entry => response.responses().add( + new OldAlterConfigsResourceResponse(). + setErrorCode(entry.getValue.error().code()). + setErrorMessage(entry.getValue.message()). + setResourceName(entry.getKey.name()). + setResourceType(entry.getKey.`type`().id()))) + requestHelper.sendResponseMaybeThrottle(request, throttleMs => + new AlterConfigsResponse(response.setThrottleTimeMs(throttleMs))) + } + }) + } + + def handleVote(request: RequestChannel.Request): Unit = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new VoteResponse(response.asInstanceOf[VoteResponseData])) } - private def handleBeginQuorumEpoch(request: RequestChannel.Request): Unit = { + def handleBeginQuorumEpoch(request: RequestChannel.Request): Unit = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new BeginQuorumEpochResponse(response.asInstanceOf[BeginQuorumEpochResponseData])) } - private def handleEndQuorumEpoch(request: RequestChannel.Request): Unit = { + def handleEndQuorumEpoch(request: RequestChannel.Request): Unit = { authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) handleRaftRequest(request, response => new EndQuorumEpochResponse(response.asInstanceOf[EndQuorumEpochResponseData])) } - private def handleDescribeQuorum(request: RequestChannel.Request): Unit = { + def handleDescribeQuorum(request: RequestChannel.Request): Unit = { authHelper.authorizeClusterOperation(request, DESCRIBE) handleRaftRequest(request, response => new DescribeQuorumResponse(response.asInstanceOf[DescribeQuorumResponseData])) } def handleAlterIsrRequest(request: RequestChannel.Request): Unit = { - authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val alterIsrRequest = request.body[AlterIsrRequest] + authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val future = controller.alterIsr(alterIsrRequest.data) future.whenComplete((result, exception) => { val response = if (exception != null) { @@ -433,7 +537,7 @@ class ControllerApis(val requestChannel: RequestChannel, } else { new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData(). setThrottleTimeMs(requestThrottleMs). - setErrorCode(Errors.NONE.code). + setErrorCode(NONE.code). setIsCaughtUp(reply.isCaughtUp). setIsFenced(reply.isFenced). setShouldShutDown(reply.shouldShutDown)) @@ -476,11 +580,11 @@ class ControllerApis(val requestChannel: RequestChannel, if (e != null) { new BrokerRegistrationResponse(new BrokerRegistrationResponseData(). setThrottleTimeMs(requestThrottleMs). - setErrorCode(Errors.forException(e).code())) + setErrorCode(Errors.forException(e).code)) } else { new BrokerRegistrationResponse(new BrokerRegistrationResponseData(). setThrottleTimeMs(requestThrottleMs). - setErrorCode(Errors.NONE.code). + setErrorCode(NONE.code). setBrokerEpoch(reply.epoch)) } } @@ -506,8 +610,7 @@ class ControllerApis(val requestChannel: RequestChannel, def handleAlterClientQuotas(request: RequestChannel.Request): Unit = { val quotaRequest = request.body[AlterClientQuotasRequest] - authHelper.authorize(request.context, ALTER_CONFIGS, CLUSTER, CLUSTER_NAME) - + authHelper.authorizeClusterOperation(request, ALTER_CONFIGS) controller.alterClientQuotas(quotaRequest.entries, quotaRequest.validateOnly) .whenComplete((results, exception) => { if (exception != null) { @@ -520,25 +623,63 @@ class ControllerApis(val requestChannel: RequestChannel, } def handleIncrementalAlterConfigs(request: RequestChannel.Request): Unit = { + val response = new IncrementalAlterConfigsResponseData() val alterConfigsRequest = request.body[IncrementalAlterConfigsRequest] - authHelper.authorize(request.context, ALTER_CONFIGS, CLUSTER, CLUSTER_NAME) - val configChanges = new util.HashMap[ConfigResource, util.Map[String, util.Map.Entry[AlterConfigOp.OpType, String]]]() + val duplicateResources = new util.HashSet[ConfigResource] + val configChanges = new util.HashMap[ConfigResource, + util.Map[String, Entry[AlterConfigOp.OpType, String]]]() alterConfigsRequest.data.resources.forEach { resource => - val configResource = new ConfigResource(ConfigResource.Type.forId(resource.resourceType), resource.resourceName()) - val altersByName = new util.HashMap[String, util.Map.Entry[AlterConfigOp.OpType, String]]() - resource.configs.forEach { config => - altersByName.put(config.name, new util.AbstractMap.SimpleEntry[AlterConfigOp.OpType, String]( - AlterConfigOp.OpType.forId(config.configOperation), config.value)) + val configResource = new ConfigResource( + ConfigResource.Type.forId(resource.resourceType), resource.resourceName()) + if (configResource.`type`().equals(ConfigResource.Type.UNKNOWN)) { + response.responses().add(new AlterConfigsResourceResponse(). + setErrorCode(UNSUPPORTED_VERSION.code()). + setErrorMessage("Unknown resource type " + resource.resourceType() + "."). + setResourceName(resource.resourceName()). + setResourceType(resource.resourceType())) + } else if (!duplicateResources.contains(configResource)) { + val altersByName = new util.HashMap[String, Entry[AlterConfigOp.OpType, String]]() + resource.configs.forEach { config => + altersByName.put(config.name, new util.AbstractMap.SimpleEntry[AlterConfigOp.OpType, String]( + AlterConfigOp.OpType.forId(config.configOperation), config.value)) + } + if (configChanges.put(configResource, altersByName) != null) { + duplicateResources.add(configResource) + configChanges.remove(configResource) + response.responses().add(new AlterConfigsResourceResponse(). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Duplicate resource."). + setResourceName(resource.resourceName()). + setResourceType(resource.resourceType())) + } + } + } + val iterator = configChanges.keySet().iterator() + while (iterator.hasNext) { + val resource = iterator.next() + val apiError = authorizeAlterResource(request.context, resource) + if (apiError.isFailure) { + response.responses().add(new AlterConfigsResourceResponse(). + setErrorCode(apiError.error().code()). + setErrorMessage(apiError.message()). + setResourceName(resource.name()). + setResourceType(resource.`type`().id())) + iterator.remove() } - configChanges.put(configResource, altersByName) } controller.incrementalAlterConfigs(configChanges, alterConfigsRequest.data.validateOnly) - .whenComplete((results, exception) => { + .whenComplete((controllerResults, exception) => { if (exception != null) { requestHelper.handleError(request, exception) } else { - requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new IncrementalAlterConfigsResponse(requestThrottleMs, results)) + controllerResults.entrySet().forEach(entry => response.responses().add( + new AlterConfigsResourceResponse(). + setErrorCode(entry.getValue.error().code()). + setErrorMessage(entry.getValue.message()). + setResourceName(entry.getKey.name()). + setResourceType(entry.getKey.`type`().id()))) + requestHelper.sendResponseMaybeThrottle(request, throttleMs => + new IncrementalAlterConfigsResponse(response.setThrottleTimeMs(throttleMs))) } }) } @@ -577,7 +718,7 @@ class ControllerApis(val requestChannel: RequestChannel, duplicateTopicNames.forEach { topicName => responses.add(new CreatePartitionsTopicResult(). setName(topicName). - setErrorCode(INVALID_REQUEST.code()). + setErrorCode(INVALID_REQUEST.code). setErrorMessage("Duplicate topic name.")) topicNames.remove(topicName) } @@ -595,7 +736,7 @@ class ControllerApis(val requestChannel: RequestChannel, } else { responses.add(new CreatePartitionsTopicResult(). setName(topicName). - setErrorCode(TOPIC_AUTHORIZATION_FAILED.code())) + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code)) } } controller.createPartitions(topics).thenApply { results => @@ -603,4 +744,20 @@ class ControllerApis(val requestChannel: RequestChannel, responses } } + + def handleAlterPartitionReassignments(request: RequestChannel.Request): Unit = { + val alterRequest = request.body[AlterPartitionReassignmentsRequest] + authHelper.authorizeClusterOperation(request, ALTER) + val response = controller.alterPartitionReassignments(alterRequest.data()).get() + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new AlterPartitionReassignmentsResponse(response)) + } + + def handleListPartitionReassignments(request: RequestChannel.Request): Unit = { + val listRequest = request.body[ListPartitionReassignmentsRequest] + authHelper.authorizeClusterOperation(request, DESCRIBE) + val response = controller.listPartitionReassignments(listRequest.data()).get() + requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => + new ListPartitionReassignmentsResponse(response)) + } } diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java index e28380516b35c..fc14145e28037 100644 --- a/core/src/test/java/kafka/test/MockController.java +++ b/core/src/test/java/kafka/test/MockController.java @@ -23,6 +23,8 @@ import org.apache.kafka.common.errors.NotControllerException; import org.apache.kafka.common.message.AlterIsrRequestData; import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData; import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; @@ -33,6 +35,8 @@ import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; import org.apache.kafka.common.message.ElectLeadersRequestData; import org.apache.kafka.common.message.ElectLeadersResponseData; +import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; @@ -45,12 +49,18 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; +import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.DELETE; +import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; +import static org.apache.kafka.common.protocol.Errors.INVALID_REQUEST; + public class MockController implements Controller { private final static NotControllerException NOT_CONTROLLER_EXCEPTION = @@ -130,6 +140,8 @@ static class MockTopic { private final Map topics = new HashMap<>(); + private final Map> configs = new HashMap<>(); + @Override synchronized public CompletableFuture>> findTopicIds(Collection topicNames) { @@ -197,15 +209,73 @@ public CompletableFuture finalizedFeatures() { @Override public CompletableFuture> incrementalAlterConfigs( - Map>> configChanges, + Map>> configChanges, boolean validateOnly) { + Map results = new HashMap<>(); + for (Entry>> entry : + configChanges.entrySet()) { + ConfigResource resource = entry.getKey(); + results.put(resource, incrementalAlterResource(resource, entry.getValue(), validateOnly)); + } + CompletableFuture> future = new CompletableFuture<>(); + future.complete(results); + return future; + } + + private ApiError incrementalAlterResource(ConfigResource resource, + Map> ops, boolean validateOnly) { + for (Entry> entry : ops.entrySet()) { + AlterConfigOp.OpType opType = entry.getValue().getKey(); + if (opType != SET && opType != DELETE) { + return new ApiError(INVALID_REQUEST, "This mock does not " + + "support the " + opType + " config operation."); + } + } + if (!validateOnly) { + for (Entry> entry : ops.entrySet()) { + String key = entry.getKey(); + AlterConfigOp.OpType op = entry.getValue().getKey(); + String value = entry.getValue().getValue(); + switch (op) { + case SET: + configs.computeIfAbsent(resource, __ -> new HashMap<>()).put(key, value); + break; + case DELETE: + configs.getOrDefault(resource, Collections.emptyMap()).remove(key); + break; + } + } + } + return ApiError.NONE; + } + + @Override + public CompletableFuture + alterPartitionReassignments(AlterPartitionReassignmentsRequestData request) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture + listPartitionReassignments(ListPartitionReassignmentsRequestData request) { throw new UnsupportedOperationException(); } @Override public CompletableFuture> legacyAlterConfigs( Map> newConfigs, boolean validateOnly) { - throw new UnsupportedOperationException(); + Map results = new HashMap<>(); + if (!validateOnly) { + for (Entry> entry : newConfigs.entrySet()) { + ConfigResource resource = entry.getKey(); + Map map = configs.computeIfAbsent(resource, __ -> new HashMap<>()); + map.clear(); + map.putAll(entry.getValue()); + } + } + CompletableFuture> future = new CompletableFuture<>(); + future.complete(results); + return future; } @Override diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index a0fb556a3f6dc..44d82e40226f5 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -27,9 +27,11 @@ import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers import kafka.test.MockController import kafka.utils.MockTime +import org.apache.kafka.clients.admin.AlterConfigOp import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid.ZERO_UUID -import org.apache.kafka.common.errors.{InvalidRequestException, NotControllerException, TopicDeletionDisabledException} +import org.apache.kafka.common.config.{ConfigResource, TopicConfig} +import org.apache.kafka.common.errors._ import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic @@ -40,7 +42,14 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCol import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult import org.apache.kafka.common.message.DeleteTopicsRequestData.DeleteTopicState import org.apache.kafka.common.message.DeleteTopicsResponseData.DeletableTopicResult -import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreatePartitionsRequestData, DeleteTopicsRequestData} +import org.apache.kafka.common.message._ +import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResourceCollection, AlterableConfig, AlterableConfigCollection, AlterConfigsResource} +import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterConfigsResourceCollection => OldAlterConfigsResourceCollection} +import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterConfigsResource => OldAlterConfigsResource} +import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterableConfigCollection => OldAlterableConfigCollection} +import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterableConfig => OldAlterableConfig} +import org.apache.kafka.common.message.AlterConfigsResponseData.{AlterConfigsResourceResponse => OldAlterConfigsResourceResponse} +import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse import org.apache.kafka.common.network.{ClientInformation, ListenerName} import org.apache.kafka.common.protocol.Errors._ import org.apache.kafka.common.protocol.ApiKeys @@ -123,6 +132,139 @@ class ControllerApisTest { requestChannelMetrics) } + def createDenyAllAuthorizer(): Authorizer = { + val authorizer = mock(classOf[Authorizer]) + mock(classOf[Authorizer]) + when(authorizer.authorize( + any(classOf[AuthorizableRequestContext]), + any(classOf[java.util.List[Action]]) + )).thenReturn( + java.util.Collections.singletonList(AuthorizationResult.DENIED) + ) + authorizer + } + + @Test + def testUnauthorizedFetch(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleFetch(buildRequest(new FetchRequest(new FetchRequestData(), 12)))) + } + + @Test + def testUnauthorizedVote(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleVote(buildRequest(new VoteRequest.Builder(new VoteRequestData()).build(0)))) + } + + @Test + def testHandleLegacyAlterConfigsErrors(): Unit = { + val requestData = new AlterConfigsRequestData().setResources( + new OldAlterConfigsResourceCollection(util.Arrays.asList( + new OldAlterConfigsResource(). + setResourceName("1"). + setResourceType(ConfigResource.Type.BROKER.id()). + setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig(). + setName(KafkaConfig.LogCleanerBackoffMsProp). + setValue("100000")).iterator())), + new OldAlterConfigsResource(). + setResourceName("2"). + setResourceType(ConfigResource.Type.BROKER.id()). + setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig(). + setName(KafkaConfig.LogCleanerBackoffMsProp). + setValue("100000")).iterator())), + new OldAlterConfigsResource(). + setResourceName("2"). + setResourceType(ConfigResource.Type.BROKER.id()). + setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig(). + setName(KafkaConfig.LogCleanerBackoffMsProp). + setValue("100000")).iterator())), + new OldAlterConfigsResource(). + setResourceName("baz"). + setResourceType(123.toByte). + setConfigs(new OldAlterableConfigCollection(util.Arrays.asList(new OldAlterableConfig(). + setName("foo"). + setValue("bar")).iterator())), + ).iterator())) + val request = buildRequest(new AlterConfigsRequest(requestData, 0)) + createControllerApis(Some(createDenyAllAuthorizer()), + new MockController.Builder().build()).handleLegacyAlterConfigs(request) + val capturedResponse: ArgumentCaptor[AbstractResponse] = + ArgumentCaptor.forClass(classOf[AbstractResponse]) + verify(requestChannel).sendResponse( + ArgumentMatchers.eq(request), + capturedResponse.capture(), + ArgumentMatchers.eq(None)) + assertNotNull(capturedResponse.getValue) + val response = capturedResponse.getValue.asInstanceOf[AlterConfigsResponse] + assertEquals(Set( + new OldAlterConfigsResourceResponse(). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Duplicate resource."). + setResourceName("2"). + setResourceType(ConfigResource.Type.BROKER.id()), + new OldAlterConfigsResourceResponse(). + setErrorCode(UNSUPPORTED_VERSION.code()). + setErrorMessage("Unknown resource type 123."). + setResourceName("baz"). + setResourceType(123.toByte), + new OldAlterConfigsResourceResponse(). + setErrorCode(CLUSTER_AUTHORIZATION_FAILED.code()). + setErrorMessage("Cluster authorization failed."). + setResourceName("1"). + setResourceType(ConfigResource.Type.BROKER.id())), + response.data().responses().asScala.toSet) + } + + @Test + def testUnauthorizedBeginQuorumEpoch(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleBeginQuorumEpoch(buildRequest(new BeginQuorumEpochRequest.Builder( + new BeginQuorumEpochRequestData()).build(0)))) + } + + @Test + def testUnauthorizedEndQuorumEpoch(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleEndQuorumEpoch(buildRequest(new EndQuorumEpochRequest.Builder( + new EndQuorumEpochRequestData()).build(0)))) + } + + @Test + def testUnauthorizedDescribeQuorum(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleDescribeQuorum(buildRequest(new DescribeQuorumRequest.Builder( + new DescribeQuorumRequestData()).build(0)))) + } + + @Test + def testUnauthorizedHandleAlterIsrRequest(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleAlterIsrRequest(buildRequest(new AlterIsrRequest.Builder( + new AlterIsrRequestData()).build(0)))) + } + + @Test + def testUnauthorizedHandleBrokerHeartBeatRequest(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleBrokerHeartBeatRequest(buildRequest(new BrokerHeartbeatRequest.Builder( + new BrokerHeartbeatRequestData()).build(0)))) + } + + @Test + def testUnauthorizedHandleUnregisterBroker(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleUnregisterBroker(buildRequest(new UnregisterBrokerRequest.Builder( + new UnregisterBrokerRequestData()).build(0)))) + } + @Test def testUnauthorizedBrokerRegistration(): Unit = { val brokerRegistrationRequest = new BrokerRegistrationRequest.Builder( @@ -134,15 +276,7 @@ class ControllerApisTest { val request = buildRequest(brokerRegistrationRequest) val capturedResponse: ArgumentCaptor[AbstractResponse] = ArgumentCaptor.forClass(classOf[AbstractResponse]) - val authorizer = mock(classOf[Authorizer]) - when(authorizer.authorize( - any(classOf[AuthorizableRequestContext]), - any(classOf[java.util.List[Action]]) - )).thenReturn( - java.util.Collections.singletonList(AuthorizationResult.DENIED) - ) - - createControllerApis(Some(authorizer), mock(classOf[Controller])).handle(request) + createControllerApis(Some(createDenyAllAuthorizer()), mock(classOf[Controller])).handle(request) verify(requestChannel).sendResponse( ArgumentMatchers.eq(request), capturedResponse.capture(), @@ -155,6 +289,136 @@ class ControllerApisTest { brokerRegistrationResponse.errorCounts().asScala) } + @Test + def testUnauthorizedHandleAlterClientQuotas(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleAlterClientQuotas(buildRequest(new AlterClientQuotasRequest( + new AlterClientQuotasRequestData(), 0)))) + } + + @Test + def testUnauthorizedHandleIncrementalAlterConfigs(): Unit = { + val requestData = new IncrementalAlterConfigsRequestData().setResources( + new AlterConfigsResourceCollection( + util.Arrays.asList(new AlterConfigsResource(). + setResourceName("1"). + setResourceType(ConfigResource.Type.BROKER.id()). + setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). + setName(KafkaConfig.LogCleanerBackoffMsProp). + setValue("100000"). + setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), + new AlterConfigsResource(). + setResourceName("foo"). + setResourceType(ConfigResource.Type.TOPIC.id()). + setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). + setName(TopicConfig.FLUSH_MS_CONFIG). + setValue("1000"). + setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), + ).iterator())) + val request = buildRequest(new IncrementalAlterConfigsRequest.Builder(requestData).build(0)) + createControllerApis(Some(createDenyAllAuthorizer()), + new MockController.Builder().build()).handleIncrementalAlterConfigs(request) + val capturedResponse: ArgumentCaptor[AbstractResponse] = + ArgumentCaptor.forClass(classOf[AbstractResponse]) + verify(requestChannel).sendResponse( + ArgumentMatchers.eq(request), + capturedResponse.capture(), + ArgumentMatchers.eq(None)) + assertNotNull(capturedResponse.getValue) + val response = capturedResponse.getValue.asInstanceOf[IncrementalAlterConfigsResponse] + assertEquals(Set(new AlterConfigsResourceResponse(). + setErrorCode(CLUSTER_AUTHORIZATION_FAILED.code()). + setErrorMessage(CLUSTER_AUTHORIZATION_FAILED.message()). + setResourceName("1"). + setResourceType(ConfigResource.Type.BROKER.id()), + new AlterConfigsResourceResponse(). + setErrorCode(TOPIC_AUTHORIZATION_FAILED.code()). + setErrorMessage(TOPIC_AUTHORIZATION_FAILED.message()). + setResourceName("foo"). + setResourceType(ConfigResource.Type.TOPIC.id())), + response.data().responses().asScala.toSet) + } + + @Test + def testInvalidIncrementalAlterConfigsResources(): Unit = { + val requestData = new IncrementalAlterConfigsRequestData().setResources( + new AlterConfigsResourceCollection(util.Arrays.asList( + new AlterConfigsResource(). + setResourceName("1"). + setResourceType(ConfigResource.Type.BROKER_LOGGER.id()). + setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). + setName("kafka.server.KafkaConfig"). + setValue("TRACE"). + setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), + new AlterConfigsResource(). + setResourceName("3"). + setResourceType(ConfigResource.Type.BROKER.id()). + setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). + setName(KafkaConfig.LogCleanerBackoffMsProp). + setValue("100000"). + setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), + new AlterConfigsResource(). + setResourceName("3"). + setResourceType(ConfigResource.Type.BROKER.id()). + setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). + setName(KafkaConfig.LogCleanerBackoffMsProp). + setValue("100000"). + setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), + new AlterConfigsResource(). + setResourceName("foo"). + setResourceType(124.toByte). + setConfigs(new AlterableConfigCollection(util.Arrays.asList(new AlterableConfig(). + setName("foo"). + setValue("bar"). + setConfigOperation(AlterConfigOp.OpType.SET.id())).iterator())), + ).iterator())) + val request = buildRequest(new IncrementalAlterConfigsRequest.Builder(requestData).build(0)) + createControllerApis(Some(createDenyAllAuthorizer()), + new MockController.Builder().build()).handleIncrementalAlterConfigs(request) + val capturedResponse: ArgumentCaptor[AbstractResponse] = + ArgumentCaptor.forClass(classOf[AbstractResponse]) + verify(requestChannel).sendResponse( + ArgumentMatchers.eq(request), + capturedResponse.capture(), + ArgumentMatchers.eq(None)) + assertNotNull(capturedResponse.getValue) + val response = capturedResponse.getValue.asInstanceOf[IncrementalAlterConfigsResponse] + assertEquals(Set( + new AlterConfigsResourceResponse(). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Unexpected resource type BROKER_LOGGER."). + setResourceName("1"). + setResourceType(ConfigResource.Type.BROKER_LOGGER.id()), + new AlterConfigsResourceResponse(). + setErrorCode(INVALID_REQUEST.code()). + setErrorMessage("Duplicate resource."). + setResourceName("3"). + setResourceType(ConfigResource.Type.BROKER.id()), + new AlterConfigsResourceResponse(). + setErrorCode(UNSUPPORTED_VERSION.code()). + setErrorMessage("Unknown resource type 124."). + setResourceName("foo"). + setResourceType(124.toByte)), + response.data().responses().asScala.toSet) + } + + @Test + def testUnauthorizedHandleAlterPartitionReassignments(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleAlterPartitionReassignments(buildRequest(new AlterPartitionReassignmentsRequest.Builder( + new AlterPartitionReassignmentsRequestData()).build()))) + } + + @Test + def testUnauthorizedHandleListPartitionReassignments(): Unit = { + assertThrows(classOf[ClusterAuthorizationException], () => createControllerApis( + Some(createDenyAllAuthorizer()), new MockController.Builder().build()). + handleListPartitionReassignments(buildRequest(new ListPartitionReassignmentsRequest.Builder( + new ListPartitionReassignmentsRequestData()).build()))) + } + @Test def testCreateTopics(): Unit = { val controller = new MockController.Builder().build() @@ -170,10 +434,10 @@ class ControllerApisTest { ).iterator())) val expectedResponse = Set(new CreatableTopicResult().setName("foo"). setErrorCode(INVALID_REQUEST.code()). - setErrorMessage("Found multiple entries for this topic."), + setErrorMessage("Duplicate topic name."), new CreatableTopicResult().setName("bar"). setErrorCode(INVALID_REQUEST.code()). - setErrorMessage("Found multiple entries for this topic."), + setErrorMessage("Duplicate topic name."), new CreatableTopicResult().setName("baz"). setErrorCode(NONE.code()). setTopicId(new Uuid(0L, 1L)), @@ -181,7 +445,7 @@ class ControllerApisTest { setErrorCode(TOPIC_AUTHORIZATION_FAILED.code())) assertEquals(expectedResponse, controllerApis.createTopics(request, false, - _ => Set("baz")).topics().asScala.toSet) + _ => Set("baz")).get().topics().asScala.toSet) } @Test @@ -202,7 +466,7 @@ class ControllerApisTest { ApiKeys.DELETE_TOPICS.latestVersion().toInt, true, _ => Set.empty, - _ => Set.empty).asScala.toSet) + _ => Set.empty).get().asScala.toSet) } @Test @@ -228,7 +492,7 @@ class ControllerApisTest { ApiKeys.DELETE_TOPICS.latestVersion().toInt, true, _ => Set.empty, - _ => Set.empty).asScala.toSet) + _ => Set.empty).get().asScala.toSet) } @Test @@ -270,7 +534,7 @@ class ControllerApisTest { ApiKeys.DELETE_TOPICS.latestVersion().toInt, false, names => names.toSet, - names => names.toSet).asScala.toSet) + names => names.toSet).get().asScala.toSet) } @Test @@ -306,7 +570,7 @@ class ControllerApisTest { ApiKeys.DELETE_TOPICS.latestVersion().toInt, false, _ => Set("foo", "baz"), - _ => Set.empty).asScala.toSet) + _ => Set.empty).get().asScala.toSet) } @Test @@ -331,7 +595,7 @@ class ControllerApisTest { ApiKeys.DELETE_TOPICS.latestVersion().toInt, false, _ => Set("foo"), - _ => Set.empty).asScala.toSet) + _ => Set.empty).get().asScala.toSet) } @Test @@ -350,7 +614,7 @@ class ControllerApisTest { ApiKeys.DELETE_TOPICS.latestVersion().toInt, false, _ => Set("foo", "bar"), - _ => Set("foo", "bar"))).getCause.getClass) + _ => Set("foo", "bar")).get()).getCause.getClass) } @Test diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java index f39af148caa27..071076474dfea 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java @@ -22,6 +22,8 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.AlterIsrRequestData; import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData; import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; @@ -30,6 +32,8 @@ import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.ElectLeadersRequestData; import org.apache.kafka.common.message.ElectLeadersResponseData; +import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.requests.ApiError; @@ -138,6 +142,26 @@ CompletableFuture> incrementalAlterConfigs( Map>> configChanges, boolean validateOnly); + /** + * Start or stop some partition reassignments. + * + * @param request The alter partition reassignments request. + * + * @return A future yielding the results. + */ + CompletableFuture + alterPartitionReassignments(AlterPartitionReassignmentsRequestData request); + + /** + * List ongoing partition reassignments. + * + * @param request The list partition reassignments request. + * + * @return A future yielding the results. + */ + CompletableFuture + listPartitionReassignments(ListPartitionReassignmentsRequestData request); + /** * Perform some configuration changes using the legacy API. * diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index f3f9ddd653205..86faa5ede8e6c 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -41,6 +41,8 @@ import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.message.AlterIsrRequestData; import org.apache.kafka.common.message.AlterIsrResponseData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData; import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; @@ -49,6 +51,8 @@ import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.ElectLeadersRequestData; import org.apache.kafka.common.message.ElectLeadersResponseData; +import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.metadata.FenceBrokerRecord; @@ -1023,6 +1027,22 @@ public CompletableFuture> incrementalAlterConfigs( }); } + @Override + public CompletableFuture + alterPartitionReassignments(AlterPartitionReassignmentsRequestData request) { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new UnsupportedOperationException()); + return future; + } + + @Override + public CompletableFuture + listPartitionReassignments(ListPartitionReassignmentsRequestData request) { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new UnsupportedOperationException()); + return future; + } + @Override public CompletableFuture> legacyAlterConfigs( Map> newConfigs, boolean validateOnly) { From fc405d792de12a50956195827eaf57bbf64444c9 Mon Sep 17 00:00:00 2001 From: Shay Elkin <2046772+shayelkin@users.noreply.github.com> Date: Thu, 15 Apr 2021 11:37:15 -0700 Subject: [PATCH 063/155] Minor: Move trogdor out of tools and into its own gradle module (#10539) Move Trogdor out of tools and into its own gradle module. This allows us to minimize the dependencies of the tools module. We still keep Trogdor in the CLASSPATH created by kafka-run-class.sh. Reviewers: Colin P. McCabe --- bin/kafka-run-class.sh | 12 +++++ build.gradle | 51 +++++++++++++++++-- settings.gradle | 3 +- .../org/apache/kafka/trogdor/agent/Agent.java | 0 .../kafka/trogdor/agent/AgentClient.java | 0 .../trogdor/agent/AgentRestResource.java | 0 .../kafka/trogdor/agent/WorkerManager.java | 0 .../apache/kafka/trogdor/basic/BasicNode.java | 0 .../kafka/trogdor/basic/BasicPlatform.java | 0 .../kafka/trogdor/basic/BasicTopology.java | 0 .../apache/kafka/trogdor/common/JsonUtil.java | 0 .../org/apache/kafka/trogdor/common/Node.java | 0 .../apache/kafka/trogdor/common/Platform.java | 0 .../kafka/trogdor/common/StringExpander.java | 0 .../kafka/trogdor/common/StringFormatter.java | 0 .../apache/kafka/trogdor/common/Topology.java | 0 .../kafka/trogdor/common/WorkerUtils.java | 0 .../trogdor/coordinator/Coordinator.java | 0 .../coordinator/CoordinatorClient.java | 0 .../coordinator/CoordinatorRestResource.java | 0 .../trogdor/coordinator/NodeManager.java | 0 .../trogdor/coordinator/TaskManager.java | 0 .../fault/DegradedNetworkFaultSpec.java | 0 .../fault/DegradedNetworkFaultWorker.java | 0 .../fault/FilesUnreadableFaultSpec.java | 0 .../apache/kafka/trogdor/fault/Kibosh.java | 0 .../trogdor/fault/KiboshFaultController.java | 0 .../trogdor/fault/KiboshFaultWorker.java | 0 .../NetworkPartitionFaultController.java | 0 .../fault/NetworkPartitionFaultSpec.java | 0 .../fault/NetworkPartitionFaultWorker.java | 0 .../fault/ProcessStopFaultController.java | 0 .../trogdor/fault/ProcessStopFaultSpec.java | 0 .../trogdor/fault/ProcessStopFaultWorker.java | 0 .../trogdor/rest/AgentStatusResponse.java | 0 .../rest/CoordinatorShutdownRequest.java | 0 .../rest/CoordinatorStatusResponse.java | 0 .../kafka/trogdor/rest/CreateTaskRequest.java | 0 .../trogdor/rest/CreateWorkerRequest.java | 0 .../trogdor/rest/DestroyTaskRequest.java | 0 .../trogdor/rest/DestroyWorkerRequest.java | 0 .../org/apache/kafka/trogdor/rest/Empty.java | 0 .../kafka/trogdor/rest/ErrorResponse.java | 0 .../kafka/trogdor/rest/JsonRestServer.java | 0 .../apache/kafka/trogdor/rest/Message.java | 0 .../rest/RequestConflictException.java | 0 .../trogdor/rest/RestExceptionMapper.java | 0 .../kafka/trogdor/rest/StopTaskRequest.java | 0 .../kafka/trogdor/rest/StopWorkerRequest.java | 0 .../apache/kafka/trogdor/rest/TaskDone.java | 0 .../kafka/trogdor/rest/TaskPending.java | 0 .../kafka/trogdor/rest/TaskRequest.java | 0 .../kafka/trogdor/rest/TaskRunning.java | 0 .../apache/kafka/trogdor/rest/TaskState.java | 0 .../kafka/trogdor/rest/TaskStateType.java | 0 .../kafka/trogdor/rest/TaskStopping.java | 0 .../kafka/trogdor/rest/TasksRequest.java | 0 .../kafka/trogdor/rest/TasksResponse.java | 0 .../kafka/trogdor/rest/UptimeResponse.java | 0 .../apache/kafka/trogdor/rest/WorkerDone.java | 0 .../kafka/trogdor/rest/WorkerReceiving.java | 0 .../kafka/trogdor/rest/WorkerRunning.java | 0 .../kafka/trogdor/rest/WorkerStarting.java | 0 .../kafka/trogdor/rest/WorkerState.java | 0 .../kafka/trogdor/rest/WorkerStopping.java | 0 .../task/AgentWorkerStatusTracker.java | 0 .../trogdor/task/NoOpTaskController.java | 0 .../kafka/trogdor/task/NoOpTaskSpec.java | 0 .../kafka/trogdor/task/NoOpTaskWorker.java | 0 .../kafka/trogdor/task/TaskController.java | 0 .../apache/kafka/trogdor/task/TaskSpec.java | 0 .../apache/kafka/trogdor/task/TaskWorker.java | 0 .../trogdor/task/WorkerStatusTracker.java | 0 .../workload/ConfigurableProducerSpec.java | 0 .../workload/ConfigurableProducerWorker.java | 0 .../workload/ConnectionStressSpec.java | 0 .../workload/ConnectionStressWorker.java | 0 .../workload/ConstantFlushGenerator.java | 0 .../workload/ConstantPayloadGenerator.java | 0 .../workload/ConstantThroughputGenerator.java | 0 .../trogdor/workload/ConsumeBenchSpec.java | 0 .../trogdor/workload/ConsumeBenchWorker.java | 0 .../trogdor/workload/ExternalCommandSpec.java | 0 .../workload/ExternalCommandWorker.java | 0 .../trogdor/workload/FlushGenerator.java | 0 .../workload/GaussianFlushGenerator.java | 0 .../workload/GaussianThroughputGenerator.java | 0 ...ussianTimestampRandomPayloadGenerator.java | 0 .../kafka/trogdor/workload/Histogram.java | 0 .../workload/NullPayloadGenerator.java | 0 .../trogdor/workload/PartitionsSpec.java | 0 .../trogdor/workload/PayloadGenerator.java | 0 .../trogdor/workload/PayloadIterator.java | 0 .../trogdor/workload/PayloadKeyType.java | 0 .../trogdor/workload/ProduceBenchSpec.java | 0 .../trogdor/workload/ProduceBenchWorker.java | 0 .../trogdor/workload/RandomComponent.java | 0 .../RandomComponentPayloadGenerator.java | 0 .../trogdor/workload/RecordProcessor.java | 0 .../trogdor/workload/RoundTripWorker.java | 0 .../workload/RoundTripWorkloadSpec.java | 0 .../workload/SequentialPayloadGenerator.java | 0 .../workload/SustainedConnectionSpec.java | 0 .../workload/SustainedConnectionWorker.java | 0 .../kafka/trogdor/workload/Throttle.java | 0 .../trogdor/workload/ThroughputGenerator.java | 0 .../TimeIntervalTransactionsGenerator.java | 0 .../TimestampRandomPayloadGenerator.java | 0 .../workload/TimestampRecordProcessor.java | 0 .../kafka/trogdor/workload/TopicsSpec.java | 0 .../workload/TransactionGenerator.java | 0 .../UniformRandomPayloadGenerator.java | 0 .../UniformTransactionsGenerator.java | 0 .../apache/kafka/trogdor/agent/AgentTest.java | 0 .../trogdor/basic/BasicPlatformTest.java | 0 .../common/CapturingCommandRunner.java | 0 .../kafka/trogdor/common/ExpectedTasks.java | 0 .../trogdor/common/JsonSerializationTest.java | 0 .../kafka/trogdor/common/JsonUtilTest.java | 0 .../trogdor/common/MiniTrogdorCluster.java | 0 .../trogdor/common/StringExpanderTest.java | 0 .../trogdor/common/StringFormatterTest.java | 0 .../kafka/trogdor/common/TopologyTest.java | 0 .../kafka/trogdor/common/WorkerUtilsTest.java | 0 .../coordinator/CoordinatorClientTest.java | 0 .../trogdor/coordinator/CoordinatorTest.java | 0 .../trogdor/rest/RestExceptionMapperTest.java | 0 .../trogdor/task/SampleTaskController.java | 0 .../kafka/trogdor/task/SampleTaskSpec.java | 0 .../kafka/trogdor/task/SampleTaskWorker.java | 0 .../kafka/trogdor/task/TaskSpecTest.java | 0 .../workload/ConsumeBenchSpecTest.java | 0 .../workload/ExternalCommandWorkerTest.java | 0 .../kafka/trogdor/workload/HistogramTest.java | 0 .../workload/PayloadGeneratorTest.java | 0 .../kafka/trogdor/workload/ThrottleTest.java | 0 ...TimeIntervalTransactionsGeneratorTest.java | 0 .../trogdor/workload/TopicsSpecTest.java | 0 trogdor/src/test/resources/log4j.properties | 22 ++++++++ 139 files changed, 84 insertions(+), 4 deletions(-) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/agent/Agent.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/common/JsonUtil.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/common/Node.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/common/Platform.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/common/StringExpander.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/common/Topology.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultController.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/DestroyTaskRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/DestroyWorkerRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/Empty.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/Message.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/RequestConflictException.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TaskStateType.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/UptimeResponse.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/AgentWorkerStatusTracker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/TaskController.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/task/WorkerStatusTracker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConstantPayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/NullPayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/PayloadIterator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/PayloadKeyType.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/RandomComponentPayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/SequentialPayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/Throttle.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/ThroughputGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/TransactionGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/UniformRandomPayloadGenerator.java (100%) rename {tools => trogdor}/src/main/java/org/apache/kafka/trogdor/workload/UniformTransactionsGenerator.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java (100%) rename {tools => trogdor}/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java (100%) create mode 100644 trogdor/src/test/resources/log4j.properties diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 3889be7e3e5f8..834ce6ba35694 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -159,6 +159,18 @@ do CLASSPATH="$CLASSPATH:$dir/*" done +for file in "$base_dir"/trogdor/build/libs/trogdor-*.jar; +do + if should_include_file "$file"; then + CLASSPATH="$CLASSPATH":"$file" + fi +done + +for dir in "$base_dir"/trogdor/build/dependant-libs-${SCALA_VERSION}*; +do + CLASSPATH="$CLASSPATH:$dir/*" +done + for cc_pkg in "api" "transforms" "runtime" "file" "mirror" "mirror-client" "json" "tools" "basic-auth-extension" do for file in "$base_dir"/connect/${cc_pkg}/build/libs/connect-${cc_pkg}*.jar; diff --git a/build.gradle b/build.gradle index 39a13ab845005..b535dd39a6933 100644 --- a/build.gradle +++ b/build.gradle @@ -428,7 +428,7 @@ subprojects { exceptionFormat = testExceptionFormat } logTestStdout.rehydrate(delegate, owner, this)() - + // The suites are for running sets of tests in IDEs. // Gradle will run each test class, so we exclude the suites to avoid redundantly running the tests twice. exclude '**/*Suite.class' @@ -761,7 +761,7 @@ tasks.create(name: "testConnect", dependsOn: connectPkgs.collect { it + ":test" project(':core') { apply plugin: 'scala' - + // scaladoc generation is configured at the sub-module level with an artifacts // block (cf. see streams-scala). If scaladoc generation is invoked explicitly // for the `core` module, this ensures the generated jar doesn't include scaladoc @@ -983,6 +983,8 @@ project(':core') { from(project.siteDocsTar) { into("site-docs/") } from(project(':tools').jar) { into("libs/") } from(project(':tools').configurations.runtimeClasspath) { into("libs/") } + from(project(':trogdor').jar) { into("libs/") } + from(project(':trogdor').configurations.runtimeClasspath) { into("libs/") } from(project(':shell').jar) { into("libs/") } from(project(':shell').configurations.runtimeClasspath) { into("libs/") } from(project(':connect:api').jar) { into("libs/") } @@ -1318,7 +1320,7 @@ project(':raft') { } } } - + compileJava.dependsOn 'processMessages' jar { @@ -1468,6 +1470,49 @@ project(':storage') { project(':tools') { archivesBaseName = "kafka-tools" + dependencies { + implementation project(':clients') + implementation project(':log4j-appender') + implementation libs.argparse4j + implementation libs.jacksonDatabind + implementation libs.jacksonJDK8Datatypes + implementation libs.slf4jApi + implementation libs.log4j + + implementation libs.jacksonJaxrsJsonProvider + + testImplementation project(':clients') + testImplementation libs.junitJupiter + testImplementation project(':clients').sourceSets.test.output + testImplementation libs.mockitoInline // supports mocking static methods, final classes, etc. + + testRuntimeOnly libs.slf4jlog4j + } + + javadoc { + enabled = false + } + + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntimeClasspath) { + include('slf4j-log4j12*') + include('log4j*jar') + } + from (configurations.runtimeClasspath) { + exclude('kafka-clients*') + } + into "$buildDir/dependant-libs-${versions.scala}" + duplicatesStrategy 'exclude' + } + + jar { + dependsOn 'copyDependantLibs' + } +} + +project(':trogdor') { + archivesBaseName = "trogdor" + dependencies { implementation project(':clients') implementation project(':log4j-appender') diff --git a/settings.gradle b/settings.gradle index 91c71f52eb8d3..8ed1d0eb19d02 100644 --- a/settings.gradle +++ b/settings.gradle @@ -50,4 +50,5 @@ include 'clients', 'streams:upgrade-system-tests-25', 'streams:upgrade-system-tests-26', 'streams:upgrade-system-tests-27', - 'tools' + 'tools', + 'trogdor' diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/Agent.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/agent/Agent.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/agent/Agent.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentRestResource.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/agent/WorkerManager.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicPlatform.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicTopology.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/JsonUtil.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/JsonUtil.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/common/JsonUtil.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/common/JsonUtil.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/Node.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/Node.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/common/Node.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/common/Node.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/Platform.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/Platform.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/common/Platform.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/common/Platform.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/StringExpander.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/StringExpander.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/common/StringExpander.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/common/StringExpander.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/Topology.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/Topology.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/common/Topology.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/common/Topology.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/common/WorkerUtils.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/Coordinator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorRestResource.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/NodeManager.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/FilesUnreadableFaultSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultController.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultController.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultController.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultController.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/KiboshFaultWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultController.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultController.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/fault/ProcessStopFaultWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/AgentStatusResponse.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorShutdownRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/CoordinatorStatusResponse.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateTaskRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/CreateWorkerRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/DestroyTaskRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/DestroyTaskRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/DestroyTaskRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/DestroyTaskRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/DestroyWorkerRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/DestroyWorkerRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/DestroyWorkerRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/DestroyWorkerRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/Empty.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/Empty.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/Empty.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/Empty.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/JsonRestServer.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/Message.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/Message.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/Message.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/Message.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/RequestConflictException.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/RequestConflictException.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/RequestConflictException.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/RequestConflictException.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/RestExceptionMapper.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/StopTaskRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/StopWorkerRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskDone.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskPending.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRunning.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskState.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStateType.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStateType.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStateType.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStateType.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskStopping.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/UptimeResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/UptimeResponse.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/UptimeResponse.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/UptimeResponse.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerDone.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerReceiving.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerRunning.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStarting.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerState.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/rest/WorkerStopping.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/AgentWorkerStatusTracker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/AgentWorkerStatusTracker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/AgentWorkerStatusTracker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/AgentWorkerStatusTracker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskController.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/NoOpTaskWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/TaskController.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskController.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/TaskController.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskController.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/task/WorkerStatusTracker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/WorkerStatusTracker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/task/WorkerStatusTracker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/task/WorkerStatusTracker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantFlushGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConstantPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantPayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConstantPayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantPayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/FlushGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/NullPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/NullPayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/NullPayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/NullPayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/PayloadIterator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadIterator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/PayloadIterator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadIterator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/PayloadKeyType.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadKeyType.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/PayloadKeyType.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadKeyType.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/RandomComponentPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponentPayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/RandomComponentPayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponentPayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/RecordProcessor.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/SequentialPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SequentialPayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/SequentialPayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/SequentialPayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/Throttle.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/Throttle.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/Throttle.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/Throttle.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/ThroughputGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ThroughputGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/ThroughputGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/ThroughputGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRandomPayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/TransactionGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TransactionGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/TransactionGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/TransactionGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/UniformRandomPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/UniformRandomPayloadGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/UniformRandomPayloadGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/UniformRandomPayloadGenerator.java diff --git a/tools/src/main/java/org/apache/kafka/trogdor/workload/UniformTransactionsGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/UniformTransactionsGenerator.java similarity index 100% rename from tools/src/main/java/org/apache/kafka/trogdor/workload/UniformTransactionsGenerator.java rename to trogdor/src/main/java/org/apache/kafka/trogdor/workload/UniformTransactionsGenerator.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/basic/BasicPlatformTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/CapturingCommandRunner.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/ExpectedTasks.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/TopologyTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorClientTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/rest/RestExceptionMapperTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java b/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskController.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java b/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java b/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskWorker.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/task/TaskSpecTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/workload/HistogramTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/workload/ThrottleTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/workload/TimeIntervalTransactionsGeneratorTest.java diff --git a/tools/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java similarity index 100% rename from tools/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java rename to trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java diff --git a/trogdor/src/test/resources/log4j.properties b/trogdor/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..abeaf1ee09e4c --- /dev/null +++ b/trogdor/src/test/resources/log4j.properties @@ -0,0 +1,22 @@ +# 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. +log4j.rootLogger=TRACE, stdout + +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n + +log4j.logger.org.apache.kafka=TRACE +log4j.logger.org.eclipse.jetty=INFO From 637c44c976c115b7e770a6fd9e62e8822051b45b Mon Sep 17 00:00:00 2001 From: Bill Bejeck Date: Thu, 15 Apr 2021 23:48:53 -0400 Subject: [PATCH 064/155] KAFKA-12672: Added config for raft testing server (#10545) Adding a property to the `raft/config/kraft.properties` for running the raft test server in development. For testing I ran `./bin/test-kraft-server-start.sh --config config/kraft.properties` and validated the test server started running with a throughput test. Reviewers: Ismael Juma --- raft/README.md | 3 +++ raft/config/kraft.properties | 1 + 2 files changed, 4 insertions(+) diff --git a/raft/README.md b/raft/README.md index f4c27293f4459..bc95c9303d12f 100644 --- a/raft/README.md +++ b/raft/README.md @@ -16,6 +16,7 @@ Create 3 separate KRaft quorum properties as the following: node.id=1 listeners=PLAINTEXT://localhost:9092 + controller.listener.names=PLAINTEXT controller.quorum.voters=1@localhost:9092,2@localhost:9093,3@localhost:9094 log.dirs=/tmp/kraft-logs-1 EOF @@ -24,6 +25,7 @@ Create 3 separate KRaft quorum properties as the following: node.id=2 listeners=PLAINTEXT://localhost:9093 + controller.listener.names=PLAINTEXT controller.quorum.voters=1@localhost:9092,2@localhost:9093,3@localhost:9094 log.dirs=/tmp/kraft-logs-2 EOF @@ -32,6 +34,7 @@ Create 3 separate KRaft quorum properties as the following: node.id=3 listeners=PLAINTEXT://localhost:9094 + controller.listener.names=PLAINTEXT controller.quorum.voters=1@localhost:9092,2@localhost:9093,3@localhost:9094 log.dirs=/tmp/kraft-logs-3 EOF diff --git a/raft/config/kraft.properties b/raft/config/kraft.properties index 674ce92bc1846..a8556db6a49eb 100644 --- a/raft/config/kraft.properties +++ b/raft/config/kraft.properties @@ -15,5 +15,6 @@ node.id=0 listeners=PLAINTEXT://localhost:9092 +controller.listener.names=PLAINTEXT controller.quorum.voters=0@localhost:9092 log.dirs=/tmp/kraft-logs From a6b22f33f140d521e0ef6b764aa210e8c1f51c24 Mon Sep 17 00:00:00 2001 From: dengziming Date: Fri, 16 Apr 2021 14:38:42 +0800 Subject: [PATCH 065/155] MINOR: fix package name in integration test (#10400) Change package name of IntegrationTestUtils ,TransactionsWithMaxInFlightOneTest, ControllerContextTest and DefaultMessageFormatterTest to kafka.server since we set the package name to kafka.xxx in all other classes. Reviewers: Chia-Ping Tsai --- .../TransactionsWithMaxInFlightOneTest.scala | 6 +++--- .../kafka/server/IntegrationTestUtils.scala | 2 +- .../tools/DefaultMessageFormatterTest.scala | 11 +++++------ .../controller/ControllerContextTest.scala | 8 ++------ .../AbstractApiVersionsRequestTest.scala | 1 - .../kafka/server/ClientQuotasRequestTest.scala | 18 ++++++++---------- .../server/SaslApiVersionsRequestTest.scala | 11 +++++------ 7 files changed, 24 insertions(+), 33 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala index d29b24573b368..20094b69d879d 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package integration.kafka.api +package kafka.api import java.util.Properties @@ -25,8 +25,8 @@ import kafka.utils.TestUtils import kafka.utils.TestUtils.consumeRecords import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer.KafkaProducer -import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import scala.collection.Seq import scala.collection.mutable.Buffer @@ -70,7 +70,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { } @Test - def testTransactionalProducerSingleBrokerMaxInFlightOne() = { + def testTransactionalProducerSingleBrokerMaxInFlightOne(): Unit = { // We want to test with one broker to verify multiple requests queued on a connection assertEquals(1, servers.size) diff --git a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala index c960e35c59031..c240cbfe1e0b6 100644 --- a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala +++ b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package integration.kafka.server +package kafka.server import kafka.network.SocketServer import kafka.utils.{NotNothing, TestUtils} diff --git a/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala b/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala index f2ca243a8495d..12bbc9481107b 100644 --- a/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala +++ b/core/src/test/scala/kafka/tools/DefaultMessageFormatterTest.scala @@ -15,12 +15,12 @@ * limitations under the License. */ -package unit.kafka.tools +package kafka.tools import java.io.{ByteArrayOutputStream, Closeable, PrintStream} import java.nio.charset.StandardCharsets import java.util -import kafka.tools.DefaultMessageFormatter + import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.common.header.Header import org.apache.kafka.common.header.internals.{RecordHeader, RecordHeaders} @@ -36,7 +36,6 @@ import scala.jdk.CollectionConverters._ class DefaultMessageFormatterTest { import DefaultMessageFormatterTest._ - @ParameterizedTest @MethodSource(Array("parameters")) def testWriteRecord(name: String, record: ConsumerRecord[Array[Byte], Array[Byte]], properties: Map[String, String], expected: String): Unit = { @@ -142,7 +141,7 @@ object DefaultMessageFormatterTest { Map("print.key" -> "true", "print.headers" -> "true", "print.value" -> "true", - "key.deserializer" -> "unit.kafka.tools.UpperCaseDeserializer"), + "key.deserializer" -> "kafka.tools.UpperCaseDeserializer"), "h1:v1,h2:v2\tSOMEKEY\tsomeValue\n"), Arguments.of( "print value with custom deserializer", @@ -150,7 +149,7 @@ object DefaultMessageFormatterTest { Map("print.key" -> "true", "print.headers" -> "true", "print.value" -> "true", - "value.deserializer" -> "unit.kafka.tools.UpperCaseDeserializer"), + "value.deserializer" -> "kafka.tools.UpperCaseDeserializer"), "h1:v1,h2:v2\tsomeKey\tSOMEVALUE\n"), Arguments.of( "print headers with custom deserializer", @@ -158,7 +157,7 @@ object DefaultMessageFormatterTest { Map("print.key" -> "true", "print.headers" -> "true", "print.value" -> "true", - "headers.deserializer" -> "unit.kafka.tools.UpperCaseDeserializer"), + "headers.deserializer" -> "kafka.tools.UpperCaseDeserializer"), "h1:V1,h2:V2\tsomeKey\tsomeValue\n"), Arguments.of( "print key and value", diff --git a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala index 389a8423eda39..e8efa5af79356 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerContextTest.scala @@ -15,19 +15,15 @@ * limitations under the License. */ -package unit.kafka.controller +package kafka.controller import kafka.api.LeaderAndIsr import kafka.cluster.{Broker, EndPoint} -import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.controller.{ControllerContext, ReplicaAssignment} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} -import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.Assertions.assertTrue -import org.junit.jupiter.api.Assertions.assertFalse class ControllerContextTest { diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 9c7b4440a32b8..6a2bacab76f93 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -18,7 +18,6 @@ package kafka.server import java.util.Properties -import integration.kafka.server.IntegrationTestUtils import kafka.test.ClusterInstance import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index e495f3e865d76..fc468e5debabe 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -17,10 +17,14 @@ package kafka.server -import integration.kafka.server.IntegrationTestUtils -import kafka.test.ClusterInstance - import java.net.InetAddress +import java.util +import java.util.concurrent.{ExecutionException, TimeUnit} + +import kafka.test.ClusterInstance +import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{ScramCredentialInfo, ScramMechanism, UserScramCredentialUpsertion} import org.apache.kafka.common.config.internals.QuotaConfigs import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException} @@ -30,12 +34,6 @@ import org.apache.kafka.common.requests.{AlterClientQuotasRequest, AlterClientQu import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.extension.ExtendWith -import java.util -import java.util.concurrent.{ExecutionException, TimeUnit} -import kafka.utils.TestUtils -import kafka.test.annotation.{ClusterTest, ClusterTestDefaults, Type} -import kafka.test.junit.ClusterTestExtensions - import scala.jdk.CollectionConverters._ @ClusterTestDefaults(clusterType = Type.BOTH) @@ -597,7 +595,7 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { sendAlterClientQuotasRequest(entries, validateOnly).complete(response) val result = response.asScala assertEquals(request.size, result.size) - request.foreach(e => assertTrue(result.get(e._1).isDefined)) + request.foreach(e => assertTrue(result.contains(e._1))) result } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 05c83d476d9f8..26609480443db 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -16,22 +16,21 @@ */ package kafka.server -import integration.kafka.server.IntegrationTestUtils - import java.net.Socket import java.util.Collections + import kafka.api.{KafkaSasl, SaslSetup} +import kafka.test.annotation.{ClusterTest, Type} +import kafka.test.junit.ClusterTestExtensions +import kafka.test.{ClusterConfig, ClusterInstance} import kafka.utils.JaasTestUtils import org.apache.kafka.common.message.SaslHandshakeRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, SaslHandshakeRequest, SaslHandshakeResponse} -import kafka.test.annotation.{ClusterTest, Type} -import kafka.test.junit.ClusterTestExtensions -import kafka.test.{ClusterConfig, ClusterInstance} import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.junit.jupiter.api.extension.ExtendWith +import org.junit.jupiter.api.{AfterEach, BeforeEach} import scala.jdk.CollectionConverters._ From f8f1769256de2a1afa92794309ecee475cd80b79 Mon Sep 17 00:00:00 2001 From: Tom Bentley Date: Fri, 16 Apr 2021 15:11:25 +0100 Subject: [PATCH 066/155] KIP-145: Add SMTs, HeaderFrom, DropHeaders and InsertHeader (#9549) These SMTs were originally specified in KIP-145 but never implemented at the time. HeaderTo is not included since its original specification doesn't deal with the fact that there can be >1 header with the same name, but a field can only have a single value (which could be an array, but not if the headers for the given name had different schemas). Reviewers: Chris Egerton , Mickael Maison --- .../connect/tools/TransformationDoc.java | 8 +- .../kafka/connect/transforms/DropHeaders.java | 74 ++++ .../kafka/connect/transforms/HeaderFrom.java | 238 ++++++++++++ .../connect/transforms/InsertHeader.java | 77 ++++ .../connect/transforms/DropHeadersTest.java | 117 ++++++ .../connect/transforms/HeaderFromTest.java | 359 ++++++++++++++++++ .../connect/transforms/InsertHeaderTest.java | 121 ++++++ docs/connect.html | 3 + 8 files changed, 996 insertions(+), 1 deletion(-) create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertHeader.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java index 82c2663714c47..5771a6b0ed757 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.java @@ -18,11 +18,14 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.transforms.Cast; +import org.apache.kafka.connect.transforms.DropHeaders; import org.apache.kafka.connect.transforms.ExtractField; import org.apache.kafka.connect.transforms.Filter; import org.apache.kafka.connect.transforms.Flatten; +import org.apache.kafka.connect.transforms.HeaderFrom; import org.apache.kafka.connect.transforms.HoistField; import org.apache.kafka.connect.transforms.InsertField; +import org.apache.kafka.connect.transforms.InsertHeader; import org.apache.kafka.connect.transforms.MaskField; import org.apache.kafka.connect.transforms.RegexRouter; import org.apache.kafka.connect.transforms.ReplaceField; @@ -62,7 +65,10 @@ private DocInfo(String transformationName, String overview, ConfigDef configDef) new DocInfo(Flatten.class.getName(), Flatten.OVERVIEW_DOC, Flatten.CONFIG_DEF), new DocInfo(Cast.class.getName(), Cast.OVERVIEW_DOC, Cast.CONFIG_DEF), new DocInfo(TimestampConverter.class.getName(), TimestampConverter.OVERVIEW_DOC, TimestampConverter.CONFIG_DEF), - new DocInfo(Filter.class.getName(), Filter.OVERVIEW_DOC, Filter.CONFIG_DEF) + new DocInfo(Filter.class.getName(), Filter.OVERVIEW_DOC, Filter.CONFIG_DEF), + new DocInfo(InsertHeader.class.getName(), InsertHeader.OVERVIEW_DOC, InsertHeader.CONFIG_DEF), + new DocInfo(DropHeaders.class.getName(), DropHeaders.OVERVIEW_DOC, DropHeaders.CONFIG_DEF), + new DocInfo(HeaderFrom.class.getName(), HeaderFrom.OVERVIEW_DOC, HeaderFrom.CONFIG_DEF) ); private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java new file mode 100644 index 0000000000000..6d1e1a49811c2 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java @@ -0,0 +1,74 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; + +public class DropHeaders> implements Transformation { + + public static final String OVERVIEW_DOC = + "Removes one or more headers from each record."; + + public static final String HEADERS_FIELD = "headers"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(HEADERS_FIELD, ConfigDef.Type.LIST, + NO_DEFAULT_VALUE, new NonEmptyListValidator(), + ConfigDef.Importance.HIGH, + "The name of the headers to be removed."); + + private Set headers; + + @Override + public R apply(R record) { + Headers updatedHeaders = new ConnectHeaders(); + for (Header header : record.headers()) { + if (!headers.contains(header.key())) { + updatedHeaders.add(header); + } + } + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), + record.valueSchema(), record.value(), record.timestamp(), updatedHeaders); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + } + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + headers = new HashSet<>(config.getList(HEADERS_FIELD)); + } +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java new file mode 100644 index 0000000000000..b32ad567e8f6a --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java @@ -0,0 +1,238 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; +import org.apache.kafka.connect.transforms.util.Requirements; +import org.apache.kafka.connect.transforms.util.SchemaUtil; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.lang.String.format; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; + +public abstract class HeaderFrom> implements Transformation { + + public static final String FIELDS_FIELD = "fields"; + public static final String HEADERS_FIELD = "headers"; + public static final String OPERATION_FIELD = "operation"; + private static final String MOVE_OPERATION = "move"; + private static final String COPY_OPERATION = "copy"; + + public static final String OVERVIEW_DOC = + "Moves or copies fields in the key/value of a record into that record's headers. " + + "Corresponding elements of " + FIELDS_FIELD + " and " + + "" + HEADERS_FIELD + " together identify a field and the header it should be " + + "moved or copied to. " + + "Use the concrete transformation type designed for the record " + + "key (" + Key.class.getName() + ") or value (" + Value.class.getName() + ")."; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FIELDS_FIELD, ConfigDef.Type.LIST, + NO_DEFAULT_VALUE, new NonEmptyListValidator(), + ConfigDef.Importance.HIGH, + "Field names in the record whose values are to be copied or moved to headers.") + .define(HEADERS_FIELD, ConfigDef.Type.LIST, + NO_DEFAULT_VALUE, new NonEmptyListValidator(), + ConfigDef.Importance.HIGH, + "Header names, in the same order as the field names listed in the fields configuration property.") + .define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE, + ConfigDef.ValidString.in(MOVE_OPERATION, COPY_OPERATION), ConfigDef.Importance.HIGH, + "Either move if the fields are to be moved to the headers (removed from the key/value), " + + "or copy if the fields are to be copied to the headers (retained in the key/value)."); + + enum Operation { + MOVE(MOVE_OPERATION), + COPY(COPY_OPERATION); + + private final String name; + + Operation(String name) { + this.name = name; + } + + static Operation fromName(String name) { + switch (name) { + case MOVE_OPERATION: + return MOVE; + case COPY_OPERATION: + return COPY; + default: + throw new IllegalArgumentException(); + } + } + + public String toString() { + return name; + } + } + + private List fields; + + private List headers; + + private Operation operation; + + private Cache moveSchemaCache = new SynchronizedCache<>(new LRUCache<>(16)); + + @Override + public R apply(R record) { + Object operatingValue = operatingValue(record); + Schema operatingSchema = operatingSchema(record); + + if (operatingSchema == null) { + return applySchemaless(record, operatingValue); + } else { + return applyWithSchema(record, operatingValue, operatingSchema); + } + } + + private R applyWithSchema(R record, Object operatingValue, Schema operatingSchema) { + Headers updatedHeaders = record.headers().duplicate(); + Struct value = Requirements.requireStruct(operatingValue, "header " + operation); + final Schema updatedSchema; + final Struct updatedValue; + if (operation == Operation.MOVE) { + updatedSchema = moveSchema(operatingSchema); + updatedValue = new Struct(updatedSchema); + for (Field field : updatedSchema.fields()) { + updatedValue.put(field, value.get(field.name())); + } + } else { + updatedSchema = operatingSchema; + updatedValue = value; + } + for (int i = 0; i < fields.size(); i++) { + String fieldName = fields.get(i); + String headerName = headers.get(i); + Object fieldValue = value.schema().field(fieldName) != null ? value.get(fieldName) : null; + Schema fieldSchema = operatingSchema.field(fieldName).schema(); + updatedHeaders.add(headerName, fieldValue, fieldSchema); + } + return newRecord(record, updatedSchema, updatedValue, updatedHeaders); + } + + private Schema moveSchema(Schema operatingSchema) { + Schema moveSchema = this.moveSchemaCache.get(operatingSchema); + if (moveSchema == null) { + final SchemaBuilder builder = SchemaUtil.copySchemaBasics(operatingSchema, SchemaBuilder.struct()); + for (Field field : operatingSchema.fields()) { + if (!fields.contains(field.name())) { + builder.field(field.name(), field.schema()); + } + } + moveSchema = builder.build(); + moveSchemaCache.put(operatingSchema, moveSchema); + } + return moveSchema; + } + + private R applySchemaless(R record, Object operatingValue) { + Headers updatedHeaders = record.headers().duplicate(); + Map value = Requirements.requireMap(operatingValue, "header " + operation); + Map updatedValue = new HashMap<>(value); + for (int i = 0; i < fields.size(); i++) { + String fieldName = fields.get(i); + Object fieldValue = value.get(fieldName); + String headerName = headers.get(i); + if (operation == Operation.MOVE) { + updatedValue.remove(fieldName); + } + updatedHeaders.add(headerName, fieldValue, null); + } + return newRecord(record, null, updatedValue, updatedHeaders); + } + + protected abstract Object operatingValue(R record); + protected abstract Schema operatingSchema(R record); + protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue, Iterable
    updatedHeaders); + + public static class Key> extends HeaderFrom { + + @Override + public Object operatingValue(R record) { + return record.key(); + } + + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue, Iterable
    updatedHeaders) { + return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, + record.valueSchema(), record.value(), record.timestamp(), updatedHeaders); + } + } + + public static class Value> extends HeaderFrom { + + @Override + public Object operatingValue(R record) { + return record.value(); + } + + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue, Iterable
    updatedHeaders) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), + updatedSchema, updatedValue, record.timestamp(), updatedHeaders); + } + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + + } + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + fields = config.getList(FIELDS_FIELD); + headers = config.getList(HEADERS_FIELD); + if (headers.size() != fields.size()) { + throw new ConfigException(format("'%s' config must have the same number of elements as '%s' config.", + FIELDS_FIELD, HEADERS_FIELD)); + } + operation = Operation.fromName(config.getString(OPERATION_FIELD)); + } +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertHeader.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertHeader.java new file mode 100644 index 0000000000000..88b20020ebac9 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertHeader.java @@ -0,0 +1,77 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.Values; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Map; + +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; + +public class InsertHeader> implements Transformation { + + public static final String OVERVIEW_DOC = + "Add a header to each record."; + + public static final String HEADER_FIELD = "header"; + public static final String VALUE_LITERAL_FIELD = "value.literal"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(HEADER_FIELD, ConfigDef.Type.STRING, + NO_DEFAULT_VALUE, new ConfigDef.NonNullValidator(), + ConfigDef.Importance.HIGH, + "The name of the header.") + .define(VALUE_LITERAL_FIELD, ConfigDef.Type.STRING, + NO_DEFAULT_VALUE, new ConfigDef.NonNullValidator(), + ConfigDef.Importance.HIGH, + "The literal value that is to be set as the header value on all records."); + + private String header; + + private SchemaAndValue literalValue; + + @Override + public R apply(R record) { + Headers updatedHeaders = record.headers().duplicate(); + updatedHeaders.add(header, literalValue); + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), + record.valueSchema(), record.value(), record.timestamp(), updatedHeaders); + } + + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + + } + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + header = config.getString(HEADER_FIELD); + literalValue = Values.parseString(config.getString(VALUE_LITERAL_FIELD)); + } +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java new file mode 100644 index 0000000000000..7d20c384e78ad --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/DropHeadersTest.java @@ -0,0 +1,117 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.source.SourceRecord; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static java.util.Arrays.asList; +import static java.util.Collections.singletonMap; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class DropHeadersTest { + + private DropHeaders xform = new DropHeaders<>(); + + private Map config(String... headers) { + Map result = new HashMap<>(); + result.put(DropHeaders.HEADERS_FIELD, asList(headers)); + return result; + } + + @Test + public void dropExistingHeader() { + xform.configure(config("to-drop")); + ConnectHeaders expected = new ConnectHeaders(); + expected.addString("existing", "existing-value"); + ConnectHeaders headers = expected.duplicate(); + headers.addString("to-drop", "existing-value"); + SourceRecord original = sourceRecord(headers); + SourceRecord xformed = xform.apply(original); + assertNonHeaders(original, xformed); + assertEquals(expected, xformed.headers()); + } + + @Test + public void dropExistingHeaderWithMultipleValues() { + xform.configure(config("to-drop")); + ConnectHeaders expected = new ConnectHeaders(); + expected.addString("existing", "existing-value"); + ConnectHeaders headers = expected.duplicate(); + headers.addString("to-drop", "existing-value"); + headers.addString("to-drop", "existing-other-value"); + + SourceRecord original = sourceRecord(headers); + SourceRecord xformed = xform.apply(original); + assertNonHeaders(original, xformed); + assertEquals(expected, xformed.headers()); + } + + @Test + public void dropNonExistingHeader() { + xform.configure(config("to-drop")); + ConnectHeaders expected = new ConnectHeaders(); + expected.addString("existing", "existing-value"); + ConnectHeaders headers = expected.duplicate(); + + SourceRecord original = sourceRecord(headers); + SourceRecord xformed = xform.apply(original); + assertNonHeaders(original, xformed); + assertEquals(expected, xformed.headers()); + } + + @Test + public void configRejectsEmptyList() { + assertThrows(ConfigException.class, () -> xform.configure(config())); + } + + private void assertNonHeaders(SourceRecord original, SourceRecord xformed) { + assertEquals(original.sourcePartition(), xformed.sourcePartition()); + assertEquals(original.sourceOffset(), xformed.sourceOffset()); + assertEquals(original.topic(), xformed.topic()); + assertEquals(original.kafkaPartition(), xformed.kafkaPartition()); + assertEquals(original.keySchema(), xformed.keySchema()); + assertEquals(original.key(), xformed.key()); + assertEquals(original.valueSchema(), xformed.valueSchema()); + assertEquals(original.value(), xformed.value()); + assertEquals(original.timestamp(), xformed.timestamp()); + } + + private SourceRecord sourceRecord(ConnectHeaders headers) { + Map sourcePartition = singletonMap("foo", "bar"); + Map sourceOffset = singletonMap("baz", "quxx"); + String topic = "topic"; + Integer partition = 0; + Schema keySchema = null; + Object key = "key"; + Schema valueSchema = null; + Object value = "value"; + Long timestamp = 0L; + + SourceRecord record = new SourceRecord(sourcePartition, sourceOffset, topic, partition, + keySchema, key, valueSchema, value, timestamp, headers); + return record; + } +} + diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java new file mode 100644 index 0000000000000..61e05757474b6 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HeaderFromTest.java @@ -0,0 +1,359 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Header; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.source.SourceRecord; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.Arrays.asList; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; +import static java.util.Collections.singletonMap; +import static org.apache.kafka.connect.data.Schema.STRING_SCHEMA; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class HeaderFromTest { + + static class RecordBuilder { + private final List fields = new ArrayList<>(2); + private final List fieldSchemas = new ArrayList<>(2); + private final List fieldValues = new ArrayList<>(2); + private final ConnectHeaders headers = new ConnectHeaders(); + + public RecordBuilder() { + } + + public RecordBuilder withField(String name, Schema schema, Object value) { + fields.add(name); + fieldSchemas.add(schema); + fieldValues.add(value); + return this; + } + + public RecordBuilder addHeader(String name, Schema schema, Object value) { + headers.add(name, new SchemaAndValue(schema, value)); + return this; + } + + public SourceRecord schemaless(boolean keyTransform) { + Map map = new HashMap<>(); + for (int i = 0; i < this.fields.size(); i++) { + String fieldName = this.fields.get(i); + map.put(fieldName, this.fieldValues.get(i)); + + } + return sourceRecord(keyTransform, null, map); + } + + private Schema schema() { + SchemaBuilder schemaBuilder = new SchemaBuilder(Schema.Type.STRUCT); + for (int i = 0; i < this.fields.size(); i++) { + String fieldName = this.fields.get(i); + schemaBuilder.field(fieldName, this.fieldSchemas.get(i)); + + } + return schemaBuilder.build(); + } + + private Struct struct(Schema schema) { + Struct struct = new Struct(schema); + for (int i = 0; i < this.fields.size(); i++) { + String fieldName = this.fields.get(i); + struct.put(fieldName, this.fieldValues.get(i)); + } + return struct; + } + + public SourceRecord withSchema(boolean keyTransform) { + Schema schema = schema(); + Struct struct = struct(schema); + return sourceRecord(keyTransform, schema, struct); + } + + private SourceRecord sourceRecord(boolean keyTransform, Schema keyOrValueSchema, Object keyOrValue) { + Map sourcePartition = singletonMap("foo", "bar"); + Map sourceOffset = singletonMap("baz", "quxx"); + String topic = "topic"; + Integer partition = 0; + Long timestamp = 0L; + + ConnectHeaders headers = this.headers; + if (keyOrValueSchema == null) { + // When doing a schemaless transformation we don't expect the header to have a schema + headers = new ConnectHeaders(); + for (Header header : this.headers) { + headers.add(header.key(), new SchemaAndValue(null, header.value())); + } + } + return new SourceRecord(sourcePartition, sourceOffset, topic, partition, + keyTransform ? keyOrValueSchema : null, + keyTransform ? keyOrValue : "key", + !keyTransform ? keyOrValueSchema : null, + !keyTransform ? keyOrValue : "value", + timestamp, headers); + } + + @Override + public String toString() { + return "RecordBuilder(" + + "fields=" + fields + + ", fieldSchemas=" + fieldSchemas + + ", fieldValues=" + fieldValues + + ", headers=" + headers + + ')'; + } + } + + public static List data() { + + List result = new ArrayList<>(); + + for (Boolean testKeyTransform : asList(true, false)) { + result.add( + Arguments.of( + "basic copy", + testKeyTransform, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value"), + singletonList("field1"), singletonList("inserted1"), HeaderFrom.Operation.COPY, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", STRING_SCHEMA, "field1-value") + )); + result.add( + Arguments.of( + "basic move", + testKeyTransform, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value"), + singletonList("field1"), singletonList("inserted1"), HeaderFrom.Operation.MOVE, + new RecordBuilder() + // field1 got moved + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", STRING_SCHEMA, "field1-value") + )); + result.add( + Arguments.of( + "copy with preexisting header", + testKeyTransform, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("inserted1", STRING_SCHEMA, "existing-value"), + singletonList("field1"), singletonList("inserted1"), HeaderFrom.Operation.COPY, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("inserted1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", STRING_SCHEMA, "field1-value") + )); + result.add( + Arguments.of( + "move with preexisting header", + testKeyTransform, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("inserted1", STRING_SCHEMA, "existing-value"), + singletonList("field1"), singletonList("inserted1"), HeaderFrom.Operation.MOVE, + new RecordBuilder() + // field1 got moved + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("inserted1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", STRING_SCHEMA, "field1-value") + )); + Schema schema = new SchemaBuilder(Schema.Type.STRUCT).field("foo", STRING_SCHEMA).build(); + Struct struct = new Struct(schema).put("foo", "foo-value"); + result.add( + Arguments.of( + "copy with struct value", + testKeyTransform, + new RecordBuilder() + .withField("field1", schema, struct) + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value"), + singletonList("field1"), singletonList("inserted1"), HeaderFrom.Operation.COPY, + new RecordBuilder() + .withField("field1", schema, struct) + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", schema, struct) + )); + result.add( + Arguments.of( + "move with struct value", + testKeyTransform, + new RecordBuilder() + .withField("field1", schema, struct) + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value"), + singletonList("field1"), singletonList("inserted1"), HeaderFrom.Operation.MOVE, + new RecordBuilder() + // field1 got moved + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", schema, struct) + )); + result.add( + Arguments.of( + "two headers from same field", + testKeyTransform, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value"), + // two headers from the same field + asList("field1", "field1"), asList("inserted1", "inserted2"), HeaderFrom.Operation.MOVE, + new RecordBuilder() + // field1 got moved + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", STRING_SCHEMA, "field1-value") + .addHeader("inserted2", STRING_SCHEMA, "field1-value") + )); + result.add( + Arguments.of( + "two fields to same header", + testKeyTransform, + new RecordBuilder() + .withField("field1", STRING_SCHEMA, "field1-value") + .withField("field2", STRING_SCHEMA, "field2-value") + .addHeader("header1", STRING_SCHEMA, "existing-value"), + // two headers from the same field + asList("field1", "field2"), asList("inserted1", "inserted1"), HeaderFrom.Operation.MOVE, + new RecordBuilder() + // field1 and field2 got moved + .addHeader("header1", STRING_SCHEMA, "existing-value") + .addHeader("inserted1", STRING_SCHEMA, "field1-value") + .addHeader("inserted1", STRING_SCHEMA, "field2-value") + )); + } + return result; + } + + private Map config(List headers, List transformFields, HeaderFrom.Operation operation) { + Map result = new HashMap<>(); + result.put(HeaderFrom.HEADERS_FIELD, headers); + result.put(HeaderFrom.FIELDS_FIELD, transformFields); + result.put(HeaderFrom.OPERATION_FIELD, operation.toString()); + return result; + } + + @ParameterizedTest + @MethodSource("data") + public void schemaless(String description, + boolean keyTransform, + RecordBuilder originalBuilder, + List transformFields, List headers1, HeaderFrom.Operation operation, + RecordBuilder expectedBuilder) { + HeaderFrom xform = keyTransform ? new HeaderFrom.Key<>() : new HeaderFrom.Value<>(); + + xform.configure(config(headers1, transformFields, operation)); + ConnectHeaders headers = new ConnectHeaders(); + headers.addString("existing", "existing-value"); + + SourceRecord originalRecord = originalBuilder.schemaless(keyTransform); + SourceRecord expectedRecord = expectedBuilder.schemaless(keyTransform); + SourceRecord xformed = xform.apply(originalRecord); + assertSameRecord(expectedRecord, xformed); + } + + @ParameterizedTest + @MethodSource("data") + public void withSchema(String description, + boolean keyTransform, + RecordBuilder originalBuilder, + List transformFields, List headers1, HeaderFrom.Operation operation, + RecordBuilder expectedBuilder) { + HeaderFrom xform = keyTransform ? new HeaderFrom.Key<>() : new HeaderFrom.Value<>(); + xform.configure(config(headers1, transformFields, operation)); + ConnectHeaders headers = new ConnectHeaders(); + headers.addString("existing", "existing-value"); + Headers expect = headers.duplicate(); + for (int i = 0; i < headers1.size(); i++) { + expect.add(headers1.get(i), originalBuilder.fieldValues.get(i), originalBuilder.fieldSchemas.get(i)); + } + + SourceRecord originalRecord = originalBuilder.withSchema(keyTransform); + SourceRecord expectedRecord = expectedBuilder.withSchema(keyTransform); + SourceRecord xformed = xform.apply(originalRecord); + assertSameRecord(expectedRecord, xformed); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void invalidConfigExtraHeaderConfig(boolean keyTransform) { + Map config = config(singletonList("foo"), asList("foo", "bar"), HeaderFrom.Operation.COPY); + HeaderFrom xform = keyTransform ? new HeaderFrom.Key<>() : new HeaderFrom.Value<>(); + assertThrows(ConfigException.class, () -> xform.configure(config)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void invalidConfigExtraFieldConfig(boolean keyTransform) { + Map config = config(asList("foo", "bar"), singletonList("foo"), HeaderFrom.Operation.COPY); + HeaderFrom xform = keyTransform ? new HeaderFrom.Key<>() : new HeaderFrom.Value<>(); + assertThrows(ConfigException.class, () -> xform.configure(config)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void invalidConfigEmptyHeadersAndFieldsConfig(boolean keyTransform) { + Map config = config(emptyList(), emptyList(), HeaderFrom.Operation.COPY); + HeaderFrom xform = keyTransform ? new HeaderFrom.Key<>() : new HeaderFrom.Value<>(); + assertThrows(ConfigException.class, () -> xform.configure(config)); + } + + private static void assertSameRecord(SourceRecord expected, SourceRecord xformed) { + assertEquals(expected.sourcePartition(), xformed.sourcePartition()); + assertEquals(expected.sourceOffset(), xformed.sourceOffset()); + assertEquals(expected.topic(), xformed.topic()); + assertEquals(expected.kafkaPartition(), xformed.kafkaPartition()); + assertEquals(expected.keySchema(), xformed.keySchema()); + assertEquals(expected.key(), xformed.key()); + assertEquals(expected.valueSchema(), xformed.valueSchema()); + assertEquals(expected.value(), xformed.value()); + assertEquals(expected.timestamp(), xformed.timestamp()); + assertEquals(expected.headers(), xformed.headers()); + } + +} + diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java new file mode 100644 index 0000000000000..97cbe5d4162b6 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/InsertHeaderTest.java @@ -0,0 +1,121 @@ +/* + * 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.connect.transforms; + +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.source.SourceRecord; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static java.util.Collections.singletonMap; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class InsertHeaderTest { + + private InsertHeader xform = new InsertHeader<>(); + + private Map config(String header, String valueLiteral) { + Map result = new HashMap<>(); + result.put(InsertHeader.HEADER_FIELD, header); + result.put(InsertHeader.VALUE_LITERAL_FIELD, valueLiteral); + return result; + } + + @Test + public void insertionWithExistingOtherHeader() { + xform.configure(config("inserted", "inserted-value")); + ConnectHeaders headers = new ConnectHeaders(); + headers.addString("existing", "existing-value"); + Headers expect = headers.duplicate().addString("inserted", "inserted-value"); + + SourceRecord original = sourceRecord(headers); + SourceRecord xformed = xform.apply(original); + assertNonHeaders(original, xformed); + assertEquals(expect, xformed.headers()); + } + + @Test + public void insertionWithExistingSameHeader() { + xform.configure(config("existing", "inserted-value")); + ConnectHeaders headers = new ConnectHeaders(); + headers.addString("existing", "preexisting-value"); + Headers expect = headers.duplicate().addString("existing", "inserted-value"); + + SourceRecord original = sourceRecord(headers); + SourceRecord xformed = xform.apply(original); + assertNonHeaders(original, xformed); + assertEquals(expect, xformed.headers()); + } + + @Test + public void insertionWithByteHeader() { + xform.configure(config("inserted", "1")); + ConnectHeaders headers = new ConnectHeaders(); + headers.addString("existing", "existing-value"); + Headers expect = headers.duplicate().addByte("inserted", (byte) 1); + + SourceRecord original = sourceRecord(headers); + SourceRecord xformed = xform.apply(original); + assertNonHeaders(original, xformed); + assertEquals(expect, xformed.headers()); + } + + @Test + public void configRejectsNullHeaderKey() { + assertThrows(ConfigException.class, () -> xform.configure(config(null, "1"))); + } + + @Test + public void configRejectsNullHeaderValue() { + assertThrows(ConfigException.class, () -> xform.configure(config("inserted", null))); + } + + private void assertNonHeaders(SourceRecord original, SourceRecord xformed) { + assertEquals(original.sourcePartition(), xformed.sourcePartition()); + assertEquals(original.sourceOffset(), xformed.sourceOffset()); + assertEquals(original.topic(), xformed.topic()); + assertEquals(original.kafkaPartition(), xformed.kafkaPartition()); + assertEquals(original.keySchema(), xformed.keySchema()); + assertEquals(original.key(), xformed.key()); + assertEquals(original.valueSchema(), xformed.valueSchema()); + assertEquals(original.value(), xformed.value()); + assertEquals(original.timestamp(), xformed.timestamp()); + } + + private SourceRecord sourceRecord(ConnectHeaders headers) { + Map sourcePartition = singletonMap("foo", "bar"); + Map sourceOffset = singletonMap("baz", "quxx"); + String topic = "topic"; + Integer partition = 0; + Schema keySchema = null; + Object key = "key"; + Schema valueSchema = null; + Object value = "value"; + Long timestamp = 0L; + + SourceRecord record = new SourceRecord(sourcePartition, sourceOffset, topic, partition, + keySchema, key, valueSchema, value, timestamp, headers); + return record; + } +} + diff --git a/docs/connect.html b/docs/connect.html index ef4cfe9668c7a..e522c8a088f22 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -183,6 +183,9 @@
    predicate to selectively filter certain messages. +
  • InsertHeader - Add a header using static data
  • +
  • HeadersFrom - Copy or move fields in the key or value to the record headers
  • +
  • DropHeaders - Remove headers by name
  • Details on how to configure each transformation are listed below:

    From df185aabc04fbd3c95327a878863a0549539ea6c Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 16 Apr 2021 22:59:22 +0800 Subject: [PATCH 067/155] MINOR: remove redundant parentheses from ControllerApis (#10546) Reviewers: Ismael Juma --- .../scala/kafka/server/ControllerApis.scala | 60 +++++++++---------- 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index 3f958f5b2dd33..cf5be56a9540b 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -176,7 +176,7 @@ class ControllerApis(val requestChannel: RequestChannel, authHelper.authorize(request.context, DELETE, CLUSTER, CLUSTER_NAME), names => authHelper.filterByAuthorized(request.context, DESCRIBE, TOPIC, names)(n => n), names => authHelper.filterByAuthorized(request.context, DELETE, TOPIC, names)(n => n)) - future.whenComplete((results, exception) => { + future.whenComplete { (results, exception) => requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { if (exception != null) { deleteTopicsRequest.getErrorResponse(throttleTimeMs, exception) @@ -187,7 +187,7 @@ class ControllerApis(val requestChannel: RequestChannel, new DeleteTopicsResponse(responseData) } }) - }) + } } def deleteTopics(request: DeleteTopicsRequestData, @@ -256,7 +256,7 @@ class ControllerApis(val requestChannel: RequestChannel, val toAuthenticate = new util.HashSet[String] toAuthenticate.addAll(providedNames) val idToName = new util.HashMap[Uuid, String] - controller.findTopicNames(providedIds).thenCompose(topicNames => { + controller.findTopicNames(providedIds).thenCompose { topicNames => topicNames.forEach { (id, nameOrError) => if (nameOrError.isError) { appendResponse(null, id, nameOrError.error()) @@ -291,7 +291,7 @@ class ControllerApis(val requestChannel: RequestChannel, } // For each topic that was provided by name, check if authentication failed. // If so, create an error response for it. Otherwise, add it to the idToName map. - controller.findTopicIds(providedNames).thenCompose(topicIds => { + controller.findTopicIds(providedNames).thenCompose { topicIds => topicIds.forEach { (name, idOrError) => if (!describeable.contains(name)) { appendResponse(name, ZERO_UUID, new ApiError(TOPIC_AUTHORIZATION_FAILED)) @@ -315,7 +315,7 @@ class ControllerApis(val requestChannel: RequestChannel, } // Finally, the idToName map contains all the topics that we are authorized to delete. // Perform the deletion and create responses for each one. - controller.deleteTopics(idToName.keySet).thenApply(idToError => { + controller.deleteTopics(idToName.keySet).thenApply { idToError => idToError.forEach { (id, error) => appendResponse(idToName.get(id), id, error) } @@ -323,9 +323,9 @@ class ControllerApis(val requestChannel: RequestChannel, // distinguish between absent topics and topics we are not permitted to see. Collections.shuffle(responses) responses - }) - }) - }) + } + } + } } def handleCreateTopics(request: RequestChannel.Request): Unit = { @@ -333,7 +333,7 @@ class ControllerApis(val requestChannel: RequestChannel, val future = createTopics(createTopicsRequest.data(), authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME), names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(identity)) - future.whenComplete((result, exception) => { + future.whenComplete { (result, exception) => requestHelper.sendResponseMaybeThrottle(request, throttleTimeMs => { if (exception != null) { createTopicsRequest.getErrorResponse(throttleTimeMs, exception) @@ -342,7 +342,7 @@ class ControllerApis(val requestChannel: RequestChannel, new CreateTopicsResponse(result) } }) - }) + } } def createTopics(request: CreateTopicsRequestData, @@ -373,7 +373,7 @@ class ControllerApis(val requestChannel: RequestChannel, iterator.remove() } } - controller.createTopics(effectiveRequest).thenApply(response => { + controller.createTopics(effectiveRequest).thenApply { response => duplicateTopicNames.forEach { name => response.topics().add(new CreatableTopicResult(). setName(name). @@ -388,7 +388,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } response - }) + } } def handleApiVersionsRequest(request: RequestChannel.Request): Unit = { @@ -472,7 +472,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } controller.legacyAlterConfigs(configChanges, alterConfigsRequest.data.validateOnly) - .whenComplete((controllerResults, exception) => { + .whenComplete { (controllerResults, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -485,7 +485,7 @@ class ControllerApis(val requestChannel: RequestChannel, requestHelper.sendResponseMaybeThrottle(request, throttleMs => new AlterConfigsResponse(response.setThrottleTimeMs(throttleMs))) } - }) + } } def handleVote(request: RequestChannel.Request): Unit = { @@ -512,21 +512,21 @@ class ControllerApis(val requestChannel: RequestChannel, val alterIsrRequest = request.body[AlterIsrRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) val future = controller.alterIsr(alterIsrRequest.data) - future.whenComplete((result, exception) => { + future.whenComplete { (result, exception) => val response = if (exception != null) { alterIsrRequest.getErrorResponse(exception) } else { new AlterIsrResponse(result) } requestHelper.sendResponseExemptThrottle(request, response) - }) + } } def handleBrokerHeartBeatRequest(request: RequestChannel.Request): Unit = { val heartbeatRequest = request.body[BrokerHeartbeatRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - controller.processBrokerHeartbeat(heartbeatRequest.data).handle[Unit]((reply, e) => { + controller.processBrokerHeartbeat(heartbeatRequest.data).handle[Unit] { (reply, e) => def createResponseCallback(requestThrottleMs: Int, reply: BrokerHeartbeatReply, e: Throwable): BrokerHeartbeatResponse = { @@ -545,14 +545,14 @@ class ControllerApis(val requestChannel: RequestChannel, } requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createResponseCallback(requestThrottleMs, reply, e)) - }) + } } def handleUnregisterBroker(request: RequestChannel.Request): Unit = { val decommissionRequest = request.body[UnregisterBrokerRequest] authHelper.authorizeClusterOperation(request, ALTER) - controller.unregisterBroker(decommissionRequest.data().brokerId()).handle[Unit]((_, e) => { + controller.unregisterBroker(decommissionRequest.data().brokerId()).handle[Unit] { (_, e) => def createResponseCallback(requestThrottleMs: Int, e: Throwable): UnregisterBrokerResponse = { if (e != null) { @@ -566,14 +566,14 @@ class ControllerApis(val requestChannel: RequestChannel, } requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createResponseCallback(requestThrottleMs, e)) - }) + } } def handleBrokerRegistration(request: RequestChannel.Request): Unit = { val registrationRequest = request.body[BrokerRegistrationRequest] authHelper.authorizeClusterOperation(request, CLUSTER_ACTION) - controller.registerBroker(registrationRequest.data).handle[Unit]((reply, e) => { + controller.registerBroker(registrationRequest.data).handle[Unit] { (reply, e) => def createResponseCallback(requestThrottleMs: Int, reply: BrokerRegistrationReply, e: Throwable): BrokerRegistrationResponse = { @@ -590,7 +590,7 @@ class ControllerApis(val requestChannel: RequestChannel, } requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => createResponseCallback(requestThrottleMs, reply, e)) - }) + } } private def handleRaftRequest(request: RequestChannel.Request, @@ -598,28 +598,28 @@ class ControllerApis(val requestChannel: RequestChannel, val requestBody = request.body[AbstractRequest] val future = raftManager.handleRequest(request.header, requestBody.data, time.milliseconds()) - future.whenComplete((responseData, exception) => { + future.whenComplete { (responseData, exception) => val response = if (exception != null) { requestBody.getErrorResponse(exception) } else { buildResponse(responseData) } requestHelper.sendResponseExemptThrottle(request, response) - }) + } } def handleAlterClientQuotas(request: RequestChannel.Request): Unit = { val quotaRequest = request.body[AlterClientQuotasRequest] authHelper.authorizeClusterOperation(request, ALTER_CONFIGS) controller.alterClientQuotas(quotaRequest.entries, quotaRequest.validateOnly) - .whenComplete((results, exception) => { + .whenComplete { (results, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => AlterClientQuotasResponse.fromQuotaEntities(results, requestThrottleMs)) } - }) + } } def handleIncrementalAlterConfigs(request: RequestChannel.Request): Unit = { @@ -668,7 +668,7 @@ class ControllerApis(val requestChannel: RequestChannel, } } controller.incrementalAlterConfigs(configChanges, alterConfigsRequest.data.validateOnly) - .whenComplete((controllerResults, exception) => { + .whenComplete { (controllerResults, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -681,14 +681,14 @@ class ControllerApis(val requestChannel: RequestChannel, requestHelper.sendResponseMaybeThrottle(request, throttleMs => new IncrementalAlterConfigsResponse(response.setThrottleTimeMs(throttleMs))) } - }) + } } def handleCreatePartitions(request: RequestChannel.Request): Unit = { val future = createPartitions(request.body[CreatePartitionsRequest].data, authHelper.authorize(request.context, CREATE, CLUSTER, CLUSTER_NAME), names => authHelper.filterByAuthorized(request.context, CREATE, TOPIC, names)(n => n)) - future.whenComplete((responses, exception) => { + future.whenComplete { (responses, exception) => if (exception != null) { requestHelper.handleError(request, exception) } else { @@ -699,7 +699,7 @@ class ControllerApis(val requestChannel: RequestChannel, new CreatePartitionsResponse(responseData) }) } - }) + } } def createPartitions(request: CreatePartitionsRequestData, From 035449eb5576bf97df888a85edaaf4428b9a19e5 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sun, 18 Apr 2021 10:46:01 -0700 Subject: [PATCH 068/155] KAFKA-12633: Remove deprecated APIs in TopologyTestDriver (#10508) As well as related test classes. Reviewers: John Roesler --- docs/streams/upgrade-guide.html | 1 + .../kafka/streams/TopologyTestDriver.java | 122 +--- .../streams/test/ConsumerRecordFactory.java | 583 ------------------ .../kafka/streams/test/OutputVerifier.java | 458 -------------- .../kafka/streams/TopologyTestDriverTest.java | 189 +----- .../test/ConsumerRecordFactoryTest.java | 273 -------- .../streams/test/OutputVerifierTest.java | 452 -------------- 7 files changed, 25 insertions(+), 2053 deletions(-) delete mode 100644 streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java delete mode 100644 streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java delete mode 100644 streams/test-utils/src/test/java/org/apache/kafka/streams/test/ConsumerRecordFactoryTest.java delete mode 100644 streams/test-utils/src/test/java/org/apache/kafka/streams/test/OutputVerifierTest.java diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 0269be9ebf95d..ea410b13e331c 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -107,6 +107,7 @@

    Streams API
  • Overloaded JoinWindows#of, before, after, SessionWindows#with, TimeWindows#of, advanceBy, UnlimitedWindows#startOn and KafkaStreams#close with long typed parameters: deprecated in Kafka 2.1.0 (KIP-358).
  • Overloaded KStream#groupBy, groupByKey and KTable#groupBy with Serialized parameter: deprecated in Kafka 2.1.0 (KIP-372).
  • Joined#named, name: deprecated in Kafka 2.3.0 (KIP-307).
  • +
  • TopologyTestDriver#pipeInput, readOutput, OutputVerifier and ConsumerRecordFactory classes (KIP-470).
  • KafkaClientSupplier#getAdminClient: deprecated in Kafka 2.4.0 (KIP-476).
  • Overloaded KStream#join, leftJoin, outerJoin with KStream and Joined parameters: deprecated in Kafka 2.4.0 (KIP-479).
  • Overloaded KafkaStreams#metadataForKey: deprecated in Kafka 2.5.0 (KIP-535).
  • diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index bd93367c6b264..118bdbbd41df9 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -192,7 +192,7 @@ * Note, that calling {@code pipeInput()} will also trigger {@link PunctuationType#STREAM_TIME event-time} base * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) punctuation} callbacks. * However, you won't trigger {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type punctuations that you must - * trigger manually via {@link #advanceWallClockTime(long)}. + * trigger manually via {@link #advanceWallClockTime(Duration)}. *

    * Finally, when completed, make sure your tests {@link #close()} the driver to release all resources and * {@link org.apache.kafka.streams.processor.Processor processors}. @@ -284,23 +284,6 @@ public TopologyTestDriver(final Topology topology, this(topology, new Properties(), initialWallClockTimeMs); } - - /** - * Create a new test diver instance. - * - * @deprecated Since 2.4 use {@link #TopologyTestDriver(Topology, Properties, Instant)} - * - * @param topology the topology to be tested - * @param config the configuration for the topology - * @param initialWallClockTimeMs the initial value of internally mocked wall-clock time - */ - @Deprecated - public TopologyTestDriver(final Topology topology, - final Properties config, - final long initialWallClockTimeMs) { - this(topology.internalTopologyBuilder, config, initialWallClockTimeMs); - } - /** * Create a new test diver instance. * @@ -553,25 +536,6 @@ private void setupTask(final StreamsConfig streamsConfig, return Collections.unmodifiableMap(metrics.metrics()); } - /** - * Send an input message with the given key, value, and timestamp on the specified topic to the topology and then - * commit the messages. - * - * @deprecated Since 2.4 use methods of {@link TestInputTopic} instead - * - * @param consumerRecord the record to be processed - */ - @Deprecated - public void pipeInput(final ConsumerRecord consumerRecord) { - pipeRecord( - consumerRecord.topic(), - consumerRecord.timestamp(), - consumerRecord.key(), - consumerRecord.value(), - consumerRecord.headers() - ); - } - private void pipeRecord(final String topicName, final long timestamp, final byte[] key, @@ -737,34 +701,6 @@ private void captureOutputsAndReEnqueueInternalResults() { } } - /** - * Send input messages to the topology and then commit each message individually. - * - * @deprecated Since 2.4 use methods of {@link TestInputTopic} instead - * - * @param records a list of records to be processed - */ - @Deprecated - public void pipeInput(final List> records) { - for (final ConsumerRecord record : records) { - pipeInput(record); - } - } - - /** - * Advances the internally mocked wall-clock time. - * This might trigger a {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type - * {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator) punctuations}. - * - * @deprecated Since 2.4 use {@link #advanceWallClockTime(Duration)} instead - * - * @param advanceMs the amount of time to advance wall-clock time in milliseconds - */ - @Deprecated - public void advanceWallClockTime(final long advanceMs) { - advanceWallClockTime(Duration.ofMillis(advanceMs)); - } - /** * Advances the internally mocked wall-clock time. * This might trigger a {@link PunctuationType#WALL_CLOCK_TIME wall-clock} type @@ -783,48 +719,6 @@ public void advanceWallClockTime(final Duration advance) { completeAllProcessableWork(); } - /** - * Read the next record from the given topic. - * These records were output by the topology during the previous calls to {@link #pipeInput(ConsumerRecord)}. - * - * @deprecated Since 2.4 use methods of {@link TestOutputTopic} instead - * - * @param topic the name of the topic - * @return the next record on that topic, or {@code null} if there is no record available - */ - @Deprecated - public ProducerRecord readOutput(final String topic) { - final Queue> outputRecords = outputRecordsByTopic.get(topic); - if (outputRecords == null) { - return null; - } - return outputRecords.poll(); - } - - /** - * Read the next record from the given topic. - * These records were output by the topology during the previous calls to {@link #pipeInput(ConsumerRecord)}. - * - * @deprecated Since 2.4 use methods of {@link TestOutputTopic} instead - * - * @param topic the name of the topic - * @param keyDeserializer the deserializer for the key type - * @param valueDeserializer the deserializer for the value type - * @return the next record on that topic, or {@code null} if there is no record available - */ - @Deprecated - public ProducerRecord readOutput(final String topic, - final Deserializer keyDeserializer, - final Deserializer valueDeserializer) { - final ProducerRecord record = readOutput(topic); - if (record == null) { - return null; - } - final K key = keyDeserializer.deserialize(record.topic(), record.headers(), record.key()); - final V value = valueDeserializer.deserialize(record.topic(), record.headers(), record.value()); - return new ProducerRecord<>(record.topic(), record.partition(), record.timestamp(), key, value, record.headers()); - } - private Queue> getRecordsQueue(final String topicName) { final Queue> outputRecords = outputRecordsByTopic.get(topicName); if (outputRecords == null && !processorTopology.sinkTopics().contains(topicName)) { @@ -966,7 +860,7 @@ final boolean isEmpty(final String topic) { * The stores can be a "regular" or global stores. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to - * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * {@link TestInputTopic#pipeInput(TestRecord)} process an input message}, and/or to check the store afterward. *

    * Note, that {@code StateStore} might be {@code null} if a store is added but not connected to any processor. *

    @@ -998,7 +892,7 @@ public Map getAllStateStores() { * For built-in stores, the corresponding typed methods like {@link #getKeyValueStore(String)} should be used. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to - * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * {@link TestInputTopic#pipeInput(TestRecord) process an input message}, and/or to check the store afterward. * * @param name the name of the store * @return the state store, or {@code null} if no store has been registered with the given name @@ -1074,7 +968,7 @@ private void throwIfBuiltInStore(final StateStore stateStore) { * {@link #getTimestampedKeyValueStore(String)} for full store access instead. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to - * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * {@link TestInputTopic#pipeInput(TestRecord) process an input message}, and/or to check the store afterward. * * @param name the name of the store * @return the key value store, or {@code null} if no {@link KeyValueStore} or {@link TimestampedKeyValueStore} @@ -1101,7 +995,7 @@ public KeyValueStore getKeyValueStore(final String name) { * The store can be a "regular" or global store. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to - * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * {@link TestInputTopic#pipeInput(TestRecord) process an input message}, and/or to check the store afterward. * * @param name the name of the store * @return the key value store, or {@code null} if no {@link TimestampedKeyValueStore} has been registered with the given name @@ -1127,7 +1021,7 @@ public KeyValueStore> getTimestampedKeyValueStore * {@link #getTimestampedWindowStore(String)} for full store access instead. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to - * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * {@link TestInputTopic#pipeInput(TestRecord) process an input message}, and/or to check the store afterward. * * @param name the name of the store * @return the key value store, or {@code null} if no {@link WindowStore} or {@link TimestampedWindowStore} @@ -1154,7 +1048,7 @@ public WindowStore getWindowStore(final String name) { * The store can be a "regular" or global store. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to - * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * {@link TestInputTopic#pipeInput(TestRecord) process an input message}, and/or to check the store afterward. * * @param name the name of the store * @return the key value store, or {@code null} if no {@link TimestampedWindowStore} has been registered with the given name @@ -1176,7 +1070,7 @@ public WindowStore> getTimestampedWindowStore(fin * The store can be a "regular" or global store. *

    * This is often useful in test cases to pre-populate the store before the test case instructs the topology to - * {@link #pipeInput(ConsumerRecord) process an input message}, and/or to check the store afterward. + * {@link TestInputTopic#pipeInput(TestRecord) process an input message}, and/or to check the store afterward. * * @param name the name of the store * @return the key value store, or {@code null} if no {@link SessionStore} has been registered with the given name diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java deleted file mode 100644 index 15164f68a4ebd..0000000000000 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/ConsumerRecordFactory.java +++ /dev/null @@ -1,583 +0,0 @@ -/* - * 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.streams.test; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.header.Headers; -import org.apache.kafka.common.header.internals.RecordHeaders; -import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.TestInputTopic; -import org.apache.kafka.streams.TopologyTestDriver; - -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import java.util.Optional; - -/** - * Factory to create {@link ConsumerRecord consumer records} for a single single-partitioned topic with given key and - * value {@link Serializer serializers}. - * - * @deprecated Since 2.4 use methods of {@link TestInputTopic} instead - * - * @param the type of the key - * @param the type of the value - * - * @see TopologyTestDriver - */ -@Deprecated -public class ConsumerRecordFactory { - private final String topicName; - private final Serializer keySerializer; - private final Serializer valueSerializer; - private long timeMs; - private final long advanceMs; - - /** - * Create a new factory for the given topic. - * Uses current system time as start timestamp. - * Auto-advance is disabled. - * - * @param keySerializer the key serializer - * @param valueSerializer the value serializer - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecordFactory(final Serializer keySerializer, - final Serializer valueSerializer) { - this(null, keySerializer, valueSerializer, System.currentTimeMillis()); - } - - /** - * Create a new factory for the given topic. - * Uses current system time as start timestamp. - * Auto-advance is disabled. - * - * @param defaultTopicName the default topic name used for all generated {@link ConsumerRecord consumer records} - * @param keySerializer the key serializer - * @param valueSerializer the value serializer - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecordFactory(final String defaultTopicName, - final Serializer keySerializer, - final Serializer valueSerializer) { - this(defaultTopicName, keySerializer, valueSerializer, System.currentTimeMillis()); - } - - /** - * Create a new factory for the given topic. - * Auto-advance is disabled. - * - * @param keySerializer the key serializer - * @param valueSerializer the value serializer - * @param startTimestampMs the initial timestamp for generated records - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecordFactory(final Serializer keySerializer, - final Serializer valueSerializer, - final long startTimestampMs) { - this(null, keySerializer, valueSerializer, startTimestampMs, 0L); - } - - /** - * Create a new factory for the given topic. - * Auto-advance is disabled. - * - * @param defaultTopicName the topic name used for all generated {@link ConsumerRecord consumer records} - * @param keySerializer the key serializer - * @param valueSerializer the value serializer - * @param startTimestampMs the initial timestamp for generated records - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecordFactory(final String defaultTopicName, - final Serializer keySerializer, - final Serializer valueSerializer, - final long startTimestampMs) { - this(defaultTopicName, keySerializer, valueSerializer, startTimestampMs, 0L); - } - - /** - * Create a new factory for the given topic. - * - * @param keySerializer the key serializer - * @param valueSerializer the value serializer - * @param startTimestampMs the initial timestamp for generated records - * @param autoAdvanceMs the time increment pre generated record - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecordFactory(final Serializer keySerializer, - final Serializer valueSerializer, - final long startTimestampMs, - final long autoAdvanceMs) { - this(null, keySerializer, valueSerializer, startTimestampMs, autoAdvanceMs); - } - - /** - * Create a new factory for the given topic. - * - * @param defaultTopicName the topic name used for all generated {@link ConsumerRecord consumer records} - * @param keySerializer the key serializer - * @param valueSerializer the value serializer - * @param startTimestampMs the initial timestamp for generated records - * @param autoAdvanceMs the time increment pre generated record - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecordFactory(final String defaultTopicName, - final Serializer keySerializer, - final Serializer valueSerializer, - final long startTimestampMs, - final long autoAdvanceMs) { - Objects.requireNonNull(keySerializer, "keySerializer cannot be null"); - Objects.requireNonNull(valueSerializer, "valueSerializer cannot be null"); - this.topicName = defaultTopicName; - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - timeMs = startTimestampMs; - advanceMs = autoAdvanceMs; - } - - /** - * Advances the internally tracked time. - * - * @param advanceMs the amount of time to advance - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public void advanceTimeMs(final long advanceMs) { - if (advanceMs < 0) { - throw new IllegalArgumentException("advanceMs must be positive"); - } - timeMs += advanceMs; - } - - /** - * Create a {@link ConsumerRecord} with the given topic name, key, value, headers, and timestamp. - * Does not auto advance internally tracked time. - * - * @param topicName the topic name - * @param key the record key - * @param value the record value - * @param headers the record headers - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final K key, - final V value, - final Headers headers, - final long timestampMs) { - Objects.requireNonNull(topicName, "topicName cannot be null."); - Objects.requireNonNull(headers, "headers cannot be null."); - final byte[] serializedKey = keySerializer.serialize(topicName, headers, key); - final byte[] serializedValue = valueSerializer.serialize(topicName, headers, value); - return new ConsumerRecord<>( - topicName, - -1, - -1L, - timestampMs, - TimestampType.CREATE_TIME, - serializedKey == null ? 0 : serializedKey.length, - serializedValue == null ? 0 : serializedValue.length, - serializedKey, - serializedValue, - headers, - Optional.empty()); - } - - /** - * Create a {@link ConsumerRecord} with the given topic name and given topic, key, value, and timestamp. - * Does not auto advance internally tracked time. - * - * @param topicName the topic name - * @param key the record key - * @param value the record value - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final K key, - final V value, - final long timestampMs) { - return create(topicName, key, value, new RecordHeaders(), timestampMs); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and given key, value, and timestamp. - * Does not auto advance internally tracked time. - * - * @param key the record key - * @param value the record value - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final K key, - final V value, - final long timestampMs) { - return create(key, value, new RecordHeaders(), timestampMs); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and given key, value, headers, and timestamp. - * Does not auto advance internally tracked time. - * - * @param key the record key - * @param value the record value - * @param headers the record headers - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final K key, - final V value, - final Headers headers, - final long timestampMs) { - if (topicName == null) { - throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + - "Use #create(String topicName, K key, V value, long timestampMs) instead."); - } - return create(topicName, key, value, headers, timestampMs); - } - - /** - * Create a {@link ConsumerRecord} with the given topic name, key, and value. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param topicName the topic name - * @param key the record key - * @param value the record value - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final K key, - final V value) { - final long timestamp = timeMs; - timeMs += advanceMs; - return create(topicName, key, value, new RecordHeaders(), timestamp); - } - - /** - * Create a {@link ConsumerRecord} with the given topic name, key, value, and headers. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param topicName the topic name - * @param key the record key - * @param value the record value - * @param headers the record headers - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final K key, - final V value, - final Headers headers) { - final long timestamp = timeMs; - timeMs += advanceMs; - return create(topicName, key, value, headers, timestamp); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and given key and value. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param key the record key - * @param value the record value - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final K key, - final V value) { - return create(key, value, new RecordHeaders()); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and given key, value, and headers. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param key the record key - * @param value the record value - * @param headers the record headers - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final K key, - final V value, - final Headers headers) { - if (topicName == null) { - throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + - "Use #create(String topicName, K key, V value) instead."); - } - return create(topicName, key, value, headers); - } - - /** - * Create a {@link ConsumerRecord} with {@code null}-key and the given topic name, value, and timestamp. - * Does not auto advance internally tracked time. - * - * @param topicName the topic name - * @param value the record value - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final V value, - final long timestampMs) { - return create(topicName, null, value, new RecordHeaders(), timestampMs); - } - - /** - * Create a {@link ConsumerRecord} with {@code null}-key and the given topic name, value, headers, and timestamp. - * Does not auto advance internally tracked time. - * - * @param topicName the topic name - * @param value the record value - * @param headers the record headers - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final V value, - final Headers headers, - final long timestampMs) { - return create(topicName, null, value, headers, timestampMs); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and {@code null}-key as well as given value and timestamp. - * Does not auto advance internally tracked time. - * - * @param value the record value - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final V value, - final long timestampMs) { - return create(value, new RecordHeaders(), timestampMs); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and {@code null}-key as well as given value, headers, and timestamp. - * Does not auto advance internally tracked time. - * - * @param value the record value - * @param headers the record headers - * @param timestampMs the record timestamp - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final V value, - final Headers headers, - final long timestampMs) { - if (topicName == null) { - throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + - "Use #create(String topicName, V value, long timestampMs) instead."); - } - return create(topicName, value, headers, timestampMs); - } - - /** - * Create a {@link ConsumerRecord} with {@code null}-key and the given topic name, value, and headers. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param topicName the topic name - * @param value the record value - * @param headers the record headers - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final V value, - final Headers headers) { - return create(topicName, null, value, headers); - } - - /** - * Create a {@link ConsumerRecord} with {@code null}-key and the given topic name and value. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param topicName the topic name - * @param value the record value - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final String topicName, - final V value) { - return create(topicName, null, value, new RecordHeaders()); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and {@code null}-key was well as given value. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param value the record value - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final V value) { - return create(value, new RecordHeaders()); - } - - /** - * Create a {@link ConsumerRecord} with default topic name and {@code null}-key was well as given value and headers. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param value the record value - * @param headers the record headers - * @return the generated {@link ConsumerRecord} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public ConsumerRecord create(final V value, - final Headers headers) { - if (topicName == null) { - throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + - "Use #create(String topicName, V value, long timestampMs) instead."); - } - return create(topicName, value, headers); - } - - /** - * Creates {@link ConsumerRecord consumer records} with the given topic name, keys, and values. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param topicName the topic name - * @param keyValues the record keys and values - * @return the generated {@link ConsumerRecord consumer records} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public List> create(final String topicName, - final List> keyValues) { - final List> records = new ArrayList<>(keyValues.size()); - - for (final KeyValue keyValue : keyValues) { - records.add(create(topicName, keyValue.key, keyValue.value)); - } - - return records; - } - - /** - * Creates {@link ConsumerRecord consumer records} with default topic name as well as given keys and values. - * The timestamp will be generated based on the constructor provided start time and time will auto advance. - * - * @param keyValues the record keys and values - * @return the generated {@link ConsumerRecord consumer records} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public List> create(final List> keyValues) { - if (topicName == null) { - throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + - "Use #create(String topicName, List> keyValues) instead."); - } - - return create(topicName, keyValues); - } - - /** - * Creates {@link ConsumerRecord consumer records} with the given topic name, keys, and values. - * Does not auto advance internally tracked time. - * - * @param topicName the topic name - * @param keyValues the record keys and values - * @param startTimestamp the timestamp for the first generated record - * @param advanceMs the time difference between two consecutive generated records - * @return the generated {@link ConsumerRecord consumer records} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public List> create(final String topicName, - final List> keyValues, - final long startTimestamp, - final long advanceMs) { - if (advanceMs < 0) { - throw new IllegalArgumentException("advanceMs must be positive"); - } - - final List> records = new ArrayList<>(keyValues.size()); - - long timestamp = startTimestamp; - for (final KeyValue keyValue : keyValues) { - records.add(create(topicName, keyValue.key, keyValue.value, new RecordHeaders(), timestamp)); - timestamp += advanceMs; - } - - return records; - } - - /** - * Creates {@link ConsumerRecord consumer records} with default topic name as well as given keys and values. - * Does not auto advance internally tracked time. - * - * @param keyValues the record keys and values - * @param startTimestamp the timestamp for the first generated record - * @param advanceMs the time difference between two consecutive generated records - * @return the generated {@link ConsumerRecord consumer records} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public List> create(final List> keyValues, - final long startTimestamp, - final long advanceMs) { - if (topicName == null) { - throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + - "Use #create(String topicName, List> keyValues, long startTimestamp, long advanceMs) instead."); - } - - return create(topicName, keyValues, startTimestamp, advanceMs); - } - - /** - * Creates {@link ConsumerRecord consumer records} with the given topic name, keys and values. - * For each generated record, the time is advanced by 1. - * Does not auto advance internally tracked time. - * - * @param topicName the topic name - * @param keyValues the record keys and values - * @param startTimestamp the timestamp for the first generated record - * @return the generated {@link ConsumerRecord consumer records} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public List> create(final String topicName, - final List> keyValues, - final long startTimestamp) { - return create(topicName, keyValues, startTimestamp, 1); - } - - /** - * Creates {@link ConsumerRecord consumer records} with the given keys and values. - * For each generated record, the time is advanced by 1. - * Does not auto advance internally tracked time. - * - * @param keyValues the record keys and values - * @param startTimestamp the timestamp for the first generated record - * @return the generated {@link ConsumerRecord consumer records} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public List> create(final List> keyValues, - final long startTimestamp) { - if (topicName == null) { - throw new IllegalStateException("ConsumerRecordFactory was created without defaultTopicName. " + - "Use #create(String topicName, List> keyValues, long startTimestamp) instead."); - } - - return create(topicName, keyValues, startTimestamp, 1); - } - -} diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java deleted file mode 100644 index af0f51cb4dd57..0000000000000 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/test/OutputVerifier.java +++ /dev/null @@ -1,458 +0,0 @@ -/* - * 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.streams.test; - -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.header.Headers; -import org.apache.kafka.streams.TestOutputTopic; -import org.apache.kafka.streams.TopologyTestDriver; - -import java.util.Objects; - -/** - * Helper class to verify topology result records. - * - * @deprecated Since 2.4 use methods of {@link TestOutputTopic} and standard assertion libraries instead - * - * @see TopologyTestDriver - */ -@Deprecated -public class OutputVerifier { - - /** - * Compares a {@link ProducerRecord} with the provided value and throws an {@link AssertionError} if the - * {@code ProducerRecord}'s value is not equal to the expected value. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedValue the expected value of the {@code ProducerRecord} - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s value is not equal to {@code expectedValue} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareValue(final ProducerRecord record, - final V expectedValue) throws AssertionError { - Objects.requireNonNull(record); - - final V recordValue = record.value(); - final AssertionError error = new AssertionError("Expected value=" + expectedValue + " but was value=" + recordValue); - - if (recordValue != null) { - if (!recordValue.equals(expectedValue)) { - throw error; - } - } else if (expectedValue != null) { - throw error; - } - } - - /** - * Compares the values of two {@link ProducerRecord}'s and throws an {@link AssertionError} if they are not equal to - * each other. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedRecord a {@code ProducerRecord} for verification - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s value is not equal to {@code expectedRecord}'s value - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareValue(final ProducerRecord record, - final ProducerRecord expectedRecord) throws AssertionError { - Objects.requireNonNull(expectedRecord); - compareValue(record, expectedRecord.value()); - } - - /** - * Compares a {@link ProducerRecord} with the provided key and value and throws an {@link AssertionError} if the - * {@code ProducerRecord}'s key or value is not equal to the expected key or value. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedKey the expected key of the {@code ProducerRecord} - * @param expectedValue the expected value of the {@code ProducerRecord} - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key or value is not equal to {@code expectedKey} or {@code expectedValue} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValue(final ProducerRecord record, - final K expectedKey, - final V expectedValue) throws AssertionError { - Objects.requireNonNull(record); - - final K recordKey = record.key(); - final V recordValue = record.value(); - final AssertionError error = new AssertionError("Expected <" + expectedKey + ", " + expectedValue + "> " + - "but was <" + recordKey + ", " + recordValue + ">"); - - if (recordKey != null) { - if (!recordKey.equals(expectedKey)) { - throw error; - } - } else if (expectedKey != null) { - throw error; - } - - if (recordValue != null) { - if (!recordValue.equals(expectedValue)) { - throw error; - } - } else if (expectedValue != null) { - throw error; - } - } - - /** - * Compares the keys and values of two {@link ProducerRecord}'s and throws an {@link AssertionError} if the keys or - * values are not equal to each other. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedRecord a {@code ProducerRecord} for verification - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key or value is not equal to {@code expectedRecord}'s key or value - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValue(final ProducerRecord record, - final ProducerRecord expectedRecord) throws AssertionError { - Objects.requireNonNull(expectedRecord); - compareKeyValue(record, expectedRecord.key(), expectedRecord.value()); - } - - /** - * Compares a {@link ProducerRecord} with the provided value and timestamp and throws an {@link AssertionError} if - * the {@code ProducerRecord}'s value or timestamp is not equal to the expected value or timestamp. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedValue the expected value of the {@code ProducerRecord} - * @param expectedTimestamp the expected timestamps of the {@code ProducerRecord} - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s value or timestamp is not equal to {@code expectedValue} or {@code expectedTimestamp} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareValueTimestamp(final ProducerRecord record, - final V expectedValue, - final long expectedTimestamp) throws AssertionError { - Objects.requireNonNull(record); - - final V recordValue = record.value(); - final long recordTimestamp = record.timestamp(); - final AssertionError error = new AssertionError("Expected value=" + expectedValue + " with timestamp=" + expectedTimestamp + - " but was value=" + recordValue + " with timestamp=" + recordTimestamp); - - if (recordValue != null) { - if (!recordValue.equals(expectedValue)) { - throw error; - } - } else if (expectedValue != null) { - throw error; - } - - if (recordTimestamp != expectedTimestamp) { - throw error; - } - } - - /** - * Compares the values and timestamps of two {@link ProducerRecord}'s and throws an {@link AssertionError} if the - * values or timestamps are not equal to each other. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedRecord a {@code ProducerRecord} for verification - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s value or timestamp is not equal to {@code expectedRecord}'s value or timestamp - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareValueTimestamp(final ProducerRecord record, - final ProducerRecord expectedRecord) throws AssertionError { - Objects.requireNonNull(expectedRecord); - compareValueTimestamp(record, expectedRecord.value(), expectedRecord.timestamp()); - } - - /** - * Compares a {@link ProducerRecord} with the provided key, value, and timestamp and throws an - * {@link AssertionError} if the {@code ProducerRecord}'s key, value, or timestamp is not equal to the expected key, - * value, or timestamp. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedKey the expected key of the {@code ProducerRecord} - * @param expectedValue the expected value of the {@code ProducerRecord} - * @param expectedTimestamp the expected timestamp of the {@code ProducerRecord} - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key, value, timestamp is not equal to {@code expectedKey}, - * {@code expectedValue}, or {@code expectedTimestamps} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValueTimestamp(final ProducerRecord record, - final K expectedKey, - final V expectedValue, - final long expectedTimestamp) throws AssertionError { - Objects.requireNonNull(record); - - final K recordKey = record.key(); - final V recordValue = record.value(); - final long recordTimestamp = record.timestamp(); - final AssertionError error = new AssertionError("Expected <" + expectedKey + ", " + expectedValue + "> with timestamp=" + expectedTimestamp + - " but was <" + recordKey + ", " + recordValue + "> with timestamp=" + recordTimestamp); - - if (recordKey != null) { - if (!recordKey.equals(expectedKey)) { - throw error; - } - } else if (expectedKey != null) { - throw error; - } - - if (recordValue != null) { - if (!recordValue.equals(expectedValue)) { - throw error; - } - } else if (expectedValue != null) { - throw error; - } - - if (recordTimestamp != expectedTimestamp) { - throw error; - } - } - - /** - * Compares the keys, values, and timestamps of two {@link ProducerRecord}'s and throws an {@link AssertionError} if - * the keys, values, or timestamps are not equal to each other. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedRecord a {@code ProducerRecord} for verification - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key, value, or timestamp is not equal to - * {@code expectedRecord}'s key, value, or timestamp - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValueTimestamp(final ProducerRecord record, - final ProducerRecord expectedRecord) throws AssertionError { - Objects.requireNonNull(expectedRecord); - compareKeyValueTimestamp(record, expectedRecord.key(), expectedRecord.value(), expectedRecord.timestamp()); - } - - /** - * Compares a {@link ProducerRecord} with the provided value and headers and throws an {@link AssertionError} if - * the {@code ProducerRecord}'s value or headers is not equal to the expected value or headers. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedValue the expected value of the {@code ProducerRecord} - * @param expectedHeaders the expected headers of the {@code ProducerRecord} - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s value or headers is not equal to {@code expectedValue} or {@code expectedHeaders} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareValueHeaders(final ProducerRecord record, - final V expectedValue, - final Headers expectedHeaders) throws AssertionError { - Objects.requireNonNull(record); - - final V recordValue = record.value(); - final Headers recordHeaders = record.headers(); - final AssertionError error = new AssertionError("Expected value=" + expectedValue + " with headers=" + expectedHeaders + - " but was value=" + recordValue + " with headers=" + recordHeaders); - - if (recordValue != null) { - if (!recordValue.equals(expectedValue)) { - throw error; - } - } else if (expectedValue != null) { - throw error; - } - - if (recordHeaders != null) { - if (!recordHeaders.equals(expectedHeaders)) { - throw error; - } - } else if (expectedHeaders != null) { - throw error; - } - } - - /** - * Compares the values and headers of two {@link ProducerRecord}'s and throws an {@link AssertionError} if the - * values or headers are not equal to each other. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedRecord a {@code ProducerRecord} for verification - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s value or headers is not equal to {@code expectedRecord}'s value or headers - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareValueHeaders(final ProducerRecord record, - final ProducerRecord expectedRecord) throws AssertionError { - Objects.requireNonNull(expectedRecord); - compareValueHeaders(record, expectedRecord.value(), expectedRecord.headers()); - } - - /** - * Compares a {@link ProducerRecord} with the provided key, value, and headers and throws an - * {@link AssertionError} if the {@code ProducerRecord}'s key, value, or headers is not equal to the expected key, - * value, or headers. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedKey the expected key of the {@code ProducerRecord} - * @param expectedValue the expected value of the {@code ProducerRecord} - * @param expectedHeaders the expected headers of the {@code ProducerRecord} - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key, value, headers is not equal to {@code expectedKey}, - * {@code expectedValue}, or {@code expectedHeaders} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValueHeaders(final ProducerRecord record, - final K expectedKey, - final V expectedValue, - final Headers expectedHeaders) throws AssertionError { - Objects.requireNonNull(record); - - final K recordKey = record.key(); - final V recordValue = record.value(); - final Headers recordHeaders = record.headers(); - final AssertionError error = new AssertionError("Expected <" + expectedKey + ", " + expectedValue + "> with headers=" + expectedHeaders + - " but was <" + recordKey + ", " + recordValue + "> with headers=" + recordHeaders); - - if (recordKey != null) { - if (!recordKey.equals(expectedKey)) { - throw error; - } - } else if (expectedKey != null) { - throw error; - } - - if (recordValue != null) { - if (!recordValue.equals(expectedValue)) { - throw error; - } - } else if (expectedValue != null) { - throw error; - } - - if (recordHeaders != null) { - if (!recordHeaders.equals(expectedHeaders)) { - throw error; - } - } else if (expectedHeaders != null) { - throw error; - } - } - - /** - * Compares the keys, values, and headers of two {@link ProducerRecord}'s and throws an {@link AssertionError} if - * the keys, values, or headers are not equal to each other. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedRecord a {@code ProducerRecord} for verification - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key, value, or headers is not equal to - * {@code expectedRecord}'s key, value, or headers - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValueHeaders(final ProducerRecord record, - final ProducerRecord expectedRecord) throws AssertionError { - Objects.requireNonNull(expectedRecord); - compareKeyValueHeaders(record, expectedRecord.key(), expectedRecord.value(), expectedRecord.headers()); - } - - /** - * Compares a {@link ProducerRecord} with the provided key, value, headers, and timestamp and throws an - * {@link AssertionError} if the {@code ProducerRecord}'s key, value, headers, or timestamp is not equal to the expected key, - * value, headers, or timestamp. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedKey the expected key of the {@code ProducerRecord} - * @param expectedValue the expected value of the {@code ProducerRecord} - * @param expectedHeaders the expected headers of the {@code ProducerRecord} - * @param expectedTimestamp the expected timestamp of the {@code ProducerRecord} - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key, value, headers is not equal to {@code expectedKey}, - * {@code expectedValue}, or {@code expectedHeaders} - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValueHeadersTimestamp(final ProducerRecord record, - final K expectedKey, - final V expectedValue, - final Headers expectedHeaders, - final long expectedTimestamp) throws AssertionError { - Objects.requireNonNull(record); - - final K recordKey = record.key(); - final V recordValue = record.value(); - final Headers recordHeaders = record.headers(); - final long recordTimestamp = record.timestamp(); - final AssertionError error = new AssertionError("Expected <" + expectedKey + ", " + expectedValue + ">" + - " with timestamp=" + expectedTimestamp + " and headers=" + expectedHeaders + - " but was <" + recordKey + ", " + recordValue + ">" + - " with timestamp=" + recordTimestamp + " and headers=" + recordHeaders); - - if (recordKey != null) { - if (!recordKey.equals(expectedKey)) { - throw error; - } - } else if (expectedKey != null) { - throw error; - } - - if (recordValue != null) { - if (!recordValue.equals(expectedValue)) { - throw error; - } - } else if (expectedValue != null) { - throw error; - } - - if (recordHeaders != null) { - if (!recordHeaders.equals(expectedHeaders)) { - throw error; - } - } else if (expectedHeaders != null) { - throw error; - } - - if (recordTimestamp != expectedTimestamp) { - throw error; - } - } - - /** - * Compares the keys, values, headers, and timestamp of two {@link ProducerRecord}'s and throws an {@link AssertionError} if - * the keys, values, headers, or timestamps are not equal to each other. - * - * @param record a output {@code ProducerRecord} for verification - * @param expectedRecord a {@code ProducerRecord} for verification - * @param the key type - * @param the value type - * @throws AssertionError if {@code ProducerRecord}'s key, value, headers, or timestamp is not equal to - * {@code expectedRecord}'s key, value, headers, or timestamp - */ - @SuppressWarnings({"WeakerAccess", "unused"}) - public static void compareKeyValueHeadersTimestamp(final ProducerRecord record, - final ProducerRecord expectedRecord) throws AssertionError { - Objects.requireNonNull(expectedRecord); - compareKeyValueHeadersTimestamp(record, expectedRecord.key(), expectedRecord.value(), expectedRecord.headers(), expectedRecord.timestamp()); - } -} diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java index d31dc5374d1a6..ad7bc169d8076 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Headers; @@ -90,7 +89,6 @@ 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; public abstract class TopologyTestDriverTest { @@ -118,17 +116,6 @@ public abstract class TopologyTestDriverTest { private final byte[] value2 = new byte[0]; private final long timestamp2 = 43L; - //Factory and records for testing already deprecated methods - @SuppressWarnings("deprecation") - private final org.apache.kafka.streams.test.ConsumerRecordFactory consumerRecordFactory = - new org.apache.kafka.streams.test.ConsumerRecordFactory<>( - new ByteArraySerializer(), - new ByteArraySerializer()); - private final ConsumerRecord consumerRecord1 = - consumerRecordFactory.create(SOURCE_TOPIC_1, key1, value1, headers, timestamp1); - private final ConsumerRecord consumerRecord2 = - consumerRecordFactory.create(SOURCE_TOPIC_2, key2, value2, timestamp2); - private TopologyTestDriver testDriver; private final Properties config; private KeyValueStore store; @@ -144,16 +131,6 @@ private final static class TTDTestRecord { private final String topic; private final Headers headers; - TTDTestRecord(final ConsumerRecord consumerRecord, - final long newOffset) { - key = consumerRecord.key(); - value = consumerRecord.value(); - timestamp = consumerRecord.timestamp(); - offset = newOffset; - topic = consumerRecord.topic(); - headers = consumerRecord.headers(); - } - TTDTestRecord(final String newTopic, final TestRecord consumerRecord, final long newOffset) { @@ -181,7 +158,12 @@ private final static class TTDTestRecord { @Override public String toString() { - return "key: " + key + ", value: " + value + ", timestamp: " + timestamp + ", offset: " + offset + ", topic: " + topic; + return "key: " + key + + ", value: " + value + + ", timestamp: " + timestamp + + ", offset: " + offset + + ", topic: " + topic + + ", num.headers: " + (headers == null ? "null" : headers.toArray().length); } @Override @@ -481,25 +463,6 @@ public void shouldThrowForMissingTime() { null)); } - @Deprecated - @Test - public void shouldThrowForUnknownTopicDeprecated() { - final String unknownTopic = "unknownTopic"; - final org.apache.kafka.streams.test.ConsumerRecordFactory consumerRecordFactory = - new org.apache.kafka.streams.test.ConsumerRecordFactory<>( - "unknownTopic", - new ByteArraySerializer(), - new ByteArraySerializer()); - - testDriver = new TopologyTestDriver(new Topology()); - try { - testDriver.pipeInput(consumerRecordFactory.create((byte[]) null)); - fail("Should have throw IllegalArgumentException"); - } catch (final IllegalArgumentException exception) { - assertEquals("Unknown topic: " + unknownTopic, exception.getMessage()); - } - } - @Test public void shouldThrowNoSuchElementExceptionForUnusedOutputTopicWithDynamicRouting() { testDriver = new TopologyTestDriver(setupSourceSinkTopology()); @@ -605,97 +568,37 @@ private void pipeRecord(final String topic, final TestRecord rec } - @Deprecated - //Test not migrated to non-deprecated methods, topic handling not based on record any more @Test - public void shouldSendRecordViaCorrectSourceTopicDeprecated() { + public void shouldSendRecordViaCorrectSourceTopic() { testDriver = new TopologyTestDriver(setupMultipleSourceTopology(SOURCE_TOPIC_1, SOURCE_TOPIC_2)); final List processedRecords1 = mockProcessors.get(0).processedRecords; final List processedRecords2 = mockProcessors.get(1).processedRecords; - testDriver.pipeInput(consumerRecord1); + final TestInputTopic inputTopic1 = testDriver.createInputTopic(SOURCE_TOPIC_1, + new ByteArraySerializer(), new ByteArraySerializer()); + final TestInputTopic inputTopic2 = testDriver.createInputTopic(SOURCE_TOPIC_2, + new ByteArraySerializer(), new ByteArraySerializer()); + + inputTopic1.pipeInput(new TestRecord<>(key1, value1, headers, timestamp1)); assertEquals(1, processedRecords1.size()); assertEquals(0, processedRecords2.size()); TTDTestRecord record = processedRecords1.get(0); - TTDTestRecord expectedResult = new TTDTestRecord(consumerRecord1, 0L); + TTDTestRecord expectedResult = new TTDTestRecord(key1, value1, headers, timestamp1, 0L, SOURCE_TOPIC_1); assertThat(record, equalTo(expectedResult)); - testDriver.pipeInput(consumerRecord2); + inputTopic2.pipeInput(new TestRecord<>(key2, value2, Instant.ofEpochMilli(timestamp2))); assertEquals(1, processedRecords1.size()); assertEquals(1, processedRecords2.size()); record = processedRecords2.get(0); - expectedResult = new TTDTestRecord(consumerRecord2, 0L); + expectedResult = new TTDTestRecord(key2, value2, new RecordHeaders((Iterable

    ) null), timestamp2, 0L, SOURCE_TOPIC_2); assertThat(record, equalTo(expectedResult)); } - @Deprecated - @Test - public void shouldUseSourceSpecificDeserializersDeprecated() { - final Topology topology = new Topology(); - - final String sourceName1 = "source-1"; - final String sourceName2 = "source-2"; - final String processor = "processor"; - - topology.addSource(sourceName1, Serdes.Long().deserializer(), Serdes.String().deserializer(), SOURCE_TOPIC_1); - topology.addSource(sourceName2, Serdes.Integer().deserializer(), Serdes.Double().deserializer(), SOURCE_TOPIC_2); - topology.addProcessor(processor, new MockProcessorSupplier(), sourceName1, sourceName2); - topology.addSink( - "sink", - SINK_TOPIC_1, - (topic, data) -> { - if (data instanceof Long) { - return Serdes.Long().serializer().serialize(topic, (Long) data); - } - return Serdes.Integer().serializer().serialize(topic, (Integer) data); - }, - (topic, data) -> { - if (data instanceof String) { - return Serdes.String().serializer().serialize(topic, (String) data); - } - return Serdes.Double().serializer().serialize(topic, (Double) data); - }, - processor); - - testDriver = new TopologyTestDriver(topology); - - final org.apache.kafka.streams.test.ConsumerRecordFactory source1Factory = - new org.apache.kafka.streams.test.ConsumerRecordFactory<>( - SOURCE_TOPIC_1, - Serdes.Long().serializer(), - Serdes.String().serializer()); - final org.apache.kafka.streams.test.ConsumerRecordFactory source2Factory = - new org.apache.kafka.streams.test.ConsumerRecordFactory<>( - SOURCE_TOPIC_2, - Serdes.Integer().serializer(), - Serdes.Double().serializer()); - - final Long source1Key = 42L; - final String source1Value = "anyString"; - final Integer source2Key = 73; - final Double source2Value = 3.14; - - final ConsumerRecord consumerRecord1 = source1Factory.create(source1Key, source1Value); - final ConsumerRecord consumerRecord2 = source2Factory.create(source2Key, source2Value); - - testDriver.pipeInput(consumerRecord1); - final ProducerRecord record1 = - testDriver.readOutput(SINK_TOPIC_1, Serdes.Long().deserializer(), Serdes.String().deserializer()); - assertThat(record1.key(), equalTo(source1Key)); - assertThat(record1.value(), equalTo(source1Value)); - - testDriver.pipeInput(consumerRecord2); - final ProducerRecord record2 = - testDriver.readOutput(SINK_TOPIC_1, Serdes.Integer().deserializer(), Serdes.Double().deserializer()); - assertThat(record2.key(), equalTo(source2Key)); - assertThat(record2.value(), equalTo(source2Value)); - } - @Test public void shouldUseSourceSpecificDeserializers() { final Topology topology = new Topology(); @@ -848,33 +751,6 @@ public void shouldUseSinkSpecificSerializers() { assertThat(result2.getValue(), equalTo(source2Value)); } - @Deprecated - //Test not migrated to non-deprecated methods, List processing now in TestInputTopic - @Test - public void shouldProcessConsumerRecordList() { - testDriver = new TopologyTestDriver(setupMultipleSourceTopology(SOURCE_TOPIC_1, SOURCE_TOPIC_2)); - - final List processedRecords1 = mockProcessors.get(0).processedRecords; - final List processedRecords2 = mockProcessors.get(1).processedRecords; - - final List> testRecords = new ArrayList<>(2); - testRecords.add(consumerRecord1); - testRecords.add(consumerRecord2); - - testDriver.pipeInput(testRecords); - - assertEquals(1, processedRecords1.size()); - assertEquals(1, processedRecords2.size()); - - TTDTestRecord record = processedRecords1.get(0); - TTDTestRecord expectedResult = new TTDTestRecord(consumerRecord1, 0L); - assertThat(record, equalTo(expectedResult)); - - record = processedRecords2.get(0); - expectedResult = new TTDTestRecord(consumerRecord2, 0L); - assertThat(record, equalTo(expectedResult)); - } - @Test public void shouldForwardRecordsFromSubtopologyToSubtopology() { testDriver = new TopologyTestDriver(setupTopologyWithTwoSubtopologies()); @@ -954,36 +830,6 @@ public void shouldPunctuateOnStreamsTime() { assertThat(mockPunctuator.punctuatedAt, equalTo(expectedPunctuations)); } - @SuppressWarnings("deprecation") - //Testing already deprecated methods until methods removed - @Test - public void shouldPunctuateOnWallClockTimeDeprecated() { - final MockPunctuator mockPunctuator = new MockPunctuator(); - testDriver = new TopologyTestDriver( - setupSingleProcessorTopology(10L, PunctuationType.WALL_CLOCK_TIME, mockPunctuator), - config, - 0); - - final List expectedPunctuations = new LinkedList<>(); - testDriver.advanceWallClockTime(5L); - assertThat(mockPunctuator.punctuatedAt, equalTo(expectedPunctuations)); - - expectedPunctuations.add(14L); - testDriver.advanceWallClockTime(9L); - assertThat(mockPunctuator.punctuatedAt, equalTo(expectedPunctuations)); - - testDriver.advanceWallClockTime(1L); - assertThat(mockPunctuator.punctuatedAt, equalTo(expectedPunctuations)); - - expectedPunctuations.add(35L); - testDriver.advanceWallClockTime(20L); - assertThat(mockPunctuator.punctuatedAt, equalTo(expectedPunctuations)); - - expectedPunctuations.add(40L); - testDriver.advanceWallClockTime(5L); - assertThat(mockPunctuator.punctuatedAt, equalTo(expectedPunctuations)); - } - @Test public void shouldPunctuateOnWallClockTime() { final MockPunctuator mockPunctuator = new MockPunctuator(); @@ -1461,7 +1307,6 @@ private static class CustomMaxAggregator implements Processor context; private KeyValueStore store; - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { this.context = context; @@ -1517,7 +1362,6 @@ public Processor get() { return new Processor() { private KeyValueStore store; - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { this.store = context.getStateStore("storeProcessorStore"); @@ -1753,7 +1597,6 @@ public void shouldApplyGlobalUpdatesCorrectlyInRecursiveTopologies() { () -> new Processor() { private KeyValueStore stateStore; - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context) { stateStore = context.getStateStore("global-store"); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/ConsumerRecordFactoryTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/ConsumerRecordFactoryTest.java deleted file mode 100644 index deccdc931acc4..0000000000000 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/ConsumerRecordFactoryTest.java +++ /dev/null @@ -1,273 +0,0 @@ -/* - * 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.streams.test; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.KeyValue; -import org.junit.jupiter.api.Test; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -import static org.junit.jupiter.api.Assertions.assertArrayEquals; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertThrows; - -@Deprecated -public class ConsumerRecordFactoryTest { - private final StringSerializer stringSerializer = new StringSerializer(); - private final IntegerSerializer integerSerializer = new IntegerSerializer(); - - private final String topicName = "topic"; - private final String otherTopicName = "otherTopic"; - private final String key = "key"; - private final Integer value = 42; - private final long timestamp = 21L; - private final byte[] rawKey = stringSerializer.serialize(topicName, key); - private final byte[] rawValue = integerSerializer.serialize(topicName, value); - - private final ConsumerRecordFactory factory = - new ConsumerRecordFactory<>(topicName, new ByteArraySerializer(), integerSerializer, 0L); - - private final ConsumerRecordFactory defaultFactory = - new ConsumerRecordFactory<>(new ByteArraySerializer(), integerSerializer); - - @Test - public void shouldAdvanceTime() { - factory.advanceTimeMs(3L); - verifyRecord(topicName, rawKey, rawValue, 3L, factory.create(topicName, rawKey, value)); - - factory.advanceTimeMs(2L); - verifyRecord(topicName, rawKey, rawValue, 5L, factory.create(topicName, rawKey, value)); - } - - @Test - public void shouldNotAllowToCreateTopicWithNullTopicName() { - assertThrows(NullPointerException.class, () -> factory.create(null, rawKey, value, timestamp)); - } - - @Test - public void shouldNotAllowToCreateTopicWithNullHeaders() { - assertThrows(NullPointerException.class, () -> factory.create(topicName, rawKey, value, null, timestamp)); - } - - @Test - public void shouldNotAllowToCreateTopicWithNullTopicNameWithDefaultTimestamp() { - assertThrows(NullPointerException.class, () -> factory.create(null, rawKey, value)); - } - - @Test - public void shouldNotAllowToCreateTopicWithNullTopicNameWithNullKey() { - assertThrows(NullPointerException.class, () -> factory.create((String) null, value, timestamp)); - } - - @Test - public void shouldNotAllowToCreateTopicWithNullTopicNameWithNullKeyAndDefaultTimestamp() { - assertThrows(NullPointerException.class, () -> factory.create((String) null, value)); - } - - @Test - public void shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairs() { - assertThrows(NullPointerException.class, () -> factory.create(null, Collections.singletonList(KeyValue.pair(rawKey, value)))); - } - - @Test - public void shouldNotAllowToCreateTopicWithNullTopicNameWithKeyValuePairsAndCustomTimestamps() { - assertThrows(NullPointerException.class, - () -> factory.create(null, Collections.singletonList(KeyValue.pair(rawKey, value)), timestamp, 2L)); - } - - @Test - public void shouldRequireCustomTopicNameIfNotDefaultFactoryTopicName() { - assertThrows(IllegalStateException.class, () -> defaultFactory.create(rawKey, value, timestamp)); - } - - @Test - public void shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithDefaultTimestamp() { - assertThrows(IllegalStateException.class, () -> defaultFactory.create(rawKey, value)); - } - - @Test - public void shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKey() { - assertThrows(IllegalStateException.class, () -> defaultFactory.create(value, timestamp)); - } - - @Test - public void shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithNullKeyAndDefaultTimestamp() { - assertThrows(IllegalStateException.class, () -> defaultFactory.create(value)); - } - - @Test - public void shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairs() { - assertThrows(IllegalStateException.class, () -> defaultFactory.create(Collections.singletonList(KeyValue.pair(rawKey, value)))); - } - - @Test - public void shouldRequireCustomTopicNameIfNotDefaultFactoryTopicNameWithKeyValuePairsAndCustomTimestamps() { - assertThrows(IllegalStateException.class, () -> defaultFactory.create(Collections.singletonList(KeyValue.pair(rawKey, value)), timestamp, 2L)); - } - - @Test - public void shouldCreateConsumerRecordWithOtherTopicNameAndTimestamp() { - verifyRecord(otherTopicName, rawKey, rawValue, timestamp, factory.create(otherTopicName, rawKey, value, timestamp)); - } - - @Test - public void shouldCreateConsumerRecordWithTimestamp() { - verifyRecord(topicName, rawKey, rawValue, timestamp, factory.create(rawKey, value, timestamp)); - } - - @Test - public void shouldCreateConsumerRecordWithOtherTopicName() { - verifyRecord(otherTopicName, rawKey, rawValue, 0L, factory.create(otherTopicName, rawKey, value)); - - factory.advanceTimeMs(3L); - verifyRecord(otherTopicName, rawKey, rawValue, 3L, factory.create(otherTopicName, rawKey, value)); - } - - @Test - public void shouldCreateConsumerRecord() { - verifyRecord(topicName, rawKey, rawValue, 0L, factory.create(rawKey, value)); - - factory.advanceTimeMs(3L); - verifyRecord(topicName, rawKey, rawValue, 3L, factory.create(rawKey, value)); - } - - @Test - public void shouldCreateNullKeyConsumerRecordWithOtherTopicNameAndTimestampWithTimetamp() { - verifyRecord(topicName, null, rawValue, timestamp, factory.create(value, timestamp)); - } - - @Test - public void shouldCreateNullKeyConsumerRecordWithTimestampWithTimestamp() { - verifyRecord(topicName, null, rawValue, timestamp, factory.create(value, timestamp)); - } - - @Test - public void shouldCreateNullKeyConsumerRecord() { - verifyRecord(topicName, null, rawValue, 0L, factory.create(value)); - - factory.advanceTimeMs(3L); - verifyRecord(topicName, null, rawValue, 3L, factory.create(value)); - } - - @SuppressWarnings("unchecked") - @Test - public void shouldCreateConsumerRecordsFromKeyValuePairs() { - final ConsumerRecordFactory factory = - new ConsumerRecordFactory<>(topicName, stringSerializer, integerSerializer, 0L); - - final KeyValue[] keyValuePairs = new KeyValue[5]; - final KeyValue[] rawKeyValuePairs = new KeyValue[keyValuePairs.length]; - - for (int i = 0; i < keyValuePairs.length; ++i) { - keyValuePairs[i] = KeyValue.pair(key + "-" + i, value + i); - rawKeyValuePairs[i] = KeyValue.pair( - stringSerializer.serialize(topicName, key + "-" + i), - integerSerializer.serialize(topicName, value + i)); - } - - final List> records = - factory.create(Arrays.>asList(keyValuePairs)); - - for (int i = 0; i < keyValuePairs.length; ++i) { - verifyRecord( - topicName, - (byte[]) rawKeyValuePairs[i].key, - (byte[]) rawKeyValuePairs[i].value, - 0L, - records.get(i)); - } - } - - @SuppressWarnings("unchecked") - @Test - public void shouldCreateConsumerRecordsFromKeyValuePairsWithTimestampAndIncrements() { - final ConsumerRecordFactory factory = - new ConsumerRecordFactory<>(topicName, stringSerializer, integerSerializer, timestamp, 2L); - - final KeyValue[] keyValuePairs = new KeyValue[5]; - final KeyValue[] rawKeyValuePairs = new KeyValue[keyValuePairs.length]; - - for (int i = 0; i < keyValuePairs.length; ++i) { - keyValuePairs[i] = KeyValue.pair(key + "-" + i, value + i); - rawKeyValuePairs[i] = KeyValue.pair( - stringSerializer.serialize(topicName, key + "-" + i), - integerSerializer.serialize(topicName, value + i)); - } - - final List> records = - factory.create(Arrays.>asList(keyValuePairs)); - - for (int i = 0; i < keyValuePairs.length; ++i) { - verifyRecord( - topicName, - (byte[]) rawKeyValuePairs[i].key, - (byte[]) rawKeyValuePairs[i].value, - timestamp + 2L * i, - records.get(i)); - } - } - - @SuppressWarnings("unchecked") - @Test - public void shouldCreateConsumerRecordsFromKeyValuePairsWithCustomTimestampAndIncrementsAndNotAdvanceTime() { - final ConsumerRecordFactory factory = - new ConsumerRecordFactory<>(topicName, stringSerializer, integerSerializer, 0L); - - final KeyValue[] keyValuePairs = new KeyValue[5]; - final KeyValue[] rawKeyValuePairs = new KeyValue[keyValuePairs.length]; - - for (int i = 0; i < keyValuePairs.length; ++i) { - keyValuePairs[i] = KeyValue.pair(key + "-" + i, value + i); - rawKeyValuePairs[i] = KeyValue.pair( - stringSerializer.serialize(topicName, key + "-" + i), - integerSerializer.serialize(topicName, value + i)); - } - - final List> records = - factory.create(Arrays.>asList(keyValuePairs), timestamp, 2L); - - for (int i = 0; i < keyValuePairs.length; ++i) { - verifyRecord( - topicName, - (byte[]) rawKeyValuePairs[i].key, - (byte[]) rawKeyValuePairs[i].value, - timestamp + 2L * i, - records.get(i)); - } - - // should not have incremented internally tracked time - verifyRecord(topicName, null, rawValue, 0L, factory.create(value)); - } - - private void verifyRecord(final String topicName, - final byte[] rawKey, - final byte[] rawValue, - final long timestamp, - final ConsumerRecord record) { - assertEquals(topicName, record.topic()); - assertArrayEquals(rawKey, record.key()); - assertArrayEquals(rawValue, record.value()); - assertEquals(timestamp, record.timestamp()); - } - -} diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/OutputVerifierTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/OutputVerifierTest.java deleted file mode 100644 index 6aebe4fa55e07..0000000000000 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/OutputVerifierTest.java +++ /dev/null @@ -1,452 +0,0 @@ -/* - * 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.streams.test; - -import org.apache.kafka.clients.producer.ProducerRecord; - -import org.junit.jupiter.api.Test; - -import static org.junit.jupiter.api.Assertions.assertThrows; - -@Deprecated -public class OutputVerifierTest { - private final byte[] key = new byte[0]; - private final byte[] value = new byte[0]; - - private final ProducerRecord producerRecord = new ProducerRecord<>( - "someTopic", - Integer.MAX_VALUE, - Long.MAX_VALUE, - key, - value - ); - - private final ProducerRecord nullKeyValueRecord = new ProducerRecord( - "someTopic", - Integer.MAX_VALUE, - Long.MAX_VALUE, - null, - null - ); - - @Test - public void shouldNotAllowNullProducerRecordForCompareValue() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareValue(null, value)); - } - - @Test - public void shouldNotAllowNullProducerRecordWithExpectedRecordForCompareValue() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareValue((ProducerRecord) null, producerRecord)); - } - - @Test - public void shouldNotAllowNullExpectedRecordForCompareValue() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareValue(producerRecord, (ProducerRecord) null)); - } - - @Test - public void shouldNotAllowNullProducerRecordForCompareKeyValue() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareKeyValue(null, key, value)); - } - - @Test - public void shouldNotAllowNullProducerRecordWithExpectedRecordForCompareKeyValue() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareKeyValue(null, producerRecord)); - } - - @Test - public void shouldNotAllowNullExpectedRecordForCompareKeyValue() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareKeyValue(producerRecord, null)); - } - - @Test - public void shouldNotAllowNullProducerRecordForCompareValueTimestamp() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareValueTimestamp(null, value, 0L)); - } - - @Test - public void shouldNotAllowNullProducerRecordWithExpectedRecordForCompareValueTimestamp() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareValueTimestamp(null, producerRecord)); - } - - @Test - public void shouldNotAllowNullExpectedRecordForCompareValueTimestamp() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareValueTimestamp(producerRecord, null)); - } - - @Test - public void shouldNotAllowNullProducerRecordForCompareKeyValueTimestamp() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareKeyValueTimestamp(null, key, value, 0L)); - } - - @Test - public void shouldNotAllowNullProducerRecordWithExpectedRecordForCompareKeyValueTimestamp() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareKeyValueTimestamp(null, producerRecord)); - } - - @Test - public void shouldNotAllowNullExpectedRecordForCompareKeyValueTimestamp() { - assertThrows(NullPointerException.class, () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, null)); - } - - @Test - public void shouldPassIfValueIsEqualForCompareValue() { - OutputVerifier.compareValue(producerRecord, value); - } - - @Test - public void shouldPassIfValueIsEqualWithNullForCompareValue() { - OutputVerifier.compareValue(nullKeyValueRecord, (byte[]) null); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValue(producerRecord, key)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValue(producerRecord, (byte[]) null)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReverseForCompareValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValue(nullKeyValueRecord, value)); - } - - @Test - public void shouldPassIfValueIsEqualForCompareValueWithProducerRecord() { - OutputVerifier.compareValue(producerRecord, new ProducerRecord<>( - "otherTopic", - 0, - 0L, - value, - value - )); - } - - @Test - public void shouldPassIfValueIsEqualWithNullForCompareValueWithProducerRecord() { - OutputVerifier.compareValue(nullKeyValueRecord, new ProducerRecord( - "otherTopic", - 0, - 0L, - value, - null - )); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValue(producerRecord, - new ProducerRecord<>("sameTopic", Integer.MAX_VALUE, Long.MAX_VALUE, value, key))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValue(producerRecord, - new ProducerRecord("sameTopic", Integer.MAX_VALUE, Long.MAX_VALUE, value, null))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReverseForCompareValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValue(nullKeyValueRecord, - new ProducerRecord<>("sameTopic", Integer.MAX_VALUE, Long.MAX_VALUE, value, value))); - } - - @Test - public void shouldPassIfKeyAndValueIsEqualForCompareKeyValue() { - OutputVerifier.compareKeyValue(producerRecord, key, value); - } - - @Test - public void shouldPassIfKeyAndValueIsEqualWithNullForCompareKeyValue() { - OutputVerifier.compareKeyValue(nullKeyValueRecord, null, null); - } - - @Test - public void shouldFailIfKeyIsDifferentForCompareKeyValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, value, value)); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullForCompareKeyValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, null, value)); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullReversForCompareKeyValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue( - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, null, value), key, value)); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareKeyValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, key, key)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareKeyValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, key, null)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReversForCompareKeyValue() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue( - new ProducerRecord("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, null), key, value)); - } - - @Test - public void shouldPassIfKeyAndValueIsEqualForCompareKeyValueWithProducerRecord() { - OutputVerifier.compareKeyValue(producerRecord, new ProducerRecord<>( - "otherTopic", - 0, - 0L, - key, - value)); - } - - @Test - public void shouldPassIfKeyAndValueIsEqualWithNullForCompareKeyValueWithProducerRecord() { - OutputVerifier.compareKeyValue(nullKeyValueRecord, new ProducerRecord( - "otherTopic", - 0, - 0L, - null, - null)); - } - - @Test - public void shouldFailIfKeyIsDifferentForCompareKeyValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, value, value))); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullForCompareKeyValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, null, value))); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullReversForCompareKeyValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue( - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, null, value), producerRecord)); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareKeyValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, key))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareKeyValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue(producerRecord, - new ProducerRecord("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, null))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReversForCompareKeyValueWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValue( - new ProducerRecord("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, null), - producerRecord)); - } - - @Test - public void shouldPassIfValueAndTimestampIsEqualForCompareValueTimestamp() { - OutputVerifier.compareValueTimestamp(producerRecord, value, Long.MAX_VALUE); - } - - @Test - public void shouldPassIfValueAndTimestampIsEqualWithNullForCompareValueTimestamp() { - OutputVerifier.compareValueTimestamp(nullKeyValueRecord, null, Long.MAX_VALUE); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareValueTimestamp(producerRecord, key, Long.MAX_VALUE)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareValueTimestamp(producerRecord, null, Long.MAX_VALUE)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReverseForCompareValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareValueTimestamp(nullKeyValueRecord, value, Long.MAX_VALUE)); - } - - @Test - public void shouldPassIfValueAndTimestampIsEqualForCompareValueTimestampWithProducerRecord() { - OutputVerifier.compareValueTimestamp(producerRecord, new ProducerRecord<>( - "otherTopic", - 0, - Long.MAX_VALUE, - value, - value - )); - } - - @Test - public void shouldPassIfValueAndTimestampIsEqualWithNullForCompareValueTimestampWithProducerRecord() { - OutputVerifier.compareValueTimestamp(nullKeyValueRecord, new ProducerRecord( - "otherTopic", - 0, - Long.MAX_VALUE, - value, - null - )); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValueTimestamp(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, key))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValueTimestamp(producerRecord, - new ProducerRecord("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, null))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReverseForCompareValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValueTimestamp(nullKeyValueRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, value))); - } - - @Test - public void shouldFailIfTimestampIsDifferentForCompareValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareValueTimestamp(producerRecord, value, 0)); - } - - @Test - public void shouldFailIfTimestampDifferentWithNullReverseForCompareValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareValueTimestamp(nullKeyValueRecord, null, 0)); - } - - @Test - public void shouldFailIfTimestampIsDifferentForCompareValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareValueTimestamp(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, 0L, key, value))); - } - - @Test - public void shouldPassIfKeyAndValueAndTimestampIsEqualForCompareKeyValueTimestamp() { - OutputVerifier.compareKeyValueTimestamp(producerRecord, key, value, Long.MAX_VALUE); - } - - @Test - public void shouldPassIfKeyAndValueIsEqualWithNullForCompareKeyValueTimestamp() { - OutputVerifier.compareKeyValueTimestamp(nullKeyValueRecord, null, null, Long.MAX_VALUE); - } - - @Test - public void shouldFailIfKeyIsDifferentForCompareKeyValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, value, value, Long.MAX_VALUE)); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullForCompareKeyValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, null, value, Long.MAX_VALUE)); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullReversForCompareKeyValueTimestamp() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp( - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, null, value), - key, value, Long.MAX_VALUE)); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareKeyValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, key, key, Long.MAX_VALUE)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareKeyValueTimestamp() { - assertThrows(AssertionError.class, - () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, key, null, Long.MAX_VALUE)); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReversForCompareKeyValueTimestamp() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp( - new ProducerRecord("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, null), - key, value, Long.MAX_VALUE)); - } - - @Test - public void shouldPassIfKeyAndValueAndTimestampIsEqualForCompareKeyValueTimestampWithProducerRecord() { - OutputVerifier.compareKeyValueTimestamp(producerRecord, new ProducerRecord<>( - "otherTopic", 0, Long.MAX_VALUE, key, value)); - } - - @Test - public void shouldPassIfKeyAndValueAndTimestampIsEqualWithNullForCompareKeyValueTimestampWithProducerRecord() { - OutputVerifier.compareKeyValueTimestamp(nullKeyValueRecord, new ProducerRecord( - "otherTopic", 0, Long.MAX_VALUE, null, null)); - } - - @Test - public void shouldFailIfKeyIsDifferentForCompareKeyValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, value, value))); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullForCompareKeyValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, null, value))); - } - - @Test - public void shouldFailIfKeyIsDifferentWithNullReversForCompareKeyValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp( - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, null, value), producerRecord)); - } - - @Test - public void shouldFailIfValueIsDifferentForCompareKeyValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, - new ProducerRecord<>("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, key))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullForCompareKeyValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp(producerRecord, - new ProducerRecord("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, null))); - } - - @Test - public void shouldFailIfValueIsDifferentWithNullReversForCompareKeyValueTimestampWithProducerRecord() { - assertThrows(AssertionError.class, () -> OutputVerifier.compareKeyValueTimestamp( - new ProducerRecord("someTopic", Integer.MAX_VALUE, Long.MAX_VALUE, key, null), producerRecord)); - } - -} From 15c24da888bc0a2582fd84d89fa1b84354fede64 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Sergio=20Pe=C3=B1a?= Date: Sun, 18 Apr 2021 13:18:09 -0500 Subject: [PATCH 069/155] KAFKA-10847: Delete Time-ordered duplicated records using deleteRange() internally (#10537) This PR changes the TimeOrderedKeySchema composite key from time-seq-key -> time-key-seq to allow deletion of duplicated time-key records using the RocksDB deleteRange API. It also removes all duplicates when put(key, null) is called. Currently, the put(key, null) was a no-op, which was causing problems because there was no way to delete any keys when duplicates are allowed. The RocksDB deleteRange(keyFrom, keyTo) deletes a range of keys from keyFrom (inclusive) to keyTo (exclusive). To make keyTo inclusive, I incremented the end key by one when calling the RocksDBAccessor. Reviewers: Guozhang Wang --- .../AbstractRocksDBSegmentedBytesStore.java | 9 + .../state/internals/KeyValueSegment.java | 6 + .../streams/state/internals/RocksDBStore.java | 26 +++ .../RocksDBTimeOrderedWindowStore.java | 29 +-- .../internals/RocksDBTimestampedStore.java | 16 ++ .../streams/state/internals/Segment.java | 1 + .../state/internals/SegmentedBytesStore.java | 20 ++ .../state/internals/TimeOrderedKeySchema.java | 106 ++++----- .../state/internals/TimestampedSegment.java | 6 + ...bstractRocksDBSegmentedBytesStoreTest.java | 2 +- .../RocksDBTimeOrderedWindowStoreTest.java | 205 ++++++++++++++++-- .../internals/TimeOrderedKeySchemaTest.java | 93 -------- 12 files changed, 325 insertions(+), 194 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java index 4c545a5cdf874..51fb3ca2fd7b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStore.java @@ -196,6 +196,15 @@ public void remove(final Bytes key) { segment.delete(key); } + @Override + public void remove(final Bytes key, final long timestamp) { + final Bytes keyBytes = keySchema.toStoreBinaryKeyPrefix(key, timestamp); + final S segment = segments.getSegmentForTimestamp(timestamp); + if (segment != null) { + segment.deleteRange(keyBytes, keyBytes); + } + } + @Override public void put(final Bytes key, final byte[] value) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegment.java index b6d65049f4d4e..66c55fc9d92cb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegment.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyValueSegment.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; @@ -40,6 +41,11 @@ public void destroy() throws IOException { Utils.delete(dbDir); } + @Override + public synchronized void deleteRange(final Bytes keyFrom, final Bytes keyTo) { + super.deleteRange(keyFrom, keyTo); + } + @Override public int compareTo(final KeyValueSegment segment) { return Long.compare(id, segment.id); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 12ba4eb5c3024..c03f86d96715d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -349,6 +349,16 @@ public synchronized byte[] delete(final Bytes key) { return oldValue; } + void deleteRange(final Bytes keyFrom, final Bytes keyTo) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); + + validateStoreOpen(); + + // End of key is exclusive, so we increment it by 1 byte to make keyTo inclusive + dbAccessor.deleteRange(keyFrom.get(), Bytes.increment(keyTo).get()); + } + @Override public synchronized KeyValueIterator range(final Bytes from, final Bytes to) { @@ -524,6 +534,12 @@ KeyValueIterator range(final Bytes from, final Bytes to, final boolean forward); + /** + * Deletes keys entries in the range ['from', 'to'], including 'from' and excluding 'to'. + */ + void deleteRange(final byte[] from, + final byte[] to); + KeyValueIterator all(final boolean forward); KeyValueIterator prefixScan(final Bytes prefix); @@ -603,6 +619,16 @@ public KeyValueIterator range(final Bytes from, ); } + @Override + public void deleteRange(final byte[] from, final byte[] to) { + try { + db.deleteRange(columnFamily, wOptions, from, to); + } catch (final RocksDBException e) { + // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores. + throw new ProcessorStateException("Error while removing key from store " + name, e); + } + } + @Override public KeyValueIterator all(final boolean forward) { final RocksIterator innerIterWithTimestamp = db.newIterator(columnFamily); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java index 3efeb321ee31a..f8ba8837258f3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStore.java @@ -52,29 +52,29 @@ public class RocksDBTimeOrderedWindowStore @Override public void put(final Bytes key, final byte[] value, final long timestamp) { - // Skip if value is null and duplicates are allowed since this delete is a no-op if (!(value == null && retainDuplicates)) { maybeUpdateSeqnumForDups(); wrapped().put(TimeOrderedKeySchema.toStoreKeyBinary(key, timestamp, seqnum), value); + } else { + // Delete all duplicates for the specified key and timestamp + wrapped().remove(key, timestamp); } } @Override public byte[] fetch(final Bytes key, final long timestamp) { - return wrapped().get(TimeOrderedKeySchema.toStoreKeyBinary(key, timestamp, seqnum)); + throw new UnsupportedOperationException(); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed + @SuppressWarnings("deprecation") @Override public WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().fetch(key, timeFrom, timeTo); - return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); + throw new UnsupportedOperationException(); } @Override public WindowStoreIterator backwardFetch(final Bytes key, final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().backwardFetch(key, timeFrom, timeTo); - return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); + throw new UnsupportedOperationException(); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @@ -83,8 +83,7 @@ public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().fetch(keyFrom, keyTo, timeFrom, timeTo); - return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + throw new UnsupportedOperationException(); } @Override @@ -92,8 +91,7 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFr final Bytes keyTo, final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); - return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + throw new UnsupportedOperationException(); } @Override @@ -104,21 +102,18 @@ public KeyValueIterator, byte[]> all() { @Override public KeyValueIterator, byte[]> backwardAll() { - final KeyValueIterator bytesIterator = wrapped().backwardAll(); - return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + throw new UnsupportedOperationException(); } @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().fetchAll(timeFrom, timeTo); - return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + throw new UnsupportedOperationException(); } @Override public KeyValueIterator, byte[]> backwardFetchAll(final long timeFrom, final long timeTo) { - final KeyValueIterator bytesIterator = wrapped().backwardFetchAll(timeFrom, timeTo); - return new TimeOrderedWindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); + throw new UnsupportedOperationException(); } private void maybeUpdateSeqnumForDups() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java index db3d17560bf3e..dafbc8f059f58 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java @@ -205,6 +205,22 @@ public KeyValueIterator range(final Bytes from, true); } + @Override + public void deleteRange(final byte[] from, final byte[] to) { + try { + db.deleteRange(oldColumnFamily, wOptions, from, to); + } catch (final RocksDBException e) { + // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores. + throw new ProcessorStateException("Error while removing key from store " + name, e); + } + try { + db.deleteRange(newColumnFamily, wOptions, from, to); + } catch (final RocksDBException e) { + // String format is happening in wrapping stores. So formatted message is thrown from wrapping stores. + throw new ProcessorStateException("Error while removing key from store " + name, e); + } + } + @Override public KeyValueIterator all(final boolean forward) { final RocksIterator innerIterWithTimestamp = db.newIterator(newColumnFamily); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java index 9fddc163b5a55..ea3b89a525e04 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segment.java @@ -25,4 +25,5 @@ public interface Segment extends KeyValueStore, BatchWritingStore void destroy() throws IOException; + void deleteRange(Bytes keyFrom, Bytes keyTo); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index 79ada1f07db05..451992935860e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java @@ -109,6 +109,14 @@ public interface SegmentedBytesStore extends StateStore { */ void remove(Bytes key); + /** + * Remove all duplicated records with the provided key in the specified timestamp. + * + * @param key the segmented key to remove + * @param timestamp the timestamp to match + */ + void remove(Bytes key, long timestamp); + /** * Write a new value to the store with the provided key. The key * should be a composite of the record key, and the timestamp information etc @@ -151,6 +159,18 @@ interface KeySchema { */ Bytes lowerRange(final Bytes key, final long from); + /** + * Given a record key and a time, construct a Segmented key to search when performing + * prefixed queries. + * + * @param key + * @param timestamp + * @return The key that represents the prefixed Segmented key in bytes. + */ + default Bytes toStoreBinaryKeyPrefix(final Bytes key, long timestamp) { + throw new UnsupportedOperationException(); + } + /** * Given a range of fixed size record keys and a time, construct a Segmented key that represents * the upper range of keys to search when performing range queries. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java index b834b3a1ab1a8..f191d2f708e39 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchema.java @@ -30,62 +30,40 @@ /** * A {@link RocksDBSegmentedBytesStore.KeySchema} to serialize/deserialize a RocksDB store - * key into a schema combined of (time,seq,key). This key schema is more efficient when doing - * range queries between a time interval. For key range queries better use {@link WindowKeySchema}. + * key into a schema combined of (time,key,seq). Since key is variable length while time/seq is + * fixed length, when formatting in this order, varying time range query would be very inefficient + * since we'd need to be very conservative in picking the from / to boundaries; however for now + * we do not expect any varying time range access at all, only fixed time range only. */ public class TimeOrderedKeySchema implements RocksDBSegmentedBytesStore.KeySchema { private static final Logger LOG = LoggerFactory.getLogger(TimeOrderedKeySchema.class); private static final int TIMESTAMP_SIZE = 8; private static final int SEQNUM_SIZE = 4; - private static final int PREFIX_SIZE = TIMESTAMP_SIZE + SEQNUM_SIZE; - /** - * {@inheritdoc} - * - * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key - * range queries may be slower. If better performance on key range queries are necessary, then - * use the {@link WindowKeySchema}. - */ @Override public Bytes upperRange(final Bytes key, final long to) { - return toStoreKeyBinary(key.get(), to, Integer.MAX_VALUE); + throw new UnsupportedOperationException(); } - /** - * {@inheritdoc} - * - * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key - * range queries may be slower. If better performance on key range queries are necessary, then - * use the {@link WindowKeySchema}. - */ @Override public Bytes lowerRange(final Bytes key, final long from) { - return toStoreKeyBinary(key.get(), from, 0); + throw new UnsupportedOperationException(); + } + + @Override + public Bytes toStoreBinaryKeyPrefix(final Bytes key, final long timestamp) { + return toStoreKeyBinaryPrefix(key, timestamp); } - /** - * {@inheritdoc} - * - * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key - * range queries may be slower. If better performance on key range queries are necessary, then - * use the {@link WindowKeySchema}. - */ @Override public Bytes upperRangeFixedSize(final Bytes key, final long to) { - return toStoreKeyBinary(key, to, Integer.MAX_VALUE); + throw new UnsupportedOperationException(); } - /** - * {@inheritdoc} - * - * Queries using the {@link TimeOrderedKeySchema} are optimized for time range queries only. Key - * range queries may be slower. If better performance on key range queries are necessary, then - * use the {@link WindowKeySchema}. - */ @Override public Bytes lowerRangeFixedSize(final Bytes key, final long from) { - return toStoreKeyBinary(key, Math.max(0, from), 0); + throw new UnsupportedOperationException(); } @Override @@ -96,31 +74,36 @@ public long segmentTimestamp(final Bytes key) { /** * {@inheritdoc} * - * This method is not optimized for {@link TimeOrderedKeySchema}. The method may do unnecessary - * checks to find the next record. + * This method is optimized for {@link RocksDBTimeOrderedWindowStore#all()} only. Key and time + * range queries are not supported. */ @Override public HasNextCondition hasNextCondition(final Bytes binaryKeyFrom, final Bytes binaryKeyTo, final long from, final long to) { - return iterator -> { - while (iterator.hasNext()) { - final Bytes bytes = iterator.peekNextKey(); - final Bytes keyBytes = Bytes.wrap(extractStoreKeyBytes(bytes.get())); - final long time = extractStoreTimestamp(bytes.get()); - if ((binaryKeyFrom == null || keyBytes.compareTo(binaryKeyFrom) >= 0) - && (binaryKeyTo == null || keyBytes.compareTo(binaryKeyTo) <= 0) - && time >= from - && time <= to) { - return true; - } - iterator.next(); - } - return false; - }; + if (binaryKeyFrom != null || binaryKeyTo != null) { + throw new IllegalArgumentException("binaryKeyFrom/binaryKeyTo keys cannot be non-null. Key and time range queries are not supported."); + } + + if (from != 0 && to != Long.MAX_VALUE) { + throw new IllegalArgumentException("from/to time ranges should be 0 to Long.MAX_VALUE. Key and time range queries are not supported."); + } + + return iterator -> iterator.hasNext(); } @Override public List segmentsToSearch(final Segments segments, final long from, final long to, final boolean forward) { - return segments.segments(from, to, forward); + throw new UnsupportedOperationException(); + } + + public static Bytes toStoreKeyBinaryPrefix(final Bytes key, + final long timestamp) { + final byte[] serializedKey = key.get(); + + final ByteBuffer buf = ByteBuffer.allocate(TIMESTAMP_SIZE + serializedKey.length); + buf.putLong(timestamp); + buf.put(serializedKey); + + return Bytes.wrap(buf.array()); } public static Bytes toStoreKeyBinary(final Bytes key, @@ -149,30 +132,23 @@ static Bytes toStoreKeyBinary(final byte[] serializedKey, final int seqnum) { final ByteBuffer buf = ByteBuffer.allocate(TIMESTAMP_SIZE + serializedKey.length + SEQNUM_SIZE); buf.putLong(timestamp); - buf.putInt(seqnum); buf.put(serializedKey); + buf.putInt(seqnum); return Bytes.wrap(buf.array()); } static byte[] extractStoreKeyBytes(final byte[] binaryKey) { final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - SEQNUM_SIZE]; - System.arraycopy(binaryKey, PREFIX_SIZE, bytes, 0, bytes.length); + System.arraycopy(binaryKey, TIMESTAMP_SIZE, bytes, 0, bytes.length); return bytes; } - static K extractStoreKey(final byte[] binaryKey, - final StateSerdes serdes) { - final byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - SEQNUM_SIZE]; - System.arraycopy(binaryKey, PREFIX_SIZE, bytes, 0, bytes.length); - return serdes.keyFrom(bytes); - } - static long extractStoreTimestamp(final byte[] binaryKey) { return ByteBuffer.wrap(binaryKey).getLong(0); } static int extractStoreSequence(final byte[] binaryKey) { - return ByteBuffer.wrap(binaryKey).getInt(TIMESTAMP_SIZE); + return ByteBuffer.wrap(binaryKey).getInt(binaryKey.length - SEQNUM_SIZE); } static Windowed fromStoreKey(final byte[] binaryKey, @@ -202,8 +178,8 @@ static Window extractStoreWindow(final byte[] binaryKey, * Safely construct a time window of the given size, * taking care of bounding endMs to Long.MAX_VALUE if necessary */ - static TimeWindow timeWindowForSize(final long startMs, - final long windowSize) { + private static TimeWindow timeWindowForSize(final long startMs, + final long windowSize) { long endMs = startMs + windowSize; if (endMs < 0) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegment.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegment.java index 36fe0e5a36530..f0e4cf6132e09 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegment.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimestampedSegment.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.state.internals.metrics.RocksDBMetricsRecorder; @@ -40,6 +41,11 @@ public void destroy() throws IOException { Utils.delete(dbDir); } + @Override + public void deleteRange(final Bytes keyFrom, final Bytes keyTo) { + throw new UnsupportedOperationException(); + } + @Override public int compareTo(final TimestampedSegment segment) { return Long.compare(id, segment.id); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java index a1344f675924e..53d184652d0b9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java @@ -93,7 +93,7 @@ public abstract class AbstractRocksDBSegmentedBytesStoreTest @Parameters(name = "{0}") public static Object[] getKeySchemas() { - return new Object[] {new SessionKeySchema(), new WindowKeySchema(), new TimeOrderedKeySchema()}; + return new Object[] {new SessionKeySchema(), new WindowKeySchema()}; } @Before diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java index 37f98f1a43d4f..2646c4cf472f4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedWindowStoreTest.java @@ -16,18 +16,68 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; -import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.test.InternalMockProcessorContext; +import org.apache.kafka.test.MockRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import java.io.File; + +import static java.util.Arrays.asList; import static org.apache.kafka.streams.state.internals.RocksDbWindowBytesStoreSupplier.WindowStoreTypes; +import static org.apache.kafka.test.StreamsTestUtils.toList; +import static org.junit.Assert.assertEquals; + +public class RocksDBTimeOrderedWindowStoreTest { + private static final long WINDOW_SIZE = 3L; + private static final long SEGMENT_INTERVAL = 60_000L; + private static final long RETENTION_PERIOD = 2 * SEGMENT_INTERVAL; + + private static final String STORE_NAME = "rocksDB time-ordered window store"; + + WindowStore windowStore; + InternalMockProcessorContext context; + MockRecordCollector recordCollector; + + final File baseDir = TestUtils.tempDirectory("test"); -public class RocksDBTimeOrderedWindowStoreTest extends RocksDBWindowStoreTest { - private static final String STORE_NAME = "rocksDB window store"; + @Before + public void setup() { + windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); + + recordCollector = new MockRecordCollector(); + context = new InternalMockProcessorContext( + baseDir, + Serdes.String(), + Serdes.Integer(), + recordCollector, + new ThreadCache( + new LogContext("testCache"), + 0, + new MockStreamsMetrics(new Metrics()))); + context.setTime(1L); + + windowStore.init((StateStoreContext) context, windowStore); + } + + @After + public void after() { + windowStore.close(); + } - @Override WindowStore buildWindowStore(final long retentionPeriod, final long windowSize, final boolean retainDuplicates, @@ -47,24 +97,143 @@ WindowStore buildWindowStore(final long retentionPeriod, .build(); } - @Override - String getMetricsScope() { - return new RocksDbWindowBytesStoreSupplier(null, 0, 0, 0, false, WindowStoreTypes.TIME_ORDERED_WINDOW_STORE).metricsScope(); + @Test + public void shouldGetAll() { + final long startTime = SEGMENT_INTERVAL - 4L; + + windowStore.put(0, "zero", startTime + 0); + windowStore.put(1, "one", startTime + 1); + windowStore.put(2, "two", startTime + 2); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + + assertEquals( + asList(zero, one, two), + toList(windowStore.all()) + ); + } + + @Test + public void shouldGetAllDuplicates() { + final long startTime = SEGMENT_INTERVAL - 4L; + + windowStore.put(0, "zero1", startTime + 0); + windowStore.put(0, "zero2", startTime + 0); + windowStore.put(0, "zero3", startTime + 0); + + final KeyValue, String> zero1 = windowedPair(0, "zero1", startTime + 0); + final KeyValue, String> zero2 = windowedPair(0, "zero2", startTime + 0); + final KeyValue, String> zero3 = windowedPair(0, "zero3", startTime + 0); + + assertEquals( + asList(zero1, zero2, zero3), + toList(windowStore.all()) + ); + } + + @Test + public void shouldGetAllNonDeletedRecords() { + final long startTime = SEGMENT_INTERVAL - 4L; + + // Add some records + windowStore.put(0, "zero", startTime + 0); + windowStore.put(1, "one", startTime + 1); + windowStore.put(2, "two", startTime + 2); + windowStore.put(3, "three", startTime + 3); + windowStore.put(4, "four", startTime + 4); + + // Delete some records + windowStore.put(1, null, startTime + 1); + windowStore.put(3, null, startTime + 3); + + // Only non-deleted records should appear in the all() iterator + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> two = windowedPair(2, "two", startTime + 2); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + + assertEquals( + asList(zero, two, four), + toList(windowStore.all()) + ); + } + + @Test + public void shouldDeleteAllDuplicates() { + final long startTime = SEGMENT_INTERVAL - 4L; + + windowStore.put(0, "zero1", startTime + 0); + windowStore.put(0, "zero2", startTime + 0); + windowStore.put(0, "zero3", startTime + 0); + windowStore.put(1, "one1", startTime + 1); + windowStore.put(1, "one2", startTime + 1); + + windowStore.put(0, null, startTime + 0); + + final KeyValue, String> one1 = windowedPair(1, "one1", startTime + 1); + final KeyValue, String> one2 = windowedPair(1, "one2", startTime + 1); + + assertEquals( + asList(one1, one2), + toList(windowStore.all()) + ); } - @Override - void setClassLoggerToDebug() { - LogCaptureAppender.setClassLoggerToDebug(AbstractRocksDBSegmentedBytesStore.class); + @Test + public void shouldGetAllReturnTimestampOrderedRecords() { + final long startTime = SEGMENT_INTERVAL - 4L; + + // Add some records in different order + windowStore.put(4, "four", startTime + 4); + windowStore.put(0, "zero", startTime + 0); + windowStore.put(2, "two1", startTime + 2); + windowStore.put(3, "three", startTime + 3); + windowStore.put(1, "one", startTime + 1); + + // Add duplicates + windowStore.put(2, "two2", startTime + 2); + + // Only non-deleted records should appear in the all() iterator + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + final KeyValue, String> two1 = windowedPair(2, "two1", startTime + 2); + final KeyValue, String> two2 = windowedPair(2, "two2", startTime + 2); + final KeyValue, String> three = windowedPair(3, "three", startTime + 3); + final KeyValue, String> four = windowedPair(4, "four", startTime + 4); + + assertEquals( + asList(zero, one, two1, two2, three, four), + toList(windowStore.all()) + ); + } + + @Test + public void shouldEarlyClosedIteratorStillGetAllRecords() { + final long startTime = SEGMENT_INTERVAL - 4L; + + windowStore.put(0, "zero", startTime + 0); + windowStore.put(1, "one", startTime + 1); + + final KeyValue, String> zero = windowedPair(0, "zero", startTime + 0); + final KeyValue, String> one = windowedPair(1, "one", startTime + 1); + + final KeyValueIterator, String> it = windowStore.all(); + assertEquals(zero, it.next()); + it.close(); + + // A new all() iterator after a previous all() iterator was closed should return all elements. + assertEquals( + asList(zero, one), + toList(windowStore.all()) + ); } - @Override - long extractStoreTimestamp(final byte[] binaryKey) { - return TimeOrderedKeySchema.extractStoreTimestamp(binaryKey); + private static KeyValue, V> windowedPair(final K key, final V value, final long timestamp) { + return windowedPair(key, value, timestamp, WINDOW_SIZE); } - @Override - K extractStoreKey(final byte[] binaryKey, - final StateSerdes serdes) { - return TimeOrderedKeySchema.extractStoreKey(binaryKey, serdes); + private static KeyValue, V> windowedPair(final K key, final V value, final long timestamp, final long windowSize) { + return KeyValue.pair(new Windowed<>(key, WindowKeySchema.timeWindowForSize(timestamp, windowSize)), value); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java index 5a5c4fc47a5a4..03f81e3d72a30 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeySchemaTest.java @@ -19,20 +19,12 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.state.StateSerdes; -import org.apache.kafka.test.KeyValueIteratorStub; import org.junit.Test; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -44,93 +36,8 @@ public class TimeOrderedKeySchemaTest { final private Window window = new TimeWindow(startTime, endTime); final private Windowed windowedKey = new Windowed<>(key, window); - final private TimeOrderedKeySchema timeOrderedKeySchema = new TimeOrderedKeySchema(); final private StateSerdes stateSerdes = new StateSerdes<>("dummy", serde, Serdes.ByteArray()); - @Test - public void testHasNextConditionUsingNullKeys() { - final List> keys = Arrays.asList( - KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0}), new TimeWindow(0, 1)), 0), 1), - KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0}), new TimeWindow(0, 1)), 0), 2), - KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0, 0}), new TimeWindow(0, 1)), 0), 3), - KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0}), new TimeWindow(10, 20)), 4), 4), - KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0}), new TimeWindow(10, 20)), 5), 5), - KeyValue.pair(TimeOrderedKeySchema.toStoreKeyBinary(new Windowed<>(Bytes.wrap(new byte[] {0, 0, 0}), new TimeWindow(10, 20)), 6), 6)); - final DelegatingPeekingKeyValueIterator iterator = new DelegatingPeekingKeyValueIterator<>("foo", new KeyValueIteratorStub<>(keys.iterator())); - - final HasNextCondition hasNextCondition = timeOrderedKeySchema.hasNextCondition(null, null, 0, Long.MAX_VALUE); - final List results = new ArrayList<>(); - while (hasNextCondition.hasNext(iterator)) { - results.add(iterator.next().value); - } - assertThat(results, equalTo(Arrays.asList(1, 2, 3, 4, 5, 6))); - } - - @Test - public void testUpperBoundWithLargeTimestamps() { - final Bytes upper = timeOrderedKeySchema.upperRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), Long.MAX_VALUE); - - assertThat( - "shorter key with max timestamp should be in range", - upper.compareTo( - TimeOrderedKeySchema.toStoreKeyBinary( - new byte[] {0xA}, - Long.MAX_VALUE, - Integer.MAX_VALUE - ) - ) >= 0 - ); - - assertThat( - "shorter key with max timestamp should be in range", - upper.compareTo( - TimeOrderedKeySchema.toStoreKeyBinary( - new byte[] {0xA, 0xB}, - Long.MAX_VALUE, - Integer.MAX_VALUE - ) - ) >= 0 - ); - - assertThat(upper, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, Long.MAX_VALUE, Integer.MAX_VALUE))); - } - - @Test - public void testUpperBoundWithZeroTimestamp() { - final Bytes upper = timeOrderedKeySchema.upperRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 0); - assertThat(upper, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, Integer.MAX_VALUE))); - } - - @Test - public void testLowerBoundWithZeroTimestamp() { - final Bytes lower = timeOrderedKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 0); - assertThat(lower, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 0, 0))); - } - - @Test - public void testLowerBoundWithNonZeroTimestamp() { - final Bytes lower = timeOrderedKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), 42); - assertThat(lower, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, 42, 0))); - } - - @Test - public void testLowerBoundMatchesTrailingZeros() { - final Bytes lower = timeOrderedKeySchema.lowerRange(Bytes.wrap(new byte[] {0xA, 0xB, 0xC}), Long.MAX_VALUE - 1); - - assertThat( - "appending zeros to key should still be in range", - lower.compareTo( - TimeOrderedKeySchema.toStoreKeyBinary( - new byte[] {0xA, 0xB, 0xC, 0, 0, 0, 0, 0, 0, 0, 0}, - Long.MAX_VALUE - 1, - 0 - ) - ) < 0 - ); - - assertThat(lower, equalTo(TimeOrderedKeySchema.toStoreKeyBinary(new byte[] {0xA, 0xB, 0xC}, Long.MAX_VALUE - 1, 0))); - } - @Test public void shouldConvertToBinaryAndBack() { final Bytes serialized = TimeOrderedKeySchema.toStoreKeyBinary(windowedKey, 0, stateSerdes); From 03b953cf5530ab32c7c907f4e3c3b253800c3f0e Mon Sep 17 00:00:00 2001 From: wenbingshen Date: Mon, 19 Apr 2021 19:54:04 +0800 Subject: [PATCH 070/155] MINOR: Fix the negative time difference value from Log (#10553) Reviewers: Lucas Bradstreet , Chia-Ping Tsai , David Jacot --- core/src/main/scala/kafka/log/Log.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index e207fd694e72e..1111d98bae1cf 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -1017,7 +1017,7 @@ class Log(@volatile private var _dir: File, } producerStateManager.updateMapEndOffset(lastOffset) producerStateManager.takeSnapshot() - info(s"Producer state recovery took ${producerStateLoadStart - segmentRecoveryStart}ms for snapshot load " + + info(s"Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " + s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset") } } From 937054bb5e424d07075a758aad46abea129d0b24 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 19 Apr 2021 21:40:07 +0800 Subject: [PATCH 071/155] MINOR: remove `checksumOrNull` and `isValid` from Record (#10498) 1. rewrite the checksum of DumpLogSegments 2. remove checksumOrNull and isValid from Record Reviewers: Ismael Juma --- .../common/record/AbstractLegacyRecordBatch.java | 5 ----- .../kafka/common/record/DefaultRecord.java | 12 ------------ .../org/apache/kafka/common/record/Record.java | 13 ------------- .../common/record/DefaultRecordBatchTest.java | 16 ++++++---------- .../kafka/common/record/MemoryRecordsTest.java | 6 +++--- .../main/scala/kafka/tools/DumpLogSegments.scala | 8 +++++--- .../scala/unit/kafka/log/LogValidatorTest.scala | 8 ++++---- 7 files changed, 18 insertions(+), 50 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java index 59b2c68388a70..35ea4d36543fd 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java @@ -112,11 +112,6 @@ public boolean hasTimestampType(TimestampType timestampType) { return outerRecord().timestampType() == timestampType; } - @Override - public Long checksumOrNull() { - return checksum(); - } - @Override public long checksum() { return outerRecord().checksum(); diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java index bbaed94a31ae6..b63773bf95931 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java +++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecord.java @@ -120,18 +120,6 @@ public byte attributes() { return attributes; } - @Override - public Long checksumOrNull() { - return null; - } - - @Override - public boolean isValid() { - // new versions of the message format (2 and above) do not contain an individual record checksum; - // instead they are validated with the checksum at the log entry level - return true; - } - @Override public void ensureValid() {} diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java index ab52befc70b08..4a387f74359b5 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/Record.java +++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java @@ -52,19 +52,6 @@ public interface Record { */ long timestamp(); - /** - * Get a checksum of the record contents. - * @return A 4-byte unsigned checksum represented as a long or null if the message format does not - * include a checksum (i.e. for v2 and above) - */ - Long checksumOrNull(); - - /** - * Check whether the record has a valid checksum. - * @return true if the record has a valid checksum, false otherwise - */ - boolean isValid(); - /** * Raise a {@link org.apache.kafka.common.errors.CorruptRecordException} if the record does not have a valid checksum. */ diff --git a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java index 9cd744a8f6300..065f1b51b39aa 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java @@ -94,9 +94,7 @@ public void buildDefaultRecordBatch() { assertEquals(RecordBatch.NO_SEQUENCE, batch.baseSequence()); assertEquals(RecordBatch.NO_SEQUENCE, batch.lastSequence()); - for (Record record : batch) { - assertTrue(record.isValid()); - } + for (Record record : batch) record.ensureValid(); } } @@ -124,9 +122,7 @@ public void buildDefaultRecordBatchWithProducerId() { assertEquals(baseSequence, batch.baseSequence()); assertEquals(baseSequence + 1, batch.lastSequence()); - for (Record record : batch) { - assertTrue(record.isValid()); - } + for (Record record : batch) record.ensureValid(); } } @@ -199,7 +195,7 @@ public void testInvalidRecordCountTooManyNonCompressedV2() { DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.NONE, 5); // force iteration through the batch to execute validation // batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets - assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid)); + assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::ensureValid)); } @Test @@ -208,7 +204,7 @@ public void testInvalidRecordCountTooLittleNonCompressedV2() { DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.NONE, 2); // force iteration through the batch to execute validation // batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets - assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid)); + assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::ensureValid)); } @Test @@ -217,7 +213,7 @@ public void testInvalidRecordCountTooManyCompressedV2() { DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP, 5); // force iteration through the batch to execute validation // batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets - assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid)); + assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::ensureValid)); } @Test @@ -226,7 +222,7 @@ public void testInvalidRecordCountTooLittleCompressedV2() { DefaultRecordBatch batch = recordsWithInvalidRecordCount(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP, 2); // force iteration through the batch to execute validation // batch validation is a part of normal workflow for LogValidator.validateMessagesAndAssignOffsets - assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::isValid)); + assertThrows(InvalidRecordException.class, () -> batch.forEach(Record::ensureValid)); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index be5b337b7686e..32945223c554a 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -161,7 +161,7 @@ public void testIterator(Args args) { int recordCount = 0; for (Record record : batch) { - assertTrue(record.isValid()); + record.ensureValid(); assertTrue(record.hasMagic(batch.magic())); assertFalse(record.isCompressed()); assertEquals(firstOffset + total, record.offset()); @@ -477,7 +477,7 @@ public void testBuildEndTxnMarker() { assertEquals(1, createdRecords.size()); Record record = createdRecords.get(0); - assertTrue(record.isValid()); + record.ensureValid(); EndTransactionMarker deserializedMarker = EndTransactionMarker.deserialize(record); assertEquals(ControlRecordType.COMMIT, deserializedMarker.controlType()); assertEquals(coordinatorEpoch, deserializedMarker.coordinatorEpoch()); @@ -514,7 +514,7 @@ public void testBuildLeaderChangeMessage() { assertEquals(1, createdRecords.size()); Record record = createdRecords.get(0); - assertTrue(record.isValid()); + record.ensureValid(); assertEquals(ControlRecordType.LEADER_CHANGE, ControlRecordType.parse(record.key())); LeaderChangeMessage deserializedMessage = ControlRecordUtils.deserializeLeaderChangeMessage(record); diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 8bc07da73e51c..75796ad13f57c 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -270,7 +270,7 @@ object DumpLogSegments { var prefix = s"${RecordIndent} " if (!skipRecordMetadata) { - print(s"${prefix}offset: ${record.offset} isValid: ${record.isValid} crc: ${record.checksumOrNull}" + + print(s"${prefix}offset: ${record.offset}" + s" keySize: ${record.keySize} valueSize: ${record.valueSize} ${batch.timestampType}: ${record.timestamp}" + s" baseOffset: ${batch.baseOffset} lastOffset: ${batch.lastOffset} baseSequence: ${batch.baseSequence}" + s" lastSequence: ${batch.lastSequence} producerEpoch: ${batch.producerEpoch} partitionLeaderEpoch: ${batch.partitionLeaderEpoch}" + @@ -279,8 +279,10 @@ object DumpLogSegments { if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) { print(" sequence: " + record.sequence + " headerKeys: " + record.headers.map(_.key).mkString("[", ",", "]")) - } else { - print(s" crc: ${record.checksumOrNull} isvalid: ${record.isValid}") + } + record match { + case r: AbstractLegacyRecordBatch => print(s" isValid: ${r.isValid} crc: ${r.checksum}}") + case _ => } if (batch.isControlBatch) { diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala index 9b335608c3ca9..b0d4e3dc7efa3 100644 --- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala @@ -368,7 +368,7 @@ class LogValidatorTest { assertEquals(isTransactional, batch.isTransactional) assertEquals(partitionLeaderEpoch, batch.partitionLeaderEpoch) for (record <- batch.asScala) { - assertTrue(record.isValid) + record.ensureValid() assertEquals(timestampSeq(i), record.timestamp) i += 1 } @@ -439,7 +439,7 @@ class LogValidatorTest { assertEquals(baseSequence, batch.baseSequence) assertEquals(partitionLeaderEpoch, batch.partitionLeaderEpoch) for (record <- batch.asScala) { - assertTrue(record.isValid) + record.ensureValid() assertEquals(timestampSeq(i), record.timestamp) i += 1 } @@ -597,7 +597,7 @@ class LogValidatorTest { assertEquals(baseSequence, batch.baseSequence) assertEquals(partitionLeaderEpoch, batch.partitionLeaderEpoch) for (record <- batch.asScala) { - assertTrue(record.isValid) + record.ensureValid() assertEquals(timestampSeq(i), record.timestamp) i += 1 } @@ -1497,7 +1497,7 @@ class LogValidatorTest { assertEquals(expectedLogAppendTime, batch.maxTimestamp, s"Unexpected max timestamp of batch $batch") maybeCheckBaseTimestamp(expectedBaseTimestamp, batch) for (record <- batch.asScala) { - assertTrue(record.isValid) + record.ensureValid() assertEquals(expectedLogAppendTime, record.timestamp, s"Unexpected timestamp of record $record") } } From 0fd3eb03746ecff0c902a7133136cf15b5095fa0 Mon Sep 17 00:00:00 2001 From: wenbingshen Date: Tue, 20 Apr 2021 10:35:52 +0800 Subject: [PATCH 072/155] MINOR: Remove redundant code from BrokerApiVersionsCommand (#10556) Reviewers: Chia-Ping Tsai --- .../admin/BrokerApiVersionsCommand.scala | 24 ++++++------------- 1 file changed, 7 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala index e1bcebb95907b..871b8c4e9fec7 100644 --- a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala +++ b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.errors.AuthenticationException import org.apache.kafka.common.internals.ClusterResourceListeners import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.Selector -import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.utils.LogContext import org.apache.kafka.common.utils.{KafkaThread, Time} import org.apache.kafka.common.Node @@ -103,8 +103,6 @@ object BrokerApiVersionsCommand { // org.apache.kafka.clients.admin.AdminClient doesn't currently expose a way to retrieve the supported api versions. // We inline the bits we need from kafka.admin.AdminClient so that we can delete it. private class AdminClient(val time: Time, - val requestTimeoutMs: Int, - val retryBackoffMs: Long, val client: ConsumerNetworkClient, val bootstrapBrokers: List[Node]) extends Logging { @@ -133,7 +131,6 @@ object BrokerApiVersionsCommand { networkThread.start() private def send(target: Node, - api: ApiKeys, request: AbstractRequest.Builder[_ <: AbstractRequest]): AbstractResponse = { val future = client.send(target, request) pendingFutures.add(future) @@ -145,22 +142,22 @@ object BrokerApiVersionsCommand { throw future.exception() } - private def sendAnyNode(api: ApiKeys, request: AbstractRequest.Builder[_ <: AbstractRequest]): AbstractResponse = { + private def sendAnyNode(request: AbstractRequest.Builder[_ <: AbstractRequest]): AbstractResponse = { bootstrapBrokers.foreach { broker => try { - return send(broker, api, request) + return send(broker, request) } catch { case e: AuthenticationException => throw e case e: Exception => - debug(s"Request $api failed against node $broker", e) + debug(s"Request ${request.apiKey()} failed against node $broker", e) } } - throw new RuntimeException(s"Request $api failed on brokers $bootstrapBrokers") + throw new RuntimeException(s"Request ${request.apiKey()} failed on brokers $bootstrapBrokers") } private def getApiVersions(node: Node): ApiVersionCollection = { - val response = send(node, ApiKeys.API_VERSIONS, new ApiVersionsRequest.Builder()).asInstanceOf[ApiVersionsResponse] + val response = send(node, new ApiVersionsRequest.Builder()).asInstanceOf[ApiVersionsResponse] Errors.forCode(response.data.errorCode).maybeThrow() response.data.apiKeys } @@ -179,7 +176,7 @@ object BrokerApiVersionsCommand { private def findAllBrokers(): List[Node] = { val request = MetadataRequest.Builder.allTopics() - val response = sendAnyNode(ApiKeys.METADATA, request).asInstanceOf[MetadataResponse] + val response = sendAnyNode(request).asInstanceOf[MetadataResponse] val errors = response.errors if (!errors.isEmpty) debug(s"Metadata request contained errors: $errors") @@ -267,11 +264,6 @@ object BrokerApiVersionsCommand { class AdminConfig(originals: Map[_,_]) extends AbstractConfig(AdminConfigDef, originals.asJava, false) - def createSimplePlaintext(brokerUrl: String): AdminClient = { - val config = Map(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> brokerUrl) - create(new AdminConfig(config)) - } - def create(props: Properties): AdminClient = create(props.asScala.toMap) def create(props: Map[String, _]): AdminClient = create(new AdminConfig(props)) @@ -330,8 +322,6 @@ object BrokerApiVersionsCommand { new AdminClient( time, - requestTimeoutMs, - retryBackoffMs, highLevelClient, metadata.fetch.nodes.asScala.toList) } From 8360db41da32bb053acd1e0c61fcb43ba1c40780 Mon Sep 17 00:00:00 2001 From: "high.lee" Date: Tue, 20 Apr 2021 14:58:15 +0900 Subject: [PATCH 073/155] MINOR: Modify unnecessary access specifiers (#9861) Reviewers: Chia-Ping Tsai --- .../state/internals/MeteredTimestampedKeyValueStore.java | 8 ++++---- .../streams/state/internals/MeteredWindowStore.java | 6 +++--- .../kafka/streams/state/internals/RocksDBStore.java | 9 ++------- 3 files changed, 9 insertions(+), 14 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java index 19d3900d98605..3b333b3e7dd1e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredTimestampedKeyValueStore.java @@ -93,10 +93,10 @@ public boolean putIfDifferentValues(final K key, } } - public static class RawAndDeserializedValue { - public final byte[] serializedValue; - public final ValueAndTimestamp value; - public RawAndDeserializedValue(final byte[] serializedValue, final ValueAndTimestamp value) { + static class RawAndDeserializedValue { + final byte[] serializedValue; + final ValueAndTimestamp value; + RawAndDeserializedValue(final byte[] serializedValue, final ValueAndTimestamp value) { this.serializedValue = serializedValue; this.value = value; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 8c77cf2b26c90..e7516d29a5068 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -46,9 +46,9 @@ public class MeteredWindowStore private final long windowSizeMs; private final String metricsScope; private final Time time; - final Serde keySerde; - final Serde valueSerde; - StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; + private StateSerdes serdes; private StreamsMetricsImpl streamsMetrics; private Sensor putSensor; private Sensor fetchSensor; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index c03f86d96715d..1abe2a5d925e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -104,9 +104,6 @@ public class RocksDBStore implements KeyValueStore, BatchWritingS private final RocksDBMetricsRecorder metricsRecorder; - // visible for testing - volatile BatchingStateRestoreCallback batchingStateRestoreCallback = null; - protected volatile boolean open = false; RocksDBStore(final String name, @@ -239,11 +236,10 @@ public void init(final ProcessorContext context, // open the DB dir metricsRecorder.init(getMetricsImpl(context), context.taskId()); openDB(context.appConfigs(), context.stateDir()); - batchingStateRestoreCallback = new RocksDBBatchingRestoreCallback(this); // value getter should always read directly from rocksDB // since it is only for values that are already flushed - context.register(root, batchingStateRestoreCallback); + context.register(root, new RocksDBBatchingRestoreCallback(this)); } @Override @@ -252,11 +248,10 @@ public void init(final StateStoreContext context, // open the DB dir metricsRecorder.init(getMetricsImpl(context), context.taskId()); openDB(context.appConfigs(), context.stateDir()); - batchingStateRestoreCallback = new RocksDBBatchingRestoreCallback(this); // value getter should always read directly from rocksDB // since it is only for values that are already flushed - context.register(root, batchingStateRestoreCallback); + context.register(root, new RocksDBBatchingRestoreCallback(this)); } @Override From 7ca13d32bb513ca8ee101b91aca8eb2eebbfcc94 Mon Sep 17 00:00:00 2001 From: wenbingshen Date: Tue, 20 Apr 2021 16:30:29 +0800 Subject: [PATCH 074/155] MINOR: Fix nonsense test line from TopicCommandTest (#10551) Reviewers: Chia-Ping Tsai --- .../unit/kafka/admin/TopicCommandWithAdminClientTest.scala | 4 ++-- .../scala/unit/kafka/admin/TopicCommandWithZKClientTest.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala index efcae11955f2b..21f2bd30e5082 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandWithAdminClientTest.scala @@ -465,7 +465,7 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) val rows = output.split("\n") assertEquals(3, rows.size) - rows(0).startsWith(s"Topic:$testTopicName\tPartitionCount:2") + assertTrue(rows(0).startsWith(s"Topic: $testTopicName")) } @Test @@ -736,7 +736,7 @@ class TopicCommandWithAdminClientTest extends KafkaServerTestHarness with Loggin topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) val rows = output.split("\n") assertEquals(2, rows.size) - rows(0).startsWith(s"Topic:$testTopicName\tPartitionCount:1") + assertTrue(rows(0).startsWith(s"Topic: $testTopicName")) } @Test diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandWithZKClientTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandWithZKClientTest.scala index 26db4584c558e..92b24d58ddcfb 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandWithZKClientTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandWithZKClientTest.scala @@ -442,7 +442,7 @@ class TopicCommandWithZKClientTest extends ZooKeeperTestHarness with Logging wit topicService.describeTopic(new TopicCommandOptions(Array("--topic", testTopicName)))) val rows = output.split("\n") assertEquals(3, rows.size) - rows(0).startsWith("Topic:testTopic\tPartitionCount:2") + assertTrue(rows(0).startsWith(s"Topic: $testTopicName")) } @Test From 0b464419e2e4f28444fca653ae5aa8dd7feae9ba Mon Sep 17 00:00:00 2001 From: Kowshik Prakasam Date: Tue, 20 Apr 2021 09:20:40 -0700 Subject: [PATCH 075/155] KAFKA-12553: Refactor recovery logic to introduce LogLoader (#10478) In this PR, I have refactored the recovery logic code introducing a new class kafka.log.LogLoader responsible for all activities related with recovery of log segments from disk. With this change, the recovery logic has been moved out of the Log class and into the new LogLoader class. Advantages: This refactor has the following advantages over the existing code: As such, the recovery logic is invoked once only during Log instantiation. Some parts of the recovery logic are fairly independent from the rest of the Log class. By moving the independent private logic to a separate LogLoader class, the existing Log class has become more modular, and the constructor behavior is a lot simpler now. Therefore, this makes the code more maintainable. This PR takes us a step closer towards the Log layer reactor work (KAFKA-12554). The Log recovery logic reads and writes to LeaderEpochFileCache and ProducerStateManager instances, so as such the logic does not fit very well into the definition of a "local log". By extracting it out of the Log class, in the future this will make it much easier to clearly define the separation of concerns between LocalLog and UnifiedLog. Reviewers: Satish Duggana , Jun Rao --- core/src/main/scala/kafka/log/Log.scala | 1070 +++++------ .../src/main/scala/kafka/log/LogCleaner.scala | 11 +- core/src/main/scala/kafka/log/LogConfig.scala | 7 + core/src/main/scala/kafka/log/LogLoader.scala | 525 ++++++ .../kafka/cluster/PartitionLockTest.scala | 47 +- .../unit/kafka/cluster/PartitionTest.scala | 48 +- .../kafka/log/LogCleanerManagerTest.scala | 37 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 40 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 1501 ++++++++++++++++ .../test/scala/unit/kafka/log/LogTest.scala | 1587 +---------------- .../scala/unit/kafka/log/LogTestUtils.scala | 242 ++- .../kafka/server/ReplicaManagerTest.scala | 44 +- .../unit/kafka/utils/SchedulerTest.scala | 27 +- 13 files changed, 3004 insertions(+), 2182 deletions(-) create mode 100644 core/src/main/scala/kafka/log/LogLoader.scala create mode 100644 core/src/test/scala/unit/kafka/log/LogLoaderTest.scala diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 1111d98bae1cf..edbabb88feef6 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -19,7 +19,7 @@ package kafka.log import java.io.{File, IOException} import java.lang.{Long => JLong} -import java.nio.file.{Files, NoSuchFileException} +import java.nio.file.Files import java.text.NumberFormat import java.util.Map.{Entry => JEntry} import java.util.Optional @@ -28,7 +28,7 @@ import java.util.concurrent.TimeUnit import java.util.regex.Pattern import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0} -import kafka.common.{LogSegmentOffsetOverflowException, LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} +import kafka.common.{LongRef, OffsetsOutOfOrderException, UnexpectedAppendOffsetException} import kafka.log.AppendOrigin.RaftLeader import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec} import kafka.metrics.KafkaMetricsGroup @@ -48,7 +48,7 @@ import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPar import scala.jdk.CollectionConverters._ import scala.collection.mutable.{ArrayBuffer, ListBuffer} -import scala.collection.{Seq, Set, mutable} +import scala.collection.{Seq, mutable} object LogAppendInfo { val UnknownLogAppendInfo = LogAppendInfo(None, -1, None, RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP, -1L, @@ -222,25 +222,31 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { * * @param _dir The directory in which log segments are created. * @param config The log configuration settings + * @param segments The non-empty log segments recovered from disk * @param logStartOffset The earliest offset allowed to be exposed to kafka client. * The logStartOffset can be updated by : * - user's DeleteRecordsRequest * - broker's log retention * - broker's log truncation + * - broker's log recovery * The logStartOffset is used to decide the following: * - Log deletion. LogSegment whose nextOffset <= log's logStartOffset can be deleted. * It may trigger log rolling if the active segment is deleted. * - Earliest offset of the log in response to ListOffsetRequest. To avoid OffsetOutOfRange exception after user seeks to earliest offset, * we make sure that logStartOffset <= log's highWatermark * Other activities such as log cleaning are not affected by logStartOffset. - * @param recoveryPoint The offset at which to begin recovery--i.e. the first offset which has not been flushed to disk + * @param recoveryPoint The offset at which to begin the next recovery i.e. the first offset which has not been flushed to disk + * @param nextOffsetMetadata The offset where the next message could be appended * @param scheduler The thread pool scheduler used for background actions * @param brokerTopicStats Container for Broker Topic Yammer Metrics * @param time The time instance used for checking the clock * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is considered expired * @param producerIdExpirationCheckIntervalMs How often to check for producer ids which need to be expired - * @param hadCleanShutdown boolean flag to indicate if the Log had a clean/graceful shutdown last time. true means - * clean shutdown whereas false means a crash. + * @param topicPartition The topic partition associated with this Log instance + * @param leaderEpochCache The LeaderEpochFileCache instance (if any) containing state associated + * with the provided logStartOffset and nextOffsetMetadata + * @param producerStateManager The ProducerStateManager instance containing state associated with the provided segments + * @param logDirFailureChannel The LogDirFailureChannel instance to asynchronously handle log directory failure * @param topicId optional Uuid to specify the topic ID for the topic if it exists. Should only be specified when * first creating the log through Partition.makeLeader or Partition.makeFollower. When reloading a log, * this field will be populated by reading the topic ID value from partition.metadata if it exists. @@ -255,17 +261,19 @@ case object SnapshotGenerated extends LogStartOffsetIncrementReason { @threadsafe class Log(@volatile private var _dir: File, @volatile var config: LogConfig, + val segments: LogSegments, @volatile var logStartOffset: Long, @volatile var recoveryPoint: Long, + @volatile var nextOffsetMetadata: LogOffsetMetadata, scheduler: Scheduler, brokerTopicStats: BrokerTopicStats, val time: Time, val maxProducerIdExpirationMs: Int, val producerIdExpirationCheckIntervalMs: Int, val topicPartition: TopicPartition, + @volatile var leaderEpochCache: Option[LeaderEpochFileCache], val producerStateManager: ProducerStateManager, logDirFailureChannel: LogDirFailureChannel, - private val hadCleanShutdown: Boolean = true, @volatile var topicId: Option[Uuid], val keepPartitionMetadataFile: Boolean) extends Logging with KafkaMetricsGroup { @@ -286,8 +294,6 @@ class Log(@volatile private var _dir: File, /* last time it was flushed */ private val lastFlushedTime = new AtomicLong(time.milliseconds) - @volatile private var nextOffsetMetadata: LogOffsetMetadata = _ - /* The earliest offset which is part of an incomplete transaction. This is used to compute the * last stable offset (LSO) in ReplicaManager. Note that it is possible that the "true" first unstable offset * gets removed from the log (through record or segment deletion). In this case, the first unstable offset @@ -308,44 +314,12 @@ class Log(@volatile private var _dir: File, */ @volatile private var highWatermarkMetadata: LogOffsetMetadata = LogOffsetMetadata(logStartOffset) - /* the actual segments of the log */ - private val segments: LogSegments = new LogSegments(topicPartition) - - // Visible for testing - @volatile var leaderEpochCache: Option[LeaderEpochFileCache] = None - @volatile var partitionMetadataFile : PartitionMetadataFile = null locally { - // create the log directory if it doesn't exist - Files.createDirectories(dir.toPath) - - initializeLeaderEpochCache() initializePartitionMetadata() - - val nextOffset = loadSegments() - - /* Calculate the offset of the next message */ - nextOffsetMetadata = LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size) - - leaderEpochCache.foreach(_.truncateFromEnd(nextOffsetMetadata.messageOffset)) - - updateLogStartOffset(math.max(logStartOffset, segments.firstSegment.get.baseOffset)) - - // The earliest leader epoch may not be flushed during a hard failure. Recover it here. - leaderEpochCache.foreach(_.truncateFromStart(logStartOffset)) - - // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here - // from scratch. - if (!producerStateManager.isEmpty) - throw new IllegalStateException("Producer state must be empty during log initialization") - - // Reload all snapshots into the ProducerStateManager cache, the intermediate ProducerStateManager used - // during log recovery may have deleted some files without the Log.producerStateManager instance witnessing the - // deletion. - producerStateManager.removeStraySnapshots(segments.baseOffsets.toSeq) - loadProducerState(logEndOffset, reloadFromCleanShutdown = hadCleanShutdown) - + updateLogStartOffset(logStartOffset) + maybeIncrementFirstUnstableOffset() // Delete partition metadata file if the version does not support topic IDs. // Recover topic ID if present and topic IDs are supported // If we were provided a topic ID when creating the log, partition metadata files are supported, and one does not yet exist @@ -372,13 +346,6 @@ class Log(@volatile private var _dir: File, def parentDirFile: File = new File(_parentDir) - def initFileSize: Int = { - if (config.preallocate) - config.segmentSize - else - 0 - } - def updateConfig(newConfig: LogConfig): Unit = { val oldConfig = this.config this.config = newConfig @@ -592,243 +559,7 @@ class Log(@volatile private var _dir: File, } private def initializeLeaderEpochCache(): Unit = lock synchronized { - val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir) - - def newLeaderEpochFileCache(): LeaderEpochFileCache = { - val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) - new LeaderEpochFileCache(topicPartition, checkpointFile) - } - - if (recordVersion.precedes(RecordVersion.V2)) { - val currentCache = if (leaderEpochFile.exists()) - Some(newLeaderEpochFileCache()) - else - None - - if (currentCache.exists(_.nonEmpty)) - warn(s"Deleting non-empty leader epoch cache due to incompatible message format $recordVersion") - - Files.deleteIfExists(leaderEpochFile.toPath) - leaderEpochCache = None - } else { - leaderEpochCache = Some(newLeaderEpochFileCache()) - } - } - - /** - * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped - * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than - * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted - * by this method. - * @return Set of .swap files that are valid to be swapped in as segment files - */ - private def removeTempFilesAndCollectSwapFiles(): Set[File] = { - - def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = { - info(s"Deleting index files with suffix $suffix for baseFile $baseFile") - val offset = offsetFromFile(baseFile) - Files.deleteIfExists(Log.offsetIndexFile(dir, offset, suffix).toPath) - Files.deleteIfExists(Log.timeIndexFile(dir, offset, suffix).toPath) - Files.deleteIfExists(Log.transactionIndexFile(dir, offset, suffix).toPath) - } - - val swapFiles = mutable.Set[File]() - val cleanFiles = mutable.Set[File]() - var minCleanedFileOffset = Long.MaxValue - - for (file <- dir.listFiles if file.isFile) { - if (!file.canRead) - throw new IOException(s"Could not read file $file") - val filename = file.getName - if (filename.endsWith(DeletedFileSuffix)) { - debug(s"Deleting stray temporary file ${file.getAbsolutePath}") - Files.deleteIfExists(file.toPath) - } else if (filename.endsWith(CleanedFileSuffix)) { - minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset) - cleanFiles += file - } else if (filename.endsWith(SwapFileSuffix)) { - // we crashed in the middle of a swap operation, to recover: - // if a log, delete the index files, complete the swap operation later - // if an index just delete the index files, they will be rebuilt - val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) - info(s"Found file ${file.getAbsolutePath} from interrupted swap operation.") - if (isIndexFile(baseFile)) { - deleteIndicesIfExist(baseFile) - } else if (isLogFile(baseFile)) { - deleteIndicesIfExist(baseFile) - swapFiles += file - } - } - } - - // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap - // files could be part of an incomplete split operation that could not complete. See Log#splitOverflowedSegment - // for more details about the split operation. - val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset) - invalidSwapFiles.foreach { file => - debug(s"Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset") - val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) - deleteIndicesIfExist(baseFile, SwapFileSuffix) - Files.deleteIfExists(file.toPath) - } - - // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files - cleanFiles.foreach { file => - debug(s"Deleting stray .clean file ${file.getAbsolutePath}") - Files.deleteIfExists(file.toPath) - } - - validSwapFiles - } - - /** - * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded - * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException - * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the - * caller is responsible for closing them appropriately, if needed. - * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset - */ - private def loadSegmentFiles(): Unit = { - // load segments in ascending order because transactional data from one segment may depend on the - // segments that come before it - for (file <- dir.listFiles.sortBy(_.getName) if file.isFile) { - if (isIndexFile(file)) { - // if it is an index file, make sure it has a corresponding .log file - val offset = offsetFromFile(file) - val logFile = Log.logFile(dir, offset) - if (!logFile.exists) { - warn(s"Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.") - Files.deleteIfExists(file.toPath) - } - } else if (isLogFile(file)) { - // if it's a log file, load the corresponding log segment - val baseOffset = offsetFromFile(file) - val timeIndexFileNewlyCreated = !Log.timeIndexFile(dir, baseOffset).exists() - val segment = LogSegment.open(dir = dir, - baseOffset = baseOffset, - config, - time = time, - fileAlreadyExists = true, - needsRecovery = !hadCleanShutdown) - - try segment.sanityCheck(timeIndexFileNewlyCreated) - catch { - case _: NoSuchFileException => - error(s"Could not find offset index file corresponding to log file ${segment.log.file.getAbsolutePath}, " + - "recovering segment and rebuilding index files...") - recoverSegment(segment) - case e: CorruptIndexException => - warn(s"Found a corrupted index file corresponding to log file ${segment.log.file.getAbsolutePath} due " + - s"to ${e.getMessage}}, recovering segment and rebuilding index files...") - recoverSegment(segment) - } - addSegment(segment) - } - } - } - - /** - * Recover the given segment. - * @param segment Segment to recover - * @param leaderEpochCache Optional cache for updating the leader epoch during recovery - * @return The number of bytes truncated from the segment - * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow - */ - private def recoverSegment(segment: LogSegment, - leaderEpochCache: Option[LeaderEpochFileCache] = None): Int = lock synchronized { - val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) - rebuildProducerState(segment.baseOffset, reloadFromCleanShutdown = false, producerStateManager) - val bytesTruncated = segment.recover(producerStateManager, leaderEpochCache) - // once we have recovered the segment's data, take a snapshot to ensure that we won't - // need to reload the same segment again while recovering another segment. - producerStateManager.takeSnapshot() - bytesTruncated - } - - /** - * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs - * are loaded. - * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to - * overflow. Note that this is currently a fatal exception as we do not have - * a way to deal with it. The exception is propagated all the way up to - * KafkaServer#startup which will cause the broker to shut down if we are in - * this situation. This is expected to be an extremely rare scenario in practice, - * and manual intervention might be required to get out of it. - */ - private def completeSwapOperations(swapFiles: Set[File]): Unit = { - for (swapFile <- swapFiles) { - val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, SwapFileSuffix, "")) - val baseOffset = offsetFromFile(logFile) - val swapSegment = LogSegment.open(swapFile.getParentFile, - baseOffset = baseOffset, - config, - time = time, - fileSuffix = SwapFileSuffix) - info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.") - recoverSegment(swapSegment) - - // We create swap files for two cases: - // (1) Log cleaning where multiple segments are merged into one, and - // (2) Log splitting where one segment is split into multiple. - // - // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment - // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion - // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to - // do a replace with an existing segment. - val oldSegments = logSegments(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment => - segment.readNextOffset > swapSegment.baseOffset - } - replaceSegments(Seq(swapSegment), oldSegments.toSeq, isRecoveredSwapFile = true) - } - } - - /** - * Load the log segments from the log files on disk and return the next offset. - * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs - * are loaded. - * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that overflow index offset; or when - * we find an unexpected number of .log files with overflow - */ - private def loadSegments(): Long = { - // first do a pass through the files in the log directory and remove any temporary files - // and find any interrupted swap operations - val swapFiles = removeTempFilesAndCollectSwapFiles() - - // Now do a second pass and load all the log and index files. - // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When - // this happens, restart loading segment files from scratch. - retryOnOffsetOverflow { - // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry - // loading of segments. In that case, we also need to close all segments that could have been left open in previous - // call to loadSegmentFiles(). - segments.close() - segments.clear() - loadSegmentFiles() - } - - // Finally, complete any interrupted swap operations. To be crash-safe, - // log files that are replaced by the swap segment should be renamed to .deleted - // before the swap file is restored as the new segment file. - completeSwapOperations(swapFiles) - - if (!dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) { - val nextOffset = retryOnOffsetOverflow { - recoverLog() - } - - // reset the index size of the currently active log segment to allow more entries - activeSegment.resizeIndexes(config.maxIndexSize) - nextOffset - } else { - if (logSegments.isEmpty) { - addSegment(LogSegment.open(dir = dir, - baseOffset = 0, - config, - time = time, - initFileSize = this.initFileSize)) - } - 0 - } + leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, recordVersion) } private def updateLogEndOffset(offset: Long): Unit = { @@ -857,174 +588,13 @@ class Log(@volatile private var _dir: File, } } - /** - * Recover the log segments and return the next offset after recovery. - * This method does not need to convert IOException to KafkaStorageException because it is only called before all - * logs are loaded. - * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow - */ - private[log] def recoverLog(): Long = { - /** return the log end offset if valid */ - def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = { - if (segments.nonEmpty) { - val logEndOffset = activeSegment.readNextOffset - if (logEndOffset >= logStartOffset) - Some(logEndOffset) - else { - warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ($logStartOffset). " + - "This could happen if segment files were deleted from the file system.") - removeAndDeleteSegments(logSegments, asyncDelete = true, LogRecovery) - leaderEpochCache.foreach(_.clearAndFlush()) - producerStateManager.truncateFullyAndStartAt(logStartOffset) - None - } - } else None - } - - // if we have the clean shutdown marker, skip recovery - if (!hadCleanShutdown) { - val unflushed = logSegments(this.recoveryPoint, Long.MaxValue).iterator - var truncated = false - - while (unflushed.hasNext && !truncated) { - val segment = unflushed.next() - info(s"Recovering unflushed segment ${segment.baseOffset}") - val truncatedBytes = - try { - recoverSegment(segment, leaderEpochCache) - } catch { - case _: InvalidOffsetException => - val startOffset = segment.baseOffset - warn("Found invalid offset during recovery. Deleting the corrupt segment and " + - s"creating an empty one with starting offset $startOffset") - segment.truncateTo(startOffset) - } - if (truncatedBytes > 0) { - // we had an invalid message, delete all remaining log - warn(s"Corruption found in segment ${segment.baseOffset}, truncating to offset ${segment.readNextOffset}") - removeAndDeleteSegments(unflushed.toList, - asyncDelete = true, - reason = LogRecovery) - truncated = true - } - } - } - - val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd() - - if (logSegments.isEmpty) { - // no existing segments, create a new mutable segment beginning at logStartOffset - addSegment(LogSegment.open(dir = dir, - baseOffset = logStartOffset, - config, - time = time, - initFileSize = this.initFileSize, - preallocate = config.preallocate)) - } - - // Update the recovery point if there was a clean shutdown and did not perform any changes to - // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end - // offset. To ensure correctness and to make it easier to reason about, it's best to only advance - // the recovery point in flush(Long). If we advanced the recovery point here, we could skip recovery for - // unflushed segments if the broker crashed after we checkpoint the recovery point and before we flush the - // segment. - (hadCleanShutdown, logEndOffsetOption) match { - case (true, Some(logEndOffset)) => - recoveryPoint = logEndOffset - logEndOffset - case _ => - val logEndOffset = logEndOffsetOption.getOrElse(activeSegment.readNextOffset) - recoveryPoint = Math.min(recoveryPoint, logEndOffset) - logEndOffset - } - } - // Rebuild producer state until lastOffset. This method may be called from the recovery code path, and thus must be // free of all side-effects, i.e. it must not update any log-specific state. private def rebuildProducerState(lastOffset: Long, - reloadFromCleanShutdown: Boolean, producerStateManager: ProducerStateManager): Unit = lock synchronized { checkIfMemoryMappedBufferClosed() - val allSegments = logSegments - val offsetsToSnapshot = - if (allSegments.nonEmpty) { - val nextLatestSegmentBaseOffset = segments.lowerSegment(allSegments.last.baseOffset).map(_.baseOffset) - Seq(nextLatestSegmentBaseOffset, Some(allSegments.last.baseOffset), Some(lastOffset)) - } else { - Seq(Some(lastOffset)) - } - info(s"Loading producer state till offset $lastOffset with message format version ${recordVersion.value}") - - // We want to avoid unnecessary scanning of the log to build the producer state when the broker is being - // upgraded. The basic idea is to use the absence of producer snapshot files to detect the upgrade case, - // but we have to be careful not to assume too much in the presence of broker failures. The two most common - // upgrade cases in which we expect to find no snapshots are the following: - // - // 1. The broker has been upgraded, but the topic is still on the old message format. - // 2. The broker has been upgraded, the topic is on the new message format, and we had a clean shutdown. - // - // If we hit either of these cases, we skip producer state loading and write a new snapshot at the log end - // offset (see below). The next time the log is reloaded, we will load producer state using this snapshot - // (or later snapshots). Otherwise, if there is no snapshot file, then we have to rebuild producer state - // from the first segment. - if (recordVersion.value < RecordBatch.MAGIC_VALUE_V2 || - (producerStateManager.latestSnapshotOffset.isEmpty && reloadFromCleanShutdown)) { - // To avoid an expensive scan through all of the segments, we take empty snapshots from the start of the - // last two segments and the last offset. This should avoid the full scan in the case that the log needs - // truncation. - offsetsToSnapshot.flatten.foreach { offset => - producerStateManager.updateMapEndOffset(offset) - producerStateManager.takeSnapshot() - } - } else { - info(s"Reloading from producer snapshot and rebuilding producer state from offset $lastOffset") - val isEmptyBeforeTruncation = producerStateManager.isEmpty && producerStateManager.mapEndOffset >= lastOffset - val producerStateLoadStart = time.milliseconds() - producerStateManager.truncateAndReload(logStartOffset, lastOffset, time.milliseconds()) - val segmentRecoveryStart = time.milliseconds() - - // Only do the potentially expensive reloading if the last snapshot offset is lower than the log end - // offset (which would be the case on first startup) and there were active producers prior to truncation - // (which could be the case if truncating after initial loading). If there weren't, then truncating - // shouldn't change that fact (although it could cause a producerId to expire earlier than expected), - // and we can skip the loading. This is an optimization for users which are not yet using - // idempotent/transactional features yet. - if (lastOffset > producerStateManager.mapEndOffset && !isEmptyBeforeTruncation) { - val segmentOfLastOffset = segments.floorSegment(lastOffset) - - logSegments(producerStateManager.mapEndOffset, lastOffset).foreach { segment => - val startOffset = Utils.max(segment.baseOffset, producerStateManager.mapEndOffset, logStartOffset) - producerStateManager.updateMapEndOffset(startOffset) - - if (offsetsToSnapshot.contains(Some(segment.baseOffset))) - producerStateManager.takeSnapshot() - - val maxPosition = if (segmentOfLastOffset.contains(segment)) { - Option(segment.translateOffset(lastOffset)) - .map(_.position) - .getOrElse(segment.size) - } else { - segment.size - } - - val fetchDataInfo = segment.read(startOffset, - maxSize = Int.MaxValue, - maxPosition = maxPosition, - minOneMessage = false) - if (fetchDataInfo != null) - loadProducersFromRecords(producerStateManager, fetchDataInfo.records) - } - } - producerStateManager.updateMapEndOffset(lastOffset) - producerStateManager.takeSnapshot() - info(s"Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " + - s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset") - } - } - - private def loadProducerState(lastOffset: Long, reloadFromCleanShutdown: Boolean): Unit = lock synchronized { - rebuildProducerState(lastOffset, reloadFromCleanShutdown, producerStateManager) - maybeIncrementFirstUnstableOffset() + Log.rebuildProducerState(producerStateManager, segments, logStartOffset, lastOffset, recordVersion, time, + reloadFromCleanShutdown = false) } def activeProducers: Seq[DescribeProducersResponseData.ProducerState] = { @@ -2068,7 +1638,7 @@ class Log(@volatile private var _dir: File, baseOffset = newOffset, config, time = time, - initFileSize = initFileSize, + initFileSize = config.initFileSize, preallocate = config.preallocate) addSegment(segment) @@ -2217,7 +1787,7 @@ class Log(@volatile private var _dir: File, baseOffset = newOffset, config = config, time = time, - initFileSize = initFileSize, + initFileSize = config.initFileSize, preallocate = config.preallocate)) leaderEpochCache.foreach(_.clearAndFlush()) producerStateManager.truncateFullyAndStartAt(newOffset) @@ -2237,7 +1807,7 @@ class Log(@volatile private var _dir: File, logStartOffset = startOffset nextOffsetMetadata = LogOffsetMetadata(endOffset, activeSegment.baseOffset, activeSegment.size) recoveryPoint = math.min(recoveryPoint, endOffset) - rebuildProducerState(endOffset, reloadFromCleanShutdown = false, producerStateManager) + rebuildProducerState(endOffset, producerStateManager) updateHighWatermark(math.min(highWatermark, endOffset)) } @@ -2315,94 +1885,16 @@ class Log(@volatile private var _dir: File, } } - /** - * Perform physical deletion for the given file. Allows the file to be deleted asynchronously or synchronously. - * - * This method assumes that the file exists and the method is not thread-safe. - * - * This method does not need to convert IOException (thrown from changeFileSuffixes) to KafkaStorageException because - * it is either called before all logs are loaded or the caller will catch and handle IOException - * - * @throws IOException if the file can't be renamed and still exists - */ private def deleteSegmentFiles(segments: Iterable[LogSegment], asyncDelete: Boolean, deleteProducerStateSnapshots: Boolean = true): Unit = { - segments.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix, false)) - - def deleteSegments(): Unit = { - info(s"Deleting segment files ${segments.mkString(",")}") - maybeHandleIOException(s"Error while deleting segments for $topicPartition in dir ${dir.getParent}") { - segments.foreach { segment => - segment.deleteIfExists() - if (deleteProducerStateSnapshots) - producerStateManager.removeAndDeleteSnapshot(segment.baseOffset) - } - } - } - - if (asyncDelete) - scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs) - else - deleteSegments() + Log.deleteSegmentFiles(segments, asyncDelete, deleteProducerStateSnapshots, dir, topicPartition, + config, scheduler, logDirFailureChannel, producerStateManager) } - /** - * Swap one or more new segment in place and delete one or more existing segments in a crash-safe manner. The old - * segments will be asynchronously deleted. - * - * This method does not need to convert IOException to KafkaStorageException because it is either called before all logs are loaded - * or the caller will catch and handle IOException - * - * The sequence of operations is: - *
      - *
    1. Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments(). - * If broker crashes at this point, the clean-and-swap operation is aborted and - * the .cleaned files are deleted on recovery in loadSegments(). - *
    2. New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the - * clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in - * loadSegments(). We detect this situation by maintaining a specific order in which files are renamed from - * .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, all .swap files - * whose offset is greater than the minimum-offset .clean file are deleted. - *
    3. If the broker crashes after all new segments were renamed to .swap, the operation is completed, the swap - * operation is resumed on recovery as described in the next step. - *
    4. Old segment files are renamed to .deleted and asynchronous delete is scheduled. - * If the broker crashes, any .deleted files left behind are deleted on recovery in loadSegments(). - * replaceSegments() is then invoked to complete the swap with newSegment recreated from - * the .swap file and oldSegments containing segments which were not renamed before the crash. - *
    5. Swap segment(s) are renamed to replace the existing segments, completing this operation. - * If the broker crashes, any .deleted files which may be left behind are deleted - * on recovery in loadSegments(). - *
    - * - * @param newSegments The new log segment to add to the log - * @param oldSegments The old log segments to delete from the log - * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash - */ private[log] def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { lock synchronized { - val sortedNewSegments = newSegments.sortBy(_.baseOffset) - // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments - // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment() - // multiple times for the same segment. - val sortedOldSegments = oldSegments.filter(seg => segments.contains(seg.baseOffset)).sortBy(_.baseOffset) - checkIfMemoryMappedBufferClosed() - // need to do this in two phases to be crash safe AND do the delete asynchronously - // if we crash in the middle of this we complete the swap in loadSegments() - if (!isRecoveredSwapFile) - sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix, false)) - sortedNewSegments.reverse.foreach(addSegment(_)) - val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet - - // delete the old files - sortedOldSegments.foreach { seg => - // remove the index entry - if (seg.baseOffset != sortedNewSegments.head.baseOffset) - segments.remove(seg.baseOffset) - // delete segment files, but do not delete producer state for segment objects which are being replaced. - deleteSegmentFiles(List(seg), asyncDelete = true, deleteProducerStateSnapshots = !newSegmentBaseOffsets.contains(seg.baseOffset)) - } - // okay we are safe now, remove the swap suffix - sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) + Log.replaceSegments(segments, newSegments, oldSegments, isRecoveredSwapFile, dir, topicPartition, + config, scheduler, logDirFailureChannel, producerStateManager) } } @@ -2438,102 +1930,23 @@ class Log(@volatile private var _dir: File, private[log] def addSegment(segment: LogSegment): LogSegment = this.segments.add(segment) private def maybeHandleIOException[T](msg: => String)(fun: => T): T = { - if (logDirFailureChannel.hasOfflineLogDir(parentDir)) { - throw new KafkaStorageException(s"The log dir $parentDir is offline due to a previous IO exception.") - } - try { + Log.maybeHandleIOException(logDirFailureChannel, parentDir, msg) { fun - } catch { - case e: IOException => - logDirFailureChannel.maybeAddOfflineLogDir(parentDir, msg, e) - throw new KafkaStorageException(msg, e) } } - private[log] def retryOnOffsetOverflow[T](fn: => T): T = { - while (true) { - try { - return fn - } catch { - case e: LogSegmentOffsetOverflowException => - info(s"Caught segment overflow error: ${e.getMessage}. Split segment and retry.") - splitOverflowedSegment(e.segment) - } - } - throw new IllegalStateException() + private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = lock synchronized { + Log.splitOverflowedSegment(segment, segments, dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager) } - /** - * Split a segment into one or more segments such that there is no offset overflow in any of them. The - * resulting segments will contain the exact same messages that are present in the input segment. On successful - * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments. - * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation. - *

    Note that this method assumes we have already determined that the segment passed in contains records that cause - * offset overflow.

    - *

    The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing - * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments - * and completeSwapOperations for the implementation to make this operation recoverable on crashes.

    - * @param segment Segment to split - * @return List of new segments that replace the input segment - */ - private[log] def splitOverflowedSegment(segment: LogSegment): List[LogSegment] = { - require(isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}") - require(segment.hasOverflow, "Split operation is only permitted for segments with overflow") - - info(s"Splitting overflowed segment $segment") +} - val newSegments = ListBuffer[LogSegment]() - try { - var position = 0 - val sourceRecords = segment.log - - while (position < sourceRecords.sizeInBytes) { - val firstBatch = sourceRecords.batchesFrom(position).asScala.head - val newSegment = LogCleaner.createNewCleanedSegment(this, firstBatch.baseOffset) - newSegments += newSegment - - val bytesAppended = newSegment.appendFromFile(sourceRecords, position) - if (bytesAppended == 0) - throw new IllegalStateException(s"Failed to append records from position $position in $segment") - - position += bytesAppended - } - - // prepare new segments - var totalSizeOfNewSegments = 0 - newSegments.foreach { splitSegment => - splitSegment.onBecomeInactiveSegment() - splitSegment.flush() - splitSegment.lastModified = segment.lastModified - totalSizeOfNewSegments += splitSegment.log.sizeInBytes - } - // size of all the new segments combined must equal size of the original segment - if (totalSizeOfNewSegments != segment.log.sizeInBytes) - throw new IllegalStateException("Inconsistent segment sizes after split" + - s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments") - - // replace old segment with new ones - info(s"Replacing overflowed segment $segment with split segments $newSegments") - replaceSegments(newSegments.toList, List(segment)) - newSegments.toList - } catch { - case e: Exception => - newSegments.foreach { splitSegment => - splitSegment.close() - splitSegment.deleteIfExists() - } - throw e - } - } -} - -/** - * Helper functions for logs - */ -object Log { - - /** a log file */ - val LogFileSuffix = ".log" +/** + * Helper functions for logs + */ +object Log extends Logging { + /** a log file */ + val LogFileSuffix = ".log" /** an index file */ val IndexFileSuffix = ".index" @@ -2587,10 +2000,29 @@ object Log { lastShutdownClean: Boolean = true, topicId: Option[Uuid], keepPartitionMetadataFile: Boolean): Log = { + // create the log directory if it doesn't exist + Files.createDirectories(dir.toPath) val topicPartition = Log.parseTopicPartitionName(dir) + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) - new Log(dir, config, logStartOffset, recoveryPoint, scheduler, brokerTopicStats, time, maxProducerIdExpirationMs, - producerIdExpirationCheckIntervalMs, topicPartition, producerStateManager, logDirFailureChannel, lastShutdownClean, topicId, keepPartitionMetadataFile) + val offsets = LogLoader.load(LoadLogParams( + dir, + topicPartition, + config, + scheduler, + time, + logDirFailureChannel, + lastShutdownClean, + segments, + logStartOffset, + recoveryPoint, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + new Log(dir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, offsets.nextOffsetMetadata, scheduler, + brokerTopicStats, time, maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs, topicPartition, + leaderEpochCache, producerStateManager, logDirFailureChannel, topicId, keepPartitionMetadataFile) } /** @@ -2748,12 +2180,12 @@ object Log { new TopicPartition(topic, partition) } - private def isIndexFile(file: File): Boolean = { + private[log] def isIndexFile(file: File): Boolean = { val filename = file.getName filename.endsWith(IndexFileSuffix) || filename.endsWith(TimeIndexFileSuffix) || filename.endsWith(TxnIndexFileSuffix) } - private def isLogFile(file: File): Boolean = + private[log] def isLogFile(file: File): Boolean = file.getPath.endsWith(LogFileSuffix) private def loadProducersFromRecords(producerStateManager: ProducerStateManager, records: Records): Unit = { @@ -2784,6 +2216,382 @@ object Log { appendInfo.append(batch, firstOffsetMetadata) } + /** + * If the recordVersion is >= RecordVersion.V2, then create and return a LeaderEpochFileCache. + * Otherwise, the message format is considered incompatible and the existing LeaderEpoch file + * is deleted. + * + * @param dir The directory in which the log will reside + * @param topicPartition The topic partition + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param recordVersion The record version + * @return The new LeaderEpochFileCache instance (if created), none otherwise + */ + def maybeCreateLeaderEpochCache(dir: File, + topicPartition: TopicPartition, + logDirFailureChannel: LogDirFailureChannel, + recordVersion: RecordVersion): Option[LeaderEpochFileCache] = { + val leaderEpochFile = LeaderEpochCheckpointFile.newFile(dir) + + def newLeaderEpochFileCache(): LeaderEpochFileCache = { + val checkpointFile = new LeaderEpochCheckpointFile(leaderEpochFile, logDirFailureChannel) + new LeaderEpochFileCache(topicPartition, checkpointFile) + } + + if (recordVersion.precedes(RecordVersion.V2)) { + val currentCache = if (leaderEpochFile.exists()) + Some(newLeaderEpochFileCache()) + else + None + + if (currentCache.exists(_.nonEmpty)) + warn(s"Deleting non-empty leader epoch cache due to incompatible message format $recordVersion") + + Files.deleteIfExists(leaderEpochFile.toPath) + None + } else { + Some(newLeaderEpochFileCache()) + } + } + + /** + * Swap one or more new segment in place and delete one or more existing segments in a crash-safe + * manner. The old segments will be asynchronously deleted. + * + * This method does not need to convert IOException to KafkaStorageException because it is either + * called before all logs are loaded or the caller will catch and handle IOException + * + * The sequence of operations is: + * + * - Cleaner creates one or more new segments with suffix .cleaned and invokes replaceSegments() on + * the Log instance. If broker crashes at this point, the clean-and-swap operation is aborted and + * the .cleaned files are deleted on recovery in LogLoader. + * - New segments are renamed .swap. If the broker crashes before all segments were renamed to .swap, the + * clean-and-swap operation is aborted - .cleaned as well as .swap files are deleted on recovery in + * in LogLoader. We detect this situation by maintaining a specific order in which files are renamed + * from .cleaned to .swap. Basically, files are renamed in descending order of offsets. On recovery, + * all .swap files whose offset is greater than the minimum-offset .clean file are deleted. + * - If the broker crashes after all new segments were renamed to .swap, the operation is completed, + * the swap operation is resumed on recovery as described in the next step. + * - Old segment files are renamed to .deleted and asynchronous delete is scheduled. If the broker + * crashes, any .deleted files left behind are deleted on recovery in LogLoader. + * replaceSegments() is then invoked to complete the swap with newSegment recreated from the + * .swap file and oldSegments containing segments which were not renamed before the crash. + * - Swap segment(s) are renamed to replace the existing segments, completing this operation. + * If the broker crashes, any .deleted files which may be left behind are deleted + * on recovery in LogLoader. + * + * @param existingSegments The existing segments of the log + * @param newSegments The new log segment to add to the log + * @param oldSegments The old log segments to delete from the log + * @param isRecoveredSwapFile true if the new segment was created from a swap file during recovery after a crash + * @param dir The directory in which the log will reside + * @param topicPartition The topic + * @param config The log configuration settings + * @param scheduler The thread pool scheduler used for background actions + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param producerStateManager The ProducerStateManager instance (if any) containing state associated + * with the existingSegments + */ + private[log] def replaceSegments(existingSegments: LogSegments, + newSegments: Seq[LogSegment], + oldSegments: Seq[LogSegment], + isRecoveredSwapFile: Boolean = false, + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + producerStateManager: ProducerStateManager): Unit = { + val sortedNewSegments = newSegments.sortBy(_.baseOffset) + // Some old segments may have been removed from index and scheduled for async deletion after the caller reads segments + // but before this method is executed. We want to filter out those segments to avoid calling asyncDeleteSegment() + // multiple times for the same segment. + val sortedOldSegments = oldSegments.filter(seg => existingSegments.contains(seg.baseOffset)).sortBy(_.baseOffset) + + // need to do this in two phases to be crash safe AND do the delete asynchronously + // if we crash in the middle of this we complete the swap in loadSegments() + if (!isRecoveredSwapFile) + sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix, false)) + sortedNewSegments.reverse.foreach(existingSegments.add(_)) + val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet + + // delete the old files + sortedOldSegments.foreach { seg => + // remove the index entry + if (seg.baseOffset != sortedNewSegments.head.baseOffset) + existingSegments.remove(seg.baseOffset) + // delete segment files, but do not delete producer state for segment objects which are being replaced. + deleteSegmentFiles( + List(seg), + asyncDelete = true, + deleteProducerStateSnapshots = !newSegmentBaseOffsets.contains(seg.baseOffset), + dir, + topicPartition, + config, + scheduler, + logDirFailureChannel, + producerStateManager) + } + // okay we are safe now, remove the swap suffix + sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) + } + + /** + * Perform physical deletion of the index, log and producer snapshot files for the given segment. + * Prior to the deletion, the index and log files are renamed by appending .deleted to the + * respective file name. Allows these files to be optionally deleted asynchronously. + * + * This method assumes that the file exists. It does not need to convert IOException + * (thrown from changeFileSuffixes) to KafkaStorageException because it is either called before + * all logs are loaded or the caller will catch and handle IOException. + * + * @param segmentsToDelete The segments to be deleted + * @param asyncDelete If true, the deletion of the segments is done asynchronously + * @param deleteProducerStateSnapshots If true, the producer state snapshot associated with a + * segment will be deleted after the segment is deleted + * @param dir The directory in which the log will reside + * @param topicPartition The topic + * @param config The log configuration settings + * @param scheduler The thread pool scheduler used for background actions + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param producerStateManager The ProducerStateManager instance (if any) containing state associated + * with the existingSegments + * + * @throws IOException if the file can't be renamed and still exists + */ + private[log] def deleteSegmentFiles(segmentsToDelete: Iterable[LogSegment], + asyncDelete: Boolean, + deleteProducerStateSnapshots: Boolean = true, + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + producerStateManager: ProducerStateManager): Unit = { + segmentsToDelete.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix, false)) + + def deleteSegments(): Unit = { + info(s"Deleting segment files ${segmentsToDelete.mkString(",")}") + val parentDir = dir.getParent + maybeHandleIOException(logDirFailureChannel, parentDir, s"Error while deleting segments for $topicPartition in dir $parentDir") { + segmentsToDelete.foreach { segment => + segment.deleteIfExists() + if (deleteProducerStateSnapshots) + producerStateManager.removeAndDeleteSnapshot(segment.baseOffset) + } + } + } + + if (asyncDelete) + scheduler.schedule("delete-file", () => deleteSegments(), delay = config.fileDeleteDelayMs) + else + deleteSegments() + } + + /** + * Invokes the provided function and handles any IOException raised by the function by marking the + * provided directory offline. + * + * @param logDirFailureChannel Used to asynchronously handle log directory failure. + * @param logDir The log directory to be marked offline during an IOException. + * @param errorMsg The error message to be used when marking the log directory offline. + * @param fun The function to be executed. + * @return The value returned by the function after a successful invocation + */ + private def maybeHandleIOException[T](logDirFailureChannel: LogDirFailureChannel, + logDir: String, + errorMsg: => String)(fun: => T): T = { + if (logDirFailureChannel.hasOfflineLogDir(logDir)) { + throw new KafkaStorageException(s"The log dir $logDir is already offline due to a previous IO exception.") + } + try { + fun + } catch { + case e: IOException => + logDirFailureChannel.maybeAddOfflineLogDir(logDir, errorMsg, e) + throw new KafkaStorageException(errorMsg, e) + } + } + + /** + * Rebuilds producer state until the provided lastOffset. This function may be called from the + * recovery code path, and thus must be free of all side-effects, i.e. it must not update any + * log-specific state. + * + * @param producerStateManager The ProducerStateManager instance to be rebuilt. + * @param segments The segments of the log whose producer state is being rebuilt + * @param logStartOffset The log start offset + * @param lastOffset The last offset upto which the producer state needs to be rebuilt + * @param recordVersion The record version + * @param time The time instance used for checking the clock + * @param reloadFromCleanShutdown True if the producer state is being built after a clean shutdown, + * false otherwise. + */ + private[log] def rebuildProducerState(producerStateManager: ProducerStateManager, + segments: LogSegments, + logStartOffset: Long, + lastOffset: Long, + recordVersion: RecordVersion, + time: Time, + reloadFromCleanShutdown: Boolean): Unit = { + val allSegments = segments.values + val offsetsToSnapshot = + if (allSegments.nonEmpty) { + val nextLatestSegmentBaseOffset = segments.lowerSegment(allSegments.last.baseOffset).map(_.baseOffset) + Seq(nextLatestSegmentBaseOffset, Some(allSegments.last.baseOffset), Some(lastOffset)) + } else { + Seq(Some(lastOffset)) + } + info(s"Loading producer state till offset $lastOffset with message format version ${recordVersion.value}") + + // We want to avoid unnecessary scanning of the log to build the producer state when the broker is being + // upgraded. The basic idea is to use the absence of producer snapshot files to detect the upgrade case, + // but we have to be careful not to assume too much in the presence of broker failures. The two most common + // upgrade cases in which we expect to find no snapshots are the following: + // + // 1. The broker has been upgraded, but the topic is still on the old message format. + // 2. The broker has been upgraded, the topic is on the new message format, and we had a clean shutdown. + // + // If we hit either of these cases, we skip producer state loading and write a new snapshot at the log end + // offset (see below). The next time the log is reloaded, we will load producer state using this snapshot + // (or later snapshots). Otherwise, if there is no snapshot file, then we have to rebuild producer state + // from the first segment. + if (recordVersion.value < RecordBatch.MAGIC_VALUE_V2 || + (producerStateManager.latestSnapshotOffset.isEmpty && reloadFromCleanShutdown)) { + // To avoid an expensive scan through all of the segments, we take empty snapshots from the start of the + // last two segments and the last offset. This should avoid the full scan in the case that the log needs + // truncation. + offsetsToSnapshot.flatten.foreach { offset => + producerStateManager.updateMapEndOffset(offset) + producerStateManager.takeSnapshot() + } + } else { + info(s"Reloading from producer snapshot and rebuilding producer state from offset $lastOffset") + val isEmptyBeforeTruncation = producerStateManager.isEmpty && producerStateManager.mapEndOffset >= lastOffset + val producerStateLoadStart = time.milliseconds() + producerStateManager.truncateAndReload(logStartOffset, lastOffset, time.milliseconds()) + val segmentRecoveryStart = time.milliseconds() + + // Only do the potentially expensive reloading if the last snapshot offset is lower than the log end + // offset (which would be the case on first startup) and there were active producers prior to truncation + // (which could be the case if truncating after initial loading). If there weren't, then truncating + // shouldn't change that fact (although it could cause a producerId to expire earlier than expected), + // and we can skip the loading. This is an optimization for users which are not yet using + // idempotent/transactional features yet. + if (lastOffset > producerStateManager.mapEndOffset && !isEmptyBeforeTruncation) { + val segmentOfLastOffset = segments.floorSegment(lastOffset) + + segments.values(producerStateManager.mapEndOffset, lastOffset).foreach { segment => + val startOffset = Utils.max(segment.baseOffset, producerStateManager.mapEndOffset, logStartOffset) + producerStateManager.updateMapEndOffset(startOffset) + + if (offsetsToSnapshot.contains(Some(segment.baseOffset))) + producerStateManager.takeSnapshot() + + val maxPosition = if (segmentOfLastOffset.contains(segment)) { + Option(segment.translateOffset(lastOffset)) + .map(_.position) + .getOrElse(segment.size) + } else { + segment.size + } + + val fetchDataInfo = segment.read(startOffset, + maxSize = Int.MaxValue, + maxPosition = maxPosition) + if (fetchDataInfo != null) + loadProducersFromRecords(producerStateManager, fetchDataInfo.records) + } + } + producerStateManager.updateMapEndOffset(lastOffset) + producerStateManager.takeSnapshot() + info(s"Producer state recovery took ${segmentRecoveryStart - producerStateLoadStart}ms for snapshot load " + + s"and ${time.milliseconds() - segmentRecoveryStart}ms for segment recovery from offset $lastOffset") + } + } + + /** + * Split a segment into one or more segments such that there is no offset overflow in any of them. The + * resulting segments will contain the exact same messages that are present in the input segment. On successful + * completion of this method, the input segment will be deleted and will be replaced by the resulting new segments. + * See replaceSegments for recovery logic, in case the broker dies in the middle of this operation. + * + * Note that this method assumes we have already determined that the segment passed in contains records that cause + * offset overflow. + * + * The split logic overloads the use of .clean files that LogCleaner typically uses to make the process of replacing + * the input segment with multiple new segments atomic and recoverable in the event of a crash. See replaceSegments + * and completeSwapOperations for the implementation to make this operation recoverable on crashes.

    + * + * @param segment Segment to split + * @param existingSegments The existing segments of the log + * @param dir The directory in which the log will reside + * @param topicPartition The topic + * @param config The log configuration settings + * @param scheduler The thread pool scheduler used for background actions + * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure + * @param producerStateManager The ProducerStateManager instance (if any) containing state associated + * with the existingSegments + * @return List of new segments that replace the input segment + */ + private[log] def splitOverflowedSegment(segment: LogSegment, + existingSegments: LogSegments, + dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + logDirFailureChannel: LogDirFailureChannel, + producerStateManager: ProducerStateManager): List[LogSegment] = { + require(Log.isLogFile(segment.log.file), s"Cannot split file ${segment.log.file.getAbsoluteFile}") + require(segment.hasOverflow, "Split operation is only permitted for segments with overflow") + + info(s"Splitting overflowed segment $segment") + + val newSegments = ListBuffer[LogSegment]() + try { + var position = 0 + val sourceRecords = segment.log + + while (position < sourceRecords.sizeInBytes) { + val firstBatch = sourceRecords.batchesFrom(position).asScala.head + val newSegment = LogCleaner.createNewCleanedSegment(dir, config, firstBatch.baseOffset) + newSegments += newSegment + + val bytesAppended = newSegment.appendFromFile(sourceRecords, position) + if (bytesAppended == 0) + throw new IllegalStateException(s"Failed to append records from position $position in $segment") + + position += bytesAppended + } + + // prepare new segments + var totalSizeOfNewSegments = 0 + newSegments.foreach { splitSegment => + splitSegment.onBecomeInactiveSegment() + splitSegment.flush() + splitSegment.lastModified = segment.lastModified + totalSizeOfNewSegments += splitSegment.log.sizeInBytes + } + // size of all the new segments combined must equal size of the original segment + if (totalSizeOfNewSegments != segment.log.sizeInBytes) + throw new IllegalStateException("Inconsistent segment sizes after split" + + s" before: ${segment.log.sizeInBytes} after: $totalSizeOfNewSegments") + + // replace old segment with new ones + info(s"Replacing overflowed segment $segment with split segments $newSegments") + replaceSegments(existingSegments, newSegments.toList, List(segment), isRecoveredSwapFile = false, + dir, topicPartition, config, scheduler, logDirFailureChannel, producerStateManager) + newSegments.toList + } catch { + case e: Exception => + newSegments.foreach { splitSegment => + splitSegment.close() + splitSegment.deleteIfExists() + } + throw e + } + } } object LogMetricNames { @@ -2834,12 +2642,6 @@ case object StartOffsetBreach extends SegmentDeletionReason { } } -case object LogRecovery extends SegmentDeletionReason { - override def logReason(log: Log, toDelete: List[LogSegment]): Unit = { - log.info(s"Deleting segments as part of log recovery: ${toDelete.mkString(",")}") - } -} - case object LogTruncation extends SegmentDeletionReason { override def logReason(log: Log, toDelete: List[LogSegment]): Unit = { log.info(s"Deleting segments as part of log truncation: ${toDelete.mkString(",")}") diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index df9722c6ee2e0..80916cddaa8ba 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -450,12 +450,11 @@ object LogCleaner { } - def createNewCleanedSegment(log: Log, baseOffset: Long): LogSegment = { - LogSegment.deleteIfExists(log.dir, baseOffset, fileSuffix = Log.CleanedFileSuffix) - LogSegment.open(log.dir, baseOffset, log.config, Time.SYSTEM, - fileSuffix = Log.CleanedFileSuffix, initFileSize = log.initFileSize, preallocate = log.config.preallocate) + def createNewCleanedSegment(dir: File, logConfig: LogConfig, baseOffset: Long): LogSegment = { + LogSegment.deleteIfExists(dir, baseOffset, fileSuffix = Log.CleanedFileSuffix) + LogSegment.open(dir, baseOffset, logConfig, Time.SYSTEM, + fileSuffix = Log.CleanedFileSuffix, initFileSize = logConfig.initFileSize, preallocate = logConfig.preallocate) } - } /** @@ -563,7 +562,7 @@ private[log] class Cleaner(val id: Int, stats: CleanerStats, transactionMetadata: CleanedTransactionMetadata): Unit = { // create a new segment with a suffix appended to the name of the log and indexes - val cleaned = LogCleaner.createNewCleanedSegment(log, segments.head.baseOffset) + val cleaned = LogCleaner.createNewCleanedSegment(log.dir, log.config, segments.head.baseOffset) transactionMetadata.cleanedIndex = Some(cleaned.txnIndex) try { diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 4a184423c7fa3..874dc13ea0af6 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -103,6 +103,13 @@ case class LogConfig(props: java.util.Map[_, _], overriddenConfigs: Set[String] if (compact && maxCompactionLagMs > 0) math.min(maxCompactionLagMs, segmentMs) else segmentMs } + + def initFileSize: Int = { + if (preallocate) + segmentSize + else + 0 + } } object LogConfig { diff --git a/core/src/main/scala/kafka/log/LogLoader.scala b/core/src/main/scala/kafka/log/LogLoader.scala new file mode 100644 index 0000000000000..6b28ec51bb91d --- /dev/null +++ b/core/src/main/scala/kafka/log/LogLoader.scala @@ -0,0 +1,525 @@ +/** + * 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 kafka.log + +import java.io.{File, IOException} +import java.nio.file.{Files, NoSuchFileException} + +import kafka.common.LogSegmentOffsetOverflowException +import kafka.log.Log.{CleanedFileSuffix, DeletedFileSuffix, SwapFileSuffix, isIndexFile, isLogFile, offsetFromFile, offsetFromFileName} +import kafka.server.{LogDirFailureChannel, LogOffsetMetadata} +import kafka.server.epoch.LeaderEpochFileCache +import kafka.utils.{CoreUtils, Logging, Scheduler} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.errors.InvalidOffsetException +import org.apache.kafka.common.utils.Time + +import scala.collection.{Seq, Set, mutable} + +case class LoadedLogOffsets(logStartOffset: Long, + recoveryPoint: Long, + nextOffsetMetadata: LogOffsetMetadata) + +/** + * @param dir The directory from which log segments need to be loaded + * @param topicPartition The topic partition associated with the log being loaded + * @param config The configuration settings for the log being loaded + * @param scheduler The thread pool scheduler used for background actions + * @param time The time instance used for checking the clock + * @param logDirFailureChannel The LogDirFailureChannel instance to asynchronously handle log + * directory failure + * @param hadCleanShutdown Boolean flag to indicate whether the associated log previously had a + * clean shutdown + * @param segments The LogSegments instance into which segments recovered from disk will be + * populated + * @param logStartOffsetCheckpoint The checkpoint of the log start offset + * @param recoveryPointCheckpoint The checkpoint of the offset at which to begin the recovery + * @param maxProducerIdExpirationMs The maximum amount of time to wait before a producer id is + * considered expired + * @param leaderEpochCache An optional LeaderEpochFileCache instance to be updated during recovery + * @param producerStateManager The ProducerStateManager instance to be updated during recovery + */ +case class LoadLogParams(dir: File, + topicPartition: TopicPartition, + config: LogConfig, + scheduler: Scheduler, + time: Time, + logDirFailureChannel: LogDirFailureChannel, + hadCleanShutdown: Boolean, + segments: LogSegments, + logStartOffsetCheckpoint: Long, + recoveryPointCheckpoint: Long, + maxProducerIdExpirationMs: Int, + leaderEpochCache: Option[LeaderEpochFileCache], + producerStateManager: ProducerStateManager) { + val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}]" +} + +/** + * This object is responsible for all activities related with recovery of log segments from disk. + */ +object LogLoader extends Logging { + /** + * Load the log segments from the log files on disk, and return the components of the loaded log. + * Additionally, it also suitably updates the provided LeaderEpochFileCache and ProducerStateManager + * to reflect the contents of the loaded log. + * + * In the context of the calling thread, this function does not need to convert IOException to + * KafkaStorageException because it is only called before all logs are loaded. + * + * @param params The parameters for the log being loaded from disk + * + * @return the offsets of the Log successfully loaded from disk + * + * @throws LogSegmentOffsetOverflowException if we encounter a .swap file with messages that + * overflow index offset + */ + def load(params: LoadLogParams): LoadedLogOffsets = { + // first do a pass through the files in the log directory and remove any temporary files + // and find any interrupted swap operations + val swapFiles = removeTempFilesAndCollectSwapFiles(params) + + // Now do a second pass and load all the log and index files. + // We might encounter legacy log segments with offset overflow (KAFKA-6264). We need to split such segments. When + // this happens, restart loading segment files from scratch. + retryOnOffsetOverflow(params, { + // In case we encounter a segment with offset overflow, the retry logic will split it after which we need to retry + // loading of segments. In that case, we also need to close all segments that could have been left open in previous + // call to loadSegmentFiles(). + params.segments.close() + params.segments.clear() + loadSegmentFiles(params) + }) + + completeSwapOperations(swapFiles, params) + + val (newRecoveryPoint: Long, nextOffset: Long) = { + if (!params.dir.getAbsolutePath.endsWith(Log.DeleteDirSuffix)) { + val (newRecoveryPoint, nextOffset) = retryOnOffsetOverflow(params, { + recoverLog(params) + }) + + // reset the index size of the currently active log segment to allow more entries + params.segments.lastSegment.get.resizeIndexes(params.config.maxIndexSize) + (newRecoveryPoint, nextOffset) + } else { + if (params.segments.isEmpty) { + params.segments.add( + LogSegment.open( + dir = params.dir, + baseOffset = 0, + params.config, + time = params.time, + initFileSize = params.config.initFileSize)) + } + (0L, 0L) + } + } + + params.leaderEpochCache.foreach(_.truncateFromEnd(nextOffset)) + val newLogStartOffset = math.max(params.logStartOffsetCheckpoint, params.segments.firstSegment.get.baseOffset) + // The earliest leader epoch may not be flushed during a hard failure. Recover it here. + params.leaderEpochCache.foreach(_.truncateFromStart(params.logStartOffsetCheckpoint)) + + // Any segment loading or recovery code must not use producerStateManager, so that we can build the full state here + // from scratch. + if (!params.producerStateManager.isEmpty) + throw new IllegalStateException("Producer state must be empty during log initialization") + + // Reload all snapshots into the ProducerStateManager cache, the intermediate ProducerStateManager used + // during log recovery may have deleted some files without the LogLoader.producerStateManager instance witnessing the + // deletion. + params.producerStateManager.removeStraySnapshots(params.segments.baseOffsets.toSeq) + Log.rebuildProducerState( + params.producerStateManager, + params.segments, + newLogStartOffset, + nextOffset, + params.config.messageFormatVersion.recordVersion, + params.time, + reloadFromCleanShutdown = params.hadCleanShutdown) + + val activeSegment = params.segments.lastSegment.get + LoadedLogOffsets( + newLogStartOffset, + newRecoveryPoint, + LogOffsetMetadata(nextOffset, activeSegment.baseOffset, activeSegment.size)) + } + + /** + * Removes any temporary files found in log directory, and creates a list of all .swap files which could be swapped + * in place of existing segment(s). For log splitting, we know that any .swap file whose base offset is higher than + * the smallest offset .clean file could be part of an incomplete split operation. Such .swap files are also deleted + * by this method. + * @param params The parameters for the log being loaded from disk + * @return Set of .swap files that are valid to be swapped in as segment files + */ + private def removeTempFilesAndCollectSwapFiles(params: LoadLogParams): Set[File] = { + + def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = { + info(s"${params.logIdentifier} Deleting index files with suffix $suffix for baseFile $baseFile") + val offset = offsetFromFile(baseFile) + Files.deleteIfExists(Log.offsetIndexFile(params.dir, offset, suffix).toPath) + Files.deleteIfExists(Log.timeIndexFile(params.dir, offset, suffix).toPath) + Files.deleteIfExists(Log.transactionIndexFile(params.dir, offset, suffix).toPath) + } + + val swapFiles = mutable.Set[File]() + val cleanFiles = mutable.Set[File]() + var minCleanedFileOffset = Long.MaxValue + + for (file <- params.dir.listFiles if file.isFile) { + if (!file.canRead) + throw new IOException(s"Could not read file $file") + val filename = file.getName + if (filename.endsWith(DeletedFileSuffix)) { + debug(s"${params.logIdentifier} Deleting stray temporary file ${file.getAbsolutePath}") + Files.deleteIfExists(file.toPath) + } else if (filename.endsWith(CleanedFileSuffix)) { + minCleanedFileOffset = Math.min(offsetFromFileName(filename), minCleanedFileOffset) + cleanFiles += file + } else if (filename.endsWith(SwapFileSuffix)) { + // we crashed in the middle of a swap operation, to recover: + // if a log, delete the index files, complete the swap operation later + // if an index just delete the index files, they will be rebuilt + val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) + info(s"${params.logIdentifier} Found file ${file.getAbsolutePath} from interrupted swap operation.") + if (Log.isIndexFile(baseFile)) { + deleteIndicesIfExist(baseFile) + } else if (Log.isLogFile(baseFile)) { + deleteIndicesIfExist(baseFile) + swapFiles += file + } + } + } + + // KAFKA-6264: Delete all .swap files whose base offset is greater than the minimum .cleaned segment offset. Such .swap + // files could be part of an incomplete split operation that could not complete. See Log#splitOverflowedSegment + // for more details about the split operation. + val (invalidSwapFiles, validSwapFiles) = swapFiles.partition(file => offsetFromFile(file) >= minCleanedFileOffset) + invalidSwapFiles.foreach { file => + debug(s"${params.logIdentifier} Deleting invalid swap file ${file.getAbsoluteFile} minCleanedFileOffset: $minCleanedFileOffset") + val baseFile = new File(CoreUtils.replaceSuffix(file.getPath, SwapFileSuffix, "")) + deleteIndicesIfExist(baseFile, SwapFileSuffix) + Files.deleteIfExists(file.toPath) + } + + // Now that we have deleted all .swap files that constitute an incomplete split operation, let's delete all .clean files + cleanFiles.foreach { file => + debug(s"${params.logIdentifier} Deleting stray .clean file ${file.getAbsolutePath}") + Files.deleteIfExists(file.toPath) + } + + validSwapFiles + } + + /** + * Retries the provided function only whenever an LogSegmentOffsetOverflowException is raised by + * it during execution. Before every retry, the overflowed segment is split into one or more segments + * such that there is no offset overflow in any of them. + * + * @param params The parameters for the log being loaded from disk + * @param fn The function to be executed + * @return The value returned by the function, if successful + * @throws Exception whenever the executed function throws any exception other than + * LogSegmentOffsetOverflowException, the same exception is raised to the caller + */ + private def retryOnOffsetOverflow[T](params: LoadLogParams, fn: => T): T = { + while (true) { + try { + return fn + } catch { + case e: LogSegmentOffsetOverflowException => + info(s"${params.logIdentifier} Caught segment overflow error: ${e.getMessage}. Split segment and retry.") + Log.splitOverflowedSegment( + e.segment, + params.segments, + params.dir, + params.topicPartition, + params.config, + params.scheduler, + params.logDirFailureChannel, + params.producerStateManager) + } + } + throw new IllegalStateException() + } + + /** + * Loads segments from disk into the provided params.segments. + * + * This method does not need to convert IOException to KafkaStorageException because it is only called before all logs are loaded. + * It is possible that we encounter a segment with index offset overflow in which case the LogSegmentOffsetOverflowException + * will be thrown. Note that any segments that were opened before we encountered the exception will remain open and the + * caller is responsible for closing them appropriately, if needed. + * + * @param params The parameters for the log being loaded from disk + * @throws LogSegmentOffsetOverflowException if the log directory contains a segment with messages that overflow the index offset + */ + private def loadSegmentFiles(params: LoadLogParams): Unit = { + // load segments in ascending order because transactional data from one segment may depend on the + // segments that come before it + for (file <- params.dir.listFiles.sortBy(_.getName) if file.isFile) { + if (isIndexFile(file)) { + // if it is an index file, make sure it has a corresponding .log file + val offset = offsetFromFile(file) + val logFile = Log.logFile(params.dir, offset) + if (!logFile.exists) { + warn(s"${params.logIdentifier} Found an orphaned index file ${file.getAbsolutePath}, with no corresponding log file.") + Files.deleteIfExists(file.toPath) + } + } else if (isLogFile(file)) { + // if it's a log file, load the corresponding log segment + val baseOffset = offsetFromFile(file) + val timeIndexFileNewlyCreated = !Log.timeIndexFile(params.dir, baseOffset).exists() + val segment = LogSegment.open( + dir = params.dir, + baseOffset = baseOffset, + params.config, + time = params.time, + fileAlreadyExists = true) + + try segment.sanityCheck(timeIndexFileNewlyCreated) + catch { + case _: NoSuchFileException => + error(s"${params.logIdentifier} Could not find offset index file corresponding to log file" + + s" ${segment.log.file.getAbsolutePath}, recovering segment and rebuilding index files...") + recoverSegment(segment, params) + case e: CorruptIndexException => + warn(s"${params.logIdentifier} Found a corrupted index file corresponding to log file" + + s" ${segment.log.file.getAbsolutePath} due to ${e.getMessage}}, recovering segment and" + + " rebuilding index files...") + recoverSegment(segment, params) + } + params.segments.add(segment) + } + } + } + + /** + * Just recovers the given segment, without adding it to the provided params.segments. + * + * @param segment Segment to recover + * @param params The parameters for the log being loaded from disk + * + * @return The number of bytes truncated from the segment + * + * @throws LogSegmentOffsetOverflowException if the segment contains messages that cause index offset overflow + */ + private def recoverSegment(segment: LogSegment, params: LoadLogParams): Int = { + val producerStateManager = new ProducerStateManager(params.topicPartition, params.dir, params.maxProducerIdExpirationMs) + Log.rebuildProducerState( + producerStateManager, + params.segments, + params.logStartOffsetCheckpoint, + segment.baseOffset, + params.config.messageFormatVersion.recordVersion, + params.time, + reloadFromCleanShutdown = false) + val bytesTruncated = segment.recover(producerStateManager, params.leaderEpochCache) + // once we have recovered the segment's data, take a snapshot to ensure that we won't + // need to reload the same segment again while recovering another segment. + producerStateManager.takeSnapshot() + bytesTruncated + } + + /** + * This method completes any interrupted swap operations. In order to be crash-safe, the log files + * that are replaced by the swap segment should be renamed to .deleted before the swap file is + * restored as the new segment file. + * + * This method does not need to convert IOException to KafkaStorageException because it is only + * called before all logs are loaded. + * + * @param swapFiles the set of swap + * @param params The parameters for the log being loaded from disk + * + * @throws LogSegmentOffsetOverflowException if the swap file contains messages that cause the log segment offset to + * overflow. Note that this is currently a fatal exception as we do not have + * a way to deal with it. The exception is propagated all the way up to + * KafkaServer#startup which will cause the broker to shut down if we are in + * this situation. This is expected to be an extremely rare scenario in practice, + * and manual intervention might be required to get out of it. + */ + private def completeSwapOperations(swapFiles: Set[File], + params: LoadLogParams): Unit = { + for (swapFile <- swapFiles) { + val logFile = new File(CoreUtils.replaceSuffix(swapFile.getPath, Log.SwapFileSuffix, "")) + val baseOffset = Log.offsetFromFile(logFile) + val swapSegment = LogSegment.open(swapFile.getParentFile, + baseOffset = baseOffset, + params.config, + time = params.time, + fileSuffix = Log.SwapFileSuffix) + info(s"Found log file ${swapFile.getPath} from interrupted swap operation, repairing.") + recoverSegment(swapSegment, params) + + // We create swap files for two cases: + // (1) Log cleaning where multiple segments are merged into one, and + // (2) Log splitting where one segment is split into multiple. + // + // Both of these mean that the resultant swap segments be composed of the original set, i.e. the swap segment + // must fall within the range of existing segment(s). If we cannot find such a segment, it means the deletion + // of that segment was successful. In such an event, we should simply rename the .swap to .log without having to + // do a replace with an existing segment. + val oldSegments = params.segments.values(swapSegment.baseOffset, swapSegment.readNextOffset).filter { segment => + segment.readNextOffset > swapSegment.baseOffset + } + Log.replaceSegments( + params.segments, + Seq(swapSegment), + oldSegments.toSeq, + isRecoveredSwapFile = true, + params.dir, + params.topicPartition, + params.config, + params.scheduler, + params.logDirFailureChannel, + params.producerStateManager) + } + } + + /** + * Recover the log segments (if there was an unclean shutdown). Ensures there is at least one + * active segment, and returns the updated recovery point and next offset after recovery. Along + * the way, the method suitably updates the LeaderEpochFileCache or ProducerStateManager inside + * the provided LogComponents. + * + * This method does not need to convert IOException to KafkaStorageException because it is only + * called before all logs are loaded. + * + * @param params The parameters for the log being loaded from disk + * + * @return a tuple containing (newRecoveryPoint, nextOffset). + * + * @throws LogSegmentOffsetOverflowException if we encountered a legacy segment with offset overflow + */ + private[log] def recoverLog(params: LoadLogParams): (Long, Long) = { + /** return the log end offset if valid */ + def deleteSegmentsIfLogStartGreaterThanLogEnd(): Option[Long] = { + if (params.segments.nonEmpty) { + val logEndOffset = params.segments.lastSegment.get.readNextOffset + if (logEndOffset >= params.logStartOffsetCheckpoint) + Some(logEndOffset) + else { + warn(s"Deleting all segments because logEndOffset ($logEndOffset) is smaller than logStartOffset ${params.logStartOffsetCheckpoint}. " + + "This could happen if segment files were deleted from the file system.") + removeAndDeleteSegmentsAsync(params.segments.values, params) + params.leaderEpochCache.foreach(_.clearAndFlush()) + params.producerStateManager.truncateFullyAndStartAt(params.logStartOffsetCheckpoint) + None + } + } else None + } + + // If we have the clean shutdown marker, skip recovery. + if (!params.hadCleanShutdown) { + val unflushed = params.segments.values(params.recoveryPointCheckpoint, Long.MaxValue).iterator + var truncated = false + + while (unflushed.hasNext && !truncated) { + val segment = unflushed.next() + info(s"${params.logIdentifier} Recovering unflushed segment ${segment.baseOffset}") + val truncatedBytes = + try { + recoverSegment(segment, params) + } catch { + case _: InvalidOffsetException => + val startOffset = segment.baseOffset + warn(s"${params.logIdentifier} Found invalid offset during recovery. Deleting the" + + s" corrupt segment and creating an empty one with starting offset $startOffset") + segment.truncateTo(startOffset) + } + if (truncatedBytes > 0) { + // we had an invalid message, delete all remaining log + warn(s"${params.logIdentifier} Corruption found in segment ${segment.baseOffset}," + + s" truncating to offset ${segment.readNextOffset}") + removeAndDeleteSegmentsAsync(unflushed.toList, params) + truncated = true + } + } + } + + val logEndOffsetOption = deleteSegmentsIfLogStartGreaterThanLogEnd() + + if (params.segments.isEmpty) { + // no existing segments, create a new mutable segment beginning at logStartOffset + params.segments.add( + LogSegment.open( + dir = params.dir, + baseOffset = params.logStartOffsetCheckpoint, + params.config, + time = params.time, + initFileSize = params.config.initFileSize, + preallocate = params.config.preallocate)) + } + + // Update the recovery point if there was a clean shutdown and did not perform any changes to + // the segment. Otherwise, we just ensure that the recovery point is not ahead of the log end + // offset. To ensure correctness and to make it easier to reason about, it's best to only advance + // the recovery point when the log is flushed. If we advanced the recovery point here, we could + // skip recovery for unflushed segments if the broker crashed after we checkpoint the recovery + // point and before we flush the segment. + (params.hadCleanShutdown, logEndOffsetOption) match { + case (true, Some(logEndOffset)) => + (logEndOffset, logEndOffset) + case _ => + val logEndOffset = logEndOffsetOption.getOrElse(params.segments.lastSegment.get.readNextOffset) + (Math.min(params.recoveryPointCheckpoint, logEndOffset), logEndOffset) + } + } + + /** + * This method deletes the given log segments and the associated producer snapshots, by doing the + * following for each of them: + * - It removes the segment from the segment map so that it will no longer be used for reads. + * - It schedules asynchronous deletion of the segments that allows reads to happen concurrently without + * synchronization and without the possibility of physically deleting a file while it is being + * read. + * + * This method does not need to convert IOException to KafkaStorageException because it is either + * called before all logs are loaded or the immediate caller will catch and handle IOException + * + * @param segmentsToDelete The log segments to schedule for deletion + * @param params The parameters for the log being loaded from disk + */ + private def removeAndDeleteSegmentsAsync(segmentsToDelete: Iterable[LogSegment], + params: LoadLogParams): Unit = { + if (segmentsToDelete.nonEmpty) { + // As most callers hold an iterator into the `params.segments` collection and + // `removeAndDeleteSegmentAsync` mutates it by removing the deleted segment, we should force + // materialization of the iterator here, so that results of the iteration remain valid and + // deterministic. + val toDelete = segmentsToDelete.toList + info(s"Deleting segments as part of log recovery: ${toDelete.mkString(",")}") + toDelete.foreach { segment => + params.segments.remove(segment.baseOffset) + } + Log.deleteSegmentFiles( + segmentsToDelete, + asyncDelete = true, + deleteProducerStateSnapshots = true, + params.dir, + params.topicPartition, + params.config, + params.scheduler, + params.logDirFailureChannel, + params.producerStateManager) + } + } +} diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index a8198c9c02e09..8b7b18b03693b 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -25,6 +25,7 @@ import kafka.api.ApiVersion import kafka.log._ import kafka.server._ import kafka.server.checkpoints.OffsetCheckpoints +import kafka.server.epoch.LeaderEpochFileCache import kafka.utils._ import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.{TopicPartition, Uuid} @@ -280,7 +281,25 @@ class PartitionLockTest extends Logging { override def createLog(isNew: Boolean, isFutureReplica: Boolean, offsetCheckpoints: OffsetCheckpoints, topicId: Option[Uuid]): Log = { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None) - new SlowLog(log, mockTime, appendSemaphore) + val logDirFailureChannel = new LogDirFailureChannel(1) + val segments = new LogSegments(log.topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, log.maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + log.dir, + log.topicPartition, + log.config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + log.maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) } } when(offsetCheckpoints.fetch(ArgumentMatchers.anyString, ArgumentMatchers.eq(topicPartition))) @@ -341,19 +360,31 @@ class PartitionLockTest extends Logging { } } - private class SlowLog(log: Log, mockTime: MockTime, appendSemaphore: Semaphore) extends Log( + private class SlowLog( + log: Log, + segments: LogSegments, + offsets: LoadedLogOffsets, + leaderEpochCache: Option[LeaderEpochFileCache], + producerStateManager: ProducerStateManager, + mockTime: MockTime, + logDirFailureChannel: LogDirFailureChannel, + appendSemaphore: Semaphore + ) extends Log( log.dir, log.config, - log.logStartOffset, - log.recoveryPoint, + segments, + offsets.logStartOffset, + offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, new BrokerTopicStats, - log.time, + mockTime, log.maxProducerIdExpirationMs, log.producerIdExpirationCheckIntervalMs, log.topicPartition, - log.producerStateManager, - new LogDirFailureChannel(1), + leaderEpochCache, + producerStateManager, + logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { @@ -362,5 +393,5 @@ class PartitionLockTest extends Logging { appendSemaphore.acquire() appendInfo } - } + } } diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index dd671e543e29c..784f70d1678b2 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -41,10 +41,12 @@ import org.mockito.ArgumentMatchers import org.mockito.ArgumentMatchers.{any, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock - import java.nio.ByteBuffer import java.util.Optional import java.util.concurrent.{CountDownLatch, Semaphore} + +import kafka.server.epoch.LeaderEpochFileCache + import scala.jdk.CollectionConverters._ class PartitionTest extends AbstractPartitionTest { @@ -232,7 +234,25 @@ class PartitionTest extends AbstractPartitionTest { override def createLog(isNew: Boolean, isFutureReplica: Boolean, offsetCheckpoints: OffsetCheckpoints, topicId: Option[Uuid]): Log = { val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None) - new SlowLog(log, mockTime, appendSemaphore) + val logDirFailureChannel = new LogDirFailureChannel(1) + val segments = new LogSegments(log.topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(log.topicPartition, log.dir, log.maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + log.dir, + log.topicPartition, + log.config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + log.maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + new SlowLog(log, segments, offsets, leaderEpochCache, producerStateManager, mockTime, logDirFailureChannel, appendSemaphore) } } @@ -1934,19 +1954,31 @@ class PartitionTest extends AbstractPartitionTest { } } - private class SlowLog(log: Log, mockTime: MockTime, appendSemaphore: Semaphore) extends Log( + private class SlowLog( + log: Log, + segments: LogSegments, + offsets: LoadedLogOffsets, + leaderEpochCache: Option[LeaderEpochFileCache], + producerStateManager: ProducerStateManager, + mockTime: MockTime, + logDirFailureChannel: LogDirFailureChannel, + appendSemaphore: Semaphore + ) extends Log( log.dir, log.config, - log.logStartOffset, - log.recoveryPoint, + segments, + offsets.logStartOffset, + offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, new BrokerTopicStats, - log.time, + mockTime, log.maxProducerIdExpirationMs, log.producerIdExpirationCheckIntervalMs, log.topicPartition, - log.producerStateManager, - new LogDirFailureChannel(1), + leaderEpochCache, + producerStateManager, + logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 16aac94dc5df6..cdaa89de9e34e 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -18,6 +18,7 @@ package kafka.log import java.io.File +import java.nio.file.Files import java.util.Properties import kafka.server.{BrokerTopicStats, LogDirFailureChannel} @@ -94,19 +95,39 @@ class LogCleanerManagerTest extends Logging { val logSegmentSize = TestUtils.singletonRecords("test".getBytes).sizeInBytes * 10 val logSegmentsCount = 2 val tpDir = new File(logDir, "A-1") - - // the exception should be catched and the partition that caused it marked as uncleanable - class LogMock(dir: File, config: LogConfig) extends Log(dir, config, 0L, 0L, - time.scheduler, new BrokerTopicStats, time, 60 * 60 * 1000, LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition, new ProducerStateManager(tp, tpDir, 60 * 60 * 1000), - new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true) { - + Files.createDirectories(tpDir.toPath) + val logDirFailureChannel = new LogDirFailureChannel(10) + val config = createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact) + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val segments = new LogSegments(tp) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + tpDir, + tp, + config, + time.scheduler, + time, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + // the exception should be caught and the partition that caused it marked as uncleanable + class LogMock(dir: File, config: LogConfig, offsets: LoadedLogOffsets) + extends Log(dir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, + offsets.nextOffsetMetadata, time.scheduler, new BrokerTopicStats, time, maxProducerIdExpirationMs, + LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, + producerStateManager, logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { // Throw an error in getFirstBatchTimestampForSegments since it is called in grabFilthiestLog() override def getFirstBatchTimestampForSegments(segments: Iterable[LogSegment]): Iterable[Long] = throw new IllegalStateException("Error!") } - val log: Log = new LogMock(tpDir, createLowRetentionLogConfig(logSegmentSize, LogConfig.Compact)) + val log: Log = new LogMock(tpDir, config, offsets) writeRecords(log = log, numBatches = logSegmentsCount * 2, recordsPerBatch = 10, diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 155bd6a776546..de55724d6ba71 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -100,19 +100,43 @@ class LogCleanerTest { val logProps = new Properties() logProps.put(LogConfig.SegmentBytesProp, 1024 : java.lang.Integer) logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact + "," + LogConfig.Delete) + val config = LogConfig.fromProps(logConfig.originals, logProps) val topicPartition = Log.parseTopicPartitionName(dir) - val producerStateManager = new ProducerStateManager(topicPartition, dir) + val logDirFailureChannel = new LogDirFailureChannel(10) + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val logSegments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, dir, maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + dir, + topicPartition, + config, + time.scheduler, + time, + logDirFailureChannel, + hadCleanShutdown = true, + logSegments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + val log = new Log(dir, - config = LogConfig.fromProps(logConfig.originals, logProps), - logStartOffset = 0L, - recoveryPoint = 0L, + config = config, + segments = logSegments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler = time.scheduler, - brokerTopicStats = new BrokerTopicStats, time, - maxProducerIdExpirationMs = 60 * 60 * 1000, + brokerTopicStats = new BrokerTopicStats, + time, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, producerStateManager = producerStateManager, - logDirFailureChannel = new LogDirFailureChannel(10), + logDirFailureChannel = logDirFailureChannel, topicId = None, keepPartitionMetadataFile = true) { override def replaceSegments(newSegments: Seq[LogSegment], oldSegments: Seq[LogSegment], isRecoveredSwapFile: Boolean = false): Unit = { @@ -1755,7 +1779,7 @@ class LogCleanerTest { private def tombstoneRecord(key: Int): MemoryRecords = record(key, null) private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): Log = { - LogTest.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) + LogTestUtils.recoverAndCheck(dir, config, expectedKeys, new BrokerTopicStats(), time, time.scheduler) } } diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala new file mode 100644 index 0000000000000..0d6e7e3dfc9d4 --- /dev/null +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -0,0 +1,1501 @@ +/** + * 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 kafka.log + +import java.io.{BufferedWriter, File, FileWriter} +import java.nio.ByteBuffer +import java.nio.file.{Files, Paths} +import java.util.Properties + +import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0} +import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} +import kafka.server.{BrokerTopicStats, FetchDataInfo, KafkaConfig, LogDirFailureChannel} +import kafka.server.metadata.CachedConfigRepository +import kafka.utils.{CoreUtils, MockTime, Scheduler, TestUtils} +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.record.{CompressionType, ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType} +import org.apache.kafka.common.utils.{Time, Utils} +import org.easymock.EasyMock +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} + +import scala.collection.mutable.ListBuffer +import scala.collection.{Iterable, Map, mutable} +import scala.jdk.CollectionConverters._ + +class LogLoaderTest { + var config: KafkaConfig = null + val brokerTopicStats = new BrokerTopicStats + val tmpDir = TestUtils.tempDir() + val logDir = TestUtils.randomPartitionLogDir(tmpDir) + val mockTime = new MockTime() + + @BeforeEach + def setUp(): Unit = { + val props = TestUtils.createBrokerConfig(0, "127.0.0.1:1", port = -1) + config = KafkaConfig.fromProps(props) + } + + @AfterEach + def tearDown(): Unit = { + brokerTopicStats.close() + Utils.delete(tmpDir) + } + + @Test + def testLogRecoveryIsCalledUponBrokerCrash(): Unit = { + // LogManager must realize correctly if the last shutdown was not clean and the logs need + // to run recovery while loading upon subsequent broker boot up. + val logDir: File = TestUtils.tempDir() + val logProps = new Properties() + val logConfig = LogConfig(logProps) + val logDirs = Seq(logDir) + val topicPartition = new TopicPartition("foo", 0) + var log: Log = null + val time = new MockTime() + var cleanShutdownInterceptedValue = false + case class SimulateError(var hasError: Boolean = false) + val simulateError = SimulateError() + + // Create a LogManager with some overridden methods to facilitate interception of clean shutdown + // flag and to inject a runtime error + def interceptedLogManager(logConfig: LogConfig, logDirs: Seq[File], simulateError: SimulateError): LogManager = { + new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], new CachedConfigRepository(), + initialDefaultConfig = logConfig, cleanerConfig = CleanerConfig(enableCleaner = false), recoveryThreadsPerDataDir = 4, + flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L, + retentionCheckMs = 1000L, maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time, + brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(logDirs.size), keepPartitionMetadataFile = config.usesTopicId) { + + override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], + logStartOffsets: Map[TopicPartition, Long], topicConfigs: Map[String, LogConfig]): Log = { + if (simulateError.hasError) { + throw new RuntimeException("Simulated error") + } + cleanShutdownInterceptedValue = hadCleanShutdown + val topicPartition = Log.parseTopicPartitionName(logDir) + val config = topicConfigs.getOrElse(topicPartition.topic, currentDefaultConfig) + val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) + val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) + val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) + val loadLogParams = LoadLogParams(logDir, topicPartition, config, time.scheduler, time, + logDirFailureChannel, hadCleanShutdown, segments, logStartOffset, logRecoveryPoint, + maxProducerIdExpirationMs, leaderEpochCache, producerStateManager) + val offsets = LogLoader.load(loadLogParams) + new Log(logDir, config, segments, offsets.logStartOffset, offsets.recoveryPoint, + offsets.nextOffsetMetadata, time.scheduler, brokerTopicStats, time, maxPidExpirationMs, + LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, leaderEpochCache, + producerStateManager, logDirFailureChannel, None, true) + } + } + } + + val cleanShutdownFile = new File(logDir, Log.CleanShutdownFile) + locally { + val logManager: LogManager = interceptedLogManager(logConfig, logDirs, simulateError) + log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) + + // Load logs after a clean shutdown + Files.createFile(cleanShutdownFile.toPath) + cleanShutdownInterceptedValue = false + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) + assertTrue(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") + assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") + // Load logs without clean shutdown file + cleanShutdownInterceptedValue = true + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) + assertFalse(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") + assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") + // Create clean shutdown file and then simulate error while loading logs such that log loading does not complete. + Files.createFile(cleanShutdownFile.toPath) + logManager.shutdown() + } + + locally { + simulateError.hasError = true + val logManager: LogManager = interceptedLogManager(logConfig, logDirs, simulateError) + log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) + + // Simulate error + assertThrows(classOf[RuntimeException], () => logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty))) + assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not have existed") + // Do not simulate error on next call to LogManager#loadLogs. LogManager must understand that log had unclean shutdown the last time. + simulateError.hasError = false + cleanShutdownInterceptedValue = true + logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) + assertFalse(cleanShutdownInterceptedValue, "Unexpected value for clean shutdown flag") + } + } + + @Test + def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = { + testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.minSupportedFor(RecordVersion.V1).version) + } + + @Test + def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = { + testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.latestVersion.version) + } + + private def createLog(dir: File, + config: LogConfig, + brokerTopicStats: BrokerTopicStats = brokerTopicStats, + logStartOffset: Long = 0L, + recoveryPoint: Long = 0L, + scheduler: Scheduler = mockTime.scheduler, + time: Time = mockTime, + maxProducerIdExpirationMs: Int = 60 * 60 * 1000, + producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, + lastShutdownClean: Boolean = true): Log = { + LogTestUtils.createLog(dir, config, brokerTopicStats, scheduler, time, logStartOffset, recoveryPoint, + maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs, lastShutdownClean) + } + + private def createLogWithOffsetOverflow(logConfig: LogConfig): (Log, LogSegment) = { + LogTestUtils.initializeLogDirWithOverflowedSegment(logDir) + + val log = createLog(logDir, logConfig, recoveryPoint = Long.MaxValue) + val segmentWithOverflow = LogTestUtils.firstOverflowSegment(log).getOrElse { + throw new AssertionError("Failed to create log with a segment which has overflowed offsets") + } + + (log, segmentWithOverflow) + } + + private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]): Log = { + // method is called only in case of recovery from hard reset + LogTestUtils.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler) + } + + /** + * Wrap a single record log buffer with leader epoch. + */ + private def singletonRecordsWithLeaderEpoch(value: Array[Byte], + key: Array[Byte] = null, + leaderEpoch: Int, + offset: Long, + codec: CompressionType = CompressionType.NONE, + timestamp: Long = RecordBatch.NO_TIMESTAMP, + magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = { + val records = Seq(new SimpleRecord(timestamp, key, value)) + + val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) + val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset, + mockTime.milliseconds, leaderEpoch) + records.foreach(builder.append) + builder.build() + } + + private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 64 * 10, messageFormatVersion = messageFormatVersion) + var log = createLog(logDir, logConfig) + assertEquals(None, log.oldestProducerSnapshotOffset) + + for (i <- 0 to 100) { + val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) + log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) + } + + assertTrue(log.logSegments.size >= 5) + val segmentOffsets = log.logSegments.toVector.map(_.baseOffset) + val activeSegmentOffset = segmentOffsets.last + + // We want the recovery point to be past the segment offset and before the last 2 segments including a gap of + // 1 segment. We collect the data before closing the log. + val offsetForSegmentAfterRecoveryPoint = segmentOffsets(segmentOffsets.size - 3) + val offsetForRecoveryPointSegment = segmentOffsets(segmentOffsets.size - 4) + val (segOffsetsBeforeRecovery, segOffsetsAfterRecovery) = segmentOffsets.toSet.partition(_ < offsetForRecoveryPointSegment) + val recoveryPoint = offsetForRecoveryPointSegment + 1 + assertTrue(recoveryPoint < offsetForSegmentAfterRecoveryPoint) + log.close() + + val segmentsWithReads = mutable.Set[LogSegment]() + val recoveredSegments = mutable.Set[LogSegment]() + val expectedSegmentsWithReads = mutable.Set[Long]() + val expectedSnapshotOffsets = mutable.Set[Long]() + + if (logConfig.messageFormatVersion < KAFKA_0_11_0_IV0) { + expectedSegmentsWithReads += activeSegmentOffset + expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset + } else { + expectedSegmentsWithReads ++= segOffsetsBeforeRecovery ++ Set(activeSegmentOffset) + expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(4) :+ log.logEndOffset + } + + def createLogWithInterceptedReads(recoveryPoint: Long) = { + val maxProducerIdExpirationMs = 60 * 60 * 1000 + val topicPartition = Log.parseTopicPartitionName(logDir) + val logDirFailureChannel = new LogDirFailureChannel(10) + // Intercept all segment read calls + val interceptedLogSegments = new LogSegments(topicPartition) { + override def add(segment: LogSegment): LogSegment = { + val wrapper = new LogSegment(segment.log, segment.lazyOffsetIndex, segment.lazyTimeIndex, segment.txnIndex, segment.baseOffset, + segment.indexIntervalBytes, segment.rollJitterMs, mockTime) { + + override def read(startOffset: Long, maxSize: Int, maxPosition: Long, minOneMessage: Boolean): FetchDataInfo = { + segmentsWithReads += this + super.read(startOffset, maxSize, maxPosition, minOneMessage) + } + + override def recover(producerStateManager: ProducerStateManager, + leaderEpochCache: Option[LeaderEpochFileCache]): Int = { + recoveredSegments += this + super.recover(producerStateManager, leaderEpochCache) + } + } + super.add(wrapper) + } + } + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) + val loadLogParams = LoadLogParams( + logDir, + topicPartition, + logConfig, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = false, + interceptedLogSegments, + 0L, + recoveryPoint, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager) + val offsets = LogLoader.load(loadLogParams) + new Log(logDir, logConfig, interceptedLogSegments, offsets.logStartOffset, offsets.recoveryPoint, + offsets.nextOffsetMetadata, mockTime.scheduler, brokerTopicStats, mockTime, + maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, + leaderEpochCache, producerStateManager, logDirFailureChannel, topicId = None, + keepPartitionMetadataFile = true) + } + + // Retain snapshots for the last 2 segments + log.producerStateManager.deleteSnapshotsBefore(segmentOffsets(segmentOffsets.size - 2)) + log = createLogWithInterceptedReads(offsetForRecoveryPointSegment) + // We will reload all segments because the recovery point is behind the producer snapshot files (pre KAFKA-5829 behaviour) + assertEquals(expectedSegmentsWithReads, segmentsWithReads.map(_.baseOffset)) + assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir).toSet) + log.close() + segmentsWithReads.clear() + recoveredSegments.clear() + + // Only delete snapshots before the base offset of the recovery point segment (post KAFKA-5829 behaviour) to + // avoid reading all segments + log.producerStateManager.deleteSnapshotsBefore(offsetForRecoveryPointSegment) + log = createLogWithInterceptedReads(recoveryPoint = recoveryPoint) + assertEquals(Set(activeSegmentOffset), segmentsWithReads.map(_.baseOffset)) + assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir).toSet) + + log.close() + } + + @Test + def testSkipLoadingIfEmptyProducerStateBeforeTruncation(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + // Load the log + EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.mapEndOffset).andStubReturn(0L) + EasyMock.expect(stateManager.isEmpty).andStubReturn(true) + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(0L), EasyMock.anyLong) + EasyMock.expectLastCall() + + EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) + val config = LogConfig(new Properties()) + val maxProducerIdExpirationMs = 300000 + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = false, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + val log = new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + + // Append some messages + EasyMock.reset(stateManager) + EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) + + stateManager.updateMapEndOffset(1L) + EasyMock.expectLastCall() + stateManager.updateMapEndOffset(2L) + EasyMock.expectLastCall() + + EasyMock.replay(stateManager) + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0) + + EasyMock.verify(stateManager) + + // Now truncate + EasyMock.reset(stateManager) + EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) + EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) + EasyMock.expect(stateManager.isEmpty).andStubReturn(true) + EasyMock.expect(stateManager.mapEndOffset).andReturn(2L) + stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(1L), EasyMock.anyLong) + EasyMock.expectLastCall() + // Truncation causes the map end offset to reset to 0 + EasyMock.expect(stateManager.mapEndOffset).andReturn(0L) + // We skip directly to updating the map end offset + EasyMock.expect(stateManager.updateMapEndOffset(1L)) + EasyMock.expect(stateManager.onHighWatermarkUpdated(0L)) + + // Finally, we take a snapshot + stateManager.takeSnapshot() + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + log.truncateTo(1L) + + EasyMock.verify(stateManager) + } + + @Test + def testRecoverAfterNonMonotonicCoordinatorEpochWrite(): Unit = { + // Due to KAFKA-9144, we may encounter a coordinator epoch which goes backwards. + // This test case verifies that recovery logic relaxes validation in this case and + // just takes the latest write. + + val producerId = 1L + val coordinatorEpoch = 5 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) + var log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val firstAppendTimestamp = mockTime.milliseconds() + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, + firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch) + assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) + + mockTime.sleep(log.maxProducerIdExpirationMs) + assertEquals(None, log.producerStateManager.lastEntry(producerId)) + + val secondAppendTimestamp = mockTime.milliseconds() + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, + secondAppendTimestamp, coordinatorEpoch = coordinatorEpoch - 1) + + log.close() + + // Force recovery by setting the recoveryPoint to the log start + log = createLog(logDir, logConfig, recoveryPoint = 0L, lastShutdownClean = false) + assertEquals(secondAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) + log.close() + } + + @Test + def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logProps = new Properties() + logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") + val config = LogConfig(logProps) + val maxProducerIdExpirationMs = 300000 + val logDirFailureChannel = null + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = false, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + } + + @Test + def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logProps = new Properties() + logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") + val config = LogConfig(logProps) + val maxProducerIdExpirationMs = 300000 + val logDirFailureChannel = null + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + } + + @Test + def testSkipTruncateAndReloadIfNewMessageFormatAndCleanShutdown(): Unit = { + val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) + EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() + + EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) + + stateManager.updateMapEndOffset(0L) + EasyMock.expectLastCall().anyTimes() + + stateManager.takeSnapshot() + EasyMock.expectLastCall().anyTimes() + + EasyMock.expect(stateManager.isEmpty).andReturn(true) + EasyMock.expectLastCall().once() + + EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) + EasyMock.expectLastCall().once() + + EasyMock.replay(stateManager) + + val topicPartition = Log.parseTopicPartitionName(logDir) + val logProps = new Properties() + logProps.put(LogConfig.MessageFormatVersionProp, "0.11.0") + val config = LogConfig(logProps) + val maxProducerIdExpirationMs = 300000 + val logDirFailureChannel = null + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion) + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + config, + mockTime.scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + stateManager)) + new Log(logDir, + config, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, + scheduler = mockTime.scheduler, + brokerTopicStats = brokerTopicStats, + time = mockTime, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = 30000, + topicPartition = topicPartition, + leaderEpochCache = leaderEpochCache, + producerStateManager = stateManager, + logDirFailureChannel = logDirFailureChannel, + topicId = None, + keepPartitionMetadataFile = true) + + EasyMock.verify(stateManager) + } + + @Test + def testLoadProducersAfterDeleteRecordsMidSegment(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) + val log = createLog(logDir, logConfig) + val pid1 = 1L + val pid2 = 2L + val epoch = 0.toShort + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + assertEquals(2, log.activeProducersWithLastSequence.size) + + log.updateHighWatermark(log.logEndOffset) + log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) + + // Deleting records should not remove producer state + assertEquals(2, log.activeProducersWithLastSequence.size) + val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) + assertTrue(retainedLastSeqOpt.isDefined) + assertEquals(0, retainedLastSeqOpt.get) + + log.close() + + // Because the log start offset did not advance, producer snapshots will still be present and the state will be rebuilt + val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) + assertEquals(2, reloadedLog.activeProducersWithLastSequence.size) + val reloadedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) + assertEquals(retainedLastSeqOpt, reloadedLastSeqOpt) + } + + @Test + def testLoadingLogKeepsLargestStrayProducerStateSnapshot(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) + val log = createLog(logDir, logConfig) + val pid1 = 1L + val epoch = 0.toShort + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + log.roll() + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 1), leaderEpoch = 0) + log.roll() + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("c".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 2), leaderEpoch = 0) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("d".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 3), leaderEpoch = 0) + + // Close the log, we should now have 3 segments + log.close() + assertEquals(log.logSegments.size, 3) + // We expect 3 snapshot files, two of which are for the first two segments, the last was written out during log closing. + assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) + // Inject a stray snapshot file within the bounds of the log at offset 3, it should be cleaned up after loading the log + val straySnapshotFile = Log.producerSnapshotFile(logDir, 3).toPath + Files.createFile(straySnapshotFile) + assertEquals(Seq(1, 2, 3, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) + + createLog(logDir, logConfig, lastShutdownClean = false) + // We should clean up the stray producer state snapshot file, but keep the largest snapshot file (4) + assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) + } + + @Test + def testLoadProducersAfterDeleteRecordsOnSegment(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) + val log = createLog(logDir, logConfig) + val pid1 = 1L + val pid2 = 2L + val epoch = 0.toShort + + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + log.roll() + log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, + producerEpoch = epoch, sequence = 0), leaderEpoch = 0) + + assertEquals(2, log.logSegments.size) + assertEquals(2, log.activeProducersWithLastSequence.size) + + log.updateHighWatermark(log.logEndOffset) + log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) + log.deleteOldSegments() + + // Deleting records should not remove producer state + assertEquals(1, log.logSegments.size) + assertEquals(2, log.activeProducersWithLastSequence.size) + val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) + assertTrue(retainedLastSeqOpt.isDefined) + assertEquals(0, retainedLastSeqOpt.get) + + log.close() + + // After reloading log, producer state should not be regenerated + val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) + assertEquals(1, reloadedLog.activeProducersWithLastSequence.size) + val reloadedEntryOpt = log.activeProducersWithLastSequence.get(pid2) + assertEquals(retainedLastSeqOpt, reloadedEntryOpt) + } + + /** + * Append a bunch of messages to a log and then re-open it both with and without recovery and check that the log re-initializes correctly. + */ + @Test + def testLogRecoversToCorrectOffset(): Unit = { + val numMessages = 100 + val messageSize = 100 + val segmentSize = 7 * messageSize + val indexInterval = 3 * messageSize + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize), + timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + assertEquals(numMessages, log.logEndOffset, + "After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages)) + val lastIndexOffset = log.activeSegment.offsetIndex.lastOffset + val numIndexEntries = log.activeSegment.offsetIndex.entries + val lastOffset = log.logEndOffset + // After segment is closed, the last entry in the time index should be (largest timestamp -> last offset). + val lastTimeIndexOffset = log.logEndOffset - 1 + val lastTimeIndexTimestamp = log.activeSegment.largestTimestamp + // Depending on when the last time index entry is inserted, an entry may or may not be inserted into the time index. + val numTimeIndexEntries = log.activeSegment.timeIndex.entries + { + if (log.activeSegment.timeIndex.lastEntry.offset == log.logEndOffset - 1) 0 else 1 + } + log.close() + + def verifyRecoveredLog(log: Log, expectedRecoveryPoint: Long): Unit = { + assertEquals(expectedRecoveryPoint, log.recoveryPoint, s"Unexpected recovery point") + assertEquals(numMessages, log.logEndOffset, s"Should have $numMessages messages when log is reopened w/o recovery") + assertEquals(lastIndexOffset, log.activeSegment.offsetIndex.lastOffset, "Should have same last index offset as before.") + assertEquals(numIndexEntries, log.activeSegment.offsetIndex.entries, "Should have same number of index entries as before.") + assertEquals(lastTimeIndexTimestamp, log.activeSegment.timeIndex.lastEntry.timestamp, "Should have same last time index timestamp") + assertEquals(lastTimeIndexOffset, log.activeSegment.timeIndex.lastEntry.offset, "Should have same last time index offset") + assertEquals(numTimeIndexEntries, log.activeSegment.timeIndex.entries, "Should have same number of time index entries as before.") + } + + log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false) + verifyRecoveredLog(log, lastOffset) + log.close() + + // test recovery case + val recoveryPoint = 10 + log = createLog(logDir, logConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) + // the recovery point should not be updated after unclean shutdown until the log is flushed + verifyRecoveredLog(log, recoveryPoint) + log.flush() + verifyRecoveredLog(log, lastOffset) + log.close() + } + + /** + * Test that if we manually delete an index segment it is rebuilt when the log is re-opened + */ + @Test + def testIndexRebuild(): Unit = { + // publish the messages and close the log + val numMessages = 200 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) + val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) + log.close() + + // delete all the index files + indexFiles.foreach(_.delete()) + timeIndexFiles.foreach(_.delete()) + + // reopen the log + log = createLog(logDir, logConfig, lastShutdownClean = false) + assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) + assertTrue(log.logSegments.head.offsetIndex.entries > 0, "The index should have been rebuilt") + assertTrue(log.logSegments.head.timeIndex.entries > 0, "The time index should have been rebuilt") + for(i <- 0 until numMessages) { + assertEquals(i, LogTestUtils.readLog(log, i, 100).records.batches.iterator.next().lastOffset) + if (i == 0) + assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + else + assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + } + log.close() + } + + /** + * Test that if messages format version of the messages in a segment is before 0.10.0, the time index should be empty. + */ + @Test + def testRebuildTimeIndexForOldMessages(): Unit = { + val numMessages = 200 + val segmentSize = 200 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = 1, messageFormatVersion = "0.9.0") + var log = createLog(logDir, logConfig) + for (i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), + timestamp = mockTime.milliseconds + i * 10, magicValue = RecordBatch.MAGIC_VALUE_V1), leaderEpoch = 0) + val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) + log.close() + + // Delete the time index. + timeIndexFiles.foreach(file => Files.delete(file.toPath)) + + // The rebuilt time index should be empty + log = createLog(logDir, logConfig, recoveryPoint = numMessages + 1, lastShutdownClean = false) + for (segment <- log.logSegments.init) { + assertEquals(0, segment.timeIndex.entries, "The time index should be empty") + assertEquals(0, segment.lazyTimeIndex.file.length, "The time index file size should be 0") + } + } + + + /** + * Test that if we have corrupted an index segment it is rebuilt when the log is re-opened + */ + @Test + def testCorruptIndexRebuild(): Unit = { + // publish the messages and close the log + val numMessages = 200 + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) + var log = createLog(logDir, logConfig) + for(i <- 0 until numMessages) + log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) + val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) + val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) + log.close() + + // corrupt all the index files + for( file <- indexFiles) { + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(" ") + bw.close() + } + + // corrupt all the index files + for( file <- timeIndexFiles) { + val bw = new BufferedWriter(new FileWriter(file)) + bw.write(" ") + bw.close() + } + + // reopen the log with recovery point=0 so that the segment recovery can be triggered + log = createLog(logDir, logConfig, lastShutdownClean = false) + assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) + for(i <- 0 until numMessages) { + assertEquals(i, LogTestUtils.readLog(log, i, 100).records.batches.iterator.next().lastOffset) + if (i == 0) + assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + else + assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) + } + log.close() + } + + /** + * When we open a log any index segments without an associated log segment should be deleted. + */ + @Test + def testBogusIndexSegmentsAreRemoved(): Unit = { + val bogusIndex1 = Log.offsetIndexFile(logDir, 0) + val bogusTimeIndex1 = Log.timeIndexFile(logDir, 0) + val bogusIndex2 = Log.offsetIndexFile(logDir, 5) + val bogusTimeIndex2 = Log.timeIndexFile(logDir, 5) + + // The files remain absent until we first access it because we are doing lazy loading for time index and offset index + // files but in this test case we need to create these files in order to test we will remove them. + bogusIndex2.createNewFile() + bogusTimeIndex2.createNewFile() + + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 1) + val log = createLog(logDir, logConfig) + + // Force the segment to access the index files because we are doing index lazy loading. + log.logSegments.toSeq.head.offsetIndex + log.logSegments.toSeq.head.timeIndex + + assertTrue(bogusIndex1.length > 0, + "The first index file should have been replaced with a larger file") + assertTrue(bogusTimeIndex1.length > 0, + "The first time index file should have been replaced with a larger file") + assertFalse(bogusIndex2.exists, + "The second index file should have been deleted.") + assertFalse(bogusTimeIndex2.exists, + "The second time index file should have been deleted.") + + // check that we can append to the log + for (_ <- 0 until 10) + log.appendAsLeader(createRecords, leaderEpoch = 0) + + log.delete() + } + + /** + * Verify that truncation works correctly after re-opening the log + */ + @Test + def testReopenThenTruncate(): Unit = { + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + // create a log + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000) + var log = createLog(logDir, logConfig) + + // add enough messages to roll over several segments then close and re-open and attempt to truncate + for (_ <- 0 until 100) + log.appendAsLeader(createRecords, leaderEpoch = 0) + log.close() + log = createLog(logDir, logConfig, lastShutdownClean = false) + log.truncateTo(3) + assertEquals(1, log.numberOfSegments, "All but one segment should be deleted.") + assertEquals(3, log.logEndOffset, "Log end offset should be 3.") + } + + /** + * Any files ending in .deleted should be removed when the log is re-opened. + */ + @Test + def testOpenDeletesObsoleteFiles(): Unit = { + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) + val logConfig = LogTestUtils.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) + var log = createLog(logDir, logConfig) + + // append some messages to create some segments + for (_ <- 0 until 100) + log.appendAsLeader(createRecords, leaderEpoch = 0) + + // expire all segments + log.updateHighWatermark(log.logEndOffset) + log.deleteOldSegments() + log.close() + log = createLog(logDir, logConfig, lastShutdownClean = false) + assertEquals(1, log.numberOfSegments, "The deleted segments should be gone.") + } + + @Test + def testCorruptLog(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + val recoveryPoint = 50L + for (_ <- 0 until 10) { + // create a log and write some messages to it + logDir.mkdirs() + var log = createLog(logDir, logConfig) + val numMessages = 50 + TestUtils.random.nextInt(50) + for (_ <- 0 until numMessages) + log.appendAsLeader(createRecords, leaderEpoch = 0) + val records = log.logSegments.flatMap(_.log.records.asScala.toList).toList + log.close() + + // corrupt index and log by appending random bytes + TestUtils.appendNonsenseToFile(log.activeSegment.lazyOffsetIndex.file, TestUtils.random.nextInt(1024) + 1) + TestUtils.appendNonsenseToFile(log.activeSegment.log.file, TestUtils.random.nextInt(1024) + 1) + + // attempt recovery + log = createLog(logDir, logConfig, brokerTopicStats, 0L, recoveryPoint, lastShutdownClean = false) + assertEquals(numMessages, log.logEndOffset) + + val recovered = log.logSegments.flatMap(_.log.records.asScala.toList).toList + assertEquals(records.size, recovered.size) + + for (i <- records.indices) { + val expected = records(i) + val actual = recovered(i) + assertEquals(expected.key, actual.key, s"Keys not equal") + assertEquals(expected.value, actual.value, s"Values not equal") + assertEquals(expected.timestamp, actual.timestamp, s"Timestamps not equal") + } + + Utils.delete(logDir) + } + } + + @Test + def testOverCompactedLogRecovery(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) + val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes())) + val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, CompressionType.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes())) + val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes())) + //Writes into an empty log with baseOffset 0 + log.appendAsFollower(set1) + assertEquals(0L, log.activeSegment.baseOffset) + //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 + log.appendAsFollower(set2) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) + //This will go into the existing log + log.appendAsFollower(set3) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + //This will go into the existing log + log.appendAsFollower(set4) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + log.close() + val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) + assertEquals(2, indexFiles.length) + for (file <- indexFiles) { + val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) + assertTrue(offsetIndex.lastOffset >= 0) + offsetIndex.close() + } + Utils.delete(logDir) + } + + @Test + def testLeaderEpochCacheClearedAfterStaticMessageFormatDowngrade(): Unit = { + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) + assertEquals(Some(5), log.latestEpoch) + log.close() + + // reopen the log with an older message format version and check the cache + val downgradedLogConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, + maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) + val reopened = createLog(logDir, downgradedLogConfig, lastShutdownClean = false) + LogTestUtils.assertLeaderEpochCacheEmpty(reopened) + + reopened.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), + magicValue = RecordVersion.V1.value), leaderEpoch = 5) + LogTestUtils.assertLeaderEpochCacheEmpty(reopened) + } + + @Test + def testOverCompactedLogRecoveryMultiRecord(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) + val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0, + new SimpleRecord("v3".getBytes(), "k3".getBytes()), + new SimpleRecord("v4".getBytes(), "k4".getBytes())) + val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, 0, + new SimpleRecord("v5".getBytes(), "k5".getBytes()), + new SimpleRecord("v6".getBytes(), "k6".getBytes())) + val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, 0, + new SimpleRecord("v7".getBytes(), "k7".getBytes()), + new SimpleRecord("v8".getBytes(), "k8".getBytes())) + //Writes into an empty log with baseOffset 0 + log.appendAsFollower(set1) + assertEquals(0L, log.activeSegment.baseOffset) + //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 + log.appendAsFollower(set2) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) + //This will go into the existing log + log.appendAsFollower(set3) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + //This will go into the existing log + log.appendAsFollower(set4) + assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) + log.close() + val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) + assertEquals(2, indexFiles.length) + for (file <- indexFiles) { + val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) + assertTrue(offsetIndex.lastOffset >= 0) + offsetIndex.close() + } + Utils.delete(logDir) + } + + @Test + def testOverCompactedLogRecoveryMultiRecordV1(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + val log = createLog(logDir, logConfig) + val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, CompressionType.NONE, + new SimpleRecord("v1".getBytes(), "k1".getBytes())) + val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, + new SimpleRecord("v3".getBytes(), "k3".getBytes()), + new SimpleRecord("v4".getBytes(), "k4".getBytes())) + val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, + new SimpleRecord("v5".getBytes(), "k5".getBytes()), + new SimpleRecord("v6".getBytes(), "k6".getBytes())) + val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, + new SimpleRecord("v7".getBytes(), "k7".getBytes()), + new SimpleRecord("v8".getBytes(), "k8".getBytes())) + //Writes into an empty log with baseOffset 0 + log.appendAsFollower(set1) + assertEquals(0L, log.activeSegment.baseOffset) + //This write will roll the segment, yielding a new segment with base offset = max(1, 3) = 3 + log.appendAsFollower(set2) + assertEquals(3, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, 3).exists) + //This will also roll the segment, yielding a new segment with base offset = max(5, Integer.MAX_VALUE+4) = Integer.MAX_VALUE+4 + log.appendAsFollower(set3) + assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) + assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 4).exists) + //This will go into the existing log + log.appendAsFollower(set4) + assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) + log.close() + val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) + assertEquals(3, indexFiles.length) + for (file <- indexFiles) { + val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) + assertTrue(offsetIndex.lastOffset >= 0) + offsetIndex.close() + } + Utils.delete(logDir) + } + + @Test + def testRecoveryOfSegmentWithOffsetOverflow(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, _) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + + // Run recovery on the log. This should split the segment underneath. Ignore .deleted files as we could have still + // have them lying around after the split. + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + + // Running split again would throw an error + + for (segment <- recoveredLog.logSegments) { + assertThrows(classOf[IllegalArgumentException], () => log.splitOverflowedSegment(segment)) + } + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase1(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery just after .cleaned file is created, before rename to .swap. On recovery, existing split + // operation is aborted but the recovery process itself kicks off split which should complete. + newSegments.reverse.foreach(segment => { + segment.changeFileSuffixes("", Log.CleanedFileSuffix) + segment.truncateTo(0) + }) + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase2(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery just after one of the new segments has been renamed to .swap. On recovery, existing split + // operation is aborted but the recovery process itself kicks off split which should complete. + newSegments.reverse.foreach { segment => + if (segment != newSegments.last) + segment.changeFileSuffixes("", Log.CleanedFileSuffix) + else + segment.changeFileSuffixes("", Log.SwapFileSuffix) + segment.truncateTo(0) + } + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase3(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after all new segments have been renamed to .swap. On recovery, existing split operation + // is completed and the old segment must be deleted. + newSegments.reverse.foreach(segment => { + segment.changeFileSuffixes("", Log.SwapFileSuffix) + }) + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + log.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase4(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after all new segments have been renamed to .swap and old segment has been deleted. On + // recovery, existing split operation is completed. + newSegments.reverse.foreach(_.changeFileSuffixes("", Log.SwapFileSuffix)) + + for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) + Utils.delete(file) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testRecoveryAfterCrashDuringSplitPhase5(): Unit = { + val logConfig = LogTestUtils.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) + val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) + val expectedKeys = LogTestUtils.keysInLog(log) + val numSegmentsInitial = log.logSegments.size + + // Split the segment + val newSegments = log.splitOverflowedSegment(segmentWithOverflow) + + // Simulate recovery right after one of the new segment has been renamed to .swap and the other to .log. On + // recovery, existing split operation is completed. + newSegments.last.changeFileSuffixes("", Log.SwapFileSuffix) + + // Truncate the old segment + segmentWithOverflow.truncateTo(0) + + val recoveredLog = recoverAndCheck(logConfig, expectedKeys) + assertEquals(expectedKeys, LogTestUtils.keysInLog(recoveredLog)) + assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) + recoveredLog.close() + } + + @Test + def testCleanShutdownFile(): Unit = { + // append some messages to create some segments + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) + def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) + + var recoveryPoint = 0L + // create a log and write some messages to it + var log = createLog(logDir, logConfig) + for (_ <- 0 until 100) + log.appendAsLeader(createRecords, leaderEpoch = 0) + log.close() + + // check if recovery was attempted. Even if the recovery point is 0L, recovery should not be attempted as the + // clean shutdown file exists. Note: Earlier, Log layer relied on the presence of clean shutdown file to determine the status + // of last shutdown. Now, LogManager checks for the presence of this file and immediately deletes the same. It passes + // down a clean shutdown flag to the Log layer as log is loaded. Recovery is attempted based on this flag. + recoveryPoint = log.logEndOffset + log = createLog(logDir, logConfig) + assertEquals(recoveryPoint, log.logEndOffset) + } + + /** + * Append a bunch of messages to a log and then re-open it with recovery and check that the leader epochs are recovered properly. + */ + @Test + def testLogRecoversForLeaderEpoch(): Unit = { + val log = createLog(logDir, LogConfig()) + val leaderEpochCache = log.leaderEpochCache.get + val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0) + log.appendAsFollower(records = firstBatch) + + val secondBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 1) + log.appendAsFollower(records = secondBatch) + + val thirdBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 2) + log.appendAsFollower(records = thirdBatch) + + val fourthBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 3, offset = 3) + log.appendAsFollower(records = fourthBatch) + + assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) + + // deliberately remove some of the epoch entries + leaderEpochCache.truncateFromEnd(2) + assertNotEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) + log.close() + + // reopen the log and recover from the beginning + val recoveredLog = createLog(logDir, LogConfig(), lastShutdownClean = false) + val recoveredLeaderEpochCache = recoveredLog.leaderEpochCache.get + + // epoch entries should be recovered + assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), recoveredLeaderEpochCache.epochEntries) + recoveredLog.close() + } + + @Test + def testFullTransactionIndexRecovery(): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val pid1 = 1L + val pid2 = 2L + val pid3 = 3L + val pid4 = 4L + + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) + + // mix transactional and non-transactional data + appendPid1(5) // nextOffset: 5 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 + appendPid2(2) // 10 + appendPid1(4) // 14 + appendPid3(3) // 17 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 + appendPid1(10) // 29 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 + appendPid2(6) // 36 + appendPid4(3) // 39 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 + appendPid3(9) // 58 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 + appendPid4(8) // 67 + appendPid2(7) // 74 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 + appendPid4(4) // 89 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 + + // delete all the offset and transaction index files to force recovery + log.logSegments.foreach { segment => + segment.offsetIndex.deleteIfExists() + segment.txnIndex.deleteIfExists() + } + + log.close() + + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLog = createLog(logDir, reloadedLogConfig, lastShutdownClean = false) + val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) + assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) + } + + @Test + def testRecoverOnlyLastSegment(): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val pid1 = 1L + val pid2 = 2L + val pid3 = 3L + val pid4 = 4L + + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) + + // mix transactional and non-transactional data + appendPid1(5) // nextOffset: 5 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 + appendPid2(2) // 10 + appendPid1(4) // 14 + appendPid3(3) // 17 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 + appendPid1(10) // 29 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 + appendPid2(6) // 36 + appendPid4(3) // 39 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 + appendPid3(9) // 58 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 + appendPid4(8) // 67 + appendPid2(7) // 74 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 + appendPid4(4) // 89 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 + + // delete the last offset and transaction index files to force recovery + val lastSegment = log.logSegments.last + val recoveryPoint = lastSegment.baseOffset + lastSegment.offsetIndex.deleteIfExists() + lastSegment.txnIndex.deleteIfExists() + + log.close() + + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) + val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) + assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) + } + + @Test + def testRecoverLastSegmentWithNoSnapshots(): Unit = { + val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) + val log = createLog(logDir, logConfig) + val epoch = 0.toShort + + val pid1 = 1L + val pid2 = 2L + val pid3 = 3L + val pid4 = 4L + + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) + + // mix transactional and non-transactional data + appendPid1(5) // nextOffset: 5 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 + appendPid2(2) // 10 + appendPid1(4) // 14 + appendPid3(3) // 17 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 + appendPid1(10) // 29 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 + appendPid2(6) // 36 + appendPid4(3) // 39 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 + appendPid3(9) // 58 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 + appendPid4(8) // 67 + appendPid2(7) // 74 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 + appendPid4(4) // 89 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 + + LogTestUtils.deleteProducerSnapshotFiles(logDir) + + // delete the last offset and transaction index files to force recovery. this should force us to rebuild + // the producer state from the start of the log + val lastSegment = log.logSegments.last + val recoveryPoint = lastSegment.baseOffset + lastSegment.offsetIndex.deleteIfExists() + lastSegment.txnIndex.deleteIfExists() + + log.close() + + val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) + val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) + val abortedTransactions = LogTestUtils.allAbortedTransactions(reloadedLog) + assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) + } +} diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 20ca8e6b761bd..c8891ebd58bc2 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -19,18 +19,16 @@ package kafka.log import java.io._ import java.nio.ByteBuffer -import java.nio.file.{Files, Paths} +import java.nio.file.Files import java.util.concurrent.{Callable, Executors} import java.util.regex.Pattern import java.util.{Collections, Optional, Properties} -import kafka.api.{ApiVersion, KAFKA_0_11_0_IV0} import kafka.common.{OffsetsOutOfOrderException, RecordValidationException, UnexpectedAppendOffsetException} import kafka.log.Log.DeleteDirSuffix import kafka.metrics.KafkaYammerMetrics import kafka.server.checkpoints.LeaderEpochCheckpointFile import kafka.server.epoch.{EpochEntry, LeaderEpochFileCache} -import kafka.server.metadata.CachedConfigRepository -import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata, PartitionMetadataFile} +import kafka.server.{BrokerTopicStats, FetchHighWatermark, FetchIsolation, FetchLogEnd, FetchTxnCommitted, KafkaConfig, LogDirFailureChannel, LogOffsetMetadata, PartitionMetadataFile} import kafka.utils._ import org.apache.kafka.common.{InvalidRecordException, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.errors._ @@ -45,7 +43,7 @@ import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} -import scala.collection.{Iterable, Map, mutable} +import scala.collection.{Iterable, Map} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ListBuffer @@ -76,82 +74,6 @@ class LogTest { } } - @Test - def testLogRecoveryIsCalledUponBrokerCrash(): Unit = { - // LogManager must realize correctly if the last shutdown was not clean and the logs need - // to run recovery while loading upon subsequent broker boot up. - val logDir: File = TestUtils.tempDir() - val logProps = new Properties() - val logConfig = LogConfig(logProps) - val logDirs = Seq(logDir) - val topicPartition = new TopicPartition("foo", 0) - var log: Log = null - val time = new MockTime() - var cleanShutdownInterceptedValue = false - var simulateError = false - - // Create a LogManager with some overridden methods to facilitate interception of clean shutdown - // flag and to inject a runtime error - def interceptedLogManager(logConfig: LogConfig, logDirs: Seq[File]): LogManager = { - new LogManager(logDirs = logDirs.map(_.getAbsoluteFile), initialOfflineDirs = Array.empty[File], new CachedConfigRepository(), - initialDefaultConfig = logConfig, cleanerConfig = CleanerConfig(enableCleaner = false), recoveryThreadsPerDataDir = 4, - flushCheckMs = 1000L, flushRecoveryOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L, - retentionCheckMs = 1000L, maxPidExpirationMs = 60 * 60 * 1000, scheduler = time.scheduler, time = time, - brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(logDirs.size), keepPartitionMetadataFile = config.usesTopicId) { - - override def loadLog(logDir: File, hadCleanShutdown: Boolean, recoveryPoints: Map[TopicPartition, Long], - logStartOffsets: Map[TopicPartition, Long], topicConfigs: Map[String, LogConfig]): Log = { - - val topicPartition = Log.parseTopicPartitionName(logDir) - val config = topicConfigs.getOrElse(topicPartition.topic, currentDefaultConfig) - val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L) - val logStartOffset = logStartOffsets.getOrElse(topicPartition, 0L) - val logDirFailureChannel: LogDirFailureChannel = new LogDirFailureChannel(1) - - val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxPidExpirationMs) - val log = new Log(logDir, config, logStartOffset, logRecoveryPoint, time.scheduler, brokerTopicStats, time, maxPidExpirationMs, - LogManager.ProducerIdExpirationCheckIntervalMs, topicPartition, producerStateManager, logDirFailureChannel, hadCleanShutdown, None, true) { - override def recoverLog(): Long = { - if (simulateError) - throw new RuntimeException - cleanShutdownInterceptedValue = hadCleanShutdown - super.recoverLog() - } - } - log - - } - - } - } - - val cleanShutdownFile = new File(logDir, Log.CleanShutdownFile) - val logManager: LogManager = interceptedLogManager(logConfig, logDirs) - log = logManager.getOrCreateLog(topicPartition, isNew = true, topicId = None) - - // Load logs after a clean shutdown - Files.createFile(cleanShutdownFile.toPath) - cleanShutdownInterceptedValue = false - logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) - assertTrue(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") - assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") - // Load logs without clean shutdown file - cleanShutdownInterceptedValue = true - logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) - assertFalse(cleanShutdownInterceptedValue, "Unexpected value intercepted for clean shutdown flag") - assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not exist after loadLogs has completed") - // Create clean shutdown file and then simulate error while loading logs such that log loading does not complete. - Files.createFile(cleanShutdownFile.toPath) - simulateError = true - assertThrows(classOf[RuntimeException], () => logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty))) - assertFalse(cleanShutdownFile.exists(), "Clean shutdown file must not have existed") - // Do not simulate error on next call to LogManager#loadLogs. LogManager must understand that log had unclean shutdown the last time. - simulateError = false - cleanShutdownInterceptedValue = true - logManager.loadLogs(logManager.fetchTopicConfigOverrides(Set.empty)) - assertFalse(cleanShutdownInterceptedValue, "Unexpected value for clean shutdown flag") - } - @Test def testHighWatermarkMetadataUpdatedAfterSegmentRoll(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024) @@ -458,7 +380,7 @@ class LogTest { // Test transactional producer state (open transaction) val producer1Epoch = 5.toShort val producerId1 = 1L - appendTransactionalAsLeader(log, producerId1, producer1Epoch)(5) + LogTestUtils.appendTransactionalAsLeader(log, producerId1, producer1Epoch, mockTime)(5) assertProducerState( producerId1, producer1Epoch, @@ -469,7 +391,7 @@ class LogTest { // Test transactional producer state (closed transaction) val coordinatorEpoch = 15 - appendEndTxnMarkerAsLeader(log, producerId1, producer1Epoch, ControlRecordType.COMMIT, coordinatorEpoch) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId1, producer1Epoch, ControlRecordType.COMMIT, mockTime.milliseconds(), coordinatorEpoch) assertProducerState( producerId1, producer1Epoch, @@ -481,7 +403,7 @@ class LogTest { // Test idempotent producer state val producer2Epoch = 5.toShort val producerId2 = 2L - appendIdempotentAsLeader(log, producerId2, producer2Epoch)(3) + LogTestUtils.appendIdempotentAsLeader(log, producerId2, producer2Epoch, mockTime)(3) assertProducerState( producerId2, producer2Epoch, @@ -500,14 +422,14 @@ class LogTest { val producerId1 = 1L val producerId2 = 2L - val appendProducer1 = appendTransactionalAsLeader(log, producerId1, epoch) - val appendProducer2 = appendTransactionalAsLeader(log, producerId2, epoch) + val appendProducer1 = LogTestUtils.appendTransactionalAsLeader(log, producerId1, epoch, mockTime) + val appendProducer2 = LogTestUtils.appendTransactionalAsLeader(log, producerId2, epoch, mockTime) appendProducer1(5) - appendNonTransactionalAsLeader(log, 3) + LogTestUtils.appendNonTransactionalAsLeader(log, 3) appendProducer2(2) appendProducer1(4) - appendNonTransactionalAsLeader(log, 2) + LogTestUtils.appendNonTransactionalAsLeader(log, 2) appendProducer1(10) def assertLsoBoundedFetches(): Unit = { @@ -525,14 +447,14 @@ class LogTest { log.updateHighWatermark(log.logEndOffset) assertLsoBoundedFetches() - appendEndTxnMarkerAsLeader(log, producerId1, epoch, ControlRecordType.COMMIT) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId1, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) assertEquals(0L, log.lastStableOffset) log.updateHighWatermark(log.logEndOffset) assertEquals(8L, log.lastStableOffset) assertLsoBoundedFetches() - appendEndTxnMarkerAsLeader(log, producerId2, epoch, ControlRecordType.ABORT) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) assertEquals(8L, log.lastStableOffset) log.updateHighWatermark(log.logEndOffset) @@ -651,8 +573,8 @@ class LogTest { log.appendAsFollower(records2) assertEquals(2, log.logEndOffset, "Expect two records in the log") - assertEquals(0, readLog(log, 0, 1).records.batches.iterator.next().lastOffset) - assertEquals(1, readLog(log, 1, 1).records.batches.iterator.next().lastOffset) + assertEquals(0, LogTestUtils.readLog(log, 0, 1).records.batches.iterator.next().lastOffset) + assertEquals(1, LogTestUtils.readLog(log, 1, 1).records.batches.iterator.next().lastOffset) // roll so that active segment is empty log.roll() @@ -666,7 +588,7 @@ class LogTest { baseOffset = 2L, partitionLeaderEpoch = 0) log.appendAsFollower(records3) assertTrue(log.activeSegment.offsetIndex.maxEntries > 1) - assertEquals(2, readLog(log, 2, 1).records.batches.iterator.next().lastOffset) + assertEquals(2, LogTestUtils.readLog(log, 2, 1).records.batches.iterator.next().lastOffset) assertEquals(2, log.numberOfSegments, "Expect two segments.") } @@ -791,74 +713,31 @@ class LogTest { val logEndOffset = log.logEndOffset log.close() - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) // Reload after clean shutdown log = createLog(logDir, logConfig, recoveryPoint = logEndOffset) var expectedSnapshotOffsets = log.logSegments.map(_.baseOffset).takeRight(2).toVector :+ log.logEndOffset - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir)) log.close() - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) // Reload after unclean shutdown with recoveryPoint set to log end offset log = createLog(logDir, logConfig, recoveryPoint = logEndOffset, lastShutdownClean = false) - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir)) log.close() - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) // Reload after unclean shutdown with recoveryPoint set to 0 log = createLog(logDir, logConfig, recoveryPoint = 0L, lastShutdownClean = false) // We progressively create a snapshot for each segment after the recovery point expectedSnapshotOffsets = log.logSegments.map(_.baseOffset).tail.toVector :+ log.logEndOffset - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets) - log.close() - } - - - @Test - def testRecoverAfterNonMonotonicCoordinatorEpochWrite(): Unit = { - // Due to KAFKA-9144, we may encounter a coordinator epoch which goes backwards. - // This test case verifies that recovery logic relaxes validation in this case and - // just takes the latest write. - - val producerId = 1L - val coordinatorEpoch = 5 - val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) - var log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val firstAppendTimestamp = mockTime.milliseconds() - appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, - timestamp = firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch) - assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) - - mockTime.sleep(log.maxProducerIdExpirationMs) - assertEquals(None, log.producerStateManager.lastEntry(producerId)) - - val secondAppendTimestamp = mockTime.milliseconds() - appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, - timestamp = secondAppendTimestamp, coordinatorEpoch = coordinatorEpoch - 1) - - log.close() - - // Force recovery by setting the recoveryPoint to the log start - log = createLog(logDir, logConfig, recoveryPoint = 0L, lastShutdownClean = false) - assertEquals(secondAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) + assertEquals(expectedSnapshotOffsets, LogTestUtils.listProducerSnapshotOffsets(logDir)) log.close() } - @Test - def testProducerSnapshotsRecoveryAfterUncleanShutdownV1(): Unit = { - testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.minSupportedFor(RecordVersion.V1).version) - } - - @Test - def testProducerSnapshotsRecoveryAfterUncleanShutdownCurrentMessageFormat(): Unit = { - testProducerSnapshotsRecoveryAfterUncleanShutdown(ApiVersion.latestVersion.version) - } - @Test def testLogReinitializeAfterManualDelete(): Unit = { val logConfig = LogTest.createLogConfig() @@ -981,94 +860,6 @@ class LogTest { } } - private def testProducerSnapshotsRecoveryAfterUncleanShutdown(messageFormatVersion: String): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 64 * 10, messageFormatVersion = messageFormatVersion) - var log = createLog(logDir, logConfig) - assertEquals(None, log.oldestProducerSnapshotOffset) - - for (i <- 0 to 100) { - val record = new SimpleRecord(mockTime.milliseconds, i.toString.getBytes) - log.appendAsLeader(TestUtils.records(List(record)), leaderEpoch = 0) - } - - assertTrue(log.logSegments.size >= 5) - val segmentOffsets = log.logSegments.toVector.map(_.baseOffset) - val activeSegmentOffset = segmentOffsets.last - - // We want the recovery point to be past the segment offset and before the last 2 segments including a gap of - // 1 segment. We collect the data before closing the log. - val offsetForSegmentAfterRecoveryPoint = segmentOffsets(segmentOffsets.size - 3) - val offsetForRecoveryPointSegment = segmentOffsets(segmentOffsets.size - 4) - val (segOffsetsBeforeRecovery, segOffsetsAfterRecovery) = segmentOffsets.toSet.partition(_ < offsetForRecoveryPointSegment) - val recoveryPoint = offsetForRecoveryPointSegment + 1 - assertTrue(recoveryPoint < offsetForSegmentAfterRecoveryPoint) - log.close() - - val segmentsWithReads = mutable.Set[LogSegment]() - val recoveredSegments = mutable.Set[LogSegment]() - val expectedSegmentsWithReads = mutable.Set[Long]() - val expectedSnapshotOffsets = mutable.Set[Long]() - - if (logConfig.messageFormatVersion < KAFKA_0_11_0_IV0) { - expectedSegmentsWithReads += activeSegmentOffset - expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(2) :+ log.logEndOffset - } else { - expectedSegmentsWithReads ++= segOffsetsBeforeRecovery ++ Set(activeSegmentOffset) - expectedSnapshotOffsets ++= log.logSegments.map(_.baseOffset).toVector.takeRight(4) :+ log.logEndOffset - } - - def createLogWithInterceptedReads(recoveryPoint: Long) = { - val maxProducerIdExpirationMs = 60 * 60 * 1000 - val topicPartition = Log.parseTopicPartitionName(logDir) - val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) - - // Intercept all segment read calls - new Log(logDir, logConfig, logStartOffset = 0, recoveryPoint = recoveryPoint, mockTime.scheduler, - brokerTopicStats, mockTime, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition, producerStateManager, new LogDirFailureChannel(10), hadCleanShutdown = false, topicId = None, keepPartitionMetadataFile = true) { - - override def addSegment(segment: LogSegment): LogSegment = { - val wrapper = new LogSegment(segment.log, segment.lazyOffsetIndex, segment.lazyTimeIndex, segment.txnIndex, segment.baseOffset, - segment.indexIntervalBytes, segment.rollJitterMs, mockTime) { - - override def read(startOffset: Long, maxSize: Int, maxPosition: Long, minOneMessage: Boolean): FetchDataInfo = { - segmentsWithReads += this - super.read(startOffset, maxSize, maxPosition, minOneMessage) - } - - override def recover(producerStateManager: ProducerStateManager, - leaderEpochCache: Option[LeaderEpochFileCache]): Int = { - recoveredSegments += this - super.recover(producerStateManager, leaderEpochCache) - } - } - super.addSegment(wrapper) - } - } - } - - // Retain snapshots for the last 2 segments - log.producerStateManager.deleteSnapshotsBefore(segmentOffsets(segmentOffsets.size - 2)) - log = createLogWithInterceptedReads(offsetForRecoveryPointSegment) - // We will reload all segments because the recovery point is behind the producer snapshot files (pre KAFKA-5829 behaviour) - assertEquals(expectedSegmentsWithReads, segmentsWithReads.map(_.baseOffset)) - assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets.toSet) - log.close() - segmentsWithReads.clear() - recoveredSegments.clear() - - // Only delete snapshots before the base offset of the recovery point segment (post KAFKA-5829 behaviour) to - // avoid reading all segments - log.producerStateManager.deleteSnapshotsBefore(offsetForRecoveryPointSegment) - log = createLogWithInterceptedReads(recoveryPoint = recoveryPoint) - assertEquals(Set(activeSegmentOffset), segmentsWithReads.map(_.baseOffset)) - assertEquals(segOffsetsAfterRecovery, recoveredSegments.map(_.baseOffset)) - assertEquals(expectedSnapshotOffsets, listProducerSnapshotOffsets.toSet) - - log.close() - } - @Test def testSizeForLargeLogs(): Unit = { val largeSize = Int.MaxValue.toLong * 2 @@ -1092,211 +883,6 @@ class LogTest { assertEquals(Some(1), log.latestProducerSnapshotOffset) } - @Test - def testSkipLoadingIfEmptyProducerStateBeforeTruncation(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - // Load the log - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.mapEndOffset).andStubReturn(0L) - EasyMock.expect(stateManager.isEmpty).andStubReturn(true) - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(0L), EasyMock.anyLong) - EasyMock.expectLastCall() - - EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) - - EasyMock.replay(stateManager) - - val config = LogConfig(new Properties()) - val log = new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = new LogDirFailureChannel(1), - hadCleanShutdown = false, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - - // Append some messages - EasyMock.reset(stateManager) - EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) - - stateManager.updateMapEndOffset(1L) - EasyMock.expectLastCall() - stateManager.updateMapEndOffset(2L) - EasyMock.expectLastCall() - - EasyMock.replay(stateManager) - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes))), leaderEpoch = 0) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes))), leaderEpoch = 0) - - EasyMock.verify(stateManager) - - // Now truncate - EasyMock.reset(stateManager) - EasyMock.expect(stateManager.firstUnstableOffset).andStubReturn(None) - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - EasyMock.expect(stateManager.isEmpty).andStubReturn(true) - EasyMock.expect(stateManager.mapEndOffset).andReturn(2L) - stateManager.truncateAndReload(EasyMock.eq(0L), EasyMock.eq(1L), EasyMock.anyLong) - EasyMock.expectLastCall() - // Truncation causes the map end offset to reset to 0 - EasyMock.expect(stateManager.mapEndOffset).andReturn(0L) - // We skip directly to updating the map end offset - EasyMock.expect(stateManager.updateMapEndOffset(1L)) - EasyMock.expect(stateManager.onHighWatermarkUpdated(0L)) - - // Finally, we take a snapshot - stateManager.takeSnapshot() - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - log.truncateTo(1L) - - EasyMock.verify(stateManager) - } - - @Test - def testSkipTruncateAndReloadIfOldMessageFormatAndNoCleanShutdown(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.isEmpty).andReturn(true) - EasyMock.expectLastCall().once() - - EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val config = LogConfig(logProps) - new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = null, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - } - - @Test - def testSkipTruncateAndReloadIfOldMessageFormatAndCleanShutdown(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.isEmpty).andReturn(true) - EasyMock.expectLastCall().once() - - EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.10.2") - val config = LogConfig(logProps) - new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = null, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - } - - @Test - def testSkipTruncateAndReloadIfNewMessageFormatAndCleanShutdown(): Unit = { - val stateManager: ProducerStateManager = EasyMock.mock(classOf[ProducerStateManager]) - EasyMock.expect(stateManager.removeStraySnapshots(EasyMock.anyObject())).anyTimes() - - EasyMock.expect(stateManager.latestSnapshotOffset).andReturn(None) - - stateManager.updateMapEndOffset(0L) - EasyMock.expectLastCall().anyTimes() - - stateManager.takeSnapshot() - EasyMock.expectLastCall().anyTimes() - - EasyMock.expect(stateManager.isEmpty).andReturn(true) - EasyMock.expectLastCall().once() - - EasyMock.expect(stateManager.firstUnstableOffset).andReturn(None) - EasyMock.expectLastCall().once() - - EasyMock.replay(stateManager) - - val logProps = new Properties() - logProps.put(LogConfig.MessageFormatVersionProp, "0.11.0") - val config = LogConfig(logProps) - new Log(logDir, - config, - logStartOffset = 0L, - recoveryPoint = 0L, - scheduler = mockTime.scheduler, - brokerTopicStats = brokerTopicStats, - time = mockTime, - maxProducerIdExpirationMs = 300000, - producerIdExpirationCheckIntervalMs = 30000, - topicPartition = Log.parseTopicPartitionName(logDir), - producerStateManager = stateManager, - logDirFailureChannel = null, - topicId = None, - keepPartitionMetadataFile = true) - - EasyMock.verify(stateManager) - } - @Test def testRebuildProducerIdMapWithCompactedData(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) @@ -1451,7 +1037,7 @@ class LogTest { log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes)), producerId = pid, producerEpoch = epoch, sequence = 1), leaderEpoch = 0) - deleteProducerSnapshotFiles() + LogTestUtils.deleteProducerSnapshotFiles(logDir) log.truncateTo(1L) assertEquals(1, log.activeProducersWithLastSequence.size) @@ -1463,38 +1049,6 @@ class LogTest { assertEquals(0, lastSeq) } - @Test - def testLoadProducersAfterDeleteRecordsMidSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) - val log = createLog(logDir, logConfig) - val pid1 = 1L - val pid2 = 2L - val epoch = 0.toShort - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - assertEquals(2, log.activeProducersWithLastSequence.size) - - log.updateHighWatermark(log.logEndOffset) - log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) - - // Deleting records should not remove producer state - assertEquals(2, log.activeProducersWithLastSequence.size) - val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) - assertTrue(retainedLastSeqOpt.isDefined) - assertEquals(0, retainedLastSeqOpt.get) - - log.close() - - // Because the log start offset did not advance, producer snapshots will still be present and the state will be rebuilt - val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) - assertEquals(2, reloadedLog.activeProducersWithLastSequence.size) - val reloadedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) - assertEquals(retainedLastSeqOpt, reloadedLastSeqOpt) - } - @Test def testRetentionDeletesProducerStateSnapshots(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) @@ -1623,73 +1177,6 @@ class LogTest { "expected producer state snapshots greater than the log end offset to be cleaned up") } - @Test - def testLoadingLogKeepsLargestStrayProducerStateSnapshot(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5, retentionBytes = 0, retentionMs = 1000 * 60, fileDeleteDelayMs = 0) - val log = createLog(logDir, logConfig) - val pid1 = 1L - val epoch = 0.toShort - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("a".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - log.roll() - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("b".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 1), leaderEpoch = 0) - log.roll() - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("c".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 2), leaderEpoch = 0) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("d".getBytes)), producerId = pid1, producerEpoch = epoch, sequence = 3), leaderEpoch = 0) - - // Close the log, we should now have 3 segments - log.close() - assertEquals(log.logSegments.size, 3) - // We expect 3 snapshot files, two of which are for the first two segments, the last was written out during log closing. - assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) - // Inject a stray snapshot file within the bounds of the log at offset 3, it should be cleaned up after loading the log - val straySnapshotFile = Log.producerSnapshotFile(logDir, 3).toPath - Files.createFile(straySnapshotFile) - assertEquals(Seq(1, 2, 3, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) - - createLog(logDir, logConfig, lastShutdownClean = false) - // We should clean up the stray producer state snapshot file, but keep the largest snapshot file (4) - assertEquals(Seq(1, 2, 4), ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted) - } - - @Test - def testLoadProducersAfterDeleteRecordsOnSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 2048 * 5) - val log = createLog(logDir, logConfig) - val pid1 = 1L - val pid2 = 2L - val epoch = 0.toShort - - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "a".getBytes)), producerId = pid1, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - log.roll() - log.appendAsLeader(TestUtils.records(List(new SimpleRecord(mockTime.milliseconds(), "b".getBytes)), producerId = pid2, - producerEpoch = epoch, sequence = 0), leaderEpoch = 0) - - assertEquals(2, log.logSegments.size) - assertEquals(2, log.activeProducersWithLastSequence.size) - - log.updateHighWatermark(log.logEndOffset) - log.maybeIncrementLogStartOffset(1L, ClientRecordDeletion) - log.deleteOldSegments() - - // Deleting records should not remove producer state - assertEquals(1, log.logSegments.size) - assertEquals(2, log.activeProducersWithLastSequence.size) - val retainedLastSeqOpt = log.activeProducersWithLastSequence.get(pid2) - assertTrue(retainedLastSeqOpt.isDefined) - assertEquals(0, retainedLastSeqOpt.get) - - log.close() - - // After reloading log, producer state should not be regenerated - val reloadedLog = createLog(logDir, logConfig, logStartOffset = 1L, lastShutdownClean = false) - assertEquals(1, reloadedLog.activeProducersWithLastSequence.size) - val reloadedEntryOpt = log.activeProducersWithLastSequence.get(pid2) - assertEquals(retainedLastSeqOpt, reloadedEntryOpt) - } - @Test def testProducerIdMapTruncateFullyAndStartAt(): Unit = { val records = TestUtils.singletonRecords("foo".getBytes) @@ -1814,7 +1301,7 @@ class LogTest { new SimpleRecord("bar".getBytes), new SimpleRecord("baz".getBytes)) log.appendAsLeader(records, leaderEpoch = 0) - val abortAppendInfo = appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT) + val abortAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) log.updateHighWatermark(abortAppendInfo.lastOffset + 1) // now there should be no first unstable offset @@ -1827,17 +1314,6 @@ class LogTest { assertEquals(None, reopenedLog.firstUnstableOffset) } - private def endTxnRecords(controlRecordType: ControlRecordType, - producerId: Long, - epoch: Short, - offset: Long = 0L, - coordinatorEpoch: Int, - partitionLeaderEpoch: Int = 0, - timestamp: Long): MemoryRecords = { - val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch) - MemoryRecords.withEndTransactionMarker(offset, timestamp, partitionLeaderEpoch, producerId, epoch, marker) - } - @Test def testPeriodicProducerIdExpiration(): Unit = { val maxProducerIdExpirationMs = 200 @@ -1969,7 +1445,7 @@ class LogTest { log.appendAsFollower(memoryRecords) log.flush() - val fetchedData = readLog(log, 0, Int.MaxValue) + val fetchedData = LogTestUtils.readLog(log, 0, Int.MaxValue) val origIterator = memoryRecords.batches.iterator() for (batch <- fetchedData.records.batches.asScala) { @@ -2170,13 +1646,13 @@ class LogTest { log.appendAsLeader(TestUtils.singletonRecords(value = value), leaderEpoch = 0) for(i <- values.indices) { - val read = readLog(log, i, 1).records.batches.iterator.next() + val read = LogTestUtils.readLog(log, i, 1).records.batches.iterator.next() assertEquals(i, read.lastOffset, "Offset read should match order appended.") val actual = read.iterator.next() assertNull(actual.key, "Key should be null") assertEquals(ByteBuffer.wrap(values(i)), actual.value, "Values not equal") } - assertEquals(0, readLog(log, values.length, 100).records.batches.asScala.size, + assertEquals(0, LogTestUtils.readLog(log, values.length, 100).records.batches.asScala.size, "Reading beyond the last message returns nothing.") } @@ -2196,7 +1672,7 @@ class LogTest { log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i))) for(i <- 50 until messageIds.max) { val idx = messageIds.indexWhere(_ >= i) - val read = readLog(log, i, 100).records.records.iterator.next() + val read = LogTestUtils.readLog(log, i, 100).records.records.iterator.next() assertEquals(messageIds(idx), read.offset, "Offset read should match message id.") assertEquals(records(idx), new SimpleRecord(read), "Message should match appended.") } @@ -2220,7 +1696,7 @@ class LogTest { // now manually truncate off all but one message from the first segment to create a gap in the messages log.logSegments.head.truncateTo(1) - assertEquals(log.logEndOffset - 1, readLog(log, 1, 200).records.batches.iterator.next().lastOffset, + assertEquals(log.logEndOffset - 1, LogTestUtils.readLog(log, 1, 200).records.batches.iterator.next().lastOffset, "A read should now return the last message in the log") } @@ -2246,9 +1722,9 @@ class LogTest { for (i <- 50 until messageIds.max) { val idx = messageIds.indexWhere(_ >= i) val reads = Seq( - readLog(log, i, 1), - readLog(log, i, 100000), - readLog(log, i, 100) + LogTestUtils.readLog(log, i, 1), + LogTestUtils.readLog(log, i, 100000), + LogTestUtils.readLog(log, i, 100) ).map(_.records.records.iterator.next()) reads.foreach { read => assertEquals(messageIds(idx), read.offset, "Offset read should match message id.") @@ -2269,14 +1745,14 @@ class LogTest { log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i))) for (i <- 50 until messageIds.max) { - assertEquals(MemoryRecords.EMPTY, readLog(log, i, maxLength = 0, minOneMessage = false).records) + assertEquals(MemoryRecords.EMPTY, LogTestUtils.readLog(log, i, maxLength = 0, minOneMessage = false).records) // we return an incomplete message instead of an empty one for the case below // we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is // larger than the fetch size // in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty // partition - val fetchInfo = readLog(log, i, maxLength = 1, minOneMessage = false) + val fetchInfo = LogTestUtils.readLog(log, i, maxLength = 1, minOneMessage = false) assertTrue(fetchInfo.firstEntryIncomplete) assertTrue(fetchInfo.records.isInstanceOf[FileRecords]) assertEquals(1, fetchInfo.records.sizeInBytes) @@ -2297,11 +1773,11 @@ class LogTest { val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.singletonRecords(value = "42".getBytes), leaderEpoch = 0) - assertEquals(0, readLog(log, 1025, 1000).records.sizeInBytes, + assertEquals(0, LogTestUtils.readLog(log, 1025, 1000).records.sizeInBytes, "Reading at the log end offset should produce 0 byte read.") - assertThrows(classOf[OffsetOutOfRangeException], () => readLog(log, 0, 1000)) - assertThrows(classOf[OffsetOutOfRangeException], () => readLog(log, 1026, 1000)) + assertThrows(classOf[OffsetOutOfRangeException], () => LogTestUtils.readLog(log, 0, 1000)) + assertThrows(classOf[OffsetOutOfRangeException], () => LogTestUtils.readLog(log, 1026, 1000)) } /** @@ -2322,7 +1798,7 @@ class LogTest { /* do successive reads to ensure all our messages are there */ var offset = 0L for(i <- 0 until numMessages) { - val messages = readLog(log, offset, 1024*1024).records.batches + val messages = LogTestUtils.readLog(log, offset, 1024*1024).records.batches val head = messages.iterator.next() assertEquals(offset, head.lastOffset, "Offsets not equal") @@ -2333,7 +1809,7 @@ class LogTest { assertEquals(expected.timestamp, actual.timestamp, s"Timestamps not equal at offset $offset") offset = head.lastOffset + 1 } - val lastRead = readLog(log, startOffset = numMessages, maxLength = 1024*1024).records + val lastRead = LogTestUtils.readLog(log, startOffset = numMessages, maxLength = 1024*1024).records assertEquals(0, lastRead.records.asScala.size, "Should be no more messages") // check that rolling the log forced a flushed, the flush is async so retry in case of failure @@ -2355,7 +1831,7 @@ class LogTest { log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0) log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0) - def read(offset: Int) = readLog(log, offset, 4096).records.records + def read(offset: Int) = LogTestUtils.readLog(log, offset, 4096).records.records /* we should always get the first message in the compressed set when reading any offset in the set */ assertEquals(0, read(0).iterator.next().offset, "Read at offset 0 should produce 0") @@ -2389,7 +1865,7 @@ class LogTest { assertEquals(0, log.deleteOldSegments(), "Further collection shouldn't delete anything") assertEquals(currOffset, log.logEndOffset, "Still no change in the logEndOffset") assertEquals( - currOffset, + currOffset, log.appendAsLeader( TestUtils.singletonRecords(value = "hello".getBytes, timestamp = mockTime.milliseconds), leaderEpoch = 0 @@ -2504,58 +1980,6 @@ class LogTest { log.appendAsFollower(second) } - /** - * Append a bunch of messages to a log and then re-open it both with and without recovery and check that the log re-initializes correctly. - */ - @Test - def testLogRecoversToCorrectOffset(): Unit = { - val numMessages = 100 - val messageSize = 100 - val segmentSize = 7 * messageSize - val indexInterval = 3 * messageSize - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = indexInterval, segmentIndexBytes = 4096) - var log = createLog(logDir, logConfig) - for(i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize), - timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) - assertEquals(numMessages, log.logEndOffset, - "After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages)) - val lastIndexOffset = log.activeSegment.offsetIndex.lastOffset - val numIndexEntries = log.activeSegment.offsetIndex.entries - val lastOffset = log.logEndOffset - // After segment is closed, the last entry in the time index should be (largest timestamp -> last offset). - val lastTimeIndexOffset = log.logEndOffset - 1 - val lastTimeIndexTimestamp = log.activeSegment.largestTimestamp - // Depending on when the last time index entry is inserted, an entry may or may not be inserted into the time index. - val numTimeIndexEntries = log.activeSegment.timeIndex.entries + { - if (log.activeSegment.timeIndex.lastEntry.offset == log.logEndOffset - 1) 0 else 1 - } - log.close() - - def verifyRecoveredLog(log: Log, expectedRecoveryPoint: Long): Unit = { - assertEquals(expectedRecoveryPoint, log.recoveryPoint, s"Unexpected recovery point") - assertEquals(numMessages, log.logEndOffset, s"Should have $numMessages messages when log is reopened w/o recovery") - assertEquals(lastIndexOffset, log.activeSegment.offsetIndex.lastOffset, "Should have same last index offset as before.") - assertEquals(numIndexEntries, log.activeSegment.offsetIndex.entries, "Should have same number of index entries as before.") - assertEquals(lastTimeIndexTimestamp, log.activeSegment.timeIndex.lastEntry.timestamp, "Should have same last time index timestamp") - assertEquals(lastTimeIndexOffset, log.activeSegment.timeIndex.lastEntry.offset, "Should have same last time index offset") - assertEquals(numTimeIndexEntries, log.activeSegment.timeIndex.entries, "Should have same number of time index entries as before.") - } - - log = createLog(logDir, logConfig, recoveryPoint = lastOffset, lastShutdownClean = false) - verifyRecoveredLog(log, lastOffset) - log.close() - - // test recovery case - val recoveryPoint = 10 - log = createLog(logDir, logConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) - // the recovery point should not be updated after unclean shutdown until the log is flushed - verifyRecoveredLog(log, recoveryPoint) - log.flush() - verifyRecoveredLog(log, lastOffset) - log.close() - } - @Test def testLogRecoversTopicId(): Unit = { val logConfig = LogTest.createLogConfig() @@ -2609,40 +2033,6 @@ class LogTest { s"The last time index entry should have timestamp ${mockTime.milliseconds + numMessages - 1}") } - /** - * Test that if we manually delete an index segment it is rebuilt when the log is re-opened - */ - @Test - def testIndexRebuild(): Unit = { - // publish the messages and close the log - val numMessages = 200 - val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) - var log = createLog(logDir, logConfig) - for(i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) - val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) - val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) - log.close() - - // delete all the index files - indexFiles.foreach(_.delete()) - timeIndexFiles.foreach(_.delete()) - - // reopen the log - log = createLog(logDir, logConfig, lastShutdownClean = false) - assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) - assertTrue(log.logSegments.head.offsetIndex.entries > 0, "The index should have been rebuilt") - assertTrue(log.logSegments.head.timeIndex.entries > 0, "The time index should have been rebuilt") - for(i <- 0 until numMessages) { - assertEquals(i, readLog(log, i, 100).records.batches.iterator.next().lastOffset) - if (i == 0) - assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - else - assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - } - log.close() - } - @Test def testFetchOffsetByTimestampIncludesLeaderEpoch(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) @@ -2682,74 +2072,6 @@ class LogTest { log.fetchOffsetByTimestamp(ListOffsetsRequest.LATEST_TIMESTAMP)) } - /** - * Test that if messages format version of the messages in a segment is before 0.10.0, the time index should be empty. - */ - @Test - def testRebuildTimeIndexForOldMessages(): Unit = { - val numMessages = 200 - val segmentSize = 200 - val logConfig = LogTest.createLogConfig(segmentBytes = segmentSize, indexIntervalBytes = 1, messageFormatVersion = "0.9.0") - var log = createLog(logDir, logConfig) - for (i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), - timestamp = mockTime.milliseconds + i * 10, magicValue = RecordBatch.MAGIC_VALUE_V1), leaderEpoch = 0) - val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) - log.close() - - // Delete the time index. - timeIndexFiles.foreach(file => Files.delete(file.toPath)) - - // The rebuilt time index should be empty - log = createLog(logDir, logConfig, recoveryPoint = numMessages + 1, lastShutdownClean = false) - for (segment <- log.logSegments.init) { - assertEquals(0, segment.timeIndex.entries, "The time index should be empty") - assertEquals(0, segment.lazyTimeIndex.file.length, "The time index file size should be 0") - } - } - - /** - * Test that if we have corrupted an index segment it is rebuilt when the log is re-opened - */ - @Test - def testCorruptIndexRebuild(): Unit = { - // publish the messages and close the log - val numMessages = 200 - val logConfig = LogTest.createLogConfig(segmentBytes = 200, indexIntervalBytes = 1) - var log = createLog(logDir, logConfig) - for(i <- 0 until numMessages) - log.appendAsLeader(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = mockTime.milliseconds + i * 10), leaderEpoch = 0) - val indexFiles = log.logSegments.map(_.lazyOffsetIndex.file) - val timeIndexFiles = log.logSegments.map(_.lazyTimeIndex.file) - log.close() - - // corrupt all the index files - for( file <- indexFiles) { - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(" ") - bw.close() - } - - // corrupt all the index files - for( file <- timeIndexFiles) { - val bw = new BufferedWriter(new FileWriter(file)) - bw.write(" ") - bw.close() - } - - // reopen the log with recovery point=0 so that the segment recovery can be triggered - log = createLog(logDir, logConfig, lastShutdownClean = false) - assertEquals(numMessages, log.logEndOffset, "Should have %d messages when log is reopened".format(numMessages)) - for(i <- 0 until numMessages) { - assertEquals(i, readLog(log, i, 100).records.batches.iterator.next().lastOffset) - if (i == 0) - assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - else - assertEquals(i, log.fetchOffsetByTimestamp(mockTime.milliseconds + i * 10).get.offset) - } - log.close() - } - /** * Test the Log truncate operations */ @@ -2847,66 +2169,7 @@ class LogTest { } /** - * When we open a log any index segments without an associated log segment should be deleted. - */ - @Test - def testBogusIndexSegmentsAreRemoved(): Unit = { - val bogusIndex1 = Log.offsetIndexFile(logDir, 0) - val bogusTimeIndex1 = Log.timeIndexFile(logDir, 0) - val bogusIndex2 = Log.offsetIndexFile(logDir, 5) - val bogusTimeIndex2 = Log.timeIndexFile(logDir, 5) - - // The files remain absent until we first access it because we are doing lazy loading for time index and offset index - // files but in this test case we need to create these files in order to test we will remove them. - bogusIndex2.createNewFile() - bogusTimeIndex2.createNewFile() - - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 1) - val log = createLog(logDir, logConfig) - - // Force the segment to access the index files because we are doing index lazy loading. - log.logSegments.toSeq.head.offsetIndex - log.logSegments.toSeq.head.timeIndex - - assertTrue(bogusIndex1.length > 0, - "The first index file should have been replaced with a larger file") - assertTrue(bogusTimeIndex1.length > 0, - "The first time index file should have been replaced with a larger file") - assertFalse(bogusIndex2.exists, - "The second index file should have been deleted.") - assertFalse(bogusTimeIndex2.exists, - "The second time index file should have been deleted.") - - // check that we can append to the log - for (_ <- 0 until 10) - log.appendAsLeader(createRecords, leaderEpoch = 0) - - log.delete() - } - - /** - * Verify that truncation works correctly after re-opening the log - */ - @Test - def testReopenThenTruncate(): Unit = { - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - // create a log - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, indexIntervalBytes = 10000) - var log = createLog(logDir, logConfig) - - // add enough messages to roll over several segments then close and re-open and attempt to truncate - for (_ <- 0 until 100) - log.appendAsLeader(createRecords, leaderEpoch = 0) - log.close() - log = createLog(logDir, logConfig, lastShutdownClean = false) - log.truncateTo(3) - assertEquals(1, log.numberOfSegments, "All but one segment should be deleted.") - assertEquals(3, log.logEndOffset, "Log end offset should be 3.") - } - - /** - * Test that deleted files are deleted after the appropriate time. + * Test that deleted files are deleted after the appropriate time. */ @Test def testAsyncDelete(): Unit = { @@ -2941,32 +2204,11 @@ class LogTest { assertTrue(deletedFiles.forall(!_.exists), "Files should all be gone.") } - /** - * Any files ending in .deleted should be removed when the log is re-opened. - */ - @Test - def testOpenDeletesObsoleteFiles(): Unit = { - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds - 1000) - val logConfig = LogTest.createLogConfig(segmentBytes = createRecords.sizeInBytes * 5, segmentIndexBytes = 1000, retentionMs = 999) - var log = createLog(logDir, logConfig) - - // append some messages to create some segments - for (_ <- 0 until 100) - log.appendAsLeader(createRecords, leaderEpoch = 0) - - // expire all segments - log.updateHighWatermark(log.logEndOffset) - log.deleteOldSegments() - log.close() - log = createLog(logDir, logConfig, lastShutdownClean = false) - assertEquals(1, log.numberOfSegments, "The deleted segments should be gone.") - } - @Test def testAppendMessageWithNullPayload(): Unit = { val log = createLog(logDir, LogConfig()) log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0) - val head = readLog(log, 0, 4096).records.records.iterator.next() + val head = LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next() assertEquals(0, head.offset) assertFalse(head.hasValue, "Message payload should be null.") } @@ -3044,87 +2286,15 @@ class LogTest { val epoch = 0.toShort val log = createLog(logDir, LogConfig()) log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0) - assertEquals(0, readLog(log, 0, 4096).records.records.iterator.next().offset) - val append = appendTransactionalAsLeader(log, pid, epoch) + assertEquals(0, LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next().offset) + val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) append(10) // Kind of a hack, but renaming the index to a directory ensures that the append // to the index will fail. log.activeSegment.txnIndex.renameTo(log.dir) - assertThrows(classOf[KafkaStorageException], () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows(classOf[KafkaStorageException], () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) assertThrows(classOf[KafkaStorageException], () => log.appendAsLeader(TestUtils.singletonRecords(value = null), leaderEpoch = 0)) - assertThrows(classOf[KafkaStorageException], () => readLog(log, 0, 4096).records.records.iterator.next().offset) - } - - @Test - def testCorruptLog(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - val recoveryPoint = 50L - for (_ <- 0 until 10) { - // create a log and write some messages to it - logDir.mkdirs() - var log = createLog(logDir, logConfig) - val numMessages = 50 + TestUtils.random.nextInt(50) - for (_ <- 0 until numMessages) - log.appendAsLeader(createRecords, leaderEpoch = 0) - val records = log.logSegments.flatMap(_.log.records.asScala.toList).toList - log.close() - - // corrupt index and log by appending random bytes - TestUtils.appendNonsenseToFile(log.activeSegment.lazyOffsetIndex.file, TestUtils.random.nextInt(1024) + 1) - TestUtils.appendNonsenseToFile(log.activeSegment.log.file, TestUtils.random.nextInt(1024) + 1) - - // attempt recovery - log = createLog(logDir, logConfig, brokerTopicStats, 0L, recoveryPoint, lastShutdownClean = false) - assertEquals(numMessages, log.logEndOffset) - - val recovered = log.logSegments.flatMap(_.log.records.asScala.toList).toList - assertEquals(records.size, recovered.size) - - for (i <- records.indices) { - val expected = records(i) - val actual = recovered(i) - assertEquals(expected.key, actual.key, s"Keys not equal") - assertEquals(expected.value, actual.value, s"Values not equal") - assertEquals(expected.timestamp, actual.timestamp, s"Timestamps not equal") - } - - Utils.delete(logDir) - } - } - - @Test - def testOverCompactedLogRecovery(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) - val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes())) - val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, CompressionType.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes())) - val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes())) - //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) - assertEquals(0L, log.activeSegment.baseOffset) - //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 - log.appendAsFollower(set2) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) - //This will go into the existing log - log.appendAsFollower(set3) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - //This will go into the existing log - log.appendAsFollower(set4) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - log.close() - val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) - assertEquals(2, indexFiles.length) - for (file <- indexFiles) { - val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) - assertTrue(offsetIndex.lastOffset >= 0) - offsetIndex.close() - } - Utils.delete(logDir) + assertThrows(classOf[KafkaStorageException], () => LogTestUtils.readLog(log, 0, 4096).records.records.iterator.next().offset) } @Test @@ -3183,25 +2353,6 @@ class LogTest { assertEquals(None, log.leaderEpochCache.flatMap(_.latestEpoch)) } - @Test - def testLeaderEpochCacheClearedAfterStaticMessageFormatDowngrade(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - log.appendAsLeader(TestUtils.records(List(new SimpleRecord("foo".getBytes()))), leaderEpoch = 5) - assertEquals(Some(5), log.latestEpoch) - log.close() - - // reopen the log with an older message format version and check the cache - val downgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, - maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) - val reopened = createLog(logDir, downgradedLogConfig, lastShutdownClean = false) - assertLeaderEpochCacheEmpty(reopened) - - reopened.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), - magicValue = RecordVersion.V1.value), leaderEpoch = 5) - assertLeaderEpochCacheEmpty(reopened) - } - @Test def testLeaderEpochCacheClearedAfterDynamicMessageFormatDowngrade(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) @@ -3212,11 +2363,11 @@ class LogTest { val downgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_10_2_IV0.shortVersion) log.updateConfig(downgradedLogConfig) - assertLeaderEpochCacheEmpty(log) + LogTestUtils.assertLeaderEpochCacheEmpty(log) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), magicValue = RecordVersion.V1.value), leaderEpoch = 5) - assertLeaderEpochCacheEmpty(log) + LogTestUtils.assertLeaderEpochCacheEmpty(log) } @Test @@ -3226,7 +2377,7 @@ class LogTest { val log = createLog(logDir, logConfig) log.appendAsLeader(TestUtils.records(List(new SimpleRecord("bar".getBytes())), magicValue = RecordVersion.V1.value), leaderEpoch = 5) - assertLeaderEpochCacheEmpty(log) + LogTestUtils.assertLeaderEpochCacheEmpty(log) val upgradedLogConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024, messageFormatVersion = kafka.api.KAFKA_0_11_0_IV0.shortVersion) @@ -3235,91 +2386,7 @@ class LogTest { assertEquals(Some(5), log.latestEpoch) } - private def assertLeaderEpochCacheEmpty(log: Log): Unit = { - assertEquals(None, log.leaderEpochCache) - assertEquals(None, log.latestEpoch) - assertFalse(LeaderEpochCheckpointFile.newFile(log.dir).exists()) - } - @Test - def testOverCompactedLogRecoveryMultiRecord(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) - val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0, - new SimpleRecord("v3".getBytes(), "k3".getBytes()), - new SimpleRecord("v4".getBytes(), "k4".getBytes())) - val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, 0, - new SimpleRecord("v5".getBytes(), "k5".getBytes()), - new SimpleRecord("v6".getBytes(), "k6".getBytes())) - val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, 0, - new SimpleRecord("v7".getBytes(), "k7".getBytes()), - new SimpleRecord("v8".getBytes(), "k8".getBytes())) - //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) - assertEquals(0L, log.activeSegment.baseOffset) - //This write will roll the segment, yielding a new segment with base offset = max(1, Integer.MAX_VALUE+2) = Integer.MAX_VALUE+2 - log.appendAsFollower(set2) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 2).exists) - //This will go into the existing log - log.appendAsFollower(set3) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - //This will go into the existing log - log.appendAsFollower(set4) - assertEquals(Integer.MAX_VALUE.toLong + 2, log.activeSegment.baseOffset) - log.close() - val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) - assertEquals(2, indexFiles.length) - for (file <- indexFiles) { - val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) - assertTrue(offsetIndex.lastOffset >= 0) - offsetIndex.close() - } - Utils.delete(logDir) - } - - @Test - def testOverCompactedLogRecoveryMultiRecordV1(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - val log = createLog(logDir, logConfig) - val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, CompressionType.NONE, - new SimpleRecord("v1".getBytes(), "k1".getBytes())) - val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, - new SimpleRecord("v3".getBytes(), "k3".getBytes()), - new SimpleRecord("v4".getBytes(), "k4".getBytes())) - val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, - new SimpleRecord("v5".getBytes(), "k5".getBytes()), - new SimpleRecord("v6".getBytes(), "k6".getBytes())) - val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, - new SimpleRecord("v7".getBytes(), "k7".getBytes()), - new SimpleRecord("v8".getBytes(), "k8".getBytes())) - //Writes into an empty log with baseOffset 0 - log.appendAsFollower(set1) - assertEquals(0L, log.activeSegment.baseOffset) - //This write will roll the segment, yielding a new segment with base offset = max(1, 3) = 3 - log.appendAsFollower(set2) - assertEquals(3, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, 3).exists) - //This will also roll the segment, yielding a new segment with base offset = max(5, Integer.MAX_VALUE+4) = Integer.MAX_VALUE+4 - log.appendAsFollower(set3) - assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) - assertTrue(Log.producerSnapshotFile(logDir, Integer.MAX_VALUE.toLong + 4).exists) - //This will go into the existing log - log.appendAsFollower(set4) - assertEquals(Integer.MAX_VALUE.toLong + 4, log.activeSegment.baseOffset) - log.close() - val indexFiles = logDir.listFiles.filter(file => file.getName.contains(".index")) - assertEquals(3, indexFiles.length) - for (file <- indexFiles) { - val offsetIndex = new OffsetIndex(file, file.getName.replace(".index","").toLong) - assertTrue(offsetIndex.lastOffset >= 0) - offsetIndex.close() - } - Utils.delete(logDir) - } @Test def testSplitOnOffsetOverflow(): Unit = { @@ -3396,175 +2463,6 @@ class LogTest { assertFalse(LogTest.hasOffsetOverflow(log)) } - @Test - def testRecoveryOfSegmentWithOffsetOverflow(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, _) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - - // Run recovery on the log. This should split the segment underneath. Ignore .deleted files as we could have still - // have them lying around after the split. - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - - // Running split again would throw an error - - for (segment <- recoveredLog.logSegments) { - assertThrows(classOf[IllegalArgumentException], () => log.splitOverflowedSegment(segment)) - } - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase1(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery just after .cleaned file is created, before rename to .swap. On recovery, existing split - // operation is aborted but the recovery process itself kicks off split which should complete. - newSegments.reverse.foreach(segment => { - segment.changeFileSuffixes("", Log.CleanedFileSuffix) - segment.truncateTo(0) - }) - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")), false) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase2(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery just after one of the new segments has been renamed to .swap. On recovery, existing split - // operation is aborted but the recovery process itself kicks off split which should complete. - newSegments.reverse.foreach { segment => - if (segment != newSegments.last) - segment.changeFileSuffixes("", Log.CleanedFileSuffix) - else - segment.changeFileSuffixes("", Log.SwapFileSuffix) - segment.truncateTo(0) - } - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")), false) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase3(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery right after all new segments have been renamed to .swap. On recovery, existing split operation - // is completed and the old segment must be deleted. - newSegments.reverse.foreach(segment => { - segment.changeFileSuffixes("", Log.SwapFileSuffix) - }) - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")), false) - - // Truncate the old segment - segmentWithOverflow.truncateTo(0) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - log.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase4(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery right after all new segments have been renamed to .swap and old segment has been deleted. On - // recovery, existing split operation is completed. - newSegments.reverse.foreach(_.changeFileSuffixes("", Log.SwapFileSuffix)) - - for (file <- logDir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) - Utils.delete(file) - - // Truncate the old segment - segmentWithOverflow.truncateTo(0) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testRecoveryAfterCrashDuringSplitPhase5(): Unit = { - val logConfig = LogTest.createLogConfig(indexIntervalBytes = 1, fileDeleteDelayMs = 1000) - val (log, segmentWithOverflow) = createLogWithOffsetOverflow(logConfig) - val expectedKeys = LogTest.keysInLog(log) - val numSegmentsInitial = log.logSegments.size - - // Split the segment - val newSegments = log.splitOverflowedSegment(segmentWithOverflow) - - // Simulate recovery right after one of the new segment has been renamed to .swap and the other to .log. On - // recovery, existing split operation is completed. - newSegments.last.changeFileSuffixes("", Log.SwapFileSuffix) - - // Truncate the old segment - segmentWithOverflow.truncateTo(0) - - val recoveredLog = recoverAndCheck(logConfig, expectedKeys) - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertEquals(numSegmentsInitial + 1, recoveredLog.logSegments.size) - recoveredLog.close() - } - - @Test - def testCleanShutdownFile(): Unit = { - // append some messages to create some segments - val logConfig = LogTest.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) - def createRecords = TestUtils.singletonRecords(value = "test".getBytes, timestamp = mockTime.milliseconds) - - var recoveryPoint = 0L - // create a log and write some messages to it - var log = createLog(logDir, logConfig) - for (_ <- 0 until 100) - log.appendAsLeader(createRecords, leaderEpoch = 0) - log.close() - - // check if recovery was attempted. Even if the recovery point is 0L, recovery should not be attempted as the - // clean shutdown file exists. Note: Earlier, Log layer relied on the presence of clean shutdown file to determine the status - // of last shutdown. Now, LogManager checks for the presence of this file and immediately deletes the same. It passes - // down a clean shutdown flag to the Log layer as log is loaded. Recovery is attempted based on this flag. - recoveryPoint = log.logEndOffset - log = createLog(logDir, logConfig) - assertEquals(recoveryPoint, log.logEndOffset) - } - @Test def testParseTopicPartitionName(): Unit = { val topic = "test_topic" @@ -3910,7 +2808,7 @@ class LogTest { //Then leader epoch should be set on messages for (i <- records.indices) { - val read = readLog(log, i, 1).records.batches.iterator.next() + val read = LogTestUtils.readLog(log, i, 1).records.batches.iterator.next() assertEquals(72, read.partitionLeaderEpoch, "Should have set leader epoch") } } @@ -4038,60 +2936,6 @@ class LogTest { assertEquals(0, cache.epochEntries.size) } - /** - * Append a bunch of messages to a log and then re-open it with recovery and check that the leader epochs are recovered properly. - */ - @Test - def testLogRecoversForLeaderEpoch(): Unit = { - val log = createLog(logDir, LogConfig()) - val leaderEpochCache = epochCache(log) - val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0) - log.appendAsFollower(records = firstBatch) - - val secondBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 1) - log.appendAsFollower(records = secondBatch) - - val thirdBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 2) - log.appendAsFollower(records = thirdBatch) - - val fourthBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 3, offset = 3) - log.appendAsFollower(records = fourthBatch) - - assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) - - // deliberately remove some of the epoch entries - leaderEpochCache.truncateFromEnd(2) - assertNotEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries) - log.close() - - // reopen the log and recover from the beginning - val recoveredLog = createLog(logDir, LogConfig(), lastShutdownClean = false) - val recoveredLeaderEpochCache = epochCache(recoveredLog) - - // epoch entries should be recovered - assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), recoveredLeaderEpochCache.epochEntries) - recoveredLog.close() - } - - /** - * Wrap a single record log buffer with leader epoch. - */ - private def singletonRecordsWithLeaderEpoch(value: Array[Byte], - key: Array[Byte] = null, - leaderEpoch: Int, - offset: Long, - codec: CompressionType = CompressionType.NONE, - timestamp: Long = RecordBatch.NO_TIMESTAMP, - magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = { - val records = Seq(new SimpleRecord(timestamp, key, value)) - - val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) - val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset, - mockTime.milliseconds, leaderEpoch) - records.foreach(builder.append) - builder.build() - } - @Test def testFirstUnstableOffsetNoTransactionalData(): Unit = { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) @@ -4133,7 +2977,7 @@ class LogTest { assertEquals(firstAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) // now transaction is committed - val commitAppendInfo = appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT) + val commitAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // first unstable offset is not updated until the high watermark is advanced assertEquals(firstAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) @@ -4151,7 +2995,7 @@ class LogTest { val producerEpoch = 0.toShort val producerId = 15L - val appendProducer = appendTransactionalAsLeader(log, producerId, producerEpoch) + val appendProducer = LogTestUtils.appendTransactionalAsLeader(log, producerId, producerEpoch, mockTime) // Thread 1 writes single-record transactions and attempts to read them // before they have been aborted, and then aborts them @@ -4171,7 +3015,7 @@ class LogTest { if (readInfo.records.sizeInBytes() > 0) nonEmptyReads += 1 - appendEndTxnMarkerAsLeader(log, producerId, producerEpoch, ControlRecordType.ABORT) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, producerEpoch, ControlRecordType.ABORT, mockTime.milliseconds()) } nonEmptyReads } @@ -4207,33 +3051,33 @@ class LogTest { val pid3 = 3L val pid4 = 4L - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) + val appendPid1 = LogTestUtils.appendTransactionalAsLeader(log, pid1, epoch, mockTime) + val appendPid2 = LogTestUtils.appendTransactionalAsLeader(log, pid2, epoch, mockTime) + val appendPid3 = LogTestUtils.appendTransactionalAsLeader(log, pid3, epoch, mockTime) + val appendPid4 = LogTestUtils.appendTransactionalAsLeader(log, pid4, epoch, mockTime) // mix transactional and non-transactional data appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 + LogTestUtils.appendNonTransactionalAsLeader(log, 3) // 8 appendPid2(2) // 10 appendPid1(4) // 14 appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 + LogTestUtils.appendNonTransactionalAsLeader(log, 2) // 19 appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 30 appendPid2(6) // 36 appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 49 appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 59 appendPid4(8) // 67 appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) // 75 + LogTestUtils.appendNonTransactionalAsLeader(log, 10) // 85 appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) // 90 - val abortedTransactions = allAbortedTransactions(log) + val abortedTransactions = LogTestUtils.allAbortedTransactions(log) val expectedTransactions = List( new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L) @@ -4251,162 +3095,6 @@ class LogTest { assertEquals(None, log.firstUnstableOffset) } - @Test - def testFullTransactionIndexRecovery(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) - val log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val pid1 = 1L - val pid2 = 2L - val pid3 = 3L - val pid4 = 4L - - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) - - // mix transactional and non-transactional data - appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 - appendPid2(2) // 10 - appendPid1(4) // 14 - appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 - appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 - appendPid2(6) // 36 - appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 - appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 - appendPid4(8) // 67 - appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 - appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 - - // delete all the offset and transaction index files to force recovery - log.logSegments.foreach { segment => - segment.offsetIndex.deleteIfExists() - segment.txnIndex.deleteIfExists() - } - - log.close() - - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) - val reloadedLog = createLog(logDir, reloadedLogConfig, lastShutdownClean = false) - val abortedTransactions = allAbortedTransactions(reloadedLog) - assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) - } - - @Test - def testRecoverOnlyLastSegment(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) - val log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val pid1 = 1L - val pid2 = 2L - val pid3 = 3L - val pid4 = 4L - - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) - - // mix transactional and non-transactional data - appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 - appendPid2(2) // 10 - appendPid1(4) // 14 - appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 - appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 - appendPid2(6) // 36 - appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 - appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 - appendPid4(8) // 67 - appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 - appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 - - // delete the last offset and transaction index files to force recovery - val lastSegment = log.logSegments.last - val recoveryPoint = lastSegment.baseOffset - lastSegment.offsetIndex.deleteIfExists() - lastSegment.txnIndex.deleteIfExists() - - log.close() - - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) - val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) - val abortedTransactions = allAbortedTransactions(reloadedLog) - assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) - } - - @Test - def testRecoverLastSegmentWithNoSnapshots(): Unit = { - val logConfig = LogTest.createLogConfig(segmentBytes = 128 * 5) - val log = createLog(logDir, logConfig) - val epoch = 0.toShort - - val pid1 = 1L - val pid2 = 2L - val pid3 = 3L - val pid4 = 4L - - val appendPid1 = appendTransactionalAsLeader(log, pid1, epoch) - val appendPid2 = appendTransactionalAsLeader(log, pid2, epoch) - val appendPid3 = appendTransactionalAsLeader(log, pid3, epoch) - val appendPid4 = appendTransactionalAsLeader(log, pid4, epoch) - - // mix transactional and non-transactional data - appendPid1(5) // nextOffset: 5 - appendNonTransactionalAsLeader(log, 3) // 8 - appendPid2(2) // 10 - appendPid1(4) // 14 - appendPid3(3) // 17 - appendNonTransactionalAsLeader(log, 2) // 19 - appendPid1(10) // 29 - appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) // 30 - appendPid2(6) // 36 - appendPid4(3) // 39 - appendNonTransactionalAsLeader(log, 10) // 49 - appendPid3(9) // 58 - appendEndTxnMarkerAsLeader(log, pid3, epoch, ControlRecordType.COMMIT) // 59 - appendPid4(8) // 67 - appendPid2(7) // 74 - appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.ABORT) // 75 - appendNonTransactionalAsLeader(log, 10) // 85 - appendPid4(4) // 89 - appendEndTxnMarkerAsLeader(log, pid4, epoch, ControlRecordType.COMMIT) // 90 - - deleteProducerSnapshotFiles() - - // delete the last offset and transaction index files to force recovery. this should force us to rebuild - // the producer state from the start of the log - val lastSegment = log.logSegments.last - val recoveryPoint = lastSegment.baseOffset - lastSegment.offsetIndex.deleteIfExists() - lastSegment.txnIndex.deleteIfExists() - - log.close() - - val reloadedLogConfig = LogTest.createLogConfig(segmentBytes = 1024 * 5) - val reloadedLog = createLog(logDir, reloadedLogConfig, recoveryPoint = recoveryPoint, lastShutdownClean = false) - val abortedTransactions = allAbortedTransactions(reloadedLog) - assertEquals(List(new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L)), abortedTransactions) - } - @Test def testTransactionIndexUpdatedThroughReplication(): Unit = { val epoch = 0.toShort @@ -4448,7 +3136,7 @@ class LogTest { appendAsFollower(log, MemoryRecords.readableRecords(buffer)) - val abortedTransactions = allAbortedTransactions(log) + val abortedTransactions = LogTestUtils.allAbortedTransactions(log) val expectedTransactions = List( new AbortedTxn(pid1, 0L, 29L, 8L), new AbortedTxn(pid2, 8L, 74L, 36L) @@ -4504,16 +3192,17 @@ class LogTest { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) - val append = appendTransactionalAsLeader(log, pid, epoch) + val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) append(10) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1) append(5) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, coordinatorEpoch = 2) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.COMMIT, mockTime.milliseconds(), coordinatorEpoch = 2) - assertThrows(classOf[TransactionCoordinatorFencedException], () => appendEndTxnMarkerAsLeader(log, pid, epoch, - ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows( + classOf[TransactionCoordinatorFencedException], + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) } @Test @@ -4526,16 +3215,15 @@ class LogTest { val buffer = ByteBuffer.allocate(256) val append = appendTransactionalToBuffer(buffer, pid, epoch, leaderEpoch = 1) append(0, 10) - appendEndTxnMarkerToBuffer(buffer, pid, epoch, 10L, ControlRecordType.COMMIT, - coordinatorEpoch = 0, leaderEpoch = 1) + appendEndTxnMarkerToBuffer(buffer, pid, epoch, 10L, ControlRecordType.COMMIT, leaderEpoch = 1) buffer.flip() log.appendAsFollower(MemoryRecords.readableRecords(buffer)) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 2, leaderEpoch = 1) - appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 2, leaderEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 2, leaderEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 2, leaderEpoch = 1) assertThrows(classOf[TransactionCoordinatorFencedException], - () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1, leaderEpoch = 1)) + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1, leaderEpoch = 1)) } @Test @@ -4544,10 +3232,10 @@ class LogTest { val epoch = 5.toShort val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) - appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1) + LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1) assertThrows(classOf[InvalidProducerEpochException], - () => appendEndTxnMarkerAsLeader(log, producerId, (epoch - 1).toShort, ControlRecordType.ABORT, coordinatorEpoch = 1)) + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, producerId, (epoch - 1).toShort, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) } @Test @@ -4556,10 +3244,10 @@ class LogTest { val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L - val appendPid = appendTransactionalAsLeader(log, pid, epoch) + val appendPid = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) appendPid(5) - appendNonTransactionalAsLeader(log, 3) + LogTestUtils.appendNonTransactionalAsLeader(log, 3) assertEquals(8L, log.logEndOffset) log.roll() @@ -4581,10 +3269,10 @@ class LogTest { val log = createLog(logDir, logConfig) val epoch = 0.toShort val pid = 1L - val appendPid = appendTransactionalAsLeader(log, pid, epoch) + val appendPid = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) appendPid(5) - appendNonTransactionalAsLeader(log, 3) + LogTestUtils.appendNonTransactionalAsLeader(log, 3) assertEquals(8L, log.logEndOffset) log.roll() @@ -4610,7 +3298,7 @@ class LogTest { val logConfig = LogTest.createLogConfig(segmentBytes = 1024 * 1024 * 5) val log = createLog(logDir, logConfig) - val append = appendTransactionalAsLeader(log, pid, epoch) + val append = LogTestUtils.appendTransactionalAsLeader(log, pid, epoch, mockTime) append(10) // Kind of a hack, but renaming the index to a directory ensures that the append @@ -4618,15 +3306,18 @@ class LogTest { log.activeSegment.txnIndex.renameTo(log.dir) // The append will be written to the log successfully, but the write to the index will fail - assertThrows(classOf[KafkaStorageException], - () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows( + classOf[KafkaStorageException], + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) assertEquals(11L, log.logEndOffset) assertEquals(0L, log.lastStableOffset) // Try the append a second time. The appended offset in the log should not increase // because the log dir is marked as failed. Nor will there be a write to the transaction // index. - assertThrows(classOf[KafkaStorageException], () => appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, coordinatorEpoch = 1)) + assertThrows( + classOf[KafkaStorageException], + () => LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds(), coordinatorEpoch = 1)) assertEquals(11L, log.logEndOffset) assertEquals(0L, log.lastStableOffset) @@ -4700,14 +3391,14 @@ class LogTest { assertEquals(firstAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) // now first producer's transaction is aborted - val abortAppendInfo = appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT) + val abortAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid1, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) log.updateHighWatermark(abortAppendInfo.lastOffset + 1) // LSO should now point to one less than the first offset of the second transaction assertEquals(secondAppendInfo.firstOffset.map(_.messageOffset), log.firstUnstableOffset) // commit the second transaction - val commitAppendInfo = appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.COMMIT) + val commitAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid2, epoch, ControlRecordType.COMMIT, mockTime.milliseconds()) log.updateHighWatermark(commitAppendInfo.lastOffset + 1) // now there should be no first unstable offset @@ -4741,7 +3432,7 @@ class LogTest { assertEquals(3L, log.logEndOffsetMetadata.segmentBaseOffset) // now abort the transaction - val abortAppendInfo = appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT) + val abortAppendInfo = LogTestUtils.appendEndTxnMarkerAsLeader(log, pid, epoch, ControlRecordType.ABORT, mockTime.milliseconds()) log.updateHighWatermark(abortAppendInfo.lastOffset + 1) assertEquals(None, log.firstUnstableOffset) @@ -4767,61 +3458,6 @@ class LogTest { assertEquals(1, log.numberOfSegments) } - private def allAbortedTransactions(log: Log) = log.logSegments.flatMap(_.txnIndex.allAbortedTxns) - - private def appendTransactionalAsLeader( - log: Log, - producerId: Long, - producerEpoch: Short - ): Int => Unit = { - appendIdempotentAsLeader(log, producerId, producerEpoch, isTransactional = true) - } - - private def appendIdempotentAsLeader( - log: Log, - producerId: Long, - producerEpoch: Short, - isTransactional: Boolean = false - ): Int => Unit = { - var sequence = 0 - numRecords: Int => { - val simpleRecords = (sequence until sequence + numRecords).map { seq => - new SimpleRecord(mockTime.milliseconds(), s"$seq".getBytes) - } - - val records = if (isTransactional) { - MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, - producerEpoch, sequence, simpleRecords: _*) - } else { - MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, - producerEpoch, sequence, simpleRecords: _*) - } - - log.appendAsLeader(records, leaderEpoch = 0) - sequence += numRecords - } - } - - private def appendEndTxnMarkerAsLeader(log: Log, - producerId: Long, - producerEpoch: Short, - controlType: ControlRecordType, - coordinatorEpoch: Int = 0, - leaderEpoch: Int = 0, - timestamp: Long = mockTime.milliseconds()): LogAppendInfo = { - val records = endTxnRecords(controlType, producerId, producerEpoch, - coordinatorEpoch = coordinatorEpoch, timestamp = timestamp) - log.appendAsLeader(records, origin = AppendOrigin.Coordinator, leaderEpoch = leaderEpoch) - } - - private def appendNonTransactionalAsLeader(log: Log, numRecords: Int): Unit = { - val simpleRecords = (0 until numRecords).map { seq => - new SimpleRecord(s"$seq".getBytes) - } - val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*) - log.appendAsLeader(records, leaderEpoch = 0) - } - private def appendTransactionalToBuffer(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, @@ -4864,14 +3500,6 @@ class LogTest { log.appendAsFollower(records) } - private def deleteProducerSnapshotFiles(): Unit = { - val files = logDir.listFiles.filter(f => f.isFile && f.getName.endsWith(Log.ProducerSnapshotFileSuffix)) - files.foreach(Utils.delete) - } - - private def listProducerSnapshotOffsets: Seq[Long] = - ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted - private def createLog(dir: File, config: LogConfig, brokerTopicStats: BrokerTopicStats = brokerTopicStats, @@ -4897,20 +3525,6 @@ class LogTest { (log, segmentWithOverflow) } - - private def recoverAndCheck(config: LogConfig, expectedKeys: Iterable[Long]) = { - // method is called only in case of recovery from hard reset - LogTest.recoverAndCheck(logDir, config, expectedKeys, brokerTopicStats, mockTime, mockTime.scheduler) - } - - private def readLog(log: Log, - startOffset: Long, - maxLength: Int, - isolation: FetchIsolation = FetchLogEnd, - minOneMessage: Boolean = true): FetchDataInfo = { - log.read(startOffset, maxLength, isolation, minOneMessage) - } - } object LogTest { @@ -5055,19 +3669,4 @@ object LogTest { record <- batch.asScala if record.hasValue && record.hasKey) yield TestUtils.readString(record.key).toLong } - - def recoverAndCheck(logDir: File, config: LogConfig, expectedKeys: Iterable[Long], brokerTopicStats: BrokerTopicStats, time: Time, scheduler: Scheduler): Log = { - // Recover log file and check that after recovery, keys are as expected - // and all temporary files have been deleted - val recoveredLog = createLog(logDir, config, brokerTopicStats, scheduler, time, lastShutdownClean = false) - time.sleep(config.fileDeleteDelayMs + 1) - for (file <- logDir.listFiles) { - assertFalse(file.getName.endsWith(Log.DeletedFileSuffix), "Unexpected .deleted file after recovery") - assertFalse(file.getName.endsWith(Log.CleanedFileSuffix), "Unexpected .cleaned file after recovery") - assertFalse(file.getName.endsWith(Log.SwapFileSuffix), "Unexpected .swap file after recovery") - } - assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) - assertFalse(LogTest.hasOffsetOverflow(recoveredLog)) - recoveredLog - } } diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index b52d8d84c5b25..29777b2dec900 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -18,9 +18,18 @@ package kafka.log import java.io.File +import java.util.Properties -import org.apache.kafka.common.record.FileRecords -import org.apache.kafka.common.utils.Time +import kafka.server.checkpoints.LeaderEpochCheckpointFile +import kafka.server.{BrokerTopicStats, FetchDataInfo, FetchIsolation, FetchLogEnd, LogDirFailureChannel} +import kafka.utils.{Scheduler, TestUtils} +import org.apache.kafka.common.Uuid +import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} +import org.apache.kafka.common.utils.{Time, Utils} +import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} + +import scala.collection.Iterable +import scala.jdk.CollectionConverters._ object LogTestUtils { /** @@ -37,4 +46,233 @@ object LogTestUtils { new LogSegment(ms, idx, timeIdx, txnIndex, offset, indexIntervalBytes, 0, time) } + + def createLogConfig(segmentMs: Long = Defaults.SegmentMs, + segmentBytes: Int = Defaults.SegmentSize, + retentionMs: Long = Defaults.RetentionMs, + retentionBytes: Long = Defaults.RetentionSize, + segmentJitterMs: Long = Defaults.SegmentJitterMs, + cleanupPolicy: String = Defaults.CleanupPolicy, + maxMessageBytes: Int = Defaults.MaxMessageSize, + indexIntervalBytes: Int = Defaults.IndexInterval, + segmentIndexBytes: Int = Defaults.MaxIndexSize, + messageFormatVersion: String = Defaults.MessageFormatVersion, + fileDeleteDelayMs: Long = Defaults.FileDeleteDelayMs): LogConfig = { + val logProps = new Properties() + + logProps.put(LogConfig.SegmentMsProp, segmentMs: java.lang.Long) + logProps.put(LogConfig.SegmentBytesProp, segmentBytes: Integer) + logProps.put(LogConfig.RetentionMsProp, retentionMs: java.lang.Long) + logProps.put(LogConfig.RetentionBytesProp, retentionBytes: java.lang.Long) + logProps.put(LogConfig.SegmentJitterMsProp, segmentJitterMs: java.lang.Long) + logProps.put(LogConfig.CleanupPolicyProp, cleanupPolicy) + logProps.put(LogConfig.MaxMessageBytesProp, maxMessageBytes: Integer) + logProps.put(LogConfig.IndexIntervalBytesProp, indexIntervalBytes: Integer) + logProps.put(LogConfig.SegmentIndexBytesProp, segmentIndexBytes: Integer) + logProps.put(LogConfig.MessageFormatVersionProp, messageFormatVersion) + logProps.put(LogConfig.FileDeleteDelayMsProp, fileDeleteDelayMs: java.lang.Long) + LogConfig(logProps) + } + + def createLog(dir: File, + config: LogConfig, + brokerTopicStats: BrokerTopicStats, + scheduler: Scheduler, + time: Time, + logStartOffset: Long = 0L, + recoveryPoint: Long = 0L, + maxProducerIdExpirationMs: Int = 60 * 60 * 1000, + producerIdExpirationCheckIntervalMs: Int = LogManager.ProducerIdExpirationCheckIntervalMs, + lastShutdownClean: Boolean = true, + topicId: Option[Uuid] = None): Log = { + Log(dir = dir, + config = config, + logStartOffset = logStartOffset, + recoveryPoint = recoveryPoint, + scheduler = scheduler, + brokerTopicStats = brokerTopicStats, + time = time, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, + producerIdExpirationCheckIntervalMs = producerIdExpirationCheckIntervalMs, + logDirFailureChannel = new LogDirFailureChannel(10), + lastShutdownClean = lastShutdownClean, + topicId = topicId, + keepPartitionMetadataFile = true) + } + + /** + * Check if the given log contains any segment with records that cause offset overflow. + * @param log Log to check + * @return true if log contains at least one segment with offset overflow; false otherwise + */ + def hasOffsetOverflow(log: Log): Boolean = firstOverflowSegment(log).isDefined + + def firstOverflowSegment(log: Log): Option[LogSegment] = { + def hasOverflow(baseOffset: Long, batch: RecordBatch): Boolean = + batch.lastOffset > baseOffset + Int.MaxValue || batch.baseOffset < baseOffset + + for (segment <- log.logSegments) { + val overflowBatch = segment.log.batches.asScala.find(batch => hasOverflow(segment.baseOffset, batch)) + if (overflowBatch.isDefined) + return Some(segment) + } + None + } + + private def rawSegment(logDir: File, baseOffset: Long): FileRecords = + FileRecords.open(Log.logFile(logDir, baseOffset)) + + /** + * Initialize the given log directory with a set of segments, one of which will have an + * offset which overflows the segment + */ + def initializeLogDirWithOverflowedSegment(logDir: File): Unit = { + def writeSampleBatches(baseOffset: Long, segment: FileRecords): Long = { + def record(offset: Long) = { + val data = offset.toString.getBytes + new SimpleRecord(data, data) + } + + segment.append(MemoryRecords.withRecords(baseOffset, CompressionType.NONE, 0, + record(baseOffset))) + segment.append(MemoryRecords.withRecords(baseOffset + 1, CompressionType.NONE, 0, + record(baseOffset + 1), + record(baseOffset + 2))) + segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, CompressionType.NONE, 0, + record(baseOffset + Int.MaxValue - 1))) + // Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment. + Log.offsetIndexFile(logDir, baseOffset).createNewFile() + Log.timeIndexFile(logDir, baseOffset).createNewFile() + baseOffset + Int.MaxValue + } + + def writeNormalSegment(baseOffset: Long): Long = { + val segment = rawSegment(logDir, baseOffset) + try writeSampleBatches(baseOffset, segment) + finally segment.close() + } + + def writeOverflowSegment(baseOffset: Long): Long = { + val segment = rawSegment(logDir, baseOffset) + try { + val nextOffset = writeSampleBatches(baseOffset, segment) + writeSampleBatches(nextOffset, segment) + } finally segment.close() + } + + // We create three segments, the second of which contains offsets which overflow + var nextOffset = 0L + nextOffset = writeNormalSegment(nextOffset) + nextOffset = writeOverflowSegment(nextOffset) + writeNormalSegment(nextOffset) + } + + /* extract all the keys from a log */ + def keysInLog(log: Log): Iterable[Long] = { + for (logSegment <- log.logSegments; + batch <- logSegment.log.batches.asScala if !batch.isControlBatch; + record <- batch.asScala if record.hasValue && record.hasKey) + yield TestUtils.readString(record.key).toLong + } + + def recoverAndCheck(logDir: File, config: LogConfig, expectedKeys: Iterable[Long], brokerTopicStats: BrokerTopicStats, time: Time, scheduler: Scheduler): Log = { + // Recover log file and check that after recovery, keys are as expected + // and all temporary files have been deleted + val recoveredLog = createLog(logDir, config, brokerTopicStats, scheduler, time, lastShutdownClean = false) + time.sleep(config.fileDeleteDelayMs + 1) + for (file <- logDir.listFiles) { + assertFalse(file.getName.endsWith(Log.DeletedFileSuffix), "Unexpected .deleted file after recovery") + assertFalse(file.getName.endsWith(Log.CleanedFileSuffix), "Unexpected .cleaned file after recovery") + assertFalse(file.getName.endsWith(Log.SwapFileSuffix), "Unexpected .swap file after recovery") + } + assertEquals(expectedKeys, LogTest.keysInLog(recoveredLog)) + assertFalse(LogTest.hasOffsetOverflow(recoveredLog)) + recoveredLog + } + + def appendEndTxnMarkerAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + controlType: ControlRecordType, + timestamp: Long, + coordinatorEpoch: Int = 0, + leaderEpoch: Int = 0): LogAppendInfo = { + val records = endTxnRecords(controlType, producerId, producerEpoch, + coordinatorEpoch = coordinatorEpoch, timestamp = timestamp) + log.appendAsLeader(records, origin = AppendOrigin.Coordinator, leaderEpoch = leaderEpoch) + } + + private def endTxnRecords(controlRecordType: ControlRecordType, + producerId: Long, + epoch: Short, + offset: Long = 0L, + coordinatorEpoch: Int, + partitionLeaderEpoch: Int = 0, + timestamp: Long): MemoryRecords = { + val marker = new EndTransactionMarker(controlRecordType, coordinatorEpoch) + MemoryRecords.withEndTransactionMarker(offset, timestamp, partitionLeaderEpoch, producerId, epoch, marker) + } + + def readLog(log: Log, + startOffset: Long, + maxLength: Int, + isolation: FetchIsolation = FetchLogEnd, + minOneMessage: Boolean = true): FetchDataInfo = { + log.read(startOffset, maxLength, isolation, minOneMessage) + } + + def allAbortedTransactions(log: Log): Iterable[AbortedTxn] = log.logSegments.flatMap(_.txnIndex.allAbortedTxns) + + def deleteProducerSnapshotFiles(logDir: File): Unit = { + val files = logDir.listFiles.filter(f => f.isFile && f.getName.endsWith(Log.ProducerSnapshotFileSuffix)) + files.foreach(Utils.delete) + } + + def listProducerSnapshotOffsets(logDir: File): Seq[Long] = + ProducerStateManager.listSnapshotFiles(logDir).map(_.offset).sorted + + def assertLeaderEpochCacheEmpty(log: Log): Unit = { + assertEquals(None, log.leaderEpochCache) + assertEquals(None, log.latestEpoch) + assertFalse(LeaderEpochCheckpointFile.newFile(log.dir).exists()) + } + + def appendNonTransactionalAsLeader(log: Log, numRecords: Int): Unit = { + val simpleRecords = (0 until numRecords).map { seq => + new SimpleRecord(s"$seq".getBytes) + } + val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*) + log.appendAsLeader(records, leaderEpoch = 0) + } + + def appendTransactionalAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + time: Time): Int => Unit = { + appendIdempotentAsLeader(log, producerId, producerEpoch, time, isTransactional = true) + } + + def appendIdempotentAsLeader(log: Log, + producerId: Long, + producerEpoch: Short, + time: Time, + isTransactional: Boolean = false): Int => Unit = { + var sequence = 0 + numRecords: Int => { + val simpleRecords = (sequence until sequence + numRecords).map { seq => + new SimpleRecord(time.milliseconds(), s"$seq".getBytes) + } + + val records = if (isTransactional) { + MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, + producerEpoch, sequence, simpleRecords: _*) + } else { + MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, + producerEpoch, sequence, simpleRecords: _*) + } + + log.appendAsLeader(records, leaderEpoch = 0) + sequence += numRecords + } + } } diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 4bbe7012c46cc..deb8827bc7dc8 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -47,12 +47,13 @@ import org.easymock.EasyMock import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.Mockito - import java.io.File import java.net.InetAddress +import java.nio.file.Files import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.{Collections, Optional, Properties} + import scala.collection.{Map, Seq, mutable} import scala.jdk.CollectionConverters._ @@ -1476,23 +1477,46 @@ class ReplicaManagerTest { props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) props.asScala ++= extraProps.asScala val config = KafkaConfig.fromProps(props) - + val logConfig = LogConfig() + val logDir = new File(new File(config.logDirs.head), s"$topic-$topicPartition") + Files.createDirectories(logDir.toPath) val mockScheduler = new MockScheduler(time) val mockBrokerTopicStats = new BrokerTopicStats val mockLogDirFailureChannel = new LogDirFailureChannel(config.logDirs.size) + val tp = new TopicPartition(topic, topicPartition) + val maxProducerIdExpirationMs = 30000 + val segments = new LogSegments(tp) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, tp, mockLogDirFailureChannel, logConfig.messageFormatVersion.recordVersion) + val producerStateManager = new ProducerStateManager(tp, logDir, maxProducerIdExpirationMs) + val offsets = LogLoader.load(LoadLogParams( + logDir, + tp, + logConfig, + mockScheduler, + time, + mockLogDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) val mockLog = new Log( - _dir = new File(new File(config.logDirs.head), s"$topic-0"), - config = LogConfig(), - logStartOffset = 0L, - recoveryPoint = 0L, + _dir = logDir, + config = logConfig, + segments = segments, + logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, + nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler = mockScheduler, brokerTopicStats = mockBrokerTopicStats, time = time, - maxProducerIdExpirationMs = 30000, + maxProducerIdExpirationMs = maxProducerIdExpirationMs, producerIdExpirationCheckIntervalMs = 30000, - topicPartition = new TopicPartition(topic, topicPartition), - producerStateManager = new ProducerStateManager(new TopicPartition(topic, topicPartition), - new File(new File(config.logDirs.head), s"$topic-$topicPartition"), 30000), + topicPartition = tp, + leaderEpochCache = leaderEpochCache, + producerStateManager = producerStateManager, logDirFailureChannel = mockLogDirFailureChannel, topicId = topicId, keepPartitionMetadataFile = true) { diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index 913d3e2449a4b..159ace8103e5a 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -19,7 +19,8 @@ package kafka.utils import java.util.Properties import java.util.concurrent.atomic._ import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} -import kafka.log.{Log, LogConfig, LogManager, ProducerStateManager} + +import kafka.log.{LoadLogParams, Log, LogConfig, LogLoader, LogManager, LogSegments, ProducerStateManager} import kafka.server.{BrokerTopicStats, LogDirFailureChannel} import kafka.utils.TestUtils.retry import org.junit.jupiter.api.Assertions._ @@ -118,13 +119,31 @@ class SchedulerTest { val logDir = TestUtils.randomPartitionLogDir(tmpDir) val logConfig = LogConfig(new Properties()) val brokerTopicStats = new BrokerTopicStats - val recoveryPoint = 0L val maxProducerIdExpirationMs = 60 * 60 * 1000 val topicPartition = Log.parseTopicPartitionName(logDir) + val logDirFailureChannel = new LogDirFailureChannel(10) + val segments = new LogSegments(topicPartition) + val leaderEpochCache = Log.maybeCreateLeaderEpochCache(logDir, topicPartition, logDirFailureChannel, logConfig.messageFormatVersion.recordVersion) val producerStateManager = new ProducerStateManager(topicPartition, logDir, maxProducerIdExpirationMs) - val log = new Log(logDir, logConfig, logStartOffset = 0, recoveryPoint = recoveryPoint, scheduler, + val offsets = LogLoader.load(LoadLogParams( + logDir, + topicPartition, + logConfig, + scheduler, + mockTime, + logDirFailureChannel, + hadCleanShutdown = true, + segments, + 0L, + 0L, + maxProducerIdExpirationMs, + leaderEpochCache, + producerStateManager)) + val log = new Log(logDir, logConfig, segments = segments, logStartOffset = offsets.logStartOffset, + recoveryPoint = offsets.recoveryPoint, nextOffsetMetadata = offsets.nextOffsetMetadata, scheduler, brokerTopicStats, mockTime, maxProducerIdExpirationMs, LogManager.ProducerIdExpirationCheckIntervalMs, - topicPartition, producerStateManager, new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true) + topicPartition, leaderEpochCache, producerStateManager, logDirFailureChannel, + topicId = None, keepPartitionMetadataFile = true) assertTrue(scheduler.taskRunning(log.producerExpireCheck)) log.close() assertFalse(scheduler.taskRunning(log.producerExpireCheck)) From a24e78b34a17c9da2c9a5236c76febacddc52aef Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Tue, 20 Apr 2021 19:14:26 -0700 Subject: [PATCH 076/155] HOTFIX: remove unimplemented SPILL_TO_DISK buffer full strategy (#10571) Remove enum for the spill-to-disk strategy since this hasn't been implemented. Reviewers: Walker Carlson --- .../streams/kstream/internals/suppress/BufferFullStrategy.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/BufferFullStrategy.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/BufferFullStrategy.java index 2da7c1418253f..870a3d18f7ad7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/BufferFullStrategy.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/BufferFullStrategy.java @@ -18,6 +18,5 @@ public enum BufferFullStrategy { EMIT, - SPILL_TO_DISK, SHUT_DOWN } From d3538ed0ab7a97cf596a366accd0a81518841e5d Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Wed, 21 Apr 2021 11:40:01 +0900 Subject: [PATCH 077/155] MINOR: Remove unthrown exceptions, fix typo, etc. (#10402) Reviewers: A. Sophie Blee-Goldman , Ryanne Dolan , Chia-Ping Tsai --- .../kafka/connect/mirror/SourceAndTarget.java | 2 +- .../kafka/connect/mirror/MirrorClientTest.java | 13 ++++++------- .../kafka/connect/mirror/MirrorHeartbeatTask.java | 2 +- .../kafka/connect/mirror/MirrorSourceConnector.java | 6 ++---- 4 files changed, 10 insertions(+), 13 deletions(-) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java index f853dc40c16ac..e2f3a3756e4af 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/SourceAndTarget.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.connect.mirror; -/** Directional pair of clustes, where source is replicated to target. */ +/** Directional pair of clusters, where source is replicated to target. */ public class SourceAndTarget { private String source; private String target; diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index 058d257ec59c8..766b91ce69944 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -23,7 +23,6 @@ import java.util.Set; import java.util.HashSet; import java.util.Arrays; -import java.util.concurrent.TimeoutException; import org.junit.jupiter.api.Test; @@ -53,7 +52,7 @@ protected Set listTopics() { } @Test - public void testIsHeartbeatTopic() throws InterruptedException, TimeoutException { + public void testIsHeartbeatTopic() { MirrorClient client = new FakeMirrorClient(); assertTrue(client.isHeartbeatTopic("heartbeats")); assertTrue(client.isHeartbeatTopic("source1.heartbeats")); @@ -65,7 +64,7 @@ public void testIsHeartbeatTopic() throws InterruptedException, TimeoutException } @Test - public void testIsCheckpointTopic() throws InterruptedException, TimeoutException { + public void testIsCheckpointTopic() { MirrorClient client = new FakeMirrorClient(); assertTrue(client.isCheckpointTopic("source1.checkpoints.internal")); assertFalse(client.isCheckpointTopic("checkpoints.internal")); @@ -76,7 +75,7 @@ public void testIsCheckpointTopic() throws InterruptedException, TimeoutExceptio } @Test - public void countHopsForTopicTest() throws InterruptedException, TimeoutException { + public void countHopsForTopicTest() { MirrorClient client = new FakeMirrorClient(); assertEquals(-1, client.countHopsForTopic("topic", "source")); assertEquals(-1, client.countHopsForTopic("source", "source")); @@ -90,7 +89,7 @@ public void countHopsForTopicTest() throws InterruptedException, TimeoutExceptio } @Test - public void heartbeatTopicsTest() throws InterruptedException, TimeoutException { + public void heartbeatTopicsTest() throws InterruptedException { MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "heartbeats", "source1.heartbeats", "source2.source1.heartbeats", "source3.heartbeats")); Set heartbeatTopics = client.heartbeatTopics(); @@ -99,7 +98,7 @@ public void heartbeatTopicsTest() throws InterruptedException, TimeoutException } @Test - public void checkpointsTopicsTest() throws InterruptedException, TimeoutException { + public void checkpointsTopicsTest() throws InterruptedException { MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "checkpoints.internal", "source1.checkpoints.internal", "source2.source1.checkpoints.internal", "source3.checkpoints.internal")); Set checkpointTopics = client.checkpointTopics(); @@ -108,7 +107,7 @@ public void checkpointsTopicsTest() throws InterruptedException, TimeoutExceptio } @Test - public void replicationHopsTest() throws InterruptedException, TimeoutException { + public void replicationHopsTest() throws InterruptedException { MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "heartbeats", "source1.heartbeats", "source1.source2.heartbeats", "source3.heartbeats")); assertEquals(1, client.replicationHops("source1")); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java index 30c497dfe1107..9f38b5999ea27 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorHeartbeatTask.java @@ -47,7 +47,7 @@ public void start(Map props) { } @Override - public void commit() throws InterruptedException { + public void commit() { // nop } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index 7b844c84d2f56..91a40a638cbf6 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -409,8 +409,7 @@ static Map configToMap(Config config) { @SuppressWarnings("deprecation") // use deprecated alterConfigs API for broker compatibility back to 0.11.0 - private void updateTopicConfigs(Map topicConfigs) - throws InterruptedException, ExecutionException { + private void updateTopicConfigs(Map topicConfigs) { Map configs = topicConfigs.entrySet().stream() .collect(Collectors.toMap(x -> new ConfigResource(ConfigResource.Type.TOPIC, x.getKey()), Entry::getValue)); @@ -422,8 +421,7 @@ private void updateTopicConfigs(Map topicConfigs) })); } - private void updateTopicAcls(List bindings) - throws InterruptedException, ExecutionException { + private void updateTopicAcls(List bindings) { log.trace("Syncing {} topic ACL bindings.", bindings.size()); targetAdminClient.createAcls(bindings).values().forEach((k, v) -> v.whenComplete((x, e) -> { if (e != null) { From a46beb9d29781e0709baf596601122f770a5fa31 Mon Sep 17 00:00:00 2001 From: Alec Thomas Date: Wed, 21 Apr 2021 19:35:17 +1000 Subject: [PATCH 078/155] MINOR: Ensure `ignorable` is a boolean value (#10567) Reviewers: David Jacot , Tom Bentley , Chia-Ping Tsai --- .../src/main/resources/common/message/DeleteTopicsResponse.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/resources/common/message/DeleteTopicsResponse.json b/clients/src/main/resources/common/message/DeleteTopicsResponse.json index cb531bb13f761..19a81630463d5 100644 --- a/clients/src/main/resources/common/message/DeleteTopicsResponse.json +++ b/clients/src/main/resources/common/message/DeleteTopicsResponse.json @@ -40,7 +40,7 @@ "about": "The results for each topic we tried to delete.", "fields": [ { "name": "Name", "type": "string", "versions": "0+", "nullableVersions": "6+", "mapKey": true, "entityType": "topicName", "about": "The topic name" }, - {"name": "TopicId", "type": "uuid", "versions": "6+", "ignorable": "true", "about": "the unique topic ID"}, + {"name": "TopicId", "type": "uuid", "versions": "6+", "ignorable": true, "about": "the unique topic ID"}, { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The deletion error, or 0 if the deletion succeeded." }, { "name": "ErrorMessage", "type": "string", "versions": "5+", "nullableVersions": "5+", "ignorable": true, "default": "null", From 87b24025cedf08c550a180819b2a5a2dbb75f020 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 21 Apr 2021 10:25:54 -0700 Subject: [PATCH 079/155] HOTFIX: typo in Streams DSL docs --- docs/streams/developer-guide/dsl-api.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streams/developer-guide/dsl-api.html b/docs/streams/developer-guide/dsl-api.html index 84fe4e7ffe7b6..2add551d61137 100644 --- a/docs/streams/developer-guide/dsl-api.html +++ b/docs/streams/developer-guide/dsl-api.html @@ -1074,7 +1074,7 @@

    aggValue = 0), “adder” aggregator (e.g., aggValue + curValue), and a window. When windowing based on sessions, you must additionally provide a “session merger” aggregator (e.g., mergedAggValue = leftAggValue + rightAggValue).

    -

    The windowed aggregate turns a TimeWindowedKStream<K, V> or SessionWindowdKStream<K, V> +

    The windowed aggregate turns a TimeWindowedKStream<K, V> or SessionWindowedKStream<K, V> into a windowed KTable<Windowed<K>, V>.

    Several variants of aggregate exist, see Javadocs for details.

    import java.time.Duration;
    
    From 3bfc9fe48683ecc43139f98c7f270280dd7a3e57 Mon Sep 17 00:00:00 2001
    From: "A. Sophie Blee-Goldman" 
    Date: Wed, 21 Apr 2021 12:50:30 -0700
    Subject: [PATCH 080/155] MINOR: Bump latest 2.6 version to 2.6.2 (#10582)
    
    Bump the version for system tests to 2.6.2
    ---
     gradle/dependencies.gradle | 2 +-
     tests/docker/Dockerfile    | 4 ++--
     tests/kafkatest/version.py | 3 ++-
     vagrant/base.sh            | 4 ++--
     4 files changed, 7 insertions(+), 6 deletions(-)
    
    diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle
    index 91df77adcb2a7..3e54681c269c4 100644
    --- a/gradle/dependencies.gradle
    +++ b/gradle/dependencies.gradle
    @@ -94,7 +94,7 @@ versions += [
       kafka_23: "2.3.1",
       kafka_24: "2.4.1",
       kafka_25: "2.5.1",
    -  kafka_26: "2.6.1",
    +  kafka_26: "2.6.2",
       kafka_27: "2.7.0",
       lz4: "1.7.1",
       mavenArtifact: "3.6.3",
    diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile
    index a2183ae16aef7..57da852c0253d 100644
    --- a/tests/docker/Dockerfile
    +++ b/tests/docker/Dockerfile
    @@ -60,7 +60,7 @@ RUN mkdir -p "/opt/kafka-2.2.2" && chmod a+rw /opt/kafka-2.2.2 && curl -s "$KAFK
     RUN mkdir -p "/opt/kafka-2.3.1" && chmod a+rw /opt/kafka-2.3.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.3.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.3.1"
     RUN mkdir -p "/opt/kafka-2.4.1" && chmod a+rw /opt/kafka-2.4.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.4.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.4.1"
     RUN mkdir -p "/opt/kafka-2.5.1" && chmod a+rw /opt/kafka-2.5.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.5.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.5.1"
    -RUN mkdir -p "/opt/kafka-2.6.1" && chmod a+rw /opt/kafka-2.6.1 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.6.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.6.1"
    +RUN mkdir -p "/opt/kafka-2.6.2" && chmod a+rw /opt/kafka-2.6.2 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.6.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.6.2"
     RUN mkdir -p "/opt/kafka-2.7.0" && chmod a+rw /opt/kafka-2.7.0 && curl -s "$KAFKA_MIRROR/kafka_2.12-2.7.0.tgz" | tar xz --strip-components=1 -C "/opt/kafka-2.7.0"
     
     # Streams test dependencies
    @@ -76,7 +76,7 @@ RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.2.2-test.jar" -o /opt/kafka-2.2.2/lib
     RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.3.1-test.jar" -o /opt/kafka-2.3.1/libs/kafka-streams-2.3.1-test.jar
     RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.4.1-test.jar" -o /opt/kafka-2.4.1/libs/kafka-streams-2.4.1-test.jar
     RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.5.1-test.jar" -o /opt/kafka-2.5.1/libs/kafka-streams-2.5.1-test.jar
    -RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.6.1-test.jar" -o /opt/kafka-2.6.1/libs/kafka-streams-2.6.1-test.jar
    +RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.6.2-test.jar" -o /opt/kafka-2.6.2/libs/kafka-streams-2.6.2-test.jar
     RUN curl -s "$KAFKA_MIRROR/kafka-streams-2.7.0-test.jar" -o /opt/kafka-2.7.0/libs/kafka-streams-2.7.0-test.jar
     
     # The version of Kibosh to use for testing.
    diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py
    index 3b062a15cecb5..a2bbd50cbaf99 100644
    --- a/tests/kafkatest/version.py
    +++ b/tests/kafkatest/version.py
    @@ -199,7 +199,8 @@ def get_version(node=None):
     # 2.6.x versions
     V_2_6_0 = KafkaVersion("2.6.0")
     V_2_6_1 = KafkaVersion("2.6.1")
    -LATEST_2_6 = V_2_6_1
    +V_2_6_2 = KafkaVersion("2.6.2")
    +LATEST_2_6 = V_2_6_2
     
     # 2.7.x versions
     V_2_7_0 = KafkaVersion("2.7.0")
    diff --git a/vagrant/base.sh b/vagrant/base.sh
    index 037f9258b8475..9cfa62c2db868 100755
    --- a/vagrant/base.sh
    +++ b/vagrant/base.sh
    @@ -142,8 +142,8 @@ get_kafka 2.4.1 2.12
     chmod a+rw /opt/kafka-2.4.1
     get_kafka 2.5.1 2.12
     chmod a+rw /opt/kafka-2.5.1
    -get_kafka 2.6.1 2.12
    -chmod a+rw /opt/kafka-2.6.1
    +get_kafka 2.6.2 2.12
    +chmod a+rw /opt/kafka-2.6.2
     get_kafka 2.7.0 2.12
     chmod a+rw /opt/kafka-2.7.0
     
    
    From 72236f343d644569c8699b2337b0b99b2c8d2752 Mon Sep 17 00:00:00 2001
    From: Walker Carlson <18128741+wcarlson5@users.noreply.github.com>
    Date: Wed, 21 Apr 2021 19:43:18 -0700
    Subject: [PATCH 081/155] KAFKA-12691: Add case where task can be considered
     idling (#10565)
    
    Reviewers: Matthias J. Sax , A. Sophie Blee-Goldman 
    ---
     .../processor/internals/StreamTask.java       | 12 ++++++++++--
     .../processor/internals/StreamTaskTest.java   | 19 +++++++++++++++++++
     2 files changed, 29 insertions(+), 2 deletions(-)
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
    index e2c5e03cb15ef..d2f7926d46d9f 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
    @@ -198,6 +198,7 @@ public StreamTask(final TaskId id,
             stateMgr.registerGlobalStateStores(topology.globalStateStores());
             this.committedOffsets = new HashMap<>();
             this.highWatermark = new HashMap<>();
    +        this.timeCurrentIdlingStarted = Optional.empty();
         }
     
         // create queues for each assigned partition and associate them
    @@ -667,8 +668,15 @@ public boolean isProcessable(final long wallClockTime) {
                 // thus, the task is not processable, even if there is available data in the record queue
                 return false;
             }
    -
    -        return partitionGroup.readyToProcess(wallClockTime);
    +        final boolean readyToProcess = partitionGroup.readyToProcess(wallClockTime);
    +        if (!readyToProcess) {
    +            if (!timeCurrentIdlingStarted.isPresent()) {
    +                timeCurrentIdlingStarted = Optional.of(wallClockTime);
    +            }
    +        } else {
    +            timeCurrentIdlingStarted = Optional.empty();
    +        }
    +        return readyToProcess;
         }
     
         /**
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
    index fa80e46e31330..9bac9cc137383 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
    @@ -1220,6 +1220,8 @@ public void shouldBeProcessableIfAllPartitionsBuffered() {
             task.initializeIfNeeded();
             task.completeRestoration(noOpResetter -> { });
     
    +        assertThat("task is not idling", !task.timeCurrentIdlingStarted().isPresent());
    +
             assertFalse(task.process(0L));
     
             final byte[] bytes = ByteBuffer.allocate(4).putInt(1).array();
    @@ -1227,10 +1229,27 @@ public void shouldBeProcessableIfAllPartitionsBuffered() {
             task.addRecords(partition1, singleton(new ConsumerRecord<>(topic1, 1, 0, bytes, bytes)));
     
             assertFalse(task.process(0L));
    +        assertThat("task is idling", task.timeCurrentIdlingStarted().isPresent());
     
             task.addRecords(partition2, singleton(new ConsumerRecord<>(topic2, 1, 0, bytes, bytes)));
     
             assertTrue(task.process(0L));
    +        assertThat("task is not idling", !task.timeCurrentIdlingStarted().isPresent());
    +
    +    }
    +
    +    @Test
    +    public void shouldBeRecordIdlingTimeIfSuspended() {
    +        task = createStatelessTask(createConfig("100"), StreamsConfig.METRICS_LATEST);
    +        task.initializeIfNeeded();
    +        task.completeRestoration(noOpResetter -> { });
    +        task.suspend();
    +
    +        assertThat("task is idling", task.timeCurrentIdlingStarted().isPresent());
    +
    +        task.resume();
    +
    +        assertThat("task is not idling", !task.timeCurrentIdlingStarted().isPresent());
         }
     
         public void shouldPunctuateSystemTimeWhenIntervalElapsed() {
    
    From 6013ac3cea29ede41d0c9f1bf1f54069aeceb2ad Mon Sep 17 00:00:00 2001
    From: wenbingshen 
    Date: Thu, 22 Apr 2021 11:10:05 +0800
    Subject: [PATCH 082/155] KAFKA-12702: Fix NPE in networkListeners from
     BrokerServer (#10575)
    
    Reviewers: Chia-Ping Tsai 
    ---
     core/src/main/scala/kafka/server/BrokerServer.scala | 5 +++--
     1 file changed, 3 insertions(+), 2 deletions(-)
    
    diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
    index 59572bb90abc2..49edbfe295068 100644
    --- a/core/src/main/scala/kafka/server/BrokerServer.scala
    +++ b/core/src/main/scala/kafka/server/BrokerServer.scala
    @@ -21,6 +21,7 @@ import java.util
     import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException}
     import java.util.concurrent.atomic.AtomicBoolean
     import java.util.concurrent.locks.ReentrantLock
    +import java.net.InetAddress
     
     import kafka.cluster.Broker.ServerInfo
     import kafka.coordinator.group.GroupCoordinator
    @@ -39,7 +40,7 @@ import org.apache.kafka.common.network.ListenerName
     import org.apache.kafka.common.security.auth.SecurityProtocol
     import org.apache.kafka.common.security.scram.internals.ScramMechanism
     import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
    -import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time}
    +import org.apache.kafka.common.utils.{AppInfoParser, LogContext, Time, Utils}
     import org.apache.kafka.common.{ClusterResource, Endpoint, KafkaException}
     import org.apache.kafka.metadata.{BrokerState, VersionRange}
     import org.apache.kafka.metalog.MetaLogManager
    @@ -272,7 +273,7 @@ class BrokerServer(
           val networkListeners = new ListenerCollection()
           config.advertisedListeners.foreach { ep =>
             networkListeners.add(new Listener().
    -          setHost(ep.host).
    +          setHost(if (Utils.isBlank(ep.host)) InetAddress.getLocalHost.getCanonicalHostName else ep.host).
               setName(ep.listenerName.value()).
               setPort(socketServer.boundPort(ep.listenerName)).
               setSecurityProtocol(ep.securityProtocol.id))
    
    From 523d63df983815fed0b9d65978edd14dcab65e05 Mon Sep 17 00:00:00 2001
    From: Luke Chen <43372967+showuon@users.noreply.github.com>
    Date: Thu, 22 Apr 2021 11:41:09 +0800
    Subject: [PATCH 083/155] KAFKA-12700: override toString method to show correct
     value in doc (#10574)
    
    Reviewers: Chia-Ping Tsai 
    ---
     .../java/org/apache/kafka/connect/runtime/WorkerConfig.java  | 5 +++++
     1 file changed, 5 insertions(+)
    
    diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
    index c384376b75359..b5b72bea456b3 100644
    --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
    +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java
    @@ -523,6 +523,11 @@ public void ensureValid(String name, Object value) {
                     }
                 }
             }
    +
    +        @Override
    +        public String toString() {
    +            return "List of comma-separated URIs, ex: http://localhost:8080,https://localhost:8443.";
    +        }
         }
     
         private static class ResponseHttpHeadersValidator implements ConfigDef.Validator {
    
    From 4de5f1267164d59c741632c870f2a7aa5153e3a7 Mon Sep 17 00:00:00 2001
    From: kpatelatwork 
    Date: Thu, 22 Apr 2021 00:12:04 -0500
    Subject: [PATCH 084/155] KAFKA-12704: Improve cache access during connector
     class instantiation in config validations (#10580)
    
    Concurrent requests to validate endpoint for the same connector type calls AbstractHerder::getConnector to get the cached connector instances and if the connector hasn't been cached yet then there is a race condition in the AbstractHerder::getConnector method that potentially fails to detect that an instance of the connector has already been created and, as a result, can create another instance
    
    Existing tests are present with enough coverage so no new tests are added.
    
    Reviewers: Chris Egerton , Konstantine Karantasis 
    ---
     .../apache/kafka/connect/runtime/AbstractHerder.java  | 11 +++--------
     1 file changed, 3 insertions(+), 8 deletions(-)
    
    diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
    index 97d9731b955ea..1a8d393913cf3 100644
    --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
    +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
    @@ -61,6 +61,7 @@
     import java.util.Map;
     import java.util.Set;
     import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ConcurrentMap;
     import java.util.concurrent.ExecutorService;
     import java.util.concurrent.Executors;
     import java.util.regex.Matcher;
    @@ -99,7 +100,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
         protected volatile boolean running = false;
         private final ExecutorService connectorExecutor;
     
    -    private Map tempConnectors = new ConcurrentHashMap<>();
    +    private ConcurrentMap tempConnectors = new ConcurrentHashMap<>();
     
         public AbstractHerder(Worker worker,
                               String workerId,
    @@ -591,13 +592,7 @@ private static ConfigValueInfo convertConfigValue(ConfigValue configValue, Type
         }
     
         protected Connector getConnector(String connType) {
    -        if (tempConnectors.containsKey(connType)) {
    -            return tempConnectors.get(connType);
    -        } else {
    -            Connector connector = plugins().newConnector(connType);
    -            tempConnectors.put(connType, connector);
    -            return connector;
    -        }
    +        return tempConnectors.computeIfAbsent(connType, k -> plugins().newConnector(k));
         }
     
         /*
    
    From 2a1b240829ed4c28708df50da281f6f4d5cfe59c Mon Sep 17 00:00:00 2001
    From: Jason Gustafson 
    Date: Thu, 22 Apr 2021 09:34:11 -0700
    Subject: [PATCH 085/155] KAFKA-12586; Add `DescribeTransactions` Admin API
     (#10483)
    
    This patch contains the `Admin` implementation of the `DescribeTransactions` APIs described in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.
    
    Reviewers: Chia-Ping Tsai , David Jacot 
    ---
     .../org/apache/kafka/clients/admin/Admin.java |  21 ++
     .../admin/DescribeTransactionsOptions.java    |  31 +++
     .../admin/DescribeTransactionsResult.java     |  81 +++++++
     .../kafka/clients/admin/KafkaAdminClient.java |  12 +-
     .../clients/admin/TransactionDescription.java | 113 +++++++++
     .../kafka/clients/admin/TransactionState.java |  56 +++++
     .../admin/internals/AdminApiHandler.java      |   4 +-
     .../internals/AdminApiLookupStrategy.java     |  16 ++
     .../admin/internals/CoordinatorKey.java       |  62 +++++
     .../admin/internals/CoordinatorStrategy.java  | 118 +++++++++
     .../internals/DescribeProducersHandler.java   |   4 +-
     .../DescribeTransactionsHandler.java          | 189 +++++++++++++++
     .../internals/PartitionLeaderStrategy.java    |   1 -
     .../kafka/common/ConsumerGroupState.java      |  15 +-
     .../clients/admin/KafkaAdminClientTest.java   | 122 ++++++++++
     .../kafka/clients/admin/MockAdminClient.java  |   5 +
     .../admin/internals/AdminApiDriverTest.java   |   4 +-
     .../internals/CoordinatorStrategyTest.java    | 140 +++++++++++
     .../DescribeTransactionsHandlerTest.java      | 229 ++++++++++++++++++
     .../transaction/TransactionCoordinator.scala  |  35 +--
     .../TransactionCoordinatorTest.scala          |  19 +-
     .../transaction/TransactionMetadataTest.scala |   6 +
     22 files changed, 1250 insertions(+), 33 deletions(-)
     create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java
     create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java
     create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java
     create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java
     create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java
     create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java
     create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java
     create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java
     create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java
    
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
    index e7704f52cff32..f420985aeb0be 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java
    @@ -1490,6 +1490,27 @@ default DescribeProducersResult describeProducers(Collection par
          */
         DescribeProducersResult describeProducers(Collection partitions, DescribeProducersOptions options);
     
    +    /**
    +     * Describe the state of a set of transactional IDs. See
    +     * {@link #describeTransactions(Collection, DescribeTransactionsOptions)} for more details.
    +     *
    +     * @param transactionalIds The set of transactional IDs to query
    +     * @return The result
    +     */
    +    default DescribeTransactionsResult describeTransactions(Collection transactionalIds) {
    +        return describeTransactions(transactionalIds, new DescribeTransactionsOptions());
    +    }
    +
    +    /**
    +     * Describe the state of a set of transactional IDs from the respective transaction coordinators,
    +     * which are dynamically discovered.
    +     *
    +     * @param transactionalIds The set of transactional IDs to query
    +     * @param options Options to control the method behavior
    +     * @return The result
    +     */
    +    DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options);
    +
         /**
          * Get the metrics kept by the adminClient
          */
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java
    new file mode 100644
    index 0000000000000..3759b49f8abca
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsOptions.java
    @@ -0,0 +1,31 @@
    +/*
    + * 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.clients.admin;
    +
    +import org.apache.kafka.common.annotation.InterfaceStability;
    +
    +import java.util.Collection;
    +
    +/**
    + * Options for {@link Admin#describeTransactions(Collection)}.
    + *
    + * The API of this class is evolving, see {@link Admin} for details.
    + */
    +@InterfaceStability.Evolving
    +public class DescribeTransactionsOptions extends AbstractOptions {
    +
    +}
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java
    new file mode 100644
    index 0000000000000..edacc9f9dc44c
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/DescribeTransactionsResult.java
    @@ -0,0 +1,81 @@
    +/*
    + * 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.clients.admin;
    +
    +import org.apache.kafka.clients.admin.internals.CoordinatorKey;
    +import org.apache.kafka.common.KafkaFuture;
    +import org.apache.kafka.common.annotation.InterfaceStability;
    +import org.apache.kafka.common.internals.KafkaFutureImpl;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Map;
    +import java.util.concurrent.ExecutionException;
    +
    +@InterfaceStability.Evolving
    +public class DescribeTransactionsResult {
    +    private final Map> futures;
    +
    +    DescribeTransactionsResult(Map> futures) {
    +        this.futures = futures;
    +    }
    +
    +    /**
    +     * Get the description of a specific transactional ID.
    +     *
    +     * @param transactionalId the transactional ID to describe
    +     * @return a future which completes when the transaction description of a particular
    +     *         transactional ID is available.
    +     * @throws IllegalArgumentException if the `transactionalId` was not included in the
    +     *         respective call to {@link Admin#describeTransactions(Collection, DescribeTransactionsOptions)}.
    +     */
    +    public KafkaFuture description(String transactionalId) {
    +        CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
    +        KafkaFuture future = futures.get(key);
    +        if (future == null) {
    +            throw new IllegalArgumentException("TransactionalId " +
    +                "`" + transactionalId + "` was not included in the request");
    +        }
    +        return future;
    +    }
    +    /**
    +     * Get a future which returns a map of the transaction descriptions requested in the respective
    +     * call to {@link Admin#describeTransactions(Collection, DescribeTransactionsOptions)}.
    +     *
    +     * If the description fails on any of the transactional IDs in the request, then this future
    +     * will also fail.
    +     *
    +     * @return a future which either completes when all transaction descriptions complete or fails
    +     *         if any of the descriptions cannot be obtained
    +     */
    +    public KafkaFuture> all() {
    +        return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0]))
    +            .thenApply(nil -> {
    +                Map results = new HashMap<>(futures.size());
    +                for (Map.Entry> entry : futures.entrySet()) {
    +                    try {
    +                        results.put(entry.getKey().idValue, entry.getValue().get());
    +                    } catch (InterruptedException | ExecutionException e) {
    +                        // This should be unreachable, because allOf ensured that all the futures completed successfully.
    +                        throw new RuntimeException(e);
    +                    }
    +                }
    +                return results;
    +            });
    +    }
    +
    +}
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
    index 61dfefe2a031e..c35fe68dc2279 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java
    @@ -31,11 +31,12 @@
     import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo;
     import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
     import org.apache.kafka.clients.admin.OffsetSpec.TimestampSpec;
    +import org.apache.kafka.clients.admin.internals.AdminApiDriver;
     import org.apache.kafka.clients.admin.internals.AdminApiHandler;
     import org.apache.kafka.clients.admin.internals.AdminMetadataManager;
    -import org.apache.kafka.clients.admin.internals.AdminApiDriver;
     import org.apache.kafka.clients.admin.internals.ConsumerGroupOperationContext;
     import org.apache.kafka.clients.admin.internals.DescribeProducersHandler;
    +import org.apache.kafka.clients.admin.internals.DescribeTransactionsHandler;
     import org.apache.kafka.clients.admin.internals.MetadataOperationContext;
     import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment;
     import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    @@ -4735,6 +4736,15 @@ public DescribeProducersResult describeProducers(Collection topi
             return new DescribeProducersResult(invokeDriver(handler, options.timeoutMs));
         }
     
    +    @Override
    +    public DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options) {
    +        DescribeTransactionsHandler handler = new DescribeTransactionsHandler(
    +            transactionalIds,
    +            logContext
    +        );
    +        return new DescribeTransactionsResult(invokeDriver(handler, options.timeoutMs));
    +    }
    +
         private  Map> invokeDriver(
             AdminApiHandler handler,
             Integer timeoutMs
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java
    new file mode 100644
    index 0000000000000..5a16919848f2f
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionDescription.java
    @@ -0,0 +1,113 @@
    +/*
    + * 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.clients.admin;
    +
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.annotation.InterfaceStability;
    +
    +import java.util.Objects;
    +import java.util.OptionalLong;
    +import java.util.Set;
    +
    +@InterfaceStability.Evolving
    +public class TransactionDescription {
    +    private final int coordinatorId;
    +    private final TransactionState state;
    +    private final long producerId;
    +    private final int producerEpoch;
    +    private final long transactionTimeoutMs;
    +    private final OptionalLong transactionStartTimeMs;
    +    private final Set topicPartitions;
    +
    +    public TransactionDescription(
    +        int coordinatorId,
    +        TransactionState state,
    +        long producerId,
    +        int producerEpoch,
    +        long transactionTimeoutMs,
    +        OptionalLong transactionStartTimeMs,
    +        Set topicPartitions
    +    ) {
    +        this.coordinatorId = coordinatorId;
    +        this.state = state;
    +        this.producerId = producerId;
    +        this.producerEpoch = producerEpoch;
    +        this.transactionTimeoutMs = transactionTimeoutMs;
    +        this.transactionStartTimeMs = transactionStartTimeMs;
    +        this.topicPartitions = topicPartitions;
    +    }
    +
    +    public int coordinatorId() {
    +        return coordinatorId;
    +    }
    +
    +    public TransactionState state() {
    +        return state;
    +    }
    +
    +    public long producerId() {
    +        return producerId;
    +    }
    +
    +    public int producerEpoch() {
    +        return producerEpoch;
    +    }
    +
    +    public long transactionTimeoutMs() {
    +        return transactionTimeoutMs;
    +    }
    +
    +    public OptionalLong transactionStartTimeMs() {
    +        return transactionStartTimeMs;
    +    }
    +
    +    public Set topicPartitions() {
    +        return topicPartitions;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) return true;
    +        if (o == null || getClass() != o.getClass()) return false;
    +        TransactionDescription that = (TransactionDescription) o;
    +        return coordinatorId == that.coordinatorId &&
    +            producerId == that.producerId &&
    +            producerEpoch == that.producerEpoch &&
    +            transactionTimeoutMs == that.transactionTimeoutMs &&
    +            state == that.state &&
    +            Objects.equals(transactionStartTimeMs, that.transactionStartTimeMs) &&
    +            Objects.equals(topicPartitions, that.topicPartitions);
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hash(coordinatorId, state, producerId, producerEpoch, transactionTimeoutMs, transactionStartTimeMs, topicPartitions);
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return "TransactionDescription(" +
    +            "coordinatorId=" + coordinatorId +
    +            ", state=" + state +
    +            ", producerId=" + producerId +
    +            ", producerEpoch=" + producerEpoch +
    +            ", transactionTimeoutMs=" + transactionTimeoutMs +
    +            ", transactionStartTimeMs=" + transactionStartTimeMs +
    +            ", topicPartitions=" + topicPartitions +
    +            ')';
    +    }
    +}
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java
    new file mode 100644
    index 0000000000000..9ff0966ae761a
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/TransactionState.java
    @@ -0,0 +1,56 @@
    +/*
    + * 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.clients.admin;
    +
    +import org.apache.kafka.common.annotation.InterfaceStability;
    +
    +import java.util.Arrays;
    +import java.util.Map;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
    +
    +@InterfaceStability.Evolving
    +public enum TransactionState {
    +    ONGOING("Ongoing"),
    +    PREPARE_ABORT("PrepareAbort"),
    +    PREPARE_COMMIT("PrepareCommit"),
    +    COMPLETE_ABORT("CompleteAbort"),
    +    COMPLETE_COMMIT("CompleteCommit"),
    +    EMPTY("Empty"),
    +    PREPARE_EPOCH_FENCE("PrepareEpochFence"),
    +    UNKNOWN("Unknown");
    +
    +    private final static Map NAME_TO_ENUM = Arrays.stream(values())
    +        .collect(Collectors.toMap(state -> state.name, Function.identity()));
    +
    +    private final String name;
    +
    +    TransactionState(String name) {
    +        this.name = name;
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return name;
    +    }
    +
    +    public static TransactionState parse(String name) {
    +        TransactionState state = NAME_TO_ENUM.get(name);
    +        return state == null ? UNKNOWN : state;
    +    }
    +
    +}
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java
    index 92e6b70e7c0b9..3811462994f46 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java
    @@ -58,7 +58,7 @@ public interface AdminApiHandler {
          *
          * @return a builder for the request containing the given keys
          */
    -    AbstractRequest.Builder buildRequest(Integer brokerId, Set keys);
    +    AbstractRequest.Builder buildRequest(int brokerId, Set keys);
     
         /**
          * Callback that is invoked when a request returns successfully.
    @@ -80,7 +80,7 @@ public interface AdminApiHandler {
          *
          * @return result indicating key completion, failure, and unmapping
          */
    -    ApiResult handleResponse(Integer brokerId, Set keys, AbstractResponse response);
    +    ApiResult handleResponse(int brokerId, Set keys, AbstractResponse response);
     
         class Keys {
             public final Map staticKeys;
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiLookupStrategy.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiLookupStrategy.java
    index 15186475290b8..7969d534a9762 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiLookupStrategy.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiLookupStrategy.java
    @@ -23,6 +23,9 @@
     import java.util.Map;
     import java.util.Set;
     
    +import static java.util.Collections.emptyMap;
    +import static java.util.Collections.singletonMap;
    +
     public interface AdminApiLookupStrategy {
     
         /**
    @@ -83,6 +86,19 @@ public LookupResult(
                 this.failedKeys = Collections.unmodifiableMap(failedKeys);
                 this.mappedKeys = Collections.unmodifiableMap(mappedKeys);
             }
    +
    +        static  LookupResult empty() {
    +            return new LookupResult<>(emptyMap(), emptyMap());
    +        }
    +
    +        static  LookupResult failed(K key, Throwable exception) {
    +            return new LookupResult<>(singletonMap(key, exception), emptyMap());
    +        }
    +
    +        static  LookupResult mapped(K key, Integer brokerId) {
    +            return new LookupResult<>(emptyMap(), singletonMap(key, brokerId));
    +        }
    +
         }
     
     }
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java
    new file mode 100644
    index 0000000000000..774161196db77
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorKey.java
    @@ -0,0 +1,62 @@
    +/*
    + * 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.clients.admin.internals;
    +
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
    +
    +import java.util.Objects;
    +
    +public class CoordinatorKey {
    +    public final String idValue;
    +    public final FindCoordinatorRequest.CoordinatorType type;
    +
    +    private CoordinatorKey(String idValue, FindCoordinatorRequest.CoordinatorType type) {
    +        this.idValue = idValue;
    +        this.type = type;
    +    }
    +
    +    @Override
    +    public boolean equals(Object o) {
    +        if (this == o) return true;
    +        if (o == null || getClass() != o.getClass()) return false;
    +        CoordinatorKey that = (CoordinatorKey) o;
    +        return Objects.equals(idValue, that.idValue) &&
    +            type == that.type;
    +    }
    +
    +    @Override
    +    public int hashCode() {
    +        return Objects.hash(idValue, type);
    +    }
    +
    +    @Override
    +    public String toString() {
    +        return "CoordinatorKey(" +
    +            "idValue='" + idValue + '\'' +
    +            ", type=" + type +
    +            ')';
    +    }
    +
    +    public static CoordinatorKey byGroupId(String groupId) {
    +        return new CoordinatorKey(groupId, FindCoordinatorRequest.CoordinatorType.GROUP);
    +    }
    +
    +    public static CoordinatorKey byTransactionalId(String transactionalId) {
    +        return new CoordinatorKey(transactionalId, FindCoordinatorRequest.CoordinatorType.TRANSACTION);
    +    }
    +
    +}
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java
    new file mode 100644
    index 0000000000000..604218f3075aa
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategy.java
    @@ -0,0 +1,118 @@
    +/*
    + * 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.clients.admin.internals;
    +
    +import org.apache.kafka.common.errors.GroupAuthorizationException;
    +import org.apache.kafka.common.errors.TransactionalIdAuthorizationException;
    +import org.apache.kafka.common.message.FindCoordinatorRequestData;
    +import org.apache.kafka.common.protocol.Errors;
    +import org.apache.kafka.common.requests.AbstractResponse;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
    +import org.apache.kafka.common.requests.FindCoordinatorResponse;
    +import org.apache.kafka.common.utils.LogContext;
    +import org.slf4j.Logger;
    +
    +import java.util.Objects;
    +import java.util.Set;
    +
    +public class CoordinatorStrategy implements AdminApiLookupStrategy {
    +    private final Logger log;
    +
    +    public CoordinatorStrategy(
    +        LogContext logContext
    +    ) {
    +        this.log = logContext.logger(CoordinatorStrategy.class);
    +    }
    +
    +    @Override
    +    public ApiRequestScope lookupScope(CoordinatorKey key) {
    +        // The `FindCoordinator` API does not support batched lookups, so we use a
    +        // separate lookup context for each coordinator key we need to lookup
    +        return new LookupRequestScope(key);
    +    }
    +
    +    @Override
    +    public FindCoordinatorRequest.Builder buildRequest(Set keys) {
    +        CoordinatorKey key = requireSingleton(keys);
    +        return new FindCoordinatorRequest.Builder(
    +            new FindCoordinatorRequestData()
    +                .setKey(key.idValue)
    +                .setKeyType(key.type.id())
    +        );
    +    }
    +
    +    @Override
    +    public LookupResult handleResponse(
    +        Set keys,
    +        AbstractResponse abstractResponse
    +    ) {
    +        CoordinatorKey key = requireSingleton(keys);
    +        FindCoordinatorResponse response = (FindCoordinatorResponse) abstractResponse;
    +        Errors error = response.error();
    +
    +        switch (error) {
    +            case NONE:
    +                return LookupResult.mapped(key, response.data().nodeId());
    +
    +            case COORDINATOR_NOT_AVAILABLE:
    +            case COORDINATOR_LOAD_IN_PROGRESS:
    +                log.debug("FindCoordinator request for key {} returned topic-level error {}. Will retry",
    +                    key, error);
    +                return LookupResult.empty();
    +
    +            case GROUP_AUTHORIZATION_FAILED:
    +                return LookupResult.failed(key, new GroupAuthorizationException("FindCoordinator request for groupId " +
    +                    "`" + key + "` failed due to authorization failure", key.idValue));
    +
    +            case TRANSACTIONAL_ID_AUTHORIZATION_FAILED:
    +                return LookupResult.failed(key, new TransactionalIdAuthorizationException("FindCoordinator request for " +
    +                    "transactionalId `" + key + "` failed due to authorization failure"));
    +
    +            default:
    +                return LookupResult.failed(key, error.exception("FindCoordinator request for key " +
    +                    "`" + key + "` failed due to an unexpected error"));
    +        }
    +    }
    +
    +    private static CoordinatorKey requireSingleton(Set keys) {
    +        if (keys.size() != 1) {
    +            throw new IllegalArgumentException("Unexpected lookup key set");
    +        }
    +        return keys.iterator().next();
    +    }
    +
    +    private static class LookupRequestScope implements ApiRequestScope {
    +        final CoordinatorKey key;
    +
    +        private LookupRequestScope(CoordinatorKey key) {
    +            this.key = key;
    +        }
    +
    +        @Override
    +        public boolean equals(Object o) {
    +            if (this == o) return true;
    +            if (o == null || getClass() != o.getClass()) return false;
    +            LookupRequestScope that = (LookupRequestScope) o;
    +            return Objects.equals(key, that.key);
    +        }
    +
    +        @Override
    +        public int hashCode() {
    +            return Objects.hash(key);
    +        }
    +    }
    +}
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java
    index 85915a4ad859c..2c62461d5c9f9 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeProducersHandler.java
    @@ -82,7 +82,7 @@ public Keys initializeKeys() {
     
         @Override
         public DescribeProducersRequest.Builder buildRequest(
    -        Integer brokerId,
    +        int brokerId,
             Set topicPartitions
         ) {
             DescribeProducersRequestData request = new DescribeProducersRequestData();
    @@ -154,7 +154,7 @@ private void handlePartitionError(
     
         @Override
         public ApiResult handleResponse(
    -        Integer brokerId,
    +        int brokerId,
             Set keys,
             AbstractResponse abstractResponse
         ) {
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java
    new file mode 100644
    index 0000000000000..edeaf1ccca41f
    --- /dev/null
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandler.java
    @@ -0,0 +1,189 @@
    +/*
    + * 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.clients.admin.internals;
    +
    +import org.apache.kafka.clients.admin.TransactionDescription;
    +import org.apache.kafka.clients.admin.TransactionState;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.TransactionalIdAuthorizationException;
    +import org.apache.kafka.common.errors.TransactionalIdNotFoundException;
    +import org.apache.kafka.common.message.DescribeTransactionsRequestData;
    +import org.apache.kafka.common.message.DescribeTransactionsResponseData;
    +import org.apache.kafka.common.protocol.Errors;
    +import org.apache.kafka.common.requests.AbstractResponse;
    +import org.apache.kafka.common.requests.DescribeTransactionsRequest;
    +import org.apache.kafka.common.requests.DescribeTransactionsResponse;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
    +import org.apache.kafka.common.utils.LogContext;
    +import org.slf4j.Logger;
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.OptionalLong;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +public class DescribeTransactionsHandler implements AdminApiHandler {
    +    private final LogContext logContext;
    +    private final Logger log;
    +    private final Set keys;
    +
    +    public DescribeTransactionsHandler(
    +        Collection transactionalIds,
    +        LogContext logContext
    +    ) {
    +        this.keys = buildKeySet(transactionalIds);
    +        this.log = logContext.logger(DescribeTransactionsHandler.class);
    +        this.logContext = logContext;
    +    }
    +
    +    private static Set buildKeySet(Collection transactionalIds) {
    +        return transactionalIds.stream()
    +            .map(CoordinatorKey::byTransactionalId)
    +            .collect(Collectors.toSet());
    +    }
    +
    +    @Override
    +    public String apiName() {
    +        return "describeTransactions";
    +    }
    +
    +    @Override
    +    public Keys initializeKeys() {
    +        return Keys.dynamicMapped(keys, new CoordinatorStrategy(logContext));
    +    }
    +
    +    @Override
    +    public DescribeTransactionsRequest.Builder buildRequest(
    +        int brokerId,
    +        Set keys
    +    ) {
    +        DescribeTransactionsRequestData request = new DescribeTransactionsRequestData();
    +        List transactionalIds = keys.stream().map(key -> {
    +            if (key.type != FindCoordinatorRequest.CoordinatorType.TRANSACTION) {
    +                throw new IllegalArgumentException("Invalid group coordinator key " + key +
    +                    " when building `DescribeTransaction` request");
    +            }
    +            return key.idValue;
    +        }).collect(Collectors.toList());
    +        request.setTransactionalIds(transactionalIds);
    +        return new DescribeTransactionsRequest.Builder(request);
    +    }
    +
    +    @Override
    +    public ApiResult handleResponse(
    +        int brokerId,
    +        Set keys,
    +        AbstractResponse abstractResponse
    +    ) {
    +        DescribeTransactionsResponse response = (DescribeTransactionsResponse) abstractResponse;
    +        Map completed = new HashMap<>();
    +        Map failed = new HashMap<>();
    +        List unmapped = new ArrayList<>();
    +
    +        for (DescribeTransactionsResponseData.TransactionState transactionState : response.data().transactionStates()) {
    +            CoordinatorKey transactionalIdKey = CoordinatorKey.byTransactionalId(
    +                transactionState.transactionalId());
    +            if (!keys.contains(transactionalIdKey)) {
    +                log.warn("Response included transactionalId `{}`, which was not requested",
    +                    transactionState.transactionalId());
    +                continue;
    +            }
    +
    +            Errors error = Errors.forCode(transactionState.errorCode());
    +            if (error != Errors.NONE) {
    +                handleError(transactionalIdKey, error, failed, unmapped);
    +                continue;
    +            }
    +
    +            OptionalLong transactionStartTimeMs = transactionState.transactionStartTimeMs() < 0 ?
    +                OptionalLong.empty() :
    +                OptionalLong.of(transactionState.transactionStartTimeMs());
    +
    +            completed.put(transactionalIdKey, new TransactionDescription(
    +                brokerId,
    +                TransactionState.parse(transactionState.transactionState()),
    +                transactionState.producerId(),
    +                transactionState.producerEpoch(),
    +                transactionState.transactionTimeoutMs(),
    +                transactionStartTimeMs,
    +                collectTopicPartitions(transactionState)
    +            ));
    +        }
    +
    +        return new ApiResult<>(completed, failed, unmapped);
    +    }
    +
    +    private Set collectTopicPartitions(
    +        DescribeTransactionsResponseData.TransactionState transactionState
    +    ) {
    +        Set res = new HashSet<>();
    +        for (DescribeTransactionsResponseData.TopicData topicData : transactionState.topics()) {
    +            String topic = topicData.topic();
    +            for (Integer partitionId : topicData.partitions()) {
    +                res.add(new TopicPartition(topic, partitionId));
    +            }
    +        }
    +        return res;
    +    }
    +
    +    private void handleError(
    +        CoordinatorKey transactionalIdKey,
    +        Errors error,
    +        Map failed,
    +        List unmapped
    +    ) {
    +        switch (error) {
    +            case TRANSACTIONAL_ID_AUTHORIZATION_FAILED:
    +                failed.put(transactionalIdKey, new TransactionalIdAuthorizationException(
    +                    "DescribeTransactions request for transactionalId `" + transactionalIdKey.idValue + "` " +
    +                        "failed due to authorization failure"));
    +                break;
    +
    +            case TRANSACTIONAL_ID_NOT_FOUND:
    +                failed.put(transactionalIdKey, new TransactionalIdNotFoundException(
    +                    "DescribeTransactions request for transactionalId `" + transactionalIdKey.idValue + "` " +
    +                        "failed because the ID could not be found"));
    +                break;
    +
    +            case COORDINATOR_LOAD_IN_PROGRESS:
    +                // If the coordinator is in the middle of loading, then we just need to retry
    +                log.debug("DescribeTransactions request for transactionalId `{}` failed because the " +
    +                        "coordinator is still in the process of loading state. Will retry",
    +                    transactionalIdKey.idValue);
    +                break;
    +
    +            case NOT_COORDINATOR:
    +            case COORDINATOR_NOT_AVAILABLE:
    +                // If the coordinator is unavailable or there was a coordinator change, then we unmap
    +                // the key so that we retry the `FindCoordinator` request
    +                unmapped.add(transactionalIdKey);
    +                log.debug("DescribeTransactions request for transactionalId `{}` returned error {}. Will attempt " +
    +                        "to find the coordinator again and retry", transactionalIdKey.idValue, error);
    +                break;
    +
    +            default:
    +                failed.put(transactionalIdKey, error.exception("DescribeTransactions request for " +
    +                    "transactionalId `" + transactionalIdKey.idValue + "` failed due to unexpected error"));
    +        }
    +    }
    +
    +}
    diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategy.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategy.java
    index d0e4cbff2a04e..18ae79a7b6267 100644
    --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategy.java
    +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/PartitionLeaderStrategy.java
    @@ -153,7 +153,6 @@ public LookupResult handleResponse(
                     continue;
                 }
     
    -
                 for (MetadataResponseData.MetadataResponsePartition partitionMetadata : topicMetadata.partitions()) {
                     TopicPartition topicPartition = new TopicPartition(topic, partitionMetadata.partitionIndex());
                     Errors partitionError = Errors.forCode(partitionMetadata.errorCode());
    diff --git a/clients/src/main/java/org/apache/kafka/common/ConsumerGroupState.java b/clients/src/main/java/org/apache/kafka/common/ConsumerGroupState.java
    index 36d1a4da36118..ebd2b53934248 100644
    --- a/clients/src/main/java/org/apache/kafka/common/ConsumerGroupState.java
    +++ b/clients/src/main/java/org/apache/kafka/common/ConsumerGroupState.java
    @@ -17,7 +17,10 @@
     
     package org.apache.kafka.common;
     
    -import java.util.HashMap;
    +import java.util.Arrays;
    +import java.util.Map;
    +import java.util.function.Function;
    +import java.util.stream.Collectors;
     
     /**
      * The consumer group state.
    @@ -30,14 +33,8 @@ public enum ConsumerGroupState {
         DEAD("Dead"),
         EMPTY("Empty");
     
    -    private final static HashMap NAME_TO_ENUM;
    -
    -    static {
    -        NAME_TO_ENUM = new HashMap<>();
    -        for (ConsumerGroupState state : ConsumerGroupState.values()) {
    -            NAME_TO_ENUM.put(state.name, state);
    -        }
    -    }
    +    private final static Map NAME_TO_ENUM = Arrays.stream(values())
    +        .collect(Collectors.toMap(state -> state.name, Function.identity()));;
     
         private final String name;
     
    diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
    index 3b9e1a6ec5056..d76641ac6a0ff 100644
    --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
    +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java
    @@ -100,10 +100,12 @@
     import org.apache.kafka.common.message.DescribeLogDirsResponseData;
     import org.apache.kafka.common.message.DescribeLogDirsResponseData.DescribeLogDirsTopic;
     import org.apache.kafka.common.message.DescribeProducersResponseData;
    +import org.apache.kafka.common.message.DescribeTransactionsResponseData;
     import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData;
     import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData.CredentialInfo;
     import org.apache.kafka.common.message.ElectLeadersResponseData.PartitionResult;
     import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult;
    +import org.apache.kafka.common.message.FindCoordinatorResponseData;
     import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData;
     import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse;
     import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity;
    @@ -157,8 +159,11 @@
     import org.apache.kafka.common.requests.DescribeLogDirsResponse;
     import org.apache.kafka.common.requests.DescribeProducersRequest;
     import org.apache.kafka.common.requests.DescribeProducersResponse;
    +import org.apache.kafka.common.requests.DescribeTransactionsRequest;
    +import org.apache.kafka.common.requests.DescribeTransactionsResponse;
     import org.apache.kafka.common.requests.DescribeUserScramCredentialsResponse;
     import org.apache.kafka.common.requests.ElectLeadersResponse;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
     import org.apache.kafka.common.requests.FindCoordinatorResponse;
     import org.apache.kafka.common.requests.IncrementalAlterConfigsResponse;
     import org.apache.kafka.common.requests.JoinGroupRequest;
    @@ -5432,6 +5437,123 @@ public void testDescribeProducersRetryAfterDisconnect() throws Exception {
             }
         }
     
    +    @Test
    +    public void testDescribeTransactions() throws Exception {
    +        try (AdminClientUnitTestEnv env = mockClientEnv()) {
    +            String transactionalId = "foo";
    +            Node coordinator = env.cluster().nodes().iterator().next();
    +            TransactionDescription expected = new TransactionDescription(
    +                coordinator.id(), TransactionState.COMPLETE_COMMIT, 12345L,
    +                15, 10000L, OptionalLong.empty(), emptySet());
    +
    +            env.kafkaClient().prepareResponse(
    +                request -> request instanceof FindCoordinatorRequest,
    +                new FindCoordinatorResponse(new FindCoordinatorResponseData()
    +                    .setErrorCode(Errors.NONE.code())
    +                    .setNodeId(coordinator.id())
    +                    .setHost(coordinator.host())
    +                    .setPort(coordinator.port()))
    +            );
    +
    +            env.kafkaClient().prepareResponseFrom(
    +                request -> request instanceof DescribeTransactionsRequest,
    +                new DescribeTransactionsResponse(new DescribeTransactionsResponseData().setTransactionStates(
    +                    singletonList(new DescribeTransactionsResponseData.TransactionState()
    +                        .setErrorCode(Errors.NONE.code())
    +                        .setProducerEpoch((short) expected.producerEpoch())
    +                        .setProducerId(expected.producerId())
    +                        .setTransactionalId(transactionalId)
    +                        .setTransactionTimeoutMs(10000)
    +                        .setTransactionStartTimeMs(-1)
    +                        .setTransactionState(expected.state().toString())
    +                    )
    +                )),
    +                coordinator
    +            );
    +
    +            DescribeTransactionsResult result = env.adminClient().describeTransactions(singleton(transactionalId));
    +            KafkaFuture future = result.description(transactionalId);
    +            assertEquals(expected, future.get());
    +        }
    +    }
    +
    +    @Test
    +    public void testRetryDescribeTransactionsAfterNotCoordinatorError() throws Exception {
    +        MockTime time = new MockTime();
    +        int retryBackoffMs = 100;
    +        Cluster cluster = mockCluster(3, 0);
    +        Map configOverride = newStrMap(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG, "" + retryBackoffMs);
    +
    +        try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, configOverride)) {
    +            String transactionalId = "foo";
    +
    +            Iterator nodeIterator = env.cluster().nodes().iterator();
    +            Node coordinator1 = nodeIterator.next();
    +            Node coordinator2 = nodeIterator.next();
    +
    +            env.kafkaClient().prepareResponse(
    +                request -> request instanceof FindCoordinatorRequest,
    +                new FindCoordinatorResponse(new FindCoordinatorResponseData()
    +                    .setErrorCode(Errors.NONE.code())
    +                    .setNodeId(coordinator1.id())
    +                    .setHost(coordinator1.host())
    +                    .setPort(coordinator1.port()))
    +            );
    +
    +            env.kafkaClient().prepareResponseFrom(
    +                request -> {
    +                    if (!(request instanceof DescribeTransactionsRequest)) {
    +                        return false;
    +                    } else {
    +                        // Backoff needed here for the retry of FindCoordinator
    +                        time.sleep(retryBackoffMs);
    +                        return true;
    +                    }
    +                },
    +                new DescribeTransactionsResponse(new DescribeTransactionsResponseData().setTransactionStates(
    +                    singletonList(new DescribeTransactionsResponseData.TransactionState()
    +                        .setErrorCode(Errors.NOT_COORDINATOR.code())
    +                        .setTransactionalId(transactionalId)
    +                    )
    +                )),
    +                coordinator1
    +            );
    +
    +            env.kafkaClient().prepareResponse(
    +                request -> request instanceof FindCoordinatorRequest,
    +                new FindCoordinatorResponse(new FindCoordinatorResponseData()
    +                    .setErrorCode(Errors.NONE.code())
    +                    .setNodeId(coordinator2.id())
    +                    .setHost(coordinator2.host())
    +                    .setPort(coordinator2.port()))
    +            );
    +
    +            TransactionDescription expected = new TransactionDescription(
    +                coordinator2.id(), TransactionState.COMPLETE_COMMIT, 12345L,
    +                15, 10000L, OptionalLong.empty(), emptySet());
    +
    +            env.kafkaClient().prepareResponseFrom(
    +                request -> request instanceof DescribeTransactionsRequest,
    +                new DescribeTransactionsResponse(new DescribeTransactionsResponseData().setTransactionStates(
    +                    singletonList(new DescribeTransactionsResponseData.TransactionState()
    +                        .setErrorCode(Errors.NONE.code())
    +                        .setProducerEpoch((short) expected.producerEpoch())
    +                        .setProducerId(expected.producerId())
    +                        .setTransactionalId(transactionalId)
    +                        .setTransactionTimeoutMs(10000)
    +                        .setTransactionStartTimeMs(-1)
    +                        .setTransactionState(expected.state().toString())
    +                    )
    +                )),
    +                coordinator2
    +            );
    +
    +            DescribeTransactionsResult result = env.adminClient().describeTransactions(singleton(transactionalId));
    +            KafkaFuture future = result.description(transactionalId);
    +            assertEquals(expected, future.get());
    +        }
    +    }
    +
         private DescribeProducersResponse buildDescribeProducersResponse(
             TopicPartition topicPartition,
             List producerStates
    diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java
    index f09a545687306..a47a945aa27bc 100644
    --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java
    +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java
    @@ -908,6 +908,11 @@ public DescribeProducersResult describeProducers(Collection part
             throw new UnsupportedOperationException("Not implemented yet");
         }
     
    +    @Override
    +    public DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options) {
    +        throw new UnsupportedOperationException("Not implemented yet");
    +    }
    +
         @Override
         synchronized public void close(Duration timeout) {}
     
    diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java
    index d24123238c06b..06e72dfdb8182 100644
    --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java
    +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminApiDriverTest.java
    @@ -678,14 +678,14 @@ public void expectRequest(Set keys, ApiResult result) {
             }
     
             @Override
    -        public AbstractRequest.Builder buildRequest(Integer brokerId, Set keys) {
    +        public AbstractRequest.Builder buildRequest(int brokerId, Set keys) {
                 // The request is just a placeholder in these tests
                 assertTrue(expectedRequests.containsKey(keys), "Unexpected fulfillment request for keys " + keys);
                 return new MetadataRequest.Builder(Collections.emptyList(), false);
             }
     
             @Override
    -        public ApiResult handleResponse(Integer brokerId, Set keys, AbstractResponse response) {
    +        public ApiResult handleResponse(int brokerId, Set keys, AbstractResponse response) {
                 return Optional.ofNullable(expectedRequests.get(keys)).orElseThrow(() ->
                     new AssertionError("Unexpected fulfillment request for keys " + keys)
                 );
    diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java
    new file mode 100644
    index 0000000000000..ea4239c4f58ba
    --- /dev/null
    +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/CoordinatorStrategyTest.java
    @@ -0,0 +1,140 @@
    +/*
    + * 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.clients.admin.internals;
    +
    +import org.apache.kafka.common.errors.GroupAuthorizationException;
    +import org.apache.kafka.common.message.FindCoordinatorResponseData;
    +import org.apache.kafka.common.protocol.Errors;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest;
    +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
    +import org.apache.kafka.common.requests.FindCoordinatorResponse;
    +import org.apache.kafka.common.utils.LogContext;
    +import org.junit.jupiter.api.Test;
    +
    +import java.util.Collections;
    +
    +import static java.util.Collections.emptyMap;
    +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.assertThrows;
    +import static org.junit.jupiter.api.Assertions.assertTrue;
    +
    +public class CoordinatorStrategyTest {
    +
    +    @Test
    +    public void testBuildLookupRequest() {
    +        CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext());
    +        FindCoordinatorRequest.Builder request = strategy.buildRequest(singleton(
    +            CoordinatorKey.byGroupId("foo")));
    +        assertEquals("foo", request.data().key());
    +        assertEquals(CoordinatorType.GROUP, CoordinatorType.forId(request.data().keyType()));
    +    }
    +
    +    @Test
    +    public void testBuildLookupRequestRequiresOneKey() {
    +        CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext());
    +        assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(Collections.emptySet()));
    +
    +        CoordinatorKey group1 = CoordinatorKey.byGroupId("foo");
    +        CoordinatorKey group2 = CoordinatorKey.byGroupId("bar");
    +        assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(mkSet(group1, group2)));
    +    }
    +
    +    @Test
    +    public void testHandleResponseRequiresOneKey() {
    +        FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(Errors.NONE.code());
    +        FindCoordinatorResponse response = new FindCoordinatorResponse(responseData);
    +
    +        CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext());
    +        assertThrows(IllegalArgumentException.class, () ->
    +            strategy.handleResponse(Collections.emptySet(), response));
    +
    +        CoordinatorKey group1 = CoordinatorKey.byGroupId("foo");
    +        CoordinatorKey group2 = CoordinatorKey.byGroupId("bar");
    +        assertThrows(IllegalArgumentException.class, () ->
    +            strategy.handleResponse(mkSet(group1, group2), response));
    +    }
    +
    +    @Test
    +    public void testSuccessfulCoordinatorLookup() {
    +        CoordinatorKey group = CoordinatorKey.byGroupId("foo");
    +
    +        FindCoordinatorResponseData responseData = new FindCoordinatorResponseData()
    +            .setErrorCode(Errors.NONE.code())
    +            .setHost("localhost")
    +            .setPort(9092)
    +            .setNodeId(1);
    +
    +        AdminApiLookupStrategy.LookupResult result = runLookup(group, responseData);
    +        assertEquals(singletonMap(group, 1), result.mappedKeys);
    +        assertEquals(emptyMap(), result.failedKeys);
    +    }
    +
    +    @Test
    +    public void testRetriableCoordinatorLookup() {
    +        testRetriableCoordinatorLookup(Errors.COORDINATOR_LOAD_IN_PROGRESS);
    +        testRetriableCoordinatorLookup(Errors.COORDINATOR_NOT_AVAILABLE);
    +    }
    +
    +    private void testRetriableCoordinatorLookup(Errors error) {
    +        CoordinatorKey group = CoordinatorKey.byGroupId("foo");
    +        FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(error.code());
    +        AdminApiLookupStrategy.LookupResult result = runLookup(group, responseData);
    +
    +        assertEquals(emptyMap(), result.failedKeys);
    +        assertEquals(emptyMap(), result.mappedKeys);
    +    }
    +
    +    @Test
    +    public void testFatalErrorLookupResponses() {
    +        CoordinatorKey group = CoordinatorKey.byTransactionalId("foo");
    +        assertFatalLookup(group, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED);
    +        assertFatalLookup(group, Errors.UNKNOWN_SERVER_ERROR);
    +
    +        Throwable throwable = assertFatalLookup(group, Errors.GROUP_AUTHORIZATION_FAILED);
    +        assertTrue(throwable instanceof GroupAuthorizationException);
    +        GroupAuthorizationException exception = (GroupAuthorizationException) throwable;
    +        assertEquals("foo", exception.groupId());
    +    }
    +
    +    public Throwable assertFatalLookup(
    +        CoordinatorKey key,
    +        Errors error
    +    ) {
    +        FindCoordinatorResponseData responseData = new FindCoordinatorResponseData().setErrorCode(error.code());
    +        AdminApiLookupStrategy.LookupResult result = runLookup(key, responseData);
    +
    +        assertEquals(emptyMap(), result.mappedKeys);
    +        assertEquals(singleton(key), result.failedKeys.keySet());
    +
    +        Throwable throwable = result.failedKeys.get(key);
    +        assertTrue(error.exception().getClass().isInstance(throwable));
    +        return throwable;
    +    }
    +
    +    private AdminApiLookupStrategy.LookupResult runLookup(
    +        CoordinatorKey key,
    +        FindCoordinatorResponseData responseData
    +    ) {
    +        CoordinatorStrategy strategy = new CoordinatorStrategy(new LogContext());
    +        FindCoordinatorResponse response = new FindCoordinatorResponse(responseData);
    +        return strategy.handleResponse(singleton(key), response);
    +    }
    +
    +}
    diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java
    new file mode 100644
    index 0000000000000..04e7851e5f1a0
    --- /dev/null
    +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/DescribeTransactionsHandlerTest.java
    @@ -0,0 +1,229 @@
    +/*
    + * 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.clients.admin.internals;
    +
    +import org.apache.kafka.clients.admin.TransactionDescription;
    +import org.apache.kafka.clients.admin.internals.AdminApiHandler.ApiResult;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.message.DescribeTransactionsResponseData;
    +import org.apache.kafka.common.protocol.Errors;
    +import org.apache.kafka.common.requests.DescribeTransactionsRequest;
    +import org.apache.kafka.common.requests.DescribeTransactionsResponse;
    +import org.apache.kafka.common.utils.LogContext;
    +import org.junit.jupiter.api.Test;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +import static java.util.Arrays.asList;
    +import static java.util.Collections.emptyList;
    +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;
    +
    +public class DescribeTransactionsHandlerTest {
    +    private final LogContext logContext = new LogContext();
    +
    +    @Test
    +    public void testBuildRequest() {
    +        String transactionalId1 = "foo";
    +        String transactionalId2 = "bar";
    +        String transactionalId3 = "baz";
    +
    +        Set transactionalIds = mkSet(transactionalId1, transactionalId2, transactionalId3);
    +        DescribeTransactionsHandler handler = new DescribeTransactionsHandler(transactionalIds, logContext);
    +
    +        assertLookup(handler, transactionalIds);
    +        assertLookup(handler, mkSet(transactionalId1));
    +        assertLookup(handler, mkSet(transactionalId2, transactionalId3));
    +    }
    +
    +    @Test
    +    public void testHandleSuccessfulResponse() {
    +        int brokerId = 1;
    +        String transactionalId1 = "foo";
    +        String transactionalId2 = "bar";
    +
    +        Set transactionalIds = mkSet(transactionalId1, transactionalId2);
    +        DescribeTransactionsHandler handler = new DescribeTransactionsHandler(transactionalIds, logContext);
    +
    +        DescribeTransactionsResponseData.TransactionState transactionState1 =
    +            sampleTransactionState1(transactionalId1);
    +        DescribeTransactionsResponseData.TransactionState transactionState2 =
    +            sampleTransactionState2(transactionalId2);
    +
    +        Set keys = coordinatorKeys(transactionalIds);
    +        DescribeTransactionsResponse response = new DescribeTransactionsResponse(new DescribeTransactionsResponseData()
    +            .setTransactionStates(asList(transactionState1, transactionState2)));
    +
    +        ApiResult result = handler.handleResponse(
    +            brokerId, keys, response);
    +
    +        assertEquals(keys, result.completedKeys.keySet());
    +        assertMatchingTransactionState(brokerId, transactionState1,
    +            result.completedKeys.get(CoordinatorKey.byTransactionalId(transactionalId1)));
    +        assertMatchingTransactionState(brokerId, transactionState2,
    +            result.completedKeys.get(CoordinatorKey.byTransactionalId(transactionalId2)));
    +    }
    +
    +    @Test
    +    public void testHandleErrorResponse() {
    +        String transactionalId = "foo";
    +        Set transactionalIds = mkSet(transactionalId);
    +        DescribeTransactionsHandler handler = new DescribeTransactionsHandler(transactionalIds, logContext);
    +        assertFatalError(handler, transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED);
    +        assertFatalError(handler, transactionalId, Errors.TRANSACTIONAL_ID_NOT_FOUND);
    +        assertFatalError(handler, transactionalId, Errors.UNKNOWN_SERVER_ERROR);
    +        assertRetriableError(handler, transactionalId, Errors.COORDINATOR_LOAD_IN_PROGRESS);
    +        assertUnmappedKey(handler, transactionalId, Errors.NOT_COORDINATOR);
    +        assertUnmappedKey(handler, transactionalId, Errors.COORDINATOR_NOT_AVAILABLE);
    +    }
    +
    +    private void assertFatalError(
    +        DescribeTransactionsHandler handler,
    +        String transactionalId,
    +        Errors error
    +    ) {
    +        CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
    +        ApiResult result = handleResponseError(handler, transactionalId, error);
    +        assertEquals(emptyList(), result.unmappedKeys);
    +        assertEquals(mkSet(key), result.failedKeys.keySet());
    +
    +        Throwable throwable = result.failedKeys.get(key);
    +        assertTrue(error.exception().getClass().isInstance(throwable));
    +    }
    +
    +    private void assertRetriableError(
    +        DescribeTransactionsHandler handler,
    +        String transactionalId,
    +        Errors error
    +    ) {
    +        ApiResult result = handleResponseError(handler, transactionalId, error);
    +        assertEquals(emptyList(), result.unmappedKeys);
    +        assertEquals(emptyMap(), result.failedKeys);
    +    }
    +
    +    private void assertUnmappedKey(
    +        DescribeTransactionsHandler handler,
    +        String transactionalId,
    +        Errors error
    +    ) {
    +        CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
    +        ApiResult result = handleResponseError(handler, transactionalId, error);
    +        assertEquals(emptyMap(), result.failedKeys);
    +        assertEquals(singletonList(key), result.unmappedKeys);
    +    }
    +
    +    private ApiResult handleResponseError(
    +        DescribeTransactionsHandler handler,
    +        String transactionalId,
    +        Errors error
    +    ) {
    +        int brokerId = 1;
    +
    +        CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
    +        Set keys = mkSet(key);
    +
    +        DescribeTransactionsResponseData.TransactionState transactionState = new DescribeTransactionsResponseData.TransactionState()
    +            .setErrorCode(error.code())
    +            .setTransactionalId(transactionalId);
    +
    +        DescribeTransactionsResponse response = new DescribeTransactionsResponse(new DescribeTransactionsResponseData()
    +            .setTransactionStates(singletonList(transactionState)));
    +
    +        ApiResult result = handler.handleResponse(brokerId, keys, response);
    +        assertEquals(emptyMap(), result.completedKeys);
    +        return result;
    +    }
    +
    +    private void assertLookup(
    +        DescribeTransactionsHandler handler,
    +        Set transactionalIds
    +    ) {
    +        Set keys = coordinatorKeys(transactionalIds);
    +        DescribeTransactionsRequest.Builder request = handler.buildRequest(1, keys);
    +        assertEquals(transactionalIds, new HashSet<>(request.data.transactionalIds()));
    +    }
    +
    +    private static Set coordinatorKeys(Set transactionalIds) {
    +        return transactionalIds.stream()
    +            .map(CoordinatorKey::byTransactionalId)
    +            .collect(Collectors.toSet());
    +    }
    +
    +    private DescribeTransactionsResponseData.TransactionState sampleTransactionState1(
    +        String transactionalId
    +    ) {
    +        return new DescribeTransactionsResponseData.TransactionState()
    +            .setErrorCode(Errors.NONE.code())
    +            .setTransactionState("Ongoing")
    +            .setTransactionalId(transactionalId)
    +            .setProducerId(12345L)
    +            .setProducerEpoch((short) 15)
    +            .setTransactionStartTimeMs(1599151791L)
    +            .setTransactionTimeoutMs(10000)
    +            .setTopics(new DescribeTransactionsResponseData.TopicDataCollection(asList(
    +                new DescribeTransactionsResponseData.TopicData()
    +                    .setTopic("foo")
    +                    .setPartitions(asList(1, 3, 5)),
    +                new DescribeTransactionsResponseData.TopicData()
    +                    .setTopic("bar")
    +                    .setPartitions(asList(1, 3, 5))
    +            ).iterator()));
    +    }
    +
    +    private DescribeTransactionsResponseData.TransactionState sampleTransactionState2(
    +        String transactionalId
    +    ) {
    +        return new DescribeTransactionsResponseData.TransactionState()
    +            .setErrorCode(Errors.NONE.code())
    +            .setTransactionState("Empty")
    +            .setTransactionalId(transactionalId)
    +            .setProducerId(98765L)
    +            .setProducerEpoch((short) 30)
    +            .setTransactionStartTimeMs(-1);
    +    }
    +
    +    private void assertMatchingTransactionState(
    +        int expectedCoordinatorId,
    +        DescribeTransactionsResponseData.TransactionState expected,
    +        TransactionDescription actual
    +    ) {
    +        assertEquals(expectedCoordinatorId, actual.coordinatorId());
    +        assertEquals(expected.producerId(), actual.producerId());
    +        assertEquals(expected.producerEpoch(), actual.producerEpoch());
    +        assertEquals(expected.transactionTimeoutMs(), actual.transactionTimeoutMs());
    +        assertEquals(expected.transactionStartTimeMs(), actual.transactionStartTimeMs().orElse(-1));
    +        assertEquals(collectTransactionPartitions(expected), actual.topicPartitions());
    +    }
    +
    +    private Set collectTransactionPartitions(
    +        DescribeTransactionsResponseData.TransactionState transactionState
    +    ) {
    +        Set topicPartitions = new HashSet<>();
    +        for (DescribeTransactionsResponseData.TopicData topicData : transactionState.topics()) {
    +            for (Integer partitionId : topicData.partitions()) {
    +                topicPartitions.add(new TopicPartition(topicData.topic(), partitionId));
    +            }
    +        }
    +        return topicPartitions;
    +    }
    +
    +}
    diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
    index 8b8fde65a61e6..0e3fa28098118 100644
    --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
    +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala
    @@ -289,23 +289,28 @@ class TransactionCoordinator(brokerId: Int,
             case Right(Some(coordinatorEpochAndMetadata)) =>
               val txnMetadata = coordinatorEpochAndMetadata.transactionMetadata
               txnMetadata.inLock {
    -            txnMetadata.topicPartitions.foreach { topicPartition =>
    -              var topicData = transactionState.topics.find(topicPartition.topic)
    -              if (topicData == null) {
    -                topicData = new DescribeTransactionsResponseData.TopicData()
    -                  .setTopic(topicPartition.topic)
    -                transactionState.topics.add(topicData)
    +            if (txnMetadata.state == Dead) {
    +              // The transaction state is being expired, so ignore it
    +              transactionState.setErrorCode(Errors.TRANSACTIONAL_ID_NOT_FOUND.code)
    +            } else {
    +              txnMetadata.topicPartitions.foreach { topicPartition =>
    +                var topicData = transactionState.topics.find(topicPartition.topic)
    +                if (topicData == null) {
    +                  topicData = new DescribeTransactionsResponseData.TopicData()
    +                    .setTopic(topicPartition.topic)
    +                  transactionState.topics.add(topicData)
    +                }
    +                topicData.partitions.add(topicPartition.partition)
                   }
    -              topicData.partitions.add(topicPartition.partition)
    -            }
     
    -            transactionState
    -              .setErrorCode(Errors.NONE.code)
    -              .setProducerId(txnMetadata.producerId)
    -              .setProducerEpoch(txnMetadata.producerEpoch)
    -              .setTransactionState(txnMetadata.state.name)
    -              .setTransactionTimeoutMs(txnMetadata.txnTimeoutMs)
    -              .setTransactionStartTimeMs(txnMetadata.txnStartTimestamp)
    +              transactionState
    +                .setErrorCode(Errors.NONE.code)
    +                .setProducerId(txnMetadata.producerId)
    +                .setProducerEpoch(txnMetadata.producerEpoch)
    +                .setTransactionState(txnMetadata.state.name)
    +                .setTransactionTimeoutMs(txnMetadata.txnTimeoutMs)
    +                .setTransactionStartTimeMs(txnMetadata.txnStartTimestamp)
    +            }
               }
           }
         }
    diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala
    index 0cc52bf6e6d58..72ccc8de6bf4d 100644
    --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala
    +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala
    @@ -50,7 +50,8 @@ class TransactionCoordinatorTest {
       private val partitions = mutable.Set[TopicPartition](new TopicPartition("topic1", 0))
       private val scheduler = new MockScheduler(time)
     
    -  val coordinator = new TransactionCoordinator(brokerId,
    +  val coordinator = new TransactionCoordinator(
    +    brokerId,
         TransactionConfig(),
         scheduler,
         () => pidGenerator,
    @@ -1087,6 +1088,22 @@ class TransactionCoordinatorTest {
         assertEquals(Errors.INVALID_REQUEST, Errors.forCode(result.errorCode))
       }
     
    +  @Test
    +  def testDescribeTransactionsWithExpiringTransactionalId(): Unit = {
    +    coordinator.startup(() => transactionStatePartitionCount, enableTransactionalIdExpiration = false)
    +
    +    val txnMetadata = new TransactionMetadata(transactionalId, producerId, producerId, producerEpoch,
    +      RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, Dead, mutable.Set.empty, time.milliseconds(),
    +      time.milliseconds())
    +    EasyMock.expect(transactionManager.getTransactionState(EasyMock.eq(transactionalId)))
    +      .andReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
    +    EasyMock.replay(transactionManager)
    +
    +    val result = coordinator.handleDescribeTransactions(transactionalId)
    +    assertEquals(transactionalId, result.transactionalId)
    +    assertEquals(Errors.TRANSACTIONAL_ID_NOT_FOUND, Errors.forCode(result.errorCode))
    +  }
    +
       @Test
       def testDescribeTransactionsWhileCoordinatorLoading(): Unit = {
         EasyMock.expect(transactionManager.getTransactionState(EasyMock.eq(transactionalId)))
    diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala
    index 92407dddb5690..4c0f456fb7c56 100644
    --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala
    +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionMetadataTest.scala
    @@ -465,6 +465,12 @@ class TransactionMetadataTest {
         for (state <- TransactionState.AllStates) {
           assertEquals(state, TransactionState.fromId(state.id))
           assertEquals(Some(state), TransactionState.fromName(state.name))
    +
    +      if (state != Dead) {
    +        val clientTransactionState = org.apache.kafka.clients.admin.TransactionState.parse(state.name)
    +        assertEquals(state.name, clientTransactionState.toString)
    +        assertNotEquals(org.apache.kafka.clients.admin.TransactionState.UNKNOWN, clientTransactionState)
    +      }
         }
       }
     
    
    From 105243afb11883cf0dbbe46e3e783b7fa697836a Mon Sep 17 00:00:00 2001
    From: "high.lee" 
    Date: Sat, 24 Apr 2021 02:06:55 +0900
    Subject: [PATCH 086/155] KAFKA-10283; Consolidate client-level and
     consumer-level assignment within ClientState (#9640)
    
    Reviewers: A. Sophie Blee-Goldman , Guozhang Wang 
    ---
     .../internals/assignment/ClientState.java     | 138 +++++++++---------
     .../internals/assignment/ClientStateTask.java |  45 ++++++
     .../internals/assignment/ClientStateTest.java |  11 ++
     3 files changed, 129 insertions(+), 65 deletions(-)
     create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java
    
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
    index ee464f3852211..0029e73337824 100644
    --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientState.java
    @@ -43,23 +43,16 @@ public class ClientState {
         private static final Logger LOG = LoggerFactory.getLogger(ClientState.class);
         public static final Comparator TOPIC_PARTITION_COMPARATOR = comparing(TopicPartition::topic).thenComparing(TopicPartition::partition);
     
    -    private final Set activeTasks = new TreeSet<>();
    -    private final Set standbyTasks = new TreeSet<>();
    -    private final Set prevActiveTasks;
    -    private final Set prevStandbyTasks;
    -
         private final Map taskOffsetSums; // contains only stateful tasks we previously owned
         private final Map taskLagTotals;  // contains lag for all stateful tasks in the app topology
         private final Map ownedPartitions = new TreeMap<>(TOPIC_PARTITION_COMPARATOR);
         private final Map> consumerToPreviousStatefulTaskIds = new TreeMap<>();
     
    -    // the following four maps are used only for logging purposes;
    -    // TODO KAFKA-10283: we could consider merging them with other book-keeping maps at client-levels
    -    //                   so that they would not be inconsistent
    -    private final Map> consumerToPreviousActiveTaskIds = new TreeMap<>();
    -    private final Map> consumerToAssignedActiveTaskIds = new TreeMap<>();
    -    private final Map> consumerToAssignedStandbyTaskIds = new TreeMap<>();
    -    private final Map> consumerToRevokingActiveTaskIds = new TreeMap<>();
    +    private final ClientStateTask assignedActiveTasks = new ClientStateTask(new TreeSet<>(), new TreeMap<>());
    +    private final ClientStateTask assignedStandbyTasks = new ClientStateTask(new TreeSet<>(), new TreeMap<>());
    +    private final ClientStateTask previousActiveTasks = new ClientStateTask(null, new TreeMap<>());
    +    private final ClientStateTask previousStandbyTasks = new ClientStateTask(null, null);
    +    private final ClientStateTask revokingActiveTasks = new ClientStateTask(null, new TreeMap<>());
     
         private int capacity;
     
    @@ -68,8 +61,9 @@ public ClientState() {
         }
     
         ClientState(final int capacity) {
    -        prevActiveTasks = new TreeSet<>();
    -        prevStandbyTasks = new TreeSet<>();
    +        previousStandbyTasks.taskIds(new TreeSet<>());
    +        previousActiveTasks.taskIds(new TreeSet<>());
    +
             taskOffsetSums = new TreeMap<>();
             taskLagTotals = new TreeMap<>();
             this.capacity = capacity;
    @@ -80,8 +74,8 @@ public ClientState(final Set previousActiveTasks,
                            final Set previousStandbyTasks,
                            final Map taskLagTotals,
                            final int capacity) {
    -        prevActiveTasks = unmodifiableSet(new TreeSet<>(previousActiveTasks));
    -        prevStandbyTasks = unmodifiableSet(new TreeSet<>(previousStandbyTasks));
    +        this.previousStandbyTasks.taskIds(unmodifiableSet(new TreeSet<>(previousStandbyTasks)));
    +        this.previousActiveTasks.taskIds(unmodifiableSet(new TreeSet<>(previousActiveTasks)));
             taskOffsetSums = emptyMap();
             this.taskLagTotals = unmodifiableMap(taskLagTotals);
             this.capacity = capacity;
    @@ -100,11 +94,11 @@ boolean reachedCapacity() {
         }
     
         public Set activeTasks() {
    -        return unmodifiableSet(activeTasks);
    +        return unmodifiableSet(assignedActiveTasks.taskIds());
         }
     
         public int activeTaskCount() {
    -        return activeTasks.size();
    +        return assignedActiveTasks.taskIds().size();
         }
     
         double activeTaskLoad() {
    @@ -112,28 +106,33 @@ public int activeTaskCount() {
         }
     
         public void assignActiveTasks(final Collection tasks) {
    -        activeTasks.addAll(tasks);
    +        assignedActiveTasks.taskIds().addAll(tasks);
         }
     
         public void assignActiveToConsumer(final TaskId task, final String consumer) {
    -        consumerToAssignedActiveTaskIds.computeIfAbsent(consumer, k -> new HashSet<>()).add(task);
    +        if (!assignedActiveTasks.taskIds().contains(task)) {
    +            throw new IllegalStateException("added not assign active task " + task + " to this client state.");
    +        }
    +        assignedActiveTasks.consumerToTaskIds()
    +                           .computeIfAbsent(consumer, k -> new HashSet<>()).add(task);
         }
     
         public void assignStandbyToConsumer(final TaskId task, final String consumer) {
    -        consumerToAssignedStandbyTaskIds.computeIfAbsent(consumer, k -> new HashSet<>()).add(task);
    +        assignedStandbyTasks.consumerToTaskIds().computeIfAbsent(consumer, k -> new HashSet<>()).add(task);
         }
     
         public void revokeActiveFromConsumer(final TaskId task, final String consumer) {
    -        consumerToRevokingActiveTaskIds.computeIfAbsent(consumer, k -> new HashSet<>()).add(task);
    +        revokingActiveTasks.consumerToTaskIds().computeIfAbsent(consumer, k -> new HashSet<>()).add(task);
         }
     
         public Map> prevOwnedActiveTasksByConsumer() {
    -        return consumerToPreviousActiveTaskIds;
    +        return previousActiveTasks.consumerToTaskIds();
         }
     
         public Map> prevOwnedStandbyByConsumer() {
             // standbys are just those stateful tasks minus active tasks
             final Map> consumerToPreviousStandbyTaskIds = new TreeMap<>();
    +        final Map> consumerToPreviousActiveTaskIds = previousActiveTasks.consumerToTaskIds();
     
             for (final Map.Entry> entry: consumerToPreviousStatefulTaskIds.entrySet()) {
                 final Set standbyTaskIds = new HashSet<>(entry.getValue());
    @@ -151,62 +150,66 @@ public Set prevOwnedStatefulTasksByConsumer(final String memberId) {
         }
     
         public Map> assignedActiveTasksByConsumer() {
    -        return consumerToAssignedActiveTaskIds;
    +        return assignedActiveTasks.consumerToTaskIds();
         }
     
         public Map> revokingActiveTasksByConsumer() {
    -        return consumerToRevokingActiveTaskIds;
    +        return revokingActiveTasks.consumerToTaskIds();
         }
     
         public Map> assignedStandbyTasksByConsumer() {
    -        return consumerToAssignedStandbyTaskIds;
    +        return assignedStandbyTasks.consumerToTaskIds();
         }
     
         public void assignActive(final TaskId task) {
             assertNotAssigned(task);
    -        activeTasks.add(task);
    +        assignedActiveTasks.taskIds().add(task);
         }
     
         public void unassignActive(final TaskId task) {
    -        if (!activeTasks.contains(task)) {
    +        final Set taskIds = assignedActiveTasks.taskIds();
    +        if (!taskIds.contains(task)) {
                 throw new IllegalArgumentException("Tried to unassign active task " + task + ", but it is not currently assigned: " + this);
             }
    -        activeTasks.remove(task);
    +        taskIds.remove(task);
         }
     
         public Set standbyTasks() {
    -        return unmodifiableSet(standbyTasks);
    +        return unmodifiableSet(assignedStandbyTasks.taskIds());
         }
     
         boolean hasStandbyTask(final TaskId taskId) {
    -        return standbyTasks.contains(taskId);
    +        return assignedStandbyTasks.taskIds().contains(taskId);
         }
     
         int standbyTaskCount() {
    -        return standbyTasks.size();
    +        return assignedStandbyTasks.taskIds().size();
         }
     
         public void assignStandby(final TaskId task) {
             assertNotAssigned(task);
    -        standbyTasks.add(task);
    +        assignedStandbyTasks.taskIds().add(task);
         }
     
         void unassignStandby(final TaskId task) {
    -        if (!standbyTasks.contains(task)) {
    +        final Set taskIds = assignedStandbyTasks.taskIds();
    +        if (!taskIds.contains(task)) {
                 throw new IllegalArgumentException("Tried to unassign standby task " + task + ", but it is not currently assigned: " + this);
             }
    -        standbyTasks.remove(task);
    +        taskIds.remove(task);
         }
     
         Set assignedTasks() {
    +        final Set assignedActiveTaskIds = assignedActiveTasks.taskIds();
    +        final Set assignedStandbyTaskIds = assignedStandbyTasks.taskIds();
             // Since we're copying it, it's not strictly necessary to make it unmodifiable also.
             // I'm just trying to prevent subtle bugs if we write code that thinks it can update
             // the assignment by updating the returned set.
             return unmodifiableSet(
                 union(
    -                () -> new HashSet<>(activeTasks.size() + standbyTasks.size()),
    -                activeTasks,
    -                standbyTasks
    +                () -> new HashSet<>(assignedActiveTaskIds.size() + assignedStandbyTaskIds.size()),
    +                assignedActiveTaskIds,
    +                assignedStandbyTaskIds
                 )
             );
         }
    @@ -220,35 +223,39 @@ public int assignedTaskCount() {
         }
     
         boolean hasAssignedTask(final TaskId taskId) {
    -        return activeTasks.contains(taskId) || standbyTasks.contains(taskId);
    +        return assignedActiveTasks.taskIds().contains(taskId) || assignedStandbyTasks.taskIds().contains(taskId);
         }
     
         Set prevActiveTasks() {
    -        return unmodifiableSet(prevActiveTasks);
    +        return unmodifiableSet(previousActiveTasks.taskIds());
         }
     
         private void addPreviousActiveTask(final TaskId task) {
    -        prevActiveTasks.add(task);
    +        previousActiveTasks.taskIds().add(task);
         }
     
         void addPreviousActiveTasks(final Set prevTasks) {
    -        prevActiveTasks.addAll(prevTasks);
    +        previousActiveTasks.taskIds().addAll(prevTasks);
         }
     
         Set prevStandbyTasks() {
    -        return unmodifiableSet(prevStandbyTasks);
    +        return unmodifiableSet(previousStandbyTasks.taskIds());
         }
     
         private void addPreviousStandbyTask(final TaskId task) {
    -        prevStandbyTasks.add(task);
    +        previousStandbyTasks.taskIds().add(task);
         }
     
         void addPreviousStandbyTasks(final Set standbyTasks) {
    -        prevStandbyTasks.addAll(standbyTasks);
    +        previousStandbyTasks.taskIds().addAll(standbyTasks);
         }
     
         Set previousAssignedTasks() {
    -        return union(() -> new HashSet<>(prevActiveTasks.size() + prevStandbyTasks.size()), prevActiveTasks, prevStandbyTasks);
    +        final Set previousActiveTaskIds = previousActiveTasks.taskIds();
    +        final Set previousStandbyTaskIds = previousStandbyTasks.taskIds();
    +        return union(() -> new HashSet<>(previousActiveTaskIds.size() + previousStandbyTaskIds.size()),
    +                     previousActiveTaskIds,
    +                     previousStandbyTaskIds);
         }
     
         // May return null
    @@ -268,7 +275,7 @@ public void addPreviousTasksAndOffsetSums(final String consumerId, final Map taskForPartitionMap) {
    -        if (!prevActiveTasks.isEmpty() || !prevStandbyTasks.isEmpty()) {
    +        if (!previousActiveTasks.taskIds().isEmpty() || !previousStandbyTasks.taskIds().isEmpty()) {
                 throw new IllegalStateException("Already added previous tasks to this client state.");
             }
             initializePrevActiveTasksFromOwnedPartitions(taskForPartitionMap);
    @@ -321,15 +328,15 @@ public long lagFor(final TaskId task) {
         }
     
         public Set statefulActiveTasks() {
    -        return activeTasks.stream().filter(this::isStateful).collect(Collectors.toSet());
    +        return assignedActiveTasks.taskIds().stream().filter(this::isStateful).collect(Collectors.toSet());
         }
     
         public Set statelessActiveTasks() {
    -        return activeTasks.stream().filter(task -> !isStateful(task)).collect(Collectors.toSet());
    +        return assignedActiveTasks.taskIds().stream().filter(task -> !isStateful(task)).collect(Collectors.toSet());
         }
     
         boolean hasUnfulfilledQuota(final int tasksPerThread) {
    -        return activeTasks.size() < capacity * tasksPerThread;
    +        return assignedActiveTasks.taskIds().size() < capacity * tasksPerThread;
         }
     
         boolean hasMoreAvailableCapacityThan(final ClientState other) {
    @@ -354,21 +361,21 @@ boolean hasMoreAvailableCapacityThan(final ClientState other) {
         }
     
         public String currentAssignment() {
    -        return "[activeTasks: (" + activeTasks +
    -                ") standbyTasks: (" + standbyTasks + ")]";
    +        return "[activeTasks: (" + assignedActiveTasks.taskIds() +
    +               ") standbyTasks: (" + assignedStandbyTasks.taskIds() + ")]";
         }
     
         @Override
         public String toString() {
    -        return "[activeTasks: (" + activeTasks +
    -            ") standbyTasks: (" + standbyTasks +
    -            ") prevActiveTasks: (" + prevActiveTasks +
    -            ") prevStandbyTasks: (" + prevStandbyTasks +
    -            ") changelogOffsetTotalsByTask: (" + taskOffsetSums.entrySet() +
    -            ") taskLagTotals: (" + taskLagTotals.entrySet() +
    -            ") capacity: " + capacity +
    -            " assigned: " + assignedTaskCount() +
    -            "]";
    +        return "[activeTasks: (" + assignedActiveTasks.taskIds() +
    +               ") standbyTasks: (" + assignedStandbyTasks.taskIds() +
    +               ") prevActiveTasks: (" + previousActiveTasks.taskIds() +
    +               ") prevStandbyTasks: (" + previousStandbyTasks.taskIds() +
    +               ") changelogOffsetTotalsByTask: (" + taskOffsetSums.entrySet() +
    +               ") taskLagTotals: (" + taskLagTotals.entrySet() +
    +               ") capacity: " + capacity +
    +               " assigned: " + assignedTaskCount() +
    +               "]";
         }
     
         private boolean isStateful(final TaskId task) {
    @@ -385,7 +392,7 @@ private void initializePrevActiveTasksFromOwnedPartitions(final Map new HashSet<>()).add(task);
    +                previousActiveTasks.consumerToTaskIds().computeIfAbsent(partitionEntry.getValue(), k -> new HashSet<>()).add(task);
                 } else {
                     LOG.error("No task found for topic partition {}", tp);
                 }
    @@ -393,13 +400,14 @@ private void initializePrevActiveTasksFromOwnedPartitions(final Map previousActiveTaskIds = previousActiveTasks.taskIds();
    +        if (previousActiveTaskIds.isEmpty() && !ownedPartitions.isEmpty()) {
                 LOG.error("Tried to process tasks in offset sum map before processing tasks from ownedPartitions = {}", ownedPartitions);
                 throw new IllegalStateException("Must initialize prevActiveTasks from ownedPartitions before initializing remaining tasks.");
             }
             for (final Map.Entry taskEntry : taskOffsetSums.entrySet()) {
                 final TaskId task = taskEntry.getKey();
    -            if (!prevActiveTasks.contains(task)) {
    +            if (!previousActiveTaskIds.contains(task)) {
                     final long offsetSum = taskEntry.getValue();
                     if (offsetSum == Task.LATEST_OFFSET) {
                         addPreviousActiveTask(task);
    @@ -411,7 +419,7 @@ private void initializeRemainingPrevTasksFromTaskOffsetSums() {
         }
     
         private void assertNotAssigned(final TaskId task) {
    -        if (standbyTasks.contains(task) || activeTasks.contains(task)) {
    +        if (assignedStandbyTasks.taskIds().contains(task) || assignedActiveTasks.taskIds().contains(task)) {
                 throw new IllegalArgumentException("Tried to assign task " + task + ", but it is already assigned: " + this);
             }
         }
    diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java
    new file mode 100644
    index 0000000000000..92769699ccccd
    --- /dev/null
    +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTask.java
    @@ -0,0 +1,45 @@
    +/*
    + * 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.streams.processor.internals.assignment;
    +
    +import java.util.Map;
    +import java.util.Set;
    +
    +import org.apache.kafka.streams.processor.TaskId;
    +
    +class ClientStateTask {
    +    private final Map> consumerToTaskIds;
    +    private Set taskIds;
    +
    +    ClientStateTask(final Set taskIds,
    +                    final Map> consumerToTaskIds) {
    +        this.taskIds = taskIds;
    +        this.consumerToTaskIds = consumerToTaskIds;
    +    }
    +
    +    void taskIds(final Set clientToTaskIds) {
    +        taskIds = clientToTaskIds;
    +    }
    +
    +    Set taskIds() {
    +        return taskIds;
    +    }
    +
    +    Map> consumerToTaskIds() {
    +        return consumerToTaskIds;
    +    }
    +}
    diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java
    index 952a9904c6aa1..e1853dcf94294 100644
    --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java
    +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/ClientStateTest.java
    @@ -21,10 +21,13 @@
     import org.apache.kafka.streams.processor.internals.Task;
     import org.junit.Test;
     
    +import java.util.ArrayList;
     import java.util.Collections;
    +import java.util.List;
     import java.util.Map;
     import java.util.Set;
     
    +import static java.util.Arrays.asList;
     import static org.apache.kafka.common.utils.Utils.mkEntry;
     import static org.apache.kafka.common.utils.Utils.mkMap;
     import static org.apache.kafka.common.utils.Utils.mkSet;
    @@ -364,6 +367,9 @@ public void shouldReturnPreviousActiveStandbyTasksForConsumer() {
     
         @Test
         public void shouldReturnAssignedTasksForConsumer() {
    +        final List allTasks = new ArrayList<>(asList(TASK_0_0, TASK_0_1, TASK_0_2));
    +        client.assignActiveTasks(allTasks);
    +
             client.assignActiveToConsumer(TASK_0_0, "c1");
             // calling it multiple tasks should be idempotent
             client.assignActiveToConsumer(TASK_0_0, "c1");
    @@ -479,4 +485,9 @@ public void shouldThrowIllegalStateExceptionIfAttemptingToInitializeNonEmptyPrev
             assertThrows(IllegalStateException.class, () -> client.initializePrevTasks(Collections.emptyMap()));
         }
     
    +    @Test
    +    public void shouldThrowIllegalStateExceptionIfAssignedTasksForConsumerToNonClientAssignActive() {
    +        assertThrows(IllegalStateException.class, () -> client.assignActiveToConsumer(TASK_0_0, "c1"));
    +    }
    +
     }
    
    From c972ac929edaa32f30dae19714180cdd23addeeb Mon Sep 17 00:00:00 2001
    From: Vito Jeng 
    Date: Sat, 24 Apr 2021 03:09:32 +0800
    Subject: [PATCH 087/155] KAFKA-5876: Apply UnknownStateStoreException for
     Interactive Queries (#9821)
    
    KIP-216: IQ should throw different exceptions for different errors, Part 2
    
    Reviewers: Matthias J. Sax , Anna Sophie Blee-Goldman , Bruno Cadonna 
    ---
     docs/streams/upgrade-guide.html                        |  3 +++
     .../java/org/apache/kafka/streams/KafkaStreams.java    |  7 ++++---
     .../org/apache/kafka/streams/KafkaStreamsTest.java     | 10 ++++++++++
     .../streams/integration/JoinStoreIntegrationTest.java  |  6 +++---
     .../integration/QueryableStateIntegrationTest.java     |  5 +++--
     5 files changed, 23 insertions(+), 8 deletions(-)
    
    diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html
    index ea410b13e331c..cd0f67773d244 100644
    --- a/docs/streams/upgrade-guide.html
    +++ b/docs/streams/upgrade-guide.html
    @@ -93,6 +93,9 @@ 

    Upgrade Guide and API Changes

    Streams API changes in 3.0.0

    +

    + A new exception may be thrown from KafkaStreams#store(). If the specified store name does not exist in the topology, an UnknownStateStoreException will be thrown instead of the former InvalidStateStoreException. See KIP-216 for more information. +

    We removed the default implementation of RocksDBConfigSetter#close().

    diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index c1b4369d503a0..b7eda15dc8e24 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -44,6 +44,7 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; +import org.apache.kafka.streams.errors.UnknownStateStoreException; import org.apache.kafka.streams.internals.metrics.ClientMetrics; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.StateRestoreListener; @@ -1515,8 +1516,8 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, * * @param storeQueryParameters the parameters used to fetch a queryable store * @return A facade wrapping the local {@link StateStore} instances - * @throws InvalidStateStoreException If the specified store name does not exist in the topology - * or if the Streams instance isn't in a queryable state. + * @throws UnknownStateStoreException If the specified store name does not exist in the topology. + * @throws InvalidStateStoreException If the Streams instance isn't in a queryable state. * If the store's type does not match the QueryableStoreType, * the Streams instance is not in a queryable state with respect * to the parameters, or if the store is not available locally, then @@ -1526,7 +1527,7 @@ public T store(final StoreQueryParameters storeQueryParameters) { final String storeName = storeQueryParameters.storeName(); if ((taskTopology == null || !taskTopology.hasStore(storeName)) && (globalTaskTopology == null || !globalTaskTopology.hasStore(storeName))) { - throw new InvalidStateStoreException( + throw new UnknownStateStoreException( "Cannot get state store " + storeName + " because no such store is registered in the topology." ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index b3dd559d88d63..82e8d6b2425c7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; +import org.apache.kafka.streams.errors.UnknownStateStoreException; import org.apache.kafka.streams.internals.metrics.ClientMetrics; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.processor.StateRestoreListener; @@ -91,6 +92,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; +import static org.apache.kafka.streams.state.QueryableStoreTypes.keyValueStore; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.easymock.EasyMock.anyInt; import static org.easymock.EasyMock.anyLong; @@ -745,6 +747,14 @@ public void shouldNotGetQueryMetadataWithPartitionerWhenNotRunningOrRebalancing( assertThrows(IllegalStateException.class, () -> streams.queryMetadataForKey("store", "key", (topic, key, value, numPartitions) -> 0)); } + @Test + public void shouldThrowUnknownStateStoreExceptionWhenStoreNotExist() { + try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { + streams.start(); + assertThrows(UnknownStateStoreException.class, () -> streams.store(StoreQueryParameters.fromNameAndType("unknown-store", keyValueStore()))); + } + } + @Test public void shouldReturnEmptyLocalStorePartitionLags() { // Mock all calls made to compute the offset lags, diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinStoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/JoinStoreIntegrationTest.java index c46af43bac1b0..f5ed891afc1f2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinStoreIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/JoinStoreIntegrationTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.errors.UnknownStateStoreException; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.JoinWindows; @@ -119,9 +119,9 @@ public void providingAJoinStoreNameShouldNotMakeTheJoinResultQueriable() throws kafkaStreams.start(); latch.await(); - final InvalidStateStoreException exception = + final UnknownStateStoreException exception = assertThrows( - InvalidStateStoreException.class, + UnknownStateStoreException.class, () -> kafkaStreams.store(fromNameAndType("join-store", keyValueStore())) ); assertThat( diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index f023977e309da..cd1dd0dca61d1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -37,6 +37,7 @@ import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.errors.UnknownStateStoreException; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.Consumed; @@ -470,8 +471,8 @@ public void shouldRejectNonExistentStoreName() throws InterruptedException { streams.store(fromNameAndType(storeName, keyValueStore())); assertThat(store, Matchers.notNullValue()); - final InvalidStateStoreException exception = assertThrows( - InvalidStateStoreException.class, + final UnknownStateStoreException exception = assertThrows( + UnknownStateStoreException.class, () -> streams.store(fromNameAndType("no-table", keyValueStore())) ); assertThat( From ea076a8d6289fa8e94e7e6876437b565f9a9f9c3 Mon Sep 17 00:00:00 2001 From: Luke Chen <43372967+showuon@users.noreply.github.com> Date: Sun, 25 Apr 2021 20:55:10 +0800 Subject: [PATCH 088/155] KAFKA-10746: Change to Warn logs when necessary to notify users (#9627) Reviewers: A. Sophie Blee-Goldman , Chia-Ping Tsai --- .../consumer/internals/AbstractCoordinator.java | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index ba26427eb9633..cd1daa84c8ad5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -1023,6 +1023,10 @@ protected void close(Timer timer) { } /** + * Sends LeaveGroupRequest and logs the {@code leaveReason}, unless this member is using static membership or is already + * not part of the group (ie does not have a valid member id, is in the UNJOINED state, or the coordinator is unknown). + * + * @param leaveReason the reason to leave the group for logging * @throws KafkaException if the rebalance callback throws exception */ public synchronized RequestFuture maybeLeaveGroup(String leaveReason) { @@ -1381,12 +1385,13 @@ public void run() { } else if (heartbeat.pollTimeoutExpired(now)) { // the poll timeout has expired, which means that the foreground thread has stalled // in between calls to poll(). - String leaveReason = "consumer poll timeout has expired. This means the time between subsequent calls to poll() " + - "was longer than the configured max.poll.interval.ms, which typically implies that " + - "the poll loop is spending too much time processing messages. " + - "You can address this either by increasing max.poll.interval.ms or by reducing " + - "the maximum size of batches returned in poll() with max.poll.records."; - maybeLeaveGroup(leaveReason); + log.warn("consumer poll timeout has expired. This means the time between subsequent calls to poll() " + + "was longer than the configured max.poll.interval.ms, which typically implies that " + + "the poll loop is spending too much time processing messages. You can address this " + + "either by increasing max.poll.interval.ms or by reducing the maximum size of batches " + + "returned in poll() with max.poll.records."); + + maybeLeaveGroup("consumer poll timeout has expired."); } else if (!heartbeat.shouldHeartbeat(now)) { // poll again after waiting for the retry backoff in case the heartbeat failed or the // coordinator disconnected From 031b7208b3b54ab03c6d93fa617967d2f1f8abe5 Mon Sep 17 00:00:00 2001 From: wenbingshen Date: Sun, 25 Apr 2021 21:15:50 +0800 Subject: [PATCH 089/155] KAFKA-12684: Fix noop set is incorrectly replaced with succeeded set from LeaderElectionCommand (#10558) Reviewers: David Jacot , Chia-Ping Tsai --- .../kafka/admin/LeaderElectionCommand.scala | 2 +- .../admin/LeaderElectionCommandTest.scala | 42 +++++++++++++++++++ 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala index 03737aa532294..92edcad003f33 100644 --- a/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala +++ b/core/src/main/scala/kafka/admin/LeaderElectionCommand.scala @@ -166,7 +166,7 @@ object LeaderElectionCommand extends Logging { } if (noop.nonEmpty) { - val partitions = succeeded.mkString(", ") + val partitions = noop.mkString(", ") println(s"Valid replica already elected for partitions $partitions") } diff --git a/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala b/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala index b95d69d7f735a..a7b91ea146a2f 100644 --- a/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/LeaderElectionCommandTest.scala @@ -273,6 +273,48 @@ final class LeaderElectionCommandTest extends ZooKeeperTestHarness { )) assertTrue(e.getCause.isInstanceOf[TimeoutException]) } + + @Test + def testElectionResultOutput(): Unit = { + TestUtils.resource(Admin.create(createConfig(servers).asJava)) { client => + val topic = "non-preferred-topic" + val partition0 = 0 + val partition1 = 1 + val assignment0 = Seq(broker2, broker3) + val assignment1 = Seq(broker3, broker2) + + TestUtils.createTopic(zkClient, topic, Map(partition0 -> assignment0, partition1 -> assignment1), servers) + + val topicPartition0 = new TopicPartition(topic, partition0) + val topicPartition1 = new TopicPartition(topic, partition1) + + TestUtils.assertLeader(client, topicPartition0, broker2) + TestUtils.assertLeader(client, topicPartition1, broker3) + + servers(broker2).shutdown() + TestUtils.assertLeader(client, topicPartition0, broker3) + servers(broker2).startup() + TestUtils.waitForBrokersInIsr(client, topicPartition0, Set(broker2)) + TestUtils.waitForBrokersInIsr(client, topicPartition1, Set(broker2)) + + val topicPartitionPath = tempTopicPartitionFile(Set(topicPartition0, topicPartition1)) + val output = TestUtils.grabConsoleOutput( + LeaderElectionCommand.main( + Array( + "--bootstrap-server", bootstrapServers(servers), + "--election-type", "preferred", + "--path-to-json-file", topicPartitionPath.toString + ) + ) + ) + + val electionResultOutputIter = output.split("\n").iterator + assertTrue(electionResultOutputIter.hasNext) + assertTrue(electionResultOutputIter.next().contains(s"Successfully completed leader election (PREFERRED) for partitions $topicPartition0")) + assertTrue(electionResultOutputIter.hasNext) + assertTrue(electionResultOutputIter.next().contains(s"Valid replica already elected for partitions $topicPartition1")) + } + } } object LeaderElectionCommandTest { From d949f1094e2dc270ecbdd7b84aa8470690c697e4 Mon Sep 17 00:00:00 2001 From: ketulgupta1995 Date: Tue, 27 Apr 2021 00:08:48 +0530 Subject: [PATCH 090/155] KAFKA-12344 Support SlidingWindows in the Scala API (#10519) Support SlidingWindows in the Scala API Reviewers: Leah Thomas , Anna Sophie Blee-Goldman --- .../scala/kstream/CogroupedKStream.scala | 18 ++++++- .../scala/kstream/KGroupedStream.scala | 11 ++++ .../streams/scala/kstream/KTableTest.scala | 50 ++++++++++++++++++- 3 files changed, 76 insertions(+), 3 deletions(-) diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/CogroupedKStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/CogroupedKStream.scala index c23deb8ddeb9d..f4fe9fc5ca852 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/CogroupedKStream.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/CogroupedKStream.scala @@ -17,7 +17,13 @@ package org.apache.kafka.streams.scala package kstream -import org.apache.kafka.streams.kstream.{CogroupedKStream => CogroupedKStreamJ, SessionWindows, Window, Windows} +import org.apache.kafka.streams.kstream.{ + SessionWindows, + SlidingWindows, + Window, + Windows, + CogroupedKStream => CogroupedKStreamJ +} import org.apache.kafka.streams.scala.FunctionsCompatConversions.{AggregatorFromFunction, InitializerFromFunction} /** @@ -82,6 +88,16 @@ class CogroupedKStream[KIn, VOut](val inner: CogroupedKStreamJ[KIn, VOut]) { def windowedBy[W <: Window](windows: Windows[W]): TimeWindowedCogroupedKStream[KIn, VOut] = new TimeWindowedCogroupedKStream(inner.windowedBy(windows)) + /** + * Create a new [[TimeWindowedCogroupedKStream]] instance that can be used to perform sliding windowed aggregations. + * + * @param windows the specification of the aggregation `SlidingWindows` + * @return an instance of [[TimeWindowedCogroupedKStream]] + * @see `org.apache.kafka.streams.kstream.CogroupedKStream#windowedBy` + */ + def windowedBy(windows: SlidingWindows): TimeWindowedCogroupedKStream[KIn, VOut] = + new TimeWindowedCogroupedKStream(inner.windowedBy(windows)) + /** * Create a new [[SessionWindowedKStream]] instance that can be used to perform session windowed aggregations. * diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala index d3f1ea096b25a..44a3e568d859e 100644 --- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala +++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KGroupedStream.scala @@ -21,6 +21,7 @@ import org.apache.kafka.streams.kstream.internals.KTableImpl import org.apache.kafka.streams.scala.serialization.Serdes import org.apache.kafka.streams.kstream.{ SessionWindows, + SlidingWindows, Window, Windows, KGroupedStream => KGroupedStreamJ, @@ -155,6 +156,16 @@ class KGroupedStream[K, V](val inner: KGroupedStreamJ[K, V]) { def windowedBy[W <: Window](windows: Windows[W]): TimeWindowedKStream[K, V] = new TimeWindowedKStream(inner.windowedBy(windows)) + /** + * Create a new [[TimeWindowedKStream]] instance that can be used to perform sliding windowed aggregations. + * + * @param windows the specification of the aggregation `SlidingWindows` + * @return an instance of [[TimeWindowedKStream]] + * @see `org.apache.kafka.streams.kstream.KGroupedStream#windowedBy` + */ + def windowedBy(windows: SlidingWindows): TimeWindowedKStream[K, V] = + new TimeWindowedKStream(inner.windowedBy(windows)) + /** * Create a new [[SessionWindowedKStream]] instance that can be used to perform session windowed aggregations. * diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala index ccf955e8889b1..15e090dc8f931 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/kstream/KTableTest.scala @@ -17,15 +17,23 @@ package org.apache.kafka.streams.scala.kstream import org.apache.kafka.streams.kstream.Suppressed.BufferConfig -import org.apache.kafka.streams.kstream.{Named, SessionWindows, TimeWindows, Windowed, Suppressed => JSuppressed} +import org.apache.kafka.streams.kstream.{ + Named, + SlidingWindows, + SessionWindows, + TimeWindows, + Windowed, + Suppressed => JSuppressed +} import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.serialization.Serdes._ import org.apache.kafka.streams.scala.utils.TestDriver import org.apache.kafka.streams.scala.{ByteArrayKeyValueStore, StreamsBuilder} import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} import org.junit.jupiter.api.Test - import java.time.Duration +import java.time.Duration.ofMillis + import scala.jdk.CollectionConverters._ class KTableTest extends TestDriver { @@ -211,6 +219,44 @@ class KTableTest extends TestDriver { testDriver.close() } + @Test + def testCorrectlyGroupByKeyWindowedBySlidingWindow(): Unit = { + val builder = new StreamsBuilder() + val sourceTopic = "source" + val sinkTopic = "sink" + val window = SlidingWindows.withTimeDifferenceAndGrace(ofMillis(1000L), ofMillis(1000L)) + val suppression = JSuppressed.untilWindowCloses(BufferConfig.unbounded()) + + val table: KTable[Windowed[String], Long] = builder + .stream[String, String](sourceTopic) + .groupByKey + .windowedBy(window) + .count() + .suppress(suppression) + + table.toStream((k, _) => s"${k.window().start()}:${k.window().end()}:${k.key()}").to(sinkTopic) + + val testDriver = createTestDriver(builder) + val testInput = testDriver.createInput[String, String](sourceTopic) + val testOutput = testDriver.createOutput[String, Long](sinkTopic) + + { + // publish key=1 @ time 0 => count==1 + testInput.pipeInput("1", "value1", 0L) + assertTrue(testOutput.isEmpty) + } + { + // move event time right past the grace period of the first window. + testInput.pipeInput("2", "value3", 5001L) + val record = testOutput.readKeyValue + assertEquals("0:1000:1", record.key) + assertEquals(1L, record.value) + } + assertTrue(testOutput.isEmpty) + + testDriver.close() + } + @Test def testCorrectlySuppressResultsUsingSuppressedUntilWindowClosesByWindowed(): Unit = { val builder = new StreamsBuilder() From bcfe91035657e471fbc041adf26d548da24c2b0e Mon Sep 17 00:00:00 2001 From: dengziming Date: Tue, 27 Apr 2021 06:41:00 +0800 Subject: [PATCH 091/155] MINOR: Remove redudant test files and close LogSegment after test (#10592) Reivewers: Kowshik Prakasam , Jun Rao --- .../unit/kafka/log/LogSegmentsTest.scala | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala index b6e686b02709d..b929b9c4b8a52 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentsTest.scala @@ -18,10 +18,11 @@ package kafka.log import java.io.File +import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.utils.Time +import org.apache.kafka.common.utils.{Time, Utils} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} class LogSegmentsTest { @@ -35,6 +36,16 @@ class LogSegmentsTest { LogTestUtils.createSegment(offset, logDir, indexIntervalBytes, time) } + @BeforeEach + def setup(): Unit = { + logDir = TestUtils.tempDir() + } + + @AfterEach + def teardown(): Unit = { + Utils.delete(logDir) + } + private def assertEntry(segment: LogSegment, tested: java.util.Map.Entry[java.lang.Long, LogSegment]): Unit = { assertEquals(segment.baseOffset, tested.getKey()) assertEquals(segment, tested.getValue()) @@ -89,11 +100,14 @@ class LogSegmentsTest { assertFalse(segments.nonEmpty) assertEquals(0, segments.numberOfSegments) assertFalse(segments.contains(offset1)) + + segments.close() } @Test def testSegmentAccess(): Unit = { val segments = new LogSegments(topicPartition) + val offset1 = 1 val seg1 = createSegment(offset1) val offset2 = 2 @@ -130,11 +144,13 @@ class LogSegmentsTest { assertEquals(Seq(), segments.values(4, 4).toSeq) assertEquals(Seq(seg4), segments.values(4, 5).toSeq) + segments.close() } @Test def testClosestMatchOperations(): Unit = { val segments = new LogSegments(topicPartition) + val seg1 = createSegment(1) val seg2 = createSegment(3) val seg3 = createSegment(5) @@ -159,5 +175,7 @@ class LogSegmentsTest { assertEntry(seg3, segments.higherEntry(4).get) assertEquals(Some(seg4), segments.higherSegment(5)) assertEntry(seg4, segments.higherEntry(5).get) + + segments.close() } } From 0ea440b2af07fe0d6465f76afc948b28509606b5 Mon Sep 17 00:00:00 2001 From: JoelWee <32009741+JoelWee@users.noreply.github.com> Date: Tue, 27 Apr 2021 23:44:53 +0100 Subject: [PATCH 092/155] KAFKA-6435: KIP-623 Add internal topics option to streamResetter (#8923) Allow user to specify subset of internal topics to clean up with application reset tool Reviewers: Boyang Chen , Anna Sophie Blee-Goldman , Walker Carlson --- .../scala/kafka/tools/StreamsResetter.java | 54 +++++++++++++------ .../developer-guide/app-reset-tool.html | 6 +++ .../AbstractResetIntegrationTest.java | 38 +++++++++++-- .../integration/ResetIntegrationTest.java | 32 +++++++++++ .../streams/tools/StreamsResetterTest.java | 8 +-- 5 files changed, 114 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/kafka/tools/StreamsResetter.java b/core/src/main/scala/kafka/tools/StreamsResetter.java index 485bcdf838f50..8c5150ab3c8c1 100644 --- a/core/src/main/scala/kafka/tools/StreamsResetter.java +++ b/core/src/main/scala/kafka/tools/StreamsResetter.java @@ -95,6 +95,7 @@ public class StreamsResetter { private static OptionSpec applicationIdOption; private static OptionSpec inputTopicsOption; private static OptionSpec intermediateTopicsOption; + private static OptionSpec internalTopicsOption; private static OptionSpec toOffsetOption; private static OptionSpec toDatetimeOption; private static OptionSpec byDurationOption; @@ -114,7 +115,8 @@ public class StreamsResetter { + "intermediate topics (topics that are input and output topics, e.g., used by deprecated through() method).\n" + "* This tool deletes the internal topics that were created by Kafka Streams (topics starting with " + "\"-\").\n" - + "You do not need to specify internal topics because the tool finds them automatically.\n" + + "The tool finds these internal topics automatically. If the topics flagged automatically for deletion by " + + "the dry-run are unsuitable, you can specify a subset with the \"--internal-topics\" option.\n" + "* This tool will not delete output topics (if you want to delete them, you need to do it yourself " + "with the bin/kafka-topics.sh command).\n" + "* This tool will not clean up the local state on the stream application instances (the persisted " @@ -126,7 +128,9 @@ public class StreamsResetter { + "members immediately. Make sure to stop all stream applications when this option is specified " + "to avoid unexpected disruptions.\n\n" + "*** Important! You will get wrong output if you don't clean up the local stores after running the " - + "reset tool!\n\n"; + + "reset tool!\n\n" + + "*** Warning! This tool makes irreversible changes to your application. It is strongly recommended that " + + "you run this once with \"--dry-run\" to preview your changes before making them.\n\n"; private OptionSet options = null; private final List allTopics = new LinkedList<>(); @@ -166,7 +170,7 @@ public int run(final String[] args, final HashMap consumerConfig = new HashMap<>(config); consumerConfig.putAll(properties); exitCode = maybeResetInputAndSeekToEndIntermediateTopicOffsets(consumerConfig, dryRun); - maybeDeleteInternalTopics(adminClient, dryRun); + exitCode |= maybeDeleteInternalTopics(adminClient, dryRun); } catch (final Throwable e) { exitCode = EXIT_CODE_ERROR; System.err.println("ERROR: " + e); @@ -224,6 +228,13 @@ private void parseArguments(final String[] args) { .ofType(String.class) .withValuesSeparatedBy(',') .describedAs("list"); + internalTopicsOption = optionParser.accepts("internal-topics", "Comma-separated list of " + + "internal topics to delete. Must be a subset of the internal topics marked for deletion by the " + + "default behaviour (do a dry-run without this option to view these topics).") + .withRequiredArg() + .ofType(String.class) + .withValuesSeparatedBy(',') + .describedAs("list"); toOffsetOption = optionParser.accepts("to-offset", "Reset offsets to a specific offset.") .withRequiredArg() .ofType(Long.class); @@ -608,22 +619,35 @@ private boolean isIntermediateTopic(final String topic) { return options.valuesOf(intermediateTopicsOption).contains(topic); } - private void maybeDeleteInternalTopics(final Admin adminClient, final boolean dryRun) { - System.out.println("Deleting all internal/auto-created topics for application " + options.valueOf(applicationIdOption)); - final List topicsToDelete = new ArrayList<>(); - for (final String listing : allTopics) { - if (isInternalTopic(listing)) { - if (!dryRun) { - topicsToDelete.add(listing); - } else { - System.out.println("Topic: " + listing); - } + private int maybeDeleteInternalTopics(final Admin adminClient, final boolean dryRun) { + final List inferredInternalTopics = allTopics.stream() + .filter(this::isInferredInternalTopic) + .collect(Collectors.toList()); + final List specifiedInternalTopics = options.valuesOf(internalTopicsOption); + final List topicsToDelete; + + if (!specifiedInternalTopics.isEmpty()) { + if (!inferredInternalTopics.containsAll(specifiedInternalTopics)) { + throw new IllegalArgumentException("Invalid topic specified in the " + + "--internal-topics option. " + + "Ensure that the topics specified are all internal topics. " + + "Do a dry run without the --internal-topics option to see the " + + "list of all internal topics that can be deleted."); } + + topicsToDelete = specifiedInternalTopics; + System.out.println("Deleting specified internal topics " + topicsToDelete); + } else { + topicsToDelete = inferredInternalTopics; + System.out.println("Deleting inferred internal topics " + topicsToDelete); } + if (!dryRun) { doDelete(topicsToDelete, adminClient); } + System.out.println("Done."); + return EXIT_CODE_SUCCESS; } // visible for testing @@ -647,7 +671,7 @@ public void doDelete(final List topicsToDelete, } } - private boolean isInternalTopic(final String topicName) { + private boolean isInferredInternalTopic(final String topicName) { // Specified input/intermediate topics might be named like internal topics (by chance). // Even is this is not expected in general, we need to exclude those topics here // and don't consider them as internal topics even if they follow the same naming schema. @@ -657,7 +681,7 @@ private boolean isInternalTopic(final String topicName) { } // visible for testing - public boolean matchesInternalTopicFormat(final String topicName) { + public static boolean matchesInternalTopicFormat(final String topicName) { return topicName.endsWith("-changelog") || topicName.endsWith("-repartition") || topicName.endsWith("-subscription-registration-topic") || topicName.endsWith("-subscription-response-topic") diff --git a/docs/streams/developer-guide/app-reset-tool.html b/docs/streams/developer-guide/app-reset-tool.html index 1b90af0cb801b..d6d07c27c97f2 100644 --- a/docs/streams/developer-guide/app-reset-tool.html +++ b/docs/streams/developer-guide/app-reset-tool.html @@ -77,6 +77,7 @@

    Step 1: Run the application reset tool

    Invoke the application reset tool from the command line

    +

    Warning! This tool makes irreversible changes to your application. It is strongly recommended that you run this once with --dry-run to preview your changes before making them.

    <path-to-kafka>/bin/kafka-streams-application-reset

    The tool accepts the following parameters:

    @@ -105,6 +106,11 @@

    Step 1: Run the application reset tool(topics used in the through() method). For these topics, the tool will skip to the end. +--internal-topics <String: list> Comma-separated list of internal topics + to delete. Must be a subset of the + internal topics marked for deletion by + the default behaviour (do a dry-run without + this option to view these topics). --shift-by <Long: number-of-offsets> Reset offsets shifting current offset by 'n', where 'n' can be positive or negative diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java index 76c53b15443de..e672ca1f8f3d6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractResetIntegrationTest.java @@ -60,6 +60,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.stream.Collectors; import static java.time.Duration.ofMillis; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForEmptyConsumerGroup; @@ -77,7 +78,7 @@ public abstract class AbstractResetIntegrationTest { abstract Map getClientSslConfig(); @Rule - public final TestName testName = new TestName(); + public final TestName testName = new TestName(); @AfterClass public static void afterClassCleanup() { @@ -205,6 +206,34 @@ private void add10InputElements() { } } + @Test + public void testResetWhenInternalTopicsAreSpecified() throws Exception { + final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); + streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, appID); + + // RUN + streams = new KafkaStreams(setupTopologyWithIntermediateTopic(true, OUTPUT_TOPIC_2), streamsConfig); + streams.start(); + IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(resultConsumerConfig, OUTPUT_TOPIC, 10); + + streams.close(); + waitForEmptyConsumerGroup(adminClient, appID, TIMEOUT_MULTIPLIER * STREAMS_CONSUMER_TIMEOUT); + + // RESET + streams.cleanUp(); + + final List internalTopics = cluster.getAllTopicsInCluster().stream() + .filter(topic -> StreamsResetter.matchesInternalTopicFormat(topic)) + .collect(Collectors.toList()); + cleanGlobal(false, + "--internal-topics", + String.join(",", internalTopics.subList(1, internalTopics.size())), + appID); + waitForEmptyConsumerGroup(adminClient, appID, TIMEOUT_MULTIPLIER * STREAMS_CONSUMER_TIMEOUT); + + assertInternalTopicsGotDeleted(internalTopics.get(0)); + } + @Test public void testReprocessingFromScratchAfterResetWithoutIntermediateUserTopic() throws Exception { final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); @@ -358,7 +387,6 @@ protected boolean tryCleanGlobal(final boolean withIntermediateTopics, final String resetScenario, final String resetScenarioArg, final String appID) throws Exception { - // leaving --zookeeper arg here to ensure tool works if users add it final List parameterList = new ArrayList<>( Arrays.asList("--application-id", appID, "--bootstrap-servers", cluster.bootstrapServers(), @@ -405,11 +433,11 @@ protected void cleanGlobal(final boolean withIntermediateTopics, Assert.assertTrue(cleanResult); } - protected void assertInternalTopicsGotDeleted(final String intermediateUserTopic) throws Exception { + protected void assertInternalTopicsGotDeleted(final String additionalExistingTopic) throws Exception { // do not use list topics request, but read from the embedded cluster's zookeeper path directly to confirm - if (intermediateUserTopic != null) { + if (additionalExistingTopic != null) { cluster.waitForRemainingTopics(30000, INPUT_TOPIC, OUTPUT_TOPIC, OUTPUT_TOPIC_2, OUTPUT_TOPIC_2_RERUN, - Topic.GROUP_METADATA_TOPIC_NAME, intermediateUserTopic); + Topic.GROUP_METADATA_TOPIC_NAME, additionalExistingTopic); } else { cluster.waitForRemainingTopics(30000, INPUT_TOPIC, OUTPUT_TOPIC, OUTPUT_TOPIC_2, OUTPUT_TOPIC_2_RERUN, Topic.GROUP_METADATA_TOPIC_NAME); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java index f355afaa7286d..5c236e6ba80e5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java @@ -151,6 +151,38 @@ public void shouldNotAllowToResetWhenIntermediateTopicAbsent() { Assert.assertEquals(1, exitCode); } + @Test + public void shouldNotAllowToResetWhenSpecifiedInternalTopicDoesNotExist() { + final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); + final String[] parameters = new String[] { + "--application-id", appID, + "--bootstrap-servers", cluster.bootstrapServers(), + "--internal-topics", NON_EXISTING_TOPIC + }; + final Properties cleanUpConfig = new Properties(); + cleanUpConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); + cleanUpConfig.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(CLEANUP_CONSUMER_TIMEOUT)); + + final int exitCode = new StreamsResetter().run(parameters, cleanUpConfig); + Assert.assertEquals(1, exitCode); + } + + @Test + public void shouldNotAllowToResetWhenSpecifiedInternalTopicIsNotInternal() { + final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); + final String[] parameters = new String[] { + "--application-id", appID, + "--bootstrap-servers", cluster.bootstrapServers(), + "--internal-topics", INPUT_TOPIC + }; + final Properties cleanUpConfig = new Properties(); + cleanUpConfig.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); + cleanUpConfig.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Integer.toString(CLEANUP_CONSUMER_TIMEOUT)); + + final int exitCode = new StreamsResetter().run(parameters, cleanUpConfig); + Assert.assertEquals(1, exitCode); + } + @Test public void testResetWhenLongSessionTimeoutConfiguredWithForceOption() throws Exception { final String appID = IntegrationTestUtils.safeUniqueTestName(getClass(), testName); diff --git a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java index 0383c3c379727..d4f7841000113 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tools/StreamsResetterTest.java @@ -265,10 +265,10 @@ public void shouldDeleteTopic() throws InterruptedException, ExecutionException @Test public void shouldDetermineInternalTopicBasedOnTopicName1() { - assertTrue(streamsResetter.matchesInternalTopicFormat("appId-named-subscription-response-topic")); - assertTrue(streamsResetter.matchesInternalTopicFormat("appId-named-subscription-registration-topic")); - assertTrue(streamsResetter.matchesInternalTopicFormat("appId-KTABLE-FK-JOIN-SUBSCRIPTION-RESPONSE-12323232-topic")); - assertTrue(streamsResetter.matchesInternalTopicFormat("appId-KTABLE-FK-JOIN-SUBSCRIPTION-REGISTRATION-12323232-topic")); + assertTrue(StreamsResetter.matchesInternalTopicFormat("appId-named-subscription-response-topic")); + assertTrue(StreamsResetter.matchesInternalTopicFormat("appId-named-subscription-registration-topic")); + assertTrue(StreamsResetter.matchesInternalTopicFormat("appId-KTABLE-FK-JOIN-SUBSCRIPTION-RESPONSE-12323232-topic")); + assertTrue(StreamsResetter.matchesInternalTopicFormat("appId-KTABLE-FK-JOIN-SUBSCRIPTION-REGISTRATION-12323232-topic")); } @Test From 56c7eb68d2cd03c256236779a3e8d53b885f6b17 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 28 Apr 2021 07:58:55 -0700 Subject: [PATCH 093/155] KAFKA-12716; Add `Admin` API to abort transactions (#10599) This patch adds the Admin API to abort transactions from KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions. The `WriteTxnMarker` API needs to be sent to partition leaders, so we are able to reuse `PartitionLeaderStrategy`, which was introduced when support for `DescribeProducers` was added. Reviewers: David Jacot --- .../admin/AbortTransactionOptions.java | 23 ++ .../clients/admin/AbortTransactionResult.java | 50 ++++ .../clients/admin/AbortTransactionSpec.java | 85 +++++++ .../org/apache/kafka/clients/admin/Admin.java | 22 ++ .../kafka/clients/admin/KafkaAdminClient.java | 10 + .../internals/AbortTransactionHandler.java | 184 +++++++++++++++ .../admin/internals/AdminApiHandler.java | 25 ++ .../requests/WriteTxnMarkersRequest.java | 5 + .../clients/admin/KafkaAdminClientTest.java | 92 ++++++++ .../kafka/clients/admin/MockAdminClient.java | 5 + .../AbortTransactionHandlerTest.java | 218 ++++++++++++++++++ 11 files changed, 719 insertions(+) create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java new file mode 100644 index 0000000000000..ad5827ac6d0f5 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionOptions.java @@ -0,0 +1,23 @@ +/* + * 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.clients.admin; + +import org.apache.kafka.common.annotation.InterfaceStability; + +@InterfaceStability.Evolving +public class AbortTransactionOptions extends AbstractOptions { +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java new file mode 100644 index 0000000000000..451595b1087ab --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionResult.java @@ -0,0 +1,50 @@ +/* + * 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.clients.admin; + +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.internals.KafkaFutureImpl; + +import java.util.Map; + +/** + * The result of {@link Admin#abortTransaction(AbortTransactionSpec, AbortTransactionOptions)}. + * + * The API of this class is evolving, see {@link Admin} for details. + */ +@InterfaceStability.Evolving +public class AbortTransactionResult { + private final Map> futures; + + AbortTransactionResult(Map> futures) { + this.futures = futures; + } + + /** + * Get a future which completes when the transaction specified by {@link AbortTransactionSpec} + * in the respective call to {@link Admin#abortTransaction(AbortTransactionSpec, AbortTransactionOptions)} + * returns successfully or fails due to an error or timeout. + * + * @return the future + */ + public KafkaFuture all() { + return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java new file mode 100644 index 0000000000000..9eb7057b6b84f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AbortTransactionSpec.java @@ -0,0 +1,85 @@ +/* + * 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.clients.admin; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.annotation.InterfaceStability; + +import java.util.Objects; + +@InterfaceStability.Evolving +public class AbortTransactionSpec { + private final TopicPartition topicPartition; + private final long producerId; + private final short producerEpoch; + private final int coordinatorEpoch; + + public AbortTransactionSpec( + TopicPartition topicPartition, + long producerId, + short producerEpoch, + int coordinatorEpoch + ) { + this.topicPartition = topicPartition; + this.producerId = producerId; + this.producerEpoch = producerEpoch; + this.coordinatorEpoch = coordinatorEpoch; + } + + public TopicPartition topicPartition() { + return topicPartition; + } + + public long producerId() { + return producerId; + } + + public short producerEpoch() { + return producerEpoch; + } + + public int coordinatorEpoch() { + return coordinatorEpoch; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + AbortTransactionSpec that = (AbortTransactionSpec) o; + return producerId == that.producerId && + producerEpoch == that.producerEpoch && + coordinatorEpoch == that.coordinatorEpoch && + Objects.equals(topicPartition, that.topicPartition); + } + + @Override + public int hashCode() { + return Objects.hash(topicPartition, producerId, producerEpoch, coordinatorEpoch); + } + + @Override + public String toString() { + return "AbortTransactionSpec(" + + "topicPartition=" + topicPartition + + ", producerId=" + producerId + + ", producerEpoch=" + producerEpoch + + ", coordinatorEpoch=" + coordinatorEpoch + + ')'; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java index f420985aeb0be..fc9dea2ddb67c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/Admin.java @@ -1511,6 +1511,28 @@ default DescribeTransactionsResult describeTransactions(Collection trans */ DescribeTransactionsResult describeTransactions(Collection transactionalIds, DescribeTransactionsOptions options); + /** + * Forcefully abort a transaction which is open on a topic partition. See + * {@link #abortTransaction(AbortTransactionSpec, AbortTransactionOptions)} for more details. + * + * @param spec The transaction specification including topic partition and producer details + * @return The result + */ + default AbortTransactionResult abortTransaction(AbortTransactionSpec spec) { + return abortTransaction(spec, new AbortTransactionOptions()); + } + + /** + * Forcefully abort a transaction which is open on a topic partition. This will + * send a `WriteTxnMarkers` request to the partition leader in order to abort the + * transaction. This requires administrative privileges. + * + * @param spec The transaction specification including topic partition and producer details + * @param options Options to control the method behavior (including filters) + * @return The result + */ + AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options); + /** * Get the metrics kept by the adminClient */ diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java index c35fe68dc2279..e49f28a62f61b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/KafkaAdminClient.java @@ -31,6 +31,7 @@ import org.apache.kafka.clients.admin.DescribeReplicaLogDirsResult.ReplicaLogDirInfo; import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; import org.apache.kafka.clients.admin.OffsetSpec.TimestampSpec; +import org.apache.kafka.clients.admin.internals.AbortTransactionHandler; import org.apache.kafka.clients.admin.internals.AdminApiDriver; import org.apache.kafka.clients.admin.internals.AdminApiHandler; import org.apache.kafka.clients.admin.internals.AdminMetadataManager; @@ -4745,6 +4746,15 @@ public DescribeTransactionsResult describeTransactions(Collection transa return new DescribeTransactionsResult(invokeDriver(handler, options.timeoutMs)); } + @Override + public AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options) { + AbortTransactionHandler handler = new AbortTransactionHandler( + spec, + logContext + ); + return new AbortTransactionResult(invokeDriver(handler, options.timeoutMs)); + } + private Map> invokeDriver( AdminApiHandler handler, Integer timeoutMs diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java new file mode 100644 index 0000000000000..09dc7be55162f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandler.java @@ -0,0 +1,184 @@ +/* + * 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.clients.admin.internals; + +import org.apache.kafka.clients.admin.AbortTransactionSpec; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ClusterAuthorizationException; +import org.apache.kafka.common.errors.InvalidProducerEpochException; +import org.apache.kafka.common.errors.TransactionCoordinatorFencedException; +import org.apache.kafka.common.message.WriteTxnMarkersRequestData; +import org.apache.kafka.common.message.WriteTxnMarkersResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.WriteTxnMarkersRequest; +import org.apache.kafka.common.requests.WriteTxnMarkersResponse; +import org.apache.kafka.common.utils.LogContext; +import org.slf4j.Logger; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; + +public class AbortTransactionHandler implements AdminApiHandler { + private final Logger log; + private final AbortTransactionSpec abortSpec; + private final PartitionLeaderStrategy lookupStrategy; + + public AbortTransactionHandler( + AbortTransactionSpec abortSpec, + LogContext logContext + ) { + this.abortSpec = abortSpec; + this.log = logContext.logger(AbortTransactionHandler.class); + this.lookupStrategy = new PartitionLeaderStrategy(logContext); + } + + @Override + public String apiName() { + return "abortTransaction"; + } + + @Override + public Keys initializeKeys() { + return Keys.dynamicMapped( + Collections.singleton(abortSpec.topicPartition()), + lookupStrategy + ); + } + + @Override + public WriteTxnMarkersRequest.Builder buildRequest( + int brokerId, + Set topicPartitions + ) { + validateTopicPartitions(topicPartitions); + + WriteTxnMarkersRequestData.WritableTxnMarker marker = new WriteTxnMarkersRequestData.WritableTxnMarker() + .setCoordinatorEpoch(abortSpec.coordinatorEpoch()) + .setProducerEpoch(abortSpec.producerEpoch()) + .setProducerId(abortSpec.producerId()) + .setTransactionResult(false); + + marker.topics().add(new WriteTxnMarkersRequestData.WritableTxnMarkerTopic() + .setName(abortSpec.topicPartition().topic()) + .setPartitionIndexes(singletonList(abortSpec.topicPartition().partition())) + ); + + WriteTxnMarkersRequestData request = new WriteTxnMarkersRequestData(); + request.markers().add(marker); + + return new WriteTxnMarkersRequest.Builder(request); + } + + @Override + public ApiResult handleResponse( + int brokerId, + Set topicPartitions, + AbstractResponse abstractResponse + ) { + validateTopicPartitions(topicPartitions); + + WriteTxnMarkersResponse response = (WriteTxnMarkersResponse) abstractResponse; + List markerResponses = response.data().markers(); + + if (markerResponses.size() != 1 || markerResponses.get(0).producerId() != abortSpec.producerId()) { + return ApiResult.failed(abortSpec.topicPartition(), new KafkaException("WriteTxnMarkers response " + + "included unexpected marker entries: " + markerResponses + "(expected to find exactly one " + + "entry with producerId " + abortSpec.producerId() + ")")); + } + + WriteTxnMarkersResponseData.WritableTxnMarkerResult markerResponse = markerResponses.get(0); + List topicResponses = markerResponse.topics(); + + if (topicResponses.size() != 1 || !topicResponses.get(0).name().equals(abortSpec.topicPartition().topic())) { + return ApiResult.failed(abortSpec.topicPartition(), new KafkaException("WriteTxnMarkers response " + + "included unexpected topic entries: " + markerResponses + "(expected to find exactly one " + + "entry with topic partition " + abortSpec.topicPartition() + ")")); + } + + WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult topicResponse = topicResponses.get(0); + List partitionResponses = + topicResponse.partitions(); + + if (partitionResponses.size() != 1 || partitionResponses.get(0).partitionIndex() != abortSpec.topicPartition().partition()) { + return ApiResult.failed(abortSpec.topicPartition(), new KafkaException("WriteTxnMarkers response " + + "included unexpected partition entries for topic " + abortSpec.topicPartition().topic() + + ": " + markerResponses + "(expected to find exactly one entry with partition " + + abortSpec.topicPartition().partition() + ")")); + } + + WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult partitionResponse = partitionResponses.get(0); + Errors error = Errors.forCode(partitionResponse.errorCode()); + + if (error != Errors.NONE) { + return handleError(error); + } else { + return ApiResult.completed(abortSpec.topicPartition(), null); + } + } + + private ApiResult handleError(Errors error) { + switch (error) { + case CLUSTER_AUTHORIZATION_FAILED: + log.error("WriteTxnMarkers request for abort spec {} failed cluster authorization", abortSpec); + return ApiResult.failed(abortSpec.topicPartition(), new ClusterAuthorizationException( + "WriteTxnMarkers request with " + abortSpec + " failed due to cluster " + + "authorization error")); + + case INVALID_PRODUCER_EPOCH: + log.error("WriteTxnMarkers request for abort spec {} failed due to an invalid producer epoch", + abortSpec); + return ApiResult.failed(abortSpec.topicPartition(), new InvalidProducerEpochException( + "WriteTxnMarkers request with " + abortSpec + " failed due an invalid producer epoch")); + + case TRANSACTION_COORDINATOR_FENCED: + log.error("WriteTxnMarkers request for abort spec {} failed because the coordinator epoch is fenced", + abortSpec); + return ApiResult.failed(abortSpec.topicPartition(), new TransactionCoordinatorFencedException( + "WriteTxnMarkers request with " + abortSpec + " failed since the provided " + + "coordinator epoch " + abortSpec.coordinatorEpoch() + " has been fenced " + + "by the active coordinator")); + + case NOT_LEADER_OR_FOLLOWER: + case REPLICA_NOT_AVAILABLE: + case BROKER_NOT_AVAILABLE: + case UNKNOWN_TOPIC_OR_PARTITION: + log.debug("WriteTxnMarkers request for abort spec {} failed due to {}. Will retry after attempting to " + + "find the leader again", abortSpec, error); + return ApiResult.unmapped(singletonList(abortSpec.topicPartition())); + + default: + log.error("WriteTxnMarkers request for abort spec {} failed due to an unexpected error {}", + abortSpec, error); + return ApiResult.failed(abortSpec.topicPartition(), error.exception( + "WriteTxnMarkers request with " + abortSpec + " failed due to unexpected error: " + error.message())); + } + } + + private void validateTopicPartitions(Set topicPartitions) { + if (!topicPartitions.equals(singleton(abortSpec.topicPartition()))) { + throw new IllegalArgumentException("Received unexpected topic partitions " + topicPartitions + + " (expected only " + singleton(abortSpec.topicPartition()) + ")"); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java index 3811462994f46..754ae89451fe0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AdminApiHandler.java @@ -135,5 +135,30 @@ public ApiResult( this.failedKeys = Collections.unmodifiableMap(failedKeys); this.unmappedKeys = Collections.unmodifiableList(unmappedKeys); } + + public static ApiResult completed(K key, V value) { + return new ApiResult<>( + Collections.singletonMap(key, value), + Collections.emptyMap(), + Collections.emptyList() + ); + } + + public static ApiResult failed(K key, Throwable t) { + return new ApiResult<>( + Collections.emptyMap(), + Collections.singletonMap(key, t), + Collections.emptyList() + ); + } + + public static ApiResult unmapped(List keys) { + return new ApiResult<>( + Collections.emptyMap(), + Collections.emptyMap(), + keys + ); + } } + } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java index 64a3df49c7b39..73a6b6c41d71d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersRequest.java @@ -105,6 +105,11 @@ public static class Builder extends AbstractRequest.Builder markers) { super(ApiKeys.WRITE_TXN_MARKERS, version); List dataMarkers = new ArrayList<>(); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index d76641ac6a0ff..4cad255ba0241 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -124,6 +124,7 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopicCollection; import org.apache.kafka.common.message.UnregisterBrokerResponseData; +import org.apache.kafka.common.message.WriteTxnMarkersResponseData; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; @@ -181,6 +182,8 @@ import org.apache.kafka.common.requests.UnregisterBrokerResponse; import org.apache.kafka.common.requests.UpdateFeaturesRequest; import org.apache.kafka.common.requests.UpdateFeaturesResponse; +import org.apache.kafka.common.requests.WriteTxnMarkersRequest; +import org.apache.kafka.common.requests.WriteTxnMarkersResponse; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourcePatternFilter; @@ -5554,6 +5557,95 @@ public void testRetryDescribeTransactionsAfterNotCoordinatorError() throws Excep } } + @Test + public void testAbortTransaction() throws Exception { + try (AdminClientUnitTestEnv env = mockClientEnv()) { + TopicPartition topicPartition = new TopicPartition("foo", 13); + AbortTransactionSpec abortSpec = new AbortTransactionSpec( + topicPartition, 12345L, (short) 15, 200); + Node leader = env.cluster().nodes().iterator().next(); + + expectMetadataRequest(env, topicPartition, leader); + + env.kafkaClient().prepareResponseFrom( + request -> request instanceof WriteTxnMarkersRequest, + writeTxnMarkersResponse(abortSpec, Errors.NONE), + leader + ); + + AbortTransactionResult result = env.adminClient().abortTransaction(abortSpec); + assertNull(result.all().get()); + } + } + + @Test + public void testAbortTransactionFindLeaderAfterDisconnect() throws Exception { + MockTime time = new MockTime(); + int retryBackoffMs = 100; + Cluster cluster = mockCluster(3, 0); + Map configOverride = newStrMap(AdminClientConfig.RETRY_BACKOFF_MS_CONFIG, "" + retryBackoffMs); + + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(time, cluster, configOverride)) { + TopicPartition topicPartition = new TopicPartition("foo", 13); + AbortTransactionSpec abortSpec = new AbortTransactionSpec( + topicPartition, 12345L, (short) 15, 200); + Iterator nodeIterator = env.cluster().nodes().iterator(); + Node firstLeader = nodeIterator.next(); + + expectMetadataRequest(env, topicPartition, firstLeader); + + WriteTxnMarkersResponse response = writeTxnMarkersResponse(abortSpec, Errors.NONE); + env.kafkaClient().prepareResponseFrom( + request -> { + // We need a sleep here because the client will attempt to + // backoff after the disconnect + time.sleep(retryBackoffMs); + return request instanceof WriteTxnMarkersRequest; + }, + response, + firstLeader, + true + ); + + Node retryLeader = nodeIterator.next(); + expectMetadataRequest(env, topicPartition, retryLeader); + + env.kafkaClient().prepareResponseFrom( + request -> request instanceof WriteTxnMarkersRequest, + response, + retryLeader + ); + + AbortTransactionResult result = env.adminClient().abortTransaction(abortSpec); + assertNull(result.all().get()); + } + } + + private WriteTxnMarkersResponse writeTxnMarkersResponse( + AbortTransactionSpec abortSpec, + Errors error + ) { + WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult partitionResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult() + .setPartitionIndex(abortSpec.topicPartition().partition()) + .setErrorCode(error.code()); + + WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult topicResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult() + .setName(abortSpec.topicPartition().topic()); + topicResponse.partitions().add(partitionResponse); + + WriteTxnMarkersResponseData.WritableTxnMarkerResult markerResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerResult() + .setProducerId(abortSpec.producerId()); + markerResponse.topics().add(topicResponse); + + WriteTxnMarkersResponseData response = new WriteTxnMarkersResponseData(); + response.markers().add(markerResponse); + + return new WriteTxnMarkersResponse(response); + } + private DescribeProducersResponse buildDescribeProducersResponse( TopicPartition topicPartition, List producerStates diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index a47a945aa27bc..369e1e6f55fe8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -913,6 +913,11 @@ public DescribeTransactionsResult describeTransactions(Collection transa throw new UnsupportedOperationException("Not implemented yet"); } + @Override + public AbortTransactionResult abortTransaction(AbortTransactionSpec spec, AbortTransactionOptions options) { + throw new UnsupportedOperationException("Not implemented yet"); + } + @Override synchronized public void close(Duration timeout) {} diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java new file mode 100644 index 0000000000000..aac18cd50bc8d --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AbortTransactionHandlerTest.java @@ -0,0 +1,218 @@ +/* + * 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.clients.admin.internals; + +import org.apache.kafka.clients.admin.AbortTransactionSpec; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ClusterAuthorizationException; +import org.apache.kafka.common.errors.InvalidProducerEpochException; +import org.apache.kafka.common.errors.TransactionCoordinatorFencedException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.message.WriteTxnMarkersRequestData; +import org.apache.kafka.common.message.WriteTxnMarkersResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.WriteTxnMarkersRequest; +import org.apache.kafka.common.requests.WriteTxnMarkersResponse; +import org.apache.kafka.common.utils.LogContext; +import org.junit.jupiter.api.Test; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +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.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(); + private final TopicPartition topicPartition = new TopicPartition("foo", 5); + private final AbortTransactionSpec abortSpec = new AbortTransactionSpec( + topicPartition, 12345L, (short) 15, 4321); + + @Test + public void testInvalidBuildRequestCall() { + AbortTransactionHandler handler = new AbortTransactionHandler(abortSpec, logContext); + assertThrows(IllegalArgumentException.class, () -> handler.buildRequest(1, + emptySet())); + assertThrows(IllegalArgumentException.class, () -> handler.buildRequest(1, + mkSet(new TopicPartition("foo", 1)))); + assertThrows(IllegalArgumentException.class, () -> handler.buildRequest(1, + mkSet(topicPartition, new TopicPartition("foo", 1)))); + } + + @Test + public void testValidBuildRequestCall() { + AbortTransactionHandler handler = new AbortTransactionHandler(abortSpec, logContext); + WriteTxnMarkersRequest.Builder request = handler.buildRequest(1, singleton(topicPartition)); + assertEquals(1, request.data.markers().size()); + + WriteTxnMarkersRequestData.WritableTxnMarker markerRequest = request.data.markers().get(0); + assertEquals(abortSpec.producerId(), markerRequest.producerId()); + assertEquals(abortSpec.producerEpoch(), markerRequest.producerEpoch()); + assertEquals(abortSpec.coordinatorEpoch(), markerRequest.coordinatorEpoch()); + assertEquals(1, markerRequest.topics().size()); + + WriteTxnMarkersRequestData.WritableTxnMarkerTopic topicRequest = markerRequest.topics().get(0); + assertEquals(abortSpec.topicPartition().topic(), topicRequest.name()); + assertEquals(singletonList(abortSpec.topicPartition().partition()), topicRequest.partitionIndexes()); + } + + @Test + public void testInvalidHandleResponseCall() { + AbortTransactionHandler handler = new AbortTransactionHandler(abortSpec, logContext); + WriteTxnMarkersResponseData response = new WriteTxnMarkersResponseData(); + assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(1, + emptySet(), new WriteTxnMarkersResponse(response))); + assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(1, + mkSet(new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response))); + assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(1, + mkSet(topicPartition, new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response))); + } + + @Test + public void testInvalidResponse() { + AbortTransactionHandler handler = new AbortTransactionHandler(abortSpec, logContext); + + WriteTxnMarkersResponseData response = new WriteTxnMarkersResponseData(); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + + WriteTxnMarkersResponseData.WritableTxnMarkerResult markerResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerResult(); + response.markers().add(markerResponse); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + + markerResponse.setProducerId(abortSpec.producerId()); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + + WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult topicResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult(); + markerResponse.topics().add(topicResponse); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + + topicResponse.setName(abortSpec.topicPartition().topic()); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + + WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult partitionResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult(); + topicResponse.partitions().add(partitionResponse); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + + partitionResponse.setPartitionIndex(abortSpec.topicPartition().partition()); + topicResponse.setName(abortSpec.topicPartition().topic() + "random"); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + + topicResponse.setName(abortSpec.topicPartition().topic()); + markerResponse.setProducerId(abortSpec.producerId() + 1); + assertFailed(KafkaException.class, topicPartition, handler.handleResponse(1, singleton(topicPartition), + new WriteTxnMarkersResponse(response))); + } + + @Test + public void testSuccessfulResponse() { + assertCompleted(abortSpec.topicPartition(), handleWithError(abortSpec, Errors.NONE)); + } + + @Test + public void testRetriableErrors() { + assertUnmapped(abortSpec.topicPartition(), handleWithError(abortSpec, Errors.NOT_LEADER_OR_FOLLOWER)); + assertUnmapped(abortSpec.topicPartition(), handleWithError(abortSpec, Errors.UNKNOWN_TOPIC_OR_PARTITION)); + assertUnmapped(abortSpec.topicPartition(), handleWithError(abortSpec, Errors.REPLICA_NOT_AVAILABLE)); + assertUnmapped(abortSpec.topicPartition(), handleWithError(abortSpec, Errors.BROKER_NOT_AVAILABLE)); + } + + @Test + public void testFatalErrors() { + assertFailed(ClusterAuthorizationException.class, abortSpec.topicPartition(), + handleWithError(abortSpec, Errors.CLUSTER_AUTHORIZATION_FAILED)); + assertFailed(InvalidProducerEpochException.class, abortSpec.topicPartition(), + handleWithError(abortSpec, Errors.INVALID_PRODUCER_EPOCH)); + assertFailed(TransactionCoordinatorFencedException.class, abortSpec.topicPartition(), + handleWithError(abortSpec, Errors.TRANSACTION_COORDINATOR_FENCED)); + assertFailed(UnknownServerException.class, abortSpec.topicPartition(), + handleWithError(abortSpec, Errors.UNKNOWN_SERVER_ERROR)); + } + + private AdminApiHandler.ApiResult handleWithError( + AbortTransactionSpec abortSpec, + Errors error + ) { + AbortTransactionHandler handler = new AbortTransactionHandler(abortSpec, logContext); + + WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult partitionResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerPartitionResult() + .setPartitionIndex(abortSpec.topicPartition().partition()) + .setErrorCode(error.code()); + + WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult topicResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerTopicResult() + .setName(abortSpec.topicPartition().topic()); + topicResponse.partitions().add(partitionResponse); + + WriteTxnMarkersResponseData.WritableTxnMarkerResult markerResponse = + new WriteTxnMarkersResponseData.WritableTxnMarkerResult() + .setProducerId(abortSpec.producerId()); + markerResponse.topics().add(topicResponse); + + WriteTxnMarkersResponseData response = new WriteTxnMarkersResponseData(); + response.markers().add(markerResponse); + + return handler.handleResponse(1, singleton(abortSpec.topicPartition()), + new WriteTxnMarkersResponse(response)); + } + + private void assertUnmapped( + TopicPartition topicPartition, + AdminApiHandler.ApiResult result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(singletonList(topicPartition), result.unmappedKeys); + } + + private void assertCompleted( + TopicPartition topicPartition, + AdminApiHandler.ApiResult result + ) { + assertEquals(emptySet(), result.failedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(topicPartition), result.completedKeys.keySet()); + assertNull(result.completedKeys.get(topicPartition)); + } + + private void assertFailed( + Class expectedExceptionType, + TopicPartition topicPartition, + AdminApiHandler.ApiResult result + ) { + assertEquals(emptySet(), result.completedKeys.keySet()); + assertEquals(emptyList(), result.unmappedKeys); + assertEquals(singleton(topicPartition), result.failedKeys.keySet()); + assertTrue(expectedExceptionType.isInstance(result.failedKeys.get(topicPartition))); + } + +} From ab06aef2b8507768413c9d2525b8d1f6ed68f28c Mon Sep 17 00:00:00 2001 From: Luke Chen <43372967+showuon@users.noreply.github.com> Date: Wed, 28 Apr 2021 23:56:48 +0800 Subject: [PATCH 094/155] KAFKA-12284: increase request timeout to make tests reliable (#10547) Reviewers: Chia-Ping Tsai --- .../MirrorConnectorsIntegrationBaseTest.java | 30 +++++++++++++----- .../util/clusters/EmbeddedKafkaCluster.java | 31 +++++++++++++------ 2 files changed, 45 insertions(+), 16 deletions(-) diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index e6bdb9679d64d..841eedc33c6e6 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.mirror.integration; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.admin.DescribeConfigsResult; import org.apache.kafka.clients.consumer.Consumer; @@ -83,7 +84,8 @@ public abstract class MirrorConnectorsIntegrationBaseTest { private static final int CHECKPOINT_DURATION_MS = 20_000; private static final int RECORD_CONSUME_DURATION_MS = 20_000; private static final int OFFSET_SYNC_DURATION_MS = 30_000; - private static final int TOPIC_SYNC_DURATION_MS = 30_000; + private static final int TOPIC_SYNC_DURATION_MS = 60_000; + private static final int REQUEST_TIMEOUT_DURATION_MS = 60_000; private static final int NUM_WORKERS = 3; private static final Duration CONSUMER_POLL_TIMEOUT_MS = Duration.ofMillis(500); protected static final String PRIMARY_CLUSTER_ALIAS = "primary"; @@ -165,11 +167,19 @@ public void startClusters() throws Exception { primary.start(); primary.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Workers of " + PRIMARY_CLUSTER_ALIAS + "-connect-cluster did not start in time."); + + waitForTopicCreated(primary, "mm2-status.backup.internal"); + waitForTopicCreated(primary, "mm2-offsets.backup.internal"); + waitForTopicCreated(primary, "mm2-configs.backup.internal"); backup.start(); backup.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did not start in time."); + waitForTopicCreated(backup, "mm2-status.primary.internal"); + waitForTopicCreated(backup, "mm2-offsets.primary.internal"); + waitForTopicCreated(backup, "mm2-configs.primary.internal"); + createTopics(); warmUpConsumer(Collections.singletonMap("group.id", "consumer-group-dummy")); @@ -602,13 +612,19 @@ private static Map basicMM2Config() { private void createTopics() { // to verify topic config will be sync-ed across clusters Map topicConfig = Collections.singletonMap(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); + Map emptyMap = Collections.emptyMap(); + + // increase admin client request timeout value to make the tests reliable. + Properties adminClientConfig = new Properties(); + adminClientConfig.put(AdminClientConfig.REQUEST_TIMEOUT_MS_CONFIG, REQUEST_TIMEOUT_DURATION_MS); + // create these topics before starting the connectors so we don't need to wait for discovery - primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig); - primary.kafka().createTopic("backup.test-topic-1", 1); - primary.kafka().createTopic("heartbeats", 1); - backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS); - backup.kafka().createTopic("primary.test-topic-1", 1); - backup.kafka().createTopic("heartbeats", 1); + primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig, adminClientConfig); + primary.kafka().createTopic("backup.test-topic-1", 1, 1, emptyMap, adminClientConfig); + primary.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic("primary.test-topic-1", 1, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); } /* diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index a8328c5069460..30d41d26a56e2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -337,18 +337,28 @@ public void createTopic(String topic) { * @param topic The name of the topic. */ public void createTopic(String topic, int partitions) { - createTopic(topic, partitions, 1, new HashMap<>()); + createTopic(topic, partitions, 1, Collections.emptyMap()); } /** - * Create a Kafka topic with the given parameters. + * Create a Kafka topic with given partition, replication factor, and topic config. * - * @param topic The name of the topic. - * @param partitions The number of partitions for this topic. - * @param replication The replication factor for (partitions of) this topic. - * @param topicConfig Additional topic-level configuration settings. + * @param topic The name of the topic. */ public void createTopic(String topic, int partitions, int replication, Map topicConfig) { + createTopic(topic, partitions, replication, topicConfig, new Properties()); + } + + /** + * Create a Kafka topic with the given parameters. + * + * @param topic The name of the topic. + * @param partitions The number of partitions for this topic. + * @param replication The replication factor for (partitions of) this topic. + * @param topicConfig Additional topic-level configuration settings. + * @param adminClientConfig Additional admin client configuration settings. + */ + public void createTopic(String topic, int partitions, int replication, Map topicConfig, Properties adminClientConfig) { if (replication > brokers.length) { throw new InvalidReplicationFactorException("Insufficient brokers (" + brokers.length + ") for desired replication (" + replication + ")"); @@ -359,7 +369,7 @@ public void createTopic(String topic, int partitions, int replication, Map Date: Wed, 28 Apr 2021 20:31:37 +0200 Subject: [PATCH 095/155] KAFKA-12417: streams copyDependentLibs should not copy testRuntime configuration jars (#10466) This also fixes a Gradle deprecation that unblocks the upgrade to Gradle 7.0. Reviewers: Ismael Juma --- build.gradle | 5 ----- 1 file changed, 5 deletions(-) diff --git a/build.gradle b/build.gradle index b535dd39a6933..2c21405394895 100644 --- a/build.gradle +++ b/build.gradle @@ -1667,11 +1667,6 @@ project(':streams') { } tasks.create(name: "copyDependantLibs", type: Copy) { - from (configurations.testRuntime) { - include('slf4j-log4j12*') - include('log4j*jar') - include('*hamcrest*') - } from (configurations.runtimeClasspath) { exclude('kafka-clients*') } From 3805f3706f8f3ebba81b80915c9259590525fb05 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Wed, 28 Apr 2021 13:22:15 -0700 Subject: [PATCH 096/155] KAFKA-12574: KIP-732, Deprecate eos-alpha and replace eos-beta with eos-v2 (#10573) Deprecates the following 1. StreamsConfig.EXACTLY_ONCE 2. StreamsConfig.EXACTLY_ONCE_BETA 3. Producer#sendOffsetsToTransaction(Map offsets, String consumerGroupId) And introduces a new StreamsConfig.EXACTLY_ONCE_V2 config. Additionally, this PR replaces usages of the term "eos-beta" throughout the code with the term "eos-v2" Reviewers: Matthias J. Sax --- checkstyle/suppressions.xml | 8 +- .../kafka/clients/producer/KafkaProducer.java | 18 ++- .../kafka/clients/producer/MockProducer.java | 17 ++- .../kafka/clients/producer/Producer.java | 1 + .../clients/producer/KafkaProducerTest.java | 2 +- .../clients/producer/MockProducerTest.java | 20 +-- .../kafka/api/AuthorizerIntegrationTest.scala | 6 +- .../kafka/api/TransactionsBounceTest.scala | 2 + .../kafka/api/TransactionsTest.scala | 10 +- docs/streams/core-concepts.html | 6 +- .../developer-guide/config-streams.html | 17 ++- docs/streams/upgrade-guide.html | 28 ++-- docs/upgrade.html | 7 + release.py | 2 +- .../apache/kafka/streams/StreamsConfig.java | 49 +++++-- .../internals/ActiveTaskCreator.java | 10 +- .../processor/internals/StreamThread.java | 32 ++-- .../processor/internals/StreamsProducer.java | 23 ++- .../processor/internals/TaskManager.java | 10 +- .../kafka/streams/StreamsConfigTest.java | 137 +++++++++++++++++- .../EOSUncleanShutdownIntegrationTest.java | 3 +- .../integration/EosIntegrationTest.java | 3 +- ....java => EosV2UpgradeIntegrationTest.java} | 85 +++++------ .../GlobalKTableEOSIntegrationTest.java | 4 +- ...ableSourceTopicRestartIntegrationTest.java | 5 +- .../ResetPartitionTimeIntegrationTest.java | 3 +- .../RocksDBMetricsIntegrationTest.java | 3 +- .../StandbyTaskEOSIntegrationTest.java | 3 +- .../SuppressionDurabilityIntegrationTest.java | 4 +- .../internals/ActiveTaskCreatorTest.java | 46 +++--- .../internals/RecordCollectorTest.java | 23 +-- .../processor/internals/StandbyTaskTest.java | 5 +- .../processor/internals/StreamTaskTest.java | 11 +- .../processor/internals/StreamThreadTest.java | 47 +++++- .../internals/StreamsProducerTest.java | 16 +- .../processor/internals/TaskManagerTest.java | 20 +-- .../tests/BrokerCompatibilityTest.java | 2 +- .../kafka/streams/tests/StreamsEosTest.java | 7 +- .../kafka/streams/tests/StreamsSmokeTest.java | 11 +- .../kafka/streams/TopologyTestDriver.java | 7 +- .../streams/TopologyTestDriverEosTest.java | 2 +- .../tools/TransactionalMessageCopier.java | 8 +- 42 files changed, 495 insertions(+), 228 deletions(-) rename streams/src/test/java/org/apache/kafka/streams/integration/{EosBetaUpgradeIntegrationTest.java => EosV2UpgradeIntegrationTest.java} (95%) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 197702d7a67bf..a0955305d0bb2 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -193,19 +193,19 @@ files="(StreamsPartitionAssignorTest|StreamThreadTest|StreamTaskTest|TopologyTestDriverTest).java"/> + files="(EosIntegrationTest|EosV2UpgradeIntegrationTest|KStreamKStreamJoinTest|RocksDBWindowStoreTest).java"/> + files="(EosV2UpgradeIntegrationTest|KStreamKStreamJoinTest|KTableKTableForeignKeyJoinIntegrationTest|RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest|RelationalSmokeTest|MockProcessorContextStateStoreTest).java"/> + files="(EosV2UpgradeIntegrationTest|KStreamKStreamJoinTest|TaskManagerTest).java"/> + files="(EosV2UpgradeIntegrationTest|EosTestDriver|KStreamKStreamJoinTest|KTableKTableForeignKeyJoinIntegrationTest|RelationalSmokeTest|MockProcessorContextStateStoreTest).java"/> diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 7f573d9203424..e50d9f4d5f3d9 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -525,6 +525,11 @@ private TransactionManager configureTransactionState(ProducerConfig config, final int transactionTimeoutMs = config.getInt(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); final long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG); final boolean autoDowngradeTxnCommit = config.getBoolean(ProducerConfig.AUTO_DOWNGRADE_TXN_COMMIT); + // Only log a warning if being used outside of Streams, which we know includes "StreamThread-" in the client id + if (autoDowngradeTxnCommit && !clientId.contains("StreamThread-")) { + log.warn("The configuration parameter `{}` is internal and not intended for public use, it will be " + + "removed in 4.0", ProducerConfig.AUTO_DOWNGRADE_TXN_COMMIT); + } transactionManager = new TransactionManager( logContext, transactionalId, @@ -642,7 +647,10 @@ public void beginTransaction() throws ProducerFencedException { * to the partition leader. See the exception for more details * @throws KafkaException if the producer has encountered a previous fatal or abortable error, or for any * other unexpected error + * + * @deprecated Since 3.0.0, please use {@link #sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} instead. */ + @Deprecated public void sendOffsetsToTransaction(Map offsets, String consumerGroupId) throws ProducerFencedException { sendOffsetsToTransaction(offsets, new ConsumerGroupMetadata(consumerGroupId)); @@ -657,8 +665,10 @@ public void sendOffsetsToTransaction(Map offs * This method should be used when you need to batch consumed and produced messages * together, typically in a consume-transform-produce pattern. Thus, the specified * {@code groupMetadata} should be extracted from the used {@link KafkaConsumer consumer} via - * {@link KafkaConsumer#groupMetadata()} to leverage consumer group metadata for stronger fencing than - * {@link #sendOffsetsToTransaction(Map, String)} which only sends with consumer group id. + * {@link KafkaConsumer#groupMetadata()} to leverage consumer group metadata. This will provide + * stronger fencing than just supplying the {@code consumerGroupId} and passing in {@code new ConsumerGroupMetadata(consumerGroupId)}, + * however note that the full set of consumer group metadata returned by {@link KafkaConsumer#groupMetadata()} + * requires the brokers to be on version 2.5 or newer to understand. * *

    * Note, that the consumer should have {@code enable.auto.commit=false} and should @@ -671,8 +681,8 @@ public void sendOffsetsToTransaction(Map offs * @throws ProducerFencedException fatal error indicating another producer with the same transactional.id is active * @throws org.apache.kafka.common.errors.UnsupportedVersionException fatal error indicating the broker * does not support transactions (i.e. if its version is lower than 0.11.0.0) or - * the broker doesn't support latest version of transactional API with consumer group metadata (i.e. if its version is - * lower than 2.5.0). + * the broker doesn't support latest version of transactional API with all consumer group metadata + * (i.e. if its version is lower than 2.5.0). * @throws org.apache.kafka.common.errors.UnsupportedForMessageFormatException fatal error indicating the message * format used for the offsets topic on the broker does not support transactions * @throws org.apache.kafka.common.errors.AuthorizationException fatal error indicating that the configured diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index f733869d7fd0d..811b630dcd596 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -179,10 +179,18 @@ public void beginTransaction() throws ProducerFencedException { this.sentOffsets = false; } + @SuppressWarnings("deprecation") @Override public void sendOffsetsToTransaction(Map offsets, String consumerGroupId) throws ProducerFencedException { Objects.requireNonNull(consumerGroupId); + sendOffsetsToTransaction(offsets, new ConsumerGroupMetadata(consumerGroupId)); + } + + @Override + public void sendOffsetsToTransaction(Map offsets, + ConsumerGroupMetadata groupMetadata) throws ProducerFencedException { + Objects.requireNonNull(groupMetadata); verifyProducerState(); verifyTransactionsInitialized(); verifyTransactionInFlight(); @@ -195,18 +203,11 @@ public void sendOffsetsToTransaction(Map offs return; } Map uncommittedOffsets = - this.uncommittedConsumerGroupOffsets.computeIfAbsent(consumerGroupId, k -> new HashMap<>()); + this.uncommittedConsumerGroupOffsets.computeIfAbsent(groupMetadata.groupId(), k -> new HashMap<>()); uncommittedOffsets.putAll(offsets); this.sentOffsets = true; } - @Override - public void sendOffsetsToTransaction(Map offsets, - ConsumerGroupMetadata groupMetadata) throws ProducerFencedException { - Objects.requireNonNull(groupMetadata); - sendOffsetsToTransaction(offsets, groupMetadata.groupId()); - } - @Override public void commitTransaction() throws ProducerFencedException { verifyProducerState(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 6a0cb8a8179d5..4f3e9ec0d7281 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -50,6 +50,7 @@ public interface Producer extends Closeable { /** * See {@link KafkaProducer#sendOffsetsToTransaction(Map, String)} */ + @Deprecated void sendOffsetsToTransaction(Map offsets, String consumerGroupId) throws ProducerFencedException; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 721db4d41f68d..e3d4e944cee78 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -900,7 +900,7 @@ public void testSendTxnOffsetsWithGroupId() { new StringSerializer(), metadata, client, null, time)) { producer.initTransactions(); producer.beginTransaction(); - producer.sendOffsetsToTransaction(Collections.emptyMap(), groupId); + producer.sendOffsetsToTransaction(Collections.emptyMap(), new ConsumerGroupMetadata(groupId)); producer.commitTransaction(); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java index 27b6787026f1c..16a3d692de12d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java @@ -160,14 +160,14 @@ public void shouldThrowOnBeginTransactionsIfTransactionInflight() { @Test public void shouldThrowOnSendOffsetsToTransactionIfTransactionsNotInitialized() { buildMockProducer(true); - assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, groupId)); + assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId))); } @Test public void shouldThrowOnSendOffsetsToTransactionTransactionIfNoTransactionGotStarted() { buildMockProducer(true); producer.initTransactions(); - assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, groupId)); + assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId))); } @Test @@ -270,7 +270,7 @@ public void shouldThrowOnSendOffsetsToTransactionByGroupIdIfProducerGotFenced() buildMockProducer(true); producer.initTransactions(); producer.fenceProducer(); - assertThrows(ProducerFencedException.class, () -> producer.sendOffsetsToTransaction(null, groupId)); + assertThrows(ProducerFencedException.class, () -> producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId))); } @Test @@ -404,8 +404,8 @@ public void shouldPublishConsumerGroupOffsetsOnlyAfterCommitIfTransactionsAreEna put(new TopicPartition(topic, 1), new OffsetAndMetadata(21L, null)); } }; - producer.sendOffsetsToTransaction(group1Commit, group1); - producer.sendOffsetsToTransaction(group2Commit, group2); + producer.sendOffsetsToTransaction(group1Commit, new ConsumerGroupMetadata(group1)); + producer.sendOffsetsToTransaction(group2Commit, new ConsumerGroupMetadata(group2)); assertTrue(producer.consumerGroupOffsetsHistory().isEmpty()); @@ -498,7 +498,7 @@ public void shouldResetSentOffsetsFlagOnlyWhenBeginningNewTransaction() { put(new TopicPartition(topic, 0), new OffsetAndMetadata(42L, null)); } }; - producer.sendOffsetsToTransaction(groupCommit, "groupId"); + producer.sendOffsetsToTransaction(groupCommit, new ConsumerGroupMetadata("groupId")); producer.commitTransaction(); // commit should not reset "sentOffsets" flag assertTrue(producer.sentOffsets()); @@ -532,7 +532,7 @@ public void shouldPublishLatestAndCumulativeConsumerGroupOffsetsOnlyAfterCommitI put(new TopicPartition(topic, 2), new OffsetAndMetadata(21L, null)); } }; - producer.sendOffsetsToTransaction(groupCommit1, group); + producer.sendOffsetsToTransaction(groupCommit1, new ConsumerGroupMetadata(group)); producer.sendOffsetsToTransaction(groupCommit2, new ConsumerGroupMetadata(group)); assertTrue(producer.consumerGroupOffsetsHistory().isEmpty()); @@ -563,7 +563,7 @@ public void shouldDropConsumerGroupOffsetsOnAbortIfTransactionsAreEnabled() { put(new TopicPartition(topic, 1), new OffsetAndMetadata(73L, null)); } }; - producer.sendOffsetsToTransaction(groupCommit, group); + producer.sendOffsetsToTransaction(groupCommit, new ConsumerGroupMetadata(group)); producer.abortTransaction(); producer.beginTransaction(); @@ -592,7 +592,7 @@ public void shouldPreserveOffsetsFromCommitByGroupIdOnAbortIfTransactionsAreEnab put(new TopicPartition(topic, 1), new OffsetAndMetadata(73L, null)); } }; - producer.sendOffsetsToTransaction(groupCommit, group); + producer.sendOffsetsToTransaction(groupCommit, new ConsumerGroupMetadata(group)); producer.commitTransaction(); producer.beginTransaction(); @@ -663,7 +663,7 @@ public void shouldThrowOnBeginTransactionIfProducerIsClosed() { public void shouldThrowSendOffsetsToTransactionByGroupIdIfProducerIsClosed() { buildMockProducer(true); producer.close(); - assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, groupId)); + assertThrows(IllegalStateException.class, () -> producer.sendOffsetsToTransaction(null, new ConsumerGroupMetadata(groupId))); } @Test diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index f0d9188bd2c66..c730c476249ca 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -1666,7 +1666,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producer.beginTransaction() assertThrows(classOf[GroupAuthorizationException], - () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, group)) + () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } @Test @@ -1680,7 +1680,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producer.beginTransaction() assertThrows(classOf[GroupAuthorizationException], - () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, group)) + () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } @Test @@ -1894,7 +1894,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { removeAllClientAcls() assertThrows(classOf[TransactionalIdAuthorizationException], () => { val offsets = Map(tp -> new OffsetAndMetadata(1L)).asJava - producer.sendOffsetsToTransaction(offsets, group) + producer.sendOffsetsToTransaction(offsets, new ConsumerGroupMetadata(group)) producer.commitTransaction() }) } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index d7470c54b156f..204ab388064bb 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.TopicPartition import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test +import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.collection.mutable @@ -71,6 +72,7 @@ class TransactionsBounceTest extends IntegrationTestHarness { override protected def brokerCount: Int = 4 + @nowarn("cat=deprecation") @Test def testWithGroupId(): Unit = { testBrokerFailure((producer, groupId, consumer) => diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index b9b8015dacf7f..5afad8578927c 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -27,13 +27,14 @@ import kafka.integration.KafkaServerTestHarness import kafka.server.KafkaConfig import kafka.utils.TestUtils import kafka.utils.TestUtils.consumeRecords -import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, OffsetAndMetadata} +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerGroupMetadata, KafkaConsumer, OffsetAndMetadata} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFencedException, TimeoutException} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} +import scala.annotation.nowarn import scala.jdk.CollectionConverters._ import scala.collection.Seq import scala.collection.mutable.Buffer @@ -226,6 +227,7 @@ class TransactionsTest extends KafkaServerTestHarness { assertEquals(3L, second.offset) } + @nowarn("cat=deprecation") @Test def testSendOffsetsWithGroupId() = { sendOffset((producer, groupId, consumer) => @@ -355,7 +357,7 @@ class TransactionsTest extends KafkaServerTestHarness { producer2.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic2, null, "2", "4", willBeCommitted = true)) assertThrows(classOf[ProducerFencedException], () => producer1.sendOffsetsToTransaction(Map(new TopicPartition("foobartopic", 0) - -> new OffsetAndMetadata(110L)).asJava, "foobarGroup")) + -> new OffsetAndMetadata(110L)).asJava, new ConsumerGroupMetadata("foobarGroup"))) producer2.commitTransaction() // ok @@ -379,7 +381,7 @@ class TransactionsTest extends KafkaServerTestHarness { producer.beginTransaction() val offsetAndMetadata = new OffsetAndMetadata(110L, Optional.of(15), "some metadata") - producer.sendOffsetsToTransaction(Map(tp -> offsetAndMetadata).asJava, groupId) + producer.sendOffsetsToTransaction(Map(tp -> offsetAndMetadata).asJava, new ConsumerGroupMetadata(groupId)) producer.commitTransaction() // ok // The call to commit the transaction may return before all markers are visible, so we initialize a second @@ -398,7 +400,7 @@ class TransactionsTest extends KafkaServerTestHarness { @Test def testSendOffsetsToTransactionTimeout(): Unit = { testTimeout(true, producer => producer.sendOffsetsToTransaction( - Map(new TopicPartition(topic1, 0) -> new OffsetAndMetadata(0)).asJava, "test-group")) + Map(new TopicPartition(topic1, 0) -> new OffsetAndMetadata(0)).asJava, new ConsumerGroupMetadata("test-group"))) } @Test diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html index 7391c022f1ed0..884b39898ed9b 100644 --- a/docs/streams/core-concepts.html +++ b/docs/streams/core-concepts.html @@ -291,16 +291,18 @@

    KIP-129.
    - As of the 2.6.0 release, Kafka Streams supports an improved implementation of exactly-once processing, named "exactly-once beta", + As of the 2.6.0 release, Kafka Streams supports an improved implementation of exactly-once processing, named "exactly-once v2", which requires broker version 2.5.0 or newer. This implementation is more efficient, because it reduces client and broker resource utilization, like client threads and used network connections, and it enables higher throughput and improved scalability. + As of the 3.0.0 release, the first version of exactly-once has been deprecated. Users are encouraged to use exactly-once v2 for + exactly-once processing from now on, and prepare by upgrading their brokers if necessary. For more information on how this is done inside the brokers and Kafka Streams, see KIP-447.
    To enable exactly-once semantics when running Kafka Streams applications, set the processing.guarantee config value (default value is at_least_once) - to exactly_once for EOS version 1 (requires brokers version 0.11.0 or newer) or exactly_once_beta for EOS version 2 (requires brokers version 2.5 or newer). + to StreamsConfig.EXACTLY_ONCE_V2 (requires brokers version 2.5 or newer). For more information, see the Kafka Streams Configs section.

    diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index 8ce2d0f87dee0..1f5002f08914c 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -293,8 +293,9 @@

    bootstrap.serversprocessing.guarantee Medium - The processing mode. Can be either "at_least_once" (default), - "exactly_once" (for EOS version 1), or "exactly_once_beta" (for EOS version 2). + The processing mode. Can be either "at_least_once" (default) + or "exactly_once_v2" (for EOS version 2, requires broker version 2.5+). Deprecated config options are + "exactly_once" (for EOS version 1) and "exactly_once_beta" (for EOS version 2, requires broker version 2.5+). See Processing Guarantee poll.ms @@ -667,12 +668,14 @@

    probing.rebalance.interval.ms

    processing.guarantee

    The processing guarantee that should be used. - Possible values are "at_least_once" (default), - "exactly_once" (for EOS version 1), + Possible values are "at_least_once" (default) + and "exactly_once_v2" (for EOS version 2). + Deprecated config options are "exactly_once" (for EOS alpha), and "exactly_once_beta" (for EOS version 2). - Using "exactly_once" requires broker - version 0.11.0 or newer, while using "exactly_once_beta" - requires broker version 2.5 or newer. + Using "exactly_once_v2" (or the deprecated + "exactly_once_beta") requires broker version 2.5 or newer, + while using the deprecated "exactly_once" + requires broker version 0.11.0 or newer. Note that if exactly-once processing is enabled, the default for parameter commit.interval.ms changes to 100ms. Additionally, consumers are configured with isolation.level="read_committed" diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index cd0f67773d244..f7221ed28821b 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -53,16 +53,18 @@

    Upgrade Guide and API Changes

    - Starting in Kafka Streams 2.6.x, a new processing mode is available, named EOS version 2, which is configurable by setting - processing.guarantee to "exactly_once_beta". - NOTE: The "exactly_once_beta" processing mode is ready for production (i.e., it's not "beta" software). + Starting in Kafka Streams 2.6.x, a new processing mode is available, named EOS version 2. This can be configured + by setting "processing.guarantee" to "exactly_once" for + application versions 3.0+, or setting it to "exactly_once_beta" for versions between 2.6 and 2.8. To use this new feature, your brokers must be on version 2.5.x or newer. - A switch from "exactly_once" to "exactly_once_beta" (or the other way around) is - only possible if the application is on version 2.6.x. - If you want to upgrade your application from an older version and enable this feature, - you first need to upgrade your application to version 2.6.x, staying on "exactly_once", - and then do second round of rolling bounces to switch to "exactly_once_beta". - For a downgrade, do the reverse: first switch the config from "exactly_once_beta" to + If you want to upgrade your EOS application from an older version and enable this feature in version 3.0+, + you first need to upgrade your application to version 3.0.x, staying on "exactly_once", + and then do second round of rolling bounces to switch to "exactly_once_v2". If you + are upgrading an EOS application from an older (pre-2.6) version to a version between 2.6 and 2.8, follow these + same steps but with the config "exactly_once_beta" instead. No special steps are required + to upgrade an application using "exactly_once_beta" from version 2.6+ to 3.0 or higher: you can + just change the config from "exactly_once_beta" to "exactly_once_v2" during the rolling upgrade. + For a downgrade, do the reverse: first switch the config from "exactly_once_v2" to "exactly_once" to disable the feature in your 2.6.x application. Afterward, you can downgrade your application to a pre-2.6.x version.

    @@ -96,6 +98,14 @@

    Streams API

    A new exception may be thrown from KafkaStreams#store(). If the specified store name does not exist in the topology, an UnknownStateStoreException will be thrown instead of the former InvalidStateStoreException. See KIP-216 for more information.

    +

    + We deprecated the StreamsConfig processing.guarantee configuration value "exactly_once" (for EOS version 1) in favor of the improved EOS version 2, formerly configured via + "exactly_once_beta. To avoid confusion about the term "beta" in the config name and highlight the production-readiness of EOS version 2, we have also renamed "eos-beta" to "eos-v2" + and deprecated the configuration value "exactly_once_beta", replacing it with a new configuration value "exactly_once_v2" + Users of exactly-once semantics should plan to migrate to the eos-v2 config and prepare for the removal of the deprecated configs in 4.0 or after at least a year + from the release of 3.0, whichever comes last. Note that eos-v2 requires broker version 2.5 or higher, like eos-beta, so users should begin to upgrade their kafka cluster if necessary. See + KIP-732 for more details. +

    We removed the default implementation of RocksDBConfigSetter#close().

    diff --git a/docs/upgrade.html b/docs/upgrade.html index 6a632e0728cef..290ade4e9d179 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -66,6 +66,13 @@

    Notable changes in 3
  • The quota.producer.default and quota.consumer.default configurations were removed (KAFKA-12591). Dynamic quota defaults must be used instead.
  • +
  • The Producer#sendOffsetsToTransaction(Map offsets, String consumerGroupId) method has been deprecated. Please use + Producer#sendOffsetsToTransaction(Map offsets, ConsumerGroupMetadata metadata) instead, where the ConsumerGroupMetadata + can be retrieved via KafkaConsumer#groupMetadata() for stronger semantics. Note that the full set of consumer group metadata is only + understood by brokers or version 2.5 or higher, so you must upgrade your kafka cluster to get the stronger semantics. Otherwise, you can just pass + in new ConsumerGroupMetadata(consumerGroupId) to work with older brokers. See KIP-732 for more details. +
  • +
    Notable changes in 2.8.0
    diff --git a/release.py b/release.py index 751e375f7a984..eacee7f2af437 100755 --- a/release.py +++ b/release.py @@ -346,7 +346,7 @@ def command_release_announcement_email(): Apache Kafka is a distributed streaming platform with four core APIs: -** The Producer API allows an application to publish a stream records to +** The Producer API allows an application to publish a stream of records to one or more Kafka topics. ** The Consumer API allows an application to subscribe to one or more diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 4aaede745dad1..b619b1f6db340 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -122,7 +122,9 @@ *
  • {@link ConsumerConfig#PARTITION_ASSIGNMENT_STRATEGY_CONFIG "partition.assignment.strategy"} (StreamsPartitionAssignor) - Streams client will always use its own partition assignor
  • * * - * If {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} is set to {@link #EXACTLY_ONCE "exactly_once"}, Kafka Streams does not allow users to overwrite the following properties (Streams setting shown in parentheses): + * If {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} is set to {@link #EXACTLY_ONCE_V2 "exactly_once_v2"}, + * {@link #EXACTLY_ONCE "exactly_once"} (deprecated), or {@link #EXACTLY_ONCE_BETA "exactly_once_beta"} (deprecated), Kafka Streams does not + * allow users to overwrite the following properties (Streams setting shown in parentheses): *
      *
    • {@link ConsumerConfig#ISOLATION_LEVEL_CONFIG "isolation.level"} (read_committed) - Consumers will always read committed data only
    • *
    • {@link ProducerConfig#ENABLE_IDEMPOTENCE_CONFIG "enable.idempotence"} (true) - Producer will always have idempotency enabled
    • @@ -291,23 +293,35 @@ public class StreamsConfig extends AbstractConfig { *

      * Enabling exactly-once processing semantics requires broker version 0.11.0 or higher. * If you enable this feature Kafka Streams will use more resources (like broker connections) - * compared to the {@link #AT_LEAST_ONCE} case. + * compared to {@link #AT_LEAST_ONCE "at_least_once"} and {@link #EXACTLY_ONCE_V2 "exactly_once_v2"}. * - * @see #EXACTLY_ONCE_BETA + * @deprecated Since 3.0.0, will be removed in 4.0. Use {@link #EXACTLY_ONCE_V2 "exactly_once_v2"} instead. */ @SuppressWarnings("WeakerAccess") + @Deprecated public static final String EXACTLY_ONCE = "exactly_once"; /** * Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for exactly-once processing guarantees. *

      * Enabling exactly-once (beta) requires broker version 2.5 or higher. - * If you enable this feature Kafka Streams will use less resources (like broker connections) - * compare to the {@link #EXACTLY_ONCE} case. + * If you enable this feature Kafka Streams will use fewer resources (like broker connections) + * compared to the {@link #EXACTLY_ONCE} (deprecated) case. + * + * @deprecated Since 3.0.0, will be removed in 4.0. Use {@link #EXACTLY_ONCE_V2 "exactly_once_v2"} instead. */ @SuppressWarnings("WeakerAccess") + @Deprecated public static final String EXACTLY_ONCE_BETA = "exactly_once_beta"; + /** + * Config value for parameter {@link #PROCESSING_GUARANTEE_CONFIG "processing.guarantee"} for exactly-once processing guarantees. + *

      + * Enabling exactly-once-v2 requires broker version 2.5 or higher. + */ + @SuppressWarnings("WeakerAccess") + public static final String EXACTLY_ONCE_V2 = "exactly_once_v2"; + /** * Config value for parameter {@link #BUILT_IN_METRICS_VERSION_CONFIG "built.in.metrics.version"} for built-in metrics from version 0.10.0. to 2.4 */ @@ -361,7 +375,7 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; private static final String COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds with which to save the position of the processor." + - " (Note, if processing.guarantee is set to " + EXACTLY_ONCE + ", the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + + " (Note, if processing.guarantee is set to " + EXACTLY_ONCE_V2 + ", " + EXACTLY_ONCE + ",the default value is " + EOS_DEFAULT_COMMIT_INTERVAL_MS + "," + " otherwise the default value is " + DEFAULT_COMMIT_INTERVAL_MS + "."; /** {@code connections.max.idle.ms} */ @@ -464,8 +478,9 @@ public class StreamsConfig extends AbstractConfig { @SuppressWarnings("WeakerAccess") public static final String PROCESSING_GUARANTEE_CONFIG = "processing.guarantee"; private static final String PROCESSING_GUARANTEE_DOC = "The processing guarantee that should be used. " + - "Possible values are " + AT_LEAST_ONCE + " (default), " + - "" + EXACTLY_ONCE + " (requires brokers version 0.11.0 or higher), " + + "Possible values are " + AT_LEAST_ONCE + " (default) " + + "and " + EXACTLY_ONCE_V2 + " (requires brokers version 2.5 or higher). " + + "Deprecated options are " + EXACTLY_ONCE + " (requires brokers version 0.11.0 or higher) " + "and " + EXACTLY_ONCE_BETA + " (requires brokers version 2.5 or higher). " + "Note that exactly-once processing requires a cluster of at least three brokers by default what is the " + "recommended setting for production; for development you can change this, by adjusting broker setting " + @@ -682,7 +697,7 @@ public class StreamsConfig extends AbstractConfig { .define(PROCESSING_GUARANTEE_CONFIG, Type.STRING, AT_LEAST_ONCE, - in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA), + in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), Importance.MEDIUM, PROCESSING_GUARANTEE_DOC) .define(SECURITY_PROTOCOL_CONFIG, @@ -1011,8 +1026,20 @@ protected StreamsConfig(final Map props, final boolean doLog) { super(CONFIG, props, doLog); eosEnabled = StreamThread.eosEnabled(this); + + final String processingModeConfig = getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG); + if (processingModeConfig.equals(EXACTLY_ONCE)) { + log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + + "Please use `{}` instead. Note that this requires broker version 2.5+ so you should prepare " + + "to upgrade your brokers if necessary.", EXACTLY_ONCE, EXACTLY_ONCE_V2); + } + if (processingModeConfig.equals(EXACTLY_ONCE_BETA)) { + log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release. " + + "Please use `{}` instead.", EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2); + } + if (props.containsKey(RETRIES_CONFIG)) { - log.warn("Configuration parameter `{}` is deprecated and will be removed in 3.0.0 release.", RETRIES_CONFIG); + log.warn("Configuration parameter `{}` is deprecated and will be removed in the 4.0.0 release.", RETRIES_CONFIG); } } @@ -1037,7 +1064,7 @@ private Map getCommonConsumerConfigs() { checkIfUnexpectedUserSpecifiedConsumerConfig(clientProvidedProps, NON_CONFIGURABLE_CONSUMER_EOS_CONFIGS); final Map consumerProps = new HashMap<>(eosEnabled ? CONSUMER_EOS_OVERRIDES : CONSUMER_DEFAULT_OVERRIDES); - if (StreamThread.processingMode(this) == StreamThread.ProcessingMode.EXACTLY_ONCE_BETA) { + if (StreamThread.processingMode(this) == StreamThread.ProcessingMode.EXACTLY_ONCE_V2) { consumerProps.put("internal.throw.on.fetch.stable.offset.unsupported", true); } consumerProps.putAll(getClientCustomProps()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java index 86b76e3d84ed7..788ae737dc00f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreator.java @@ -46,7 +46,7 @@ import static org.apache.kafka.streams.processor.internals.ClientUtils.getTaskProducerClientId; import static org.apache.kafka.streams.processor.internals.ClientUtils.getThreadProducerClientId; import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_ALPHA; -import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_BETA; +import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_V2; class ActiveTaskCreator { private final InternalTopologyBuilder builder; @@ -92,7 +92,7 @@ class ActiveTaskCreator { if (processingMode == EXACTLY_ONCE_ALPHA) { threadProducer = null; taskProducers = new HashMap<>(); - } else { // non-eos and eos-beta + } else { // non-eos and eos-v2 log.info("Creating thread producer client"); final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName()); @@ -115,7 +115,7 @@ public void reInitializeThreadProducer() { StreamsProducer streamsProducerForTask(final TaskId taskId) { if (processingMode != EXACTLY_ONCE_ALPHA) { - throw new IllegalStateException("Producer per thread is used."); + throw new IllegalStateException("Expected EXACTLY_ONCE to be enabled, but the processing mode was " + processingMode); } final StreamsProducer taskProducer = taskProducers.get(taskId); @@ -126,8 +126,8 @@ StreamsProducer streamsProducerForTask(final TaskId taskId) { } StreamsProducer threadProducer() { - if (processingMode != EXACTLY_ONCE_BETA) { - throw new IllegalStateException("Exactly-once beta is not enabled."); + if (processingMode != EXACTLY_ONCE_V2) { + throw new IllegalStateException("Expected EXACTLY_ONCE_V2 to be enabled, but the processing mode was " + processingMode); } return threadProducer; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index c378de6526e68..fd6f9f87f2d05 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -63,8 +63,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE; -import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_BETA; import static org.apache.kafka.streams.processor.internals.ClientUtils.getConsumerClientId; import static org.apache.kafka.streams.processor.internals.ClientUtils.getRestoreConsumerClientId; import static org.apache.kafka.streams.processor.internals.ClientUtils.getSharedAdminClientId; @@ -304,10 +302,8 @@ public boolean isRunning() { private Runnable shutdownErrorHook; private AtomicInteger assignmentErrorCode; private AtomicLong cacheResizeSize; - private final ProcessingMode processingMode; private AtomicBoolean leaveGroupRequested; - public static StreamThread create(final InternalTopologyBuilder builder, final StreamsConfig config, final KafkaClientSupplier clientSupplier, @@ -382,7 +378,7 @@ public static StreamThread create(final InternalTopologyBuilder builder, builder, adminClient, stateDirectory, - StreamThread.processingMode(config) + processingMode(config) ); referenceContainer.taskManager = taskManager; @@ -430,7 +426,7 @@ public enum ProcessingMode { EXACTLY_ONCE_ALPHA("EXACTLY_ONCE_ALPHA"), - EXACTLY_ONCE_BETA("EXACTLY_ONCE_BETA"); + EXACTLY_ONCE_V2("EXACTLY_ONCE_V2"); public final String name; @@ -439,20 +435,28 @@ public enum ProcessingMode { } } + // Note: the below two methods are static methods here instead of methods on StreamsConfig because it's a public API + + @SuppressWarnings("deprecation") public static ProcessingMode processingMode(final StreamsConfig config) { - if (EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))) { + if (StreamsConfig.EXACTLY_ONCE.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))) { return StreamThread.ProcessingMode.EXACTLY_ONCE_ALPHA; - } else if (EXACTLY_ONCE_BETA.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))) { - return StreamThread.ProcessingMode.EXACTLY_ONCE_BETA; + } else if (StreamsConfig.EXACTLY_ONCE_BETA.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))) { + return StreamThread.ProcessingMode.EXACTLY_ONCE_V2; + } else if (StreamsConfig.EXACTLY_ONCE_V2.equals(config.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG))) { + return StreamThread.ProcessingMode.EXACTLY_ONCE_V2; } else { return StreamThread.ProcessingMode.AT_LEAST_ONCE; } } public static boolean eosEnabled(final StreamsConfig config) { - final ProcessingMode processingMode = processingMode(config); + return eosEnabled(processingMode(config)); + } + + public static boolean eosEnabled(final ProcessingMode processingMode) { return processingMode == ProcessingMode.EXACTLY_ONCE_ALPHA || - processingMode == ProcessingMode.EXACTLY_ONCE_BETA; + processingMode == ProcessingMode.EXACTLY_ONCE_V2; } public StreamThread(final Time time, @@ -494,7 +498,6 @@ public StreamThread(final Time time, this.shutdownErrorHook = shutdownErrorHook; this.streamsUncaughtExceptionHandler = streamsUncaughtExceptionHandler; this.cacheResizer = cacheResizer; - this.processingMode = processingMode(config); // The following sensors are created here but their references are not stored in this object, since within // this object they are not recorded. The sensors are created here so that the stream threads starts with all @@ -567,6 +570,7 @@ public void run() { * @throws IllegalStateException If store gets registered after initialized is already finished * @throws StreamsException if the store's change log does not contain the partition */ + @SuppressWarnings("deprecation") // Needed to include StreamsConfig.EXACTLY_ONCE_BETA in error log for UnsupportedVersionException boolean runLoop() { subscribeConsumer(); @@ -601,9 +605,9 @@ boolean runLoop() { errorMessage.startsWith("Broker unexpectedly doesn't support requireStable flag on version ")) { log.error("Shutting down because the Kafka cluster seems to be on a too old version. " + - "Setting {}=\"{}\" requires broker version 2.5 or higher.", + "Setting {}=\"{}\"/\"{}\" requires broker version 2.5 or higher.", StreamsConfig.PROCESSING_GUARANTEE_CONFIG, - EXACTLY_ONCE_BETA); + StreamsConfig.EXACTLY_ONCE_V2, StreamsConfig.EXACTLY_ONCE_BETA); } failedStreamThreadSensor.record(); this.streamsUncaughtExceptionHandler.accept(e); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java index 1102aab88241e..9b27c8b55f3bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsProducer.java @@ -50,7 +50,7 @@ import static org.apache.kafka.streams.processor.internals.ClientUtils.getTaskProducerClientId; import static org.apache.kafka.streams.processor.internals.ClientUtils.getThreadProducerClientId; -import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_BETA; +import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_V2; /** * {@code StreamsProducer} manages the producers within a Kafka Streams application. @@ -64,7 +64,7 @@ public class StreamsProducer { private final Logger log; private final String logPrefix; - private final Map eosBetaProducerConfigs; + private final Map eosV2ProducerConfigs; private final KafkaClientSupplier clientSupplier; private final StreamThread.ProcessingMode processingMode; @@ -90,7 +90,7 @@ public StreamsProducer(final StreamsConfig config, switch (processingMode) { case AT_LEAST_ONCE: { producerConfigs = config.getProducerConfigs(getThreadProducerClientId(threadId)); - eosBetaProducerConfigs = null; + eosV2ProducerConfigs = null; break; } @@ -105,21 +105,21 @@ public StreamsProducer(final StreamsConfig config, final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); producerConfigs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, applicationId + "-" + taskId); - eosBetaProducerConfigs = null; + eosV2ProducerConfigs = null; break; } - case EXACTLY_ONCE_BETA: { + case EXACTLY_ONCE_V2: { producerConfigs = config.getProducerConfigs(getThreadProducerClientId(threadId)); final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG); producerConfigs.put( ProducerConfig.TRANSACTIONAL_ID_CONFIG, applicationId + "-" + - Objects.requireNonNull(processId, "processId cannot be null for exactly-once beta") + + Objects.requireNonNull(processId, "processId cannot be null for exactly-once v2") + "-" + threadId.split("-StreamThread-")[1]); - eosBetaProducerConfigs = producerConfigs; + eosV2ProducerConfigs = producerConfigs; break; } @@ -135,8 +135,7 @@ private String formatException(final String message) { } boolean eosEnabled() { - return processingMode == StreamThread.ProcessingMode.EXACTLY_ONCE_ALPHA || - processingMode == StreamThread.ProcessingMode.EXACTLY_ONCE_BETA; + return StreamThread.eosEnabled(processingMode); } /** @@ -175,13 +174,13 @@ void initTransaction() { } public void resetProducer() { - if (processingMode != EXACTLY_ONCE_BETA) { - throw new IllegalStateException(formatException("Exactly-once beta is not enabled")); + if (processingMode != EXACTLY_ONCE_V2) { + throw new IllegalStateException("Expected eos-v2 to be enabled, but the processing mode was " + processingMode); } producer.close(); - producer = clientSupplier.getProducer(eosBetaProducerConfigs); + producer = clientSupplier.getProducer(eosV2ProducerConfigs); transactionInitialized = false; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index a7ef75ace067c..ffa0f563d4c2e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -65,7 +65,7 @@ import static org.apache.kafka.common.utils.Utils.intersection; import static org.apache.kafka.common.utils.Utils.union; import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_ALPHA; -import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_BETA; +import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_V2; public class TaskManager { // initialize the task list @@ -490,7 +490,7 @@ boolean tryToCompleteRestoration(final long now, final java.util.function.Consum * {@link #handleAssignment(Map, Map)} is called. Note that only active task partitions are passed in from the * rebalance listener, so we only need to consider/commit active tasks here * - * If eos-beta is used, we must commit ALL tasks. Otherwise, we can just commit those (active) tasks which are revoked + * If eos-v2 is used, we must commit ALL tasks. Otherwise, we can just commit those (active) tasks which are revoked * * @throws TaskMigratedException if the task producer got fenced (EOS only) */ @@ -629,7 +629,7 @@ void handleLostAll() { } } - if (processingMode == EXACTLY_ONCE_BETA) { + if (processingMode == EXACTLY_ONCE_V2) { tasks.reInitializeThreadProducer(); } } @@ -847,7 +847,7 @@ private Collection tryCloseCleanAllActiveTasks(final boolean clean, } // If any active tasks can't be committed, none of them can be, and all that need a commit must be closed dirty - if (processingMode == EXACTLY_ONCE_BETA && !tasksToCloseDirty.isEmpty()) { + if (processingMode == EXACTLY_ONCE_V2 && !tasksToCloseDirty.isEmpty()) { tasksToCloseClean.removeAll(tasksToCommit); tasksToCloseDirty.addAll(tasksToCommit); } else { @@ -1104,7 +1104,7 @@ private void commitOffsetsOrTransaction(final Map allOffsets = offsetsPerTask.values().stream() .flatMap(e -> e.entrySet().stream()).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - if (processingMode == EXACTLY_ONCE_BETA) { + if (processingMode == EXACTLY_ONCE_V2) { try { tasks.threadProducer().commitTransaction(allOffsets, mainConsumer.groupMetadata()); updateTaskMetadata(allOffsets); diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 85a1872a8ba90..684b60861f32b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -49,6 +49,7 @@ import static org.apache.kafka.common.IsolationLevel.READ_UNCOMMITTED; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE; import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_BETA; +import static org.apache.kafka.streams.StreamsConfig.EXACTLY_ONCE_V2; import static org.apache.kafka.streams.StreamsConfig.STATE_DIR_CONFIG; import static org.apache.kafka.streams.StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG; import static org.apache.kafka.streams.StreamsConfig.adminClientPrefix; @@ -491,6 +492,7 @@ public void shouldNotSetInternalThrowOnFetchStableOffsetUnsupportedConfigToFalse assertThat(consumerConfigs.get("internal.throw.on.fetch.stable.offset.unsupported"), is(nullValue())); } + @SuppressWarnings("deprecation") @Test public void shouldNotSetInternalThrowOnFetchStableOffsetUnsupportedConfigToFalseInConsumerForEosAlpha() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); @@ -499,6 +501,7 @@ public void shouldNotSetInternalThrowOnFetchStableOffsetUnsupportedConfigToFalse assertThat(consumerConfigs.get("internal.throw.on.fetch.stable.offset.unsupported"), is(nullValue())); } + @SuppressWarnings("deprecation") @Test public void shouldNotSetInternalThrowOnFetchStableOffsetUnsupportedConfigToFalseInConsumerForEosBeta() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); @@ -507,12 +510,21 @@ public void shouldNotSetInternalThrowOnFetchStableOffsetUnsupportedConfigToFalse assertThat(consumerConfigs.get("internal.throw.on.fetch.stable.offset.unsupported"), is(true)); } + @Test + public void shouldNotSetInternalThrowOnFetchStableOffsetUnsupportedConfigToFalseInConsumerForEosV2() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + final StreamsConfig streamsConfig = new StreamsConfig(props); + final Map consumerConfigs = streamsConfig.getMainConsumerConfigs(groupId, clientId, threadIdx); + assertThat(consumerConfigs.get("internal.throw.on.fetch.stable.offset.unsupported"), is(true)); + } + @Test public void shouldNotSetInternalAutoDowngradeTxnCommitToTrueInProducerForEosDisabled() { final Map producerConfigs = streamsConfig.getProducerConfigs(clientId); assertThat(producerConfigs.get("internal.auto.downgrade.txn.commit"), is(nullValue())); } + @SuppressWarnings("deprecation") @Test public void shouldSetInternalAutoDowngradeTxnCommitToTrueInProducerForEosAlpha() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); @@ -521,6 +533,7 @@ public void shouldSetInternalAutoDowngradeTxnCommitToTrueInProducerForEosAlpha() assertThat(producerConfigs.get("internal.auto.downgrade.txn.commit"), is(true)); } + @SuppressWarnings("deprecation") @Test public void shouldNotSetInternalAutoDowngradeTxnCommitToTrueInProducerForEosBeta() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); @@ -529,6 +542,14 @@ public void shouldNotSetInternalAutoDowngradeTxnCommitToTrueInProducerForEosBeta assertThat(producerConfigs.get("internal.auto.downgrade.txn.commit"), is(nullValue())); } + @Test + public void shouldNotSetInternalAutoDowngradeTxnCommitToTrueInProducerForEosV2() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + final StreamsConfig streamsConfig = new StreamsConfig(props); + final Map producerConfigs = streamsConfig.getProducerConfigs(clientId); + assertThat(producerConfigs.get("internal.auto.downgrade.txn.commit"), is(nullValue())); + } + @Test public void shouldAcceptAtLeastOnce() { // don't use `StreamsConfig.AT_LEAST_ONCE` to actually do a useful test @@ -587,18 +608,26 @@ public void shouldThrowIfBuiltInMetricsVersionInvalid() { ); } + @SuppressWarnings("deprecation") @Test public void shouldResetToDefaultIfConsumerIsolationLevelIsOverriddenIfEosAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldResetToDefaultIfConsumerIsolationLevelIsOverriddenIfEosEnabled(); } + @SuppressWarnings("deprecation") @Test public void shouldResetToDefaultIfConsumerIsolationLevelIsOverriddenIfEosBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldResetToDefaultIfConsumerIsolationLevelIsOverriddenIfEosEnabled(); } + @Test + public void shouldResetToDefaultIfConsumerIsolationLevelIsOverriddenIfEosV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldResetToDefaultIfConsumerIsolationLevelIsOverriddenIfEosEnabled(); + } + private void shouldResetToDefaultIfConsumerIsolationLevelIsOverriddenIfEosEnabled() { props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "anyValue"); final StreamsConfig streamsConfig = new StreamsConfig(props); @@ -620,18 +649,26 @@ public void shouldAllowSettingConsumerIsolationLevelIfEosDisabled() { ); } + @SuppressWarnings("deprecation") @Test public void shouldResetToDefaultIfProducerEnableIdempotenceIsOverriddenIfEosAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldResetToDefaultIfProducerEnableIdempotenceIsOverriddenIfEosEnabled(); } + @SuppressWarnings("deprecation") @Test public void shouldResetToDefaultIfProducerEnableIdempotenceIsOverriddenIfEosBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldResetToDefaultIfProducerEnableIdempotenceIsOverriddenIfEosEnabled(); } + @Test + public void shouldResetToDefaultIfProducerEnableIdempotenceIsOverriddenIfEosV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldResetToDefaultIfProducerEnableIdempotenceIsOverriddenIfEosEnabled(); + } + private void shouldResetToDefaultIfProducerEnableIdempotenceIsOverriddenIfEosEnabled() { props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "anyValue"); final StreamsConfig streamsConfig = new StreamsConfig(props); @@ -647,18 +684,26 @@ public void shouldAllowSettingProducerEnableIdempotenceIfEosDisabled() { assertThat(producerConfigs.get(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG), equalTo(false)); } + @SuppressWarnings("deprecation") @Test public void shouldSetDifferentDefaultsIfEosAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldSetDifferentDefaultsIfEosEnabled(); } + @SuppressWarnings("deprecation") @Test public void shouldSetDifferentDefaultsIfEosBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldSetDifferentDefaultsIfEosEnabled(); } + @Test + public void shouldSetDifferentDefaultsIfEosV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldSetDifferentDefaultsIfEosEnabled(); + } + private void shouldSetDifferentDefaultsIfEosEnabled() { final StreamsConfig streamsConfig = new StreamsConfig(props); @@ -675,18 +720,26 @@ private void shouldSetDifferentDefaultsIfEosEnabled() { assertThat(streamsConfig.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG), equalTo(100L)); } + @SuppressWarnings("deprecation") @Test public void shouldOverrideUserConfigTransactionalIdIfEosAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldOverrideUserConfigTransactionalIdIfEosEnable(); } + @SuppressWarnings("deprecation") @Test - public void shouldOverrideUserConfigTransactionalIdIfEosBeatEnabled() { + public void shouldOverrideUserConfigTransactionalIdIfEosBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldOverrideUserConfigTransactionalIdIfEosEnable(); } + @Test + public void shouldOverrideUserConfigTransactionalIdIfEosV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldOverrideUserConfigTransactionalIdIfEosEnable(); + } + private void shouldOverrideUserConfigTransactionalIdIfEosEnable() { props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "user-TxId"); final StreamsConfig streamsConfig = new StreamsConfig(props); @@ -696,18 +749,26 @@ private void shouldOverrideUserConfigTransactionalIdIfEosEnable() { assertThat(producerConfigs.get(ProducerConfig.TRANSACTIONAL_ID_CONFIG), is(nullValue())); } + @SuppressWarnings("deprecation") @Test public void shouldNotOverrideUserConfigRetriesIfExactlyAlphaOnceEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldNotOverrideUserConfigRetriesIfExactlyOnceEnabled(); } + @SuppressWarnings("deprecation") @Test public void shouldNotOverrideUserConfigRetriesIfExactlyBetaOnceEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldNotOverrideUserConfigRetriesIfExactlyOnceEnabled(); } + @Test + public void shouldNotOverrideUserConfigRetriesIfExactlyV2OnceEnabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldNotOverrideUserConfigRetriesIfExactlyOnceEnabled(); + } + private void shouldNotOverrideUserConfigRetriesIfExactlyOnceEnabled() { final int numberOfRetries = 42; props.put(ProducerConfig.RETRIES_CONFIG, numberOfRetries); @@ -718,18 +779,26 @@ private void shouldNotOverrideUserConfigRetriesIfExactlyOnceEnabled() { assertThat(producerConfigs.get(ProducerConfig.RETRIES_CONFIG), equalTo(numberOfRetries)); } + @SuppressWarnings("deprecation") @Test public void shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceEnabled(); } + @SuppressWarnings("deprecation") @Test public void shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceEnabled(); } + @Test + public void shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceEnabled(); + } + private void shouldNotOverrideUserConfigCommitIntervalMsIfExactlyOnceEnabled() { final long commitIntervalMs = 73L; props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, commitIntervalMs); @@ -806,18 +875,26 @@ public void shouldSpecifyCorrectValueSerdeClassOnError() { } } + @SuppressWarnings("deprecation") @Test public void shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosEnabled(); } + @SuppressWarnings("deprecation") @Test public void shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosEnabled(); } + @Test + public void shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosEnabled(); + } + private void shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosEnabled() { props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 7); final StreamsConfig streamsConfig = new StreamsConfig(props); @@ -833,36 +910,52 @@ private void shouldThrowExceptionIfMaxInFlightRequestsGreaterThanFiveIfEosEnable } } + @SuppressWarnings("deprecation") @Test public void shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosEnabled(); } + @SuppressWarnings("deprecation") @Test - public void shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosBeataEnabled() { + public void shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosEnabled(); } + @Test + public void shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosEnabled(); + } + private void shouldAllowToSpecifyMaxInFlightRequestsPerConnectionAsStringIfEosEnabled() { props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "3"); new StreamsConfig(props).getProducerConfigs(clientId); } + @SuppressWarnings("deprecation") @Test public void shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosAlphaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE); shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosEnabled(); } + @SuppressWarnings("deprecation") @Test public void shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosBetaEnabled() { props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_BETA); shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosEnabled(); } + @Test + public void shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosV2Enabled() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, EXACTLY_ONCE_V2); + shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosEnabled(); + } + private void shouldThrowConfigExceptionIfMaxInFlightRequestsPerConnectionIsInvalidStringIfEosEnabled() { props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "not-a-number"); @@ -907,6 +1000,44 @@ public void shouldThrowConfigExceptionWhenOptimizationConfigNotValueInRange() { assertThrows(ConfigException.class, () -> new StreamsConfig(props)); } + @SuppressWarnings("deprecation") + @Test + public void shouldLogWarningWhenEosAlphaIsUsed() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); + + LogCaptureAppender.setClassLoggerToDebug(StreamsConfig.class); + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StreamsConfig.class)) { + new StreamsConfig(props); + + assertThat( + appender.getMessages(), + hasItem("Configuration parameter `" + StreamsConfig.EXACTLY_ONCE + + "` is deprecated and will be removed in the 4.0.0 release. " + + "Please use `" + StreamsConfig.EXACTLY_ONCE_V2 + "` instead. " + + "Note that this requires broker version 2.5+ so you should prepare " + + "to upgrade your brokers if necessary.") + ); + } + } + + @SuppressWarnings("deprecation") + @Test + public void shouldLogWarningWhenEosBetaIsUsed() { + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + + LogCaptureAppender.setClassLoggerToDebug(StreamsConfig.class); + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StreamsConfig.class)) { + new StreamsConfig(props); + + assertThat( + appender.getMessages(), + hasItem("Configuration parameter `" + StreamsConfig.EXACTLY_ONCE_BETA + + "` is deprecated and will be removed in the 4.0.0 release. " + + "Please use `" + StreamsConfig.EXACTLY_ONCE_V2 + "` instead.") + ); + } + } + @SuppressWarnings("deprecation") @Test public void shouldLogWarningWhenRetriesIsUsed() { @@ -919,7 +1050,7 @@ public void shouldLogWarningWhenRetriesIsUsed() { assertThat( appender.getMessages(), hasItem("Configuration parameter `" + StreamsConfig.RETRIES_CONFIG + - "` is deprecated and will be removed in 3.0.0 release.") + "` is deprecated and will be removed in the 4.0.0 release.") ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EOSUncleanShutdownIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EOSUncleanShutdownIntegrationTest.java index 025f1057ddcf3..ab6e2a3cd88d5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EOSUncleanShutdownIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EOSUncleanShutdownIntegrationTest.java @@ -67,11 +67,12 @@ @Category(IntegrationTest.class) public class EOSUncleanShutdownIntegrationTest { + @SuppressWarnings("deprecation") @Parameterized.Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new String[][] { {StreamsConfig.EXACTLY_ONCE}, - {StreamsConfig.EXACTLY_ONCE_BETA} + {StreamsConfig.EXACTLY_ONCE_V2} }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java index bb04267963644..41febbc21476f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java @@ -135,11 +135,12 @@ public static void closeCluster() { private volatile boolean hasUnexpectedError = false; + @SuppressWarnings("deprecation") @Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new String[][] { {StreamsConfig.EXACTLY_ONCE}, - {StreamsConfig.EXACTLY_ONCE_BETA} + {StreamsConfig.EXACTLY_ONCE_V2} }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/EosBetaUpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java similarity index 95% rename from streams/src/test/java/org/apache/kafka/streams/integration/EosBetaUpgradeIntegrationTest.java rename to streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java index 02b71551ee813..6a3f27fabbd8e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/EosBetaUpgradeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/EosV2UpgradeIntegrationTest.java @@ -89,7 +89,7 @@ @RunWith(Parameterized.class) @Category({IntegrationTest.class}) -public class EosBetaUpgradeIntegrationTest { +public class EosV2UpgradeIntegrationTest { @Parameterized.Parameters(name = "{0}") public static Collection data() { @@ -180,15 +180,16 @@ public void createTopics() throws Exception { CLUSTER.createTopic(MULTI_PARTITION_OUTPUT_TOPIC, NUM_TOPIC_PARTITIONS, 1); } + @SuppressWarnings("deprecation") @Test - public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { - // We use two KafkaStreams clients that we upgrade from eos-alpha to eos-beta. During the upgrade, + public void shouldUpgradeFromEosAlphaToEosV2() throws Exception { + // We use two KafkaStreams clients that we upgrade from eos-alpha to eos-V2. During the upgrade, // we ensure that there are pending transaction and verify that data is processed correctly. // // We either close clients cleanly (`injectError = false`) or let them crash (`injectError = true`) during // the upgrade. For both cases, EOS should not be violated. // - // Additionally, we inject errors while one client is on eos-alpha while the other client is on eos-beta: + // Additionally, we inject errors while one client is on eos-alpha while the other client is on eos-V2: // For this case, we inject the error during task commit phase, i.e., after offsets are appended to a TX, // and before the TX is committed. The goal is to verify that the written but uncommitted offsets are not // picked up, i.e., GroupCoordinator fencing works correctly. @@ -209,22 +210,22 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { // - crash case: // * the pending transactions of the crashed client got aborted // * the second client will have four pending transactions - // 5. restart the first client with eos-beta enabled and wait until rebalance stabilizes + // 5. restart the first client with eos-V2 enabled and wait until rebalance stabilizes // - the rebalance should result in a commit of all tasks // 6. write 5 record per input topic partition // - stop case: // * verify that the result was committed // - crash case: // * fail the second (i.e., eos-alpha) client during commit - // * the eos-beta client should not pickup the pending offsets + // * the eos-V2 client should not pickup the pending offsets // * verify uncommitted and committed result // 7. only for crash case: // 7a. restart the second client in eos-alpha mode and wait until rebalance stabilizes // 7b. write 10 records per input topic partition - // * fail the first (i.e., eos-beta) client during commit + // * fail the first (i.e., eos-V2) client during commit // * the eos-alpha client should not pickup the pending offsets // * verify uncommitted and committed result - // 7c. restart the first client in eos-beta mode and wait until rebalance stabilizes + // 7c. restart the first client in eos-V2 mode and wait until rebalance stabilizes // 8. write 5 records per input topic partition to get pending transactions (verified via "read_uncommitted" mode) // - 2 transaction are base on a task producer; one transaction is based on a thread producer // - we will get 4 pending writes for the crash case as we crash processing the 5th record @@ -235,19 +236,19 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { // - crash case: // * the pending transactions of the crashed client got aborted // * the first client will have one pending transactions - // 10. restart the second client with eos-beta enabled and wait until rebalance stabilizes + // 10. restart the second client with eos-V2 enabled and wait until rebalance stabilizes // - the rebalance should result in a commit of all tasks // 11. write 5 record per input topic partition and verify that the result was committed final List> stateTransitions1 = new LinkedList<>(); KafkaStreams streams1Alpha = null; - KafkaStreams streams1Beta = null; - KafkaStreams streams1BetaTwo = null; + KafkaStreams streams1V2 = null; + KafkaStreams streams1V2Two = null; final List> stateTransitions2 = new LinkedList<>(); KafkaStreams streams2Alpha = null; KafkaStreams streams2AlphaTwo = null; - KafkaStreams streams2Beta = null; + KafkaStreams streams2V2 = null; try { // phase 1: start both clients @@ -428,17 +429,17 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { commitRequested.set(0); stateTransitions1.clear(); stateTransitions2.clear(); - streams1Beta = getKafkaStreams(APP_DIR_1, StreamsConfig.EXACTLY_ONCE_BETA); - streams1Beta.setStateListener((newState, oldState) -> stateTransitions1.add(KeyValue.pair(oldState, newState))); + streams1V2 = getKafkaStreams(APP_DIR_1, StreamsConfig.EXACTLY_ONCE_V2); + streams1V2.setStateListener((newState, oldState) -> stateTransitions1.add(KeyValue.pair(oldState, newState))); assignmentListener.prepareForRebalance(); - streams1Beta.start(); + streams1V2.start(); assignmentListener.waitForNextStableAssignment(MAX_WAIT_TIME_MS); waitForRunning(stateTransitions1); waitForRunning(stateTransitions2); final Set newlyCommittedKeys; if (!injectError) { - newlyCommittedKeys = keysFromInstance(streams1Beta); + newlyCommittedKeys = keysFromInstance(streams1V2); newlyCommittedKeys.removeAll(keysFirstClientAlpha); } else { newlyCommittedKeys = mkSet(0L, 1L, 2L, 3L); @@ -464,7 +465,7 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { // p-2: 10 rec + C + 5 rec ---> 5 rec + C // p-3: 10 rec + C + 5 rec ---> 5 rec + C // crash case: (second/alpha client fails and both TX are aborted) - // (first/beta client reprocessed the 10 records and commits TX) + // (first/V2 client reprocessed the 10 records and commits TX) // p-0: 10 rec + C + 4 rec + A + 5 rec + C ---> 5 rec + C // p-1: 10 rec + C + 5 rec + A + 5 rec + C ---> 5 rec + C // p-2: 10 rec + C + 5 rec + C ---> 5 rec + A + 5 rec + C @@ -491,11 +492,11 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { computeExpectedResult(committedInputDataDuringUpgrade, committedState); verifyCommitted(expectedCommittedResult); } else { - final Set keysFirstClientBeta = keysFromInstance(streams1Beta); + final Set keysFirstClientV2 = keysFromInstance(streams1V2); final Set keysSecondClientAlpha = keysFromInstance(streams2Alpha); final List> committedInputDataAfterFirstUpgrade = - prepareData(15L, 20L, keysFirstClientBeta.toArray(new Long[0])); + prepareData(15L, 20L, keysFirstClientV2.toArray(new Long[0])); writeInputData(committedInputDataAfterFirstUpgrade); final List> expectedCommittedResultBeforeFailure = @@ -554,10 +555,10 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { // 7. only for crash case: // 7a. restart the failed second client in eos-alpha mode and wait until rebalance stabilizes // 7b. write third batch of input data - // * fail the first (i.e., eos-beta) client during commit + // * fail the first (i.e., eos-V2) client during commit // * the eos-alpha client should not pickup the pending offsets // * verify uncommitted and committed result - // 7c. restart the first client in eos-beta mode and wait until rebalance stabilizes + // 7c. restart the first client in eos-V2 mode and wait until rebalance stabilizes // // crash case: // p-0: 10 rec + C + 4 rec + A + 5 rec + C + 5 rec + C ---> 10 rec + A + 10 rec + C @@ -583,7 +584,7 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { waitForRunning(stateTransitions2); // 7b. write third batch of input data - final Set keysFirstClientBeta = keysFromInstance(streams1Beta); + final Set keysFirstClientV2 = keysFromInstance(streams1V2); final Set keysSecondClientAlphaTwo = keysFromInstance(streams2AlphaTwo); final List> committedInputDataBetweenUpgrades = @@ -597,12 +598,12 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { commitCounterClient2.set(0); - final Iterator it = keysFirstClientBeta.iterator(); + final Iterator it = keysFirstClientV2.iterator(); final Long otherKey = it.next(); final Long failingKey = it.next(); final List> uncommittedInputDataBetweenUpgrade = - prepareData(20L, 29L, keysFirstClientBeta.toArray(new Long[0])); + prepareData(20L, 29L, keysFirstClientV2.toArray(new Long[0])); uncommittedInputDataBetweenUpgrade.addAll(prepareData(29L, 30L, otherKey)); writeInputData(uncommittedInputDataBetweenUpgrade); @@ -633,7 +634,7 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { commitErrorInjectedClient1.set(false); stateTransitions1.clear(); - streams1Beta.close(); + streams1V2.close(); assertFalse(UNEXPECTED_EXCEPTION_MSG, hasUnexpectedError); final List> expectedCommittedResultAfterFailure = @@ -641,13 +642,13 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { verifyCommitted(expectedCommittedResultAfterFailure); expectedUncommittedResult.addAll(expectedCommittedResultAfterFailure); - // 7c. restart the first client in eos-beta mode and wait until rebalance stabilizes + // 7c. restart the first client in eos-V2 mode and wait until rebalance stabilizes stateTransitions1.clear(); stateTransitions2.clear(); - streams1BetaTwo = getKafkaStreams(APP_DIR_1, StreamsConfig.EXACTLY_ONCE_BETA); - streams1BetaTwo.setStateListener((newState, oldState) -> stateTransitions1.add(KeyValue.pair(oldState, newState))); + streams1V2Two = getKafkaStreams(APP_DIR_1, StreamsConfig.EXACTLY_ONCE_V2); + streams1V2Two.setStateListener((newState, oldState) -> stateTransitions1.add(KeyValue.pair(oldState, newState))); assignmentListener.prepareForRebalance(); - streams1BetaTwo.start(); + streams1V2Two.start(); assignmentListener.waitForNextStableAssignment(MAX_WAIT_TIME_MS); waitForRunning(stateTransitions1); waitForRunning(stateTransitions2); @@ -765,12 +766,12 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { // the state below indicate the case for which the "original" tasks of client2 are migrated back to client2 // if a task "switch" happens, we might get additional commits (omitted in the comment for brevity) // - // stop case: (client 1 (beta) will commit all four tasks if at least one revoked and migrate task needs committing back to client 2) + // stop case: (client 1 (V2) will commit all four tasks if at least one revoked and migrate task needs committing back to client 2) // p-0: 10 rec + C + 5 rec + C + 5 rec + C + 5 rec ---> C // p-1: 10 rec + C + 5 rec + C + 5 rec + C + 5 rec ---> C // p-2: 10 rec + C + 5 rec + C + 5 rec + C + 5 rec + C // p-3: 10 rec + C + 5 rec + C + 5 rec + C + 5 rec + C - // crash case: (client 1 (beta) will commit all four tasks even only two are migrate back to client 2) + // crash case: (client 1 (V2) will commit all four tasks even only two are migrate back to client 2) // p-0: 10 rec + C + 4 rec + A + 5 rec + C + 5 rec + C + 10 rec + A + 10 rec + C + 5 rec ---> C // p-1: 10 rec + C + 5 rec + A + 5 rec + C + 5 rec + C + 10 rec + A + 10 rec + C + 5 rec ---> C // p-2: 10 rec + C + 5 rec + C + 5 rec + A + 5 rec + C + 10 rec + C + 4 rec + A + 5 rec ---> C @@ -778,19 +779,19 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { commitRequested.set(0); stateTransitions1.clear(); stateTransitions2.clear(); - streams2Beta = getKafkaStreams(APP_DIR_1, StreamsConfig.EXACTLY_ONCE_BETA); - streams2Beta.setStateListener( + streams2V2 = getKafkaStreams(APP_DIR_1, StreamsConfig.EXACTLY_ONCE_V2); + streams2V2.setStateListener( (newState, oldState) -> stateTransitions2.add(KeyValue.pair(oldState, newState)) ); assignmentListener.prepareForRebalance(); - streams2Beta.start(); + streams2V2.start(); assignmentListener.waitForNextStableAssignment(MAX_WAIT_TIME_MS); waitForRunning(stateTransitions1); waitForRunning(stateTransitions2); newlyCommittedKeys.clear(); if (!injectError) { - newlyCommittedKeys.addAll(keysFromInstance(streams2Beta)); + newlyCommittedKeys.addAll(keysFromInstance(streams2V2)); newlyCommittedKeys.removeAll(keysSecondClientAlphaTwo); } else { newlyCommittedKeys.addAll(mkSet(0L, 1L, 2L, 3L)); @@ -843,11 +844,11 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { if (streams1Alpha != null) { streams1Alpha.close(); } - if (streams1Beta != null) { - streams1Beta.close(); + if (streams1V2 != null) { + streams1V2.close(); } - if (streams1BetaTwo != null) { - streams1BetaTwo.close(); + if (streams1V2Two != null) { + streams1V2Two.close(); } if (streams2Alpha != null) { streams2Alpha.close(); @@ -855,8 +856,8 @@ public void shouldUpgradeFromEosAlphaToEosBeta() throws Exception { if (streams2AlphaTwo != null) { streams2AlphaTwo.close(); } - if (streams2Beta != null) { - streams2Beta.close(); + if (streams2V2 != null) { + streams2V2.close(); } } } @@ -886,7 +887,7 @@ public Transformer> get() { @Override public void init(final ProcessorContext context) { this.context = context; - state = (KeyValueStore) context.getStateStore(storeName); + state = context.getStateStore(storeName); final String clientId = context.appConfigs().get(StreamsConfig.CLIENT_ID_CONFIG).toString(); if (APP_DIR_1.equals(clientId)) { crash = errorInjectedClient1; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java index afdd150bb1c30..2316d75547bfb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableEOSIntegrationTest.java @@ -87,12 +87,12 @@ public static void closeCluster() { CLUSTER.stop(); } - + @SuppressWarnings("deprecation") @Parameterized.Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new String[][] { {StreamsConfig.EXACTLY_ONCE}, - {StreamsConfig.EXACTLY_ONCE_BETA} + {StreamsConfig.EXACTLY_ONCE_V2} }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java index f544b5f0b0709..dd3873c7ee6bf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableSourceTopicRestartIntegrationTest.java @@ -145,6 +145,7 @@ public void shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosDi } } + @SuppressWarnings("deprecation") @Test public void shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosAlphaEnabled() throws Exception { STREAMS_CONFIG.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -152,8 +153,8 @@ public void shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosAl } @Test - public void shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosBetaEnabled() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosV2Enabled() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); shouldRestoreAndProgressWhenTopicWrittenToDuringRestorationWithEosEnabled(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java index 407c222286a77..435ce9fd1d4ca 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetPartitionTimeIntegrationTest.java @@ -91,12 +91,13 @@ public static void closeCluster() { private static final int DEFAULT_TIMEOUT = 100; private static long lastRecordedTimestamp = -2L; + @SuppressWarnings("deprecation") @Parameterized.Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new String[][] { {StreamsConfig.AT_LEAST_ONCE}, {StreamsConfig.EXACTLY_ONCE}, - {StreamsConfig.EXACTLY_ONCE_BETA} + {StreamsConfig.EXACTLY_ONCE_V2} }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java index d22b7adb4ae7e..648cfdad31b1b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/RocksDBMetricsIntegrationTest.java @@ -134,12 +134,13 @@ public static void closeCluster() { private static final String ESTIMATED_MEMORY_OF_TABLE_READERS = "estimate-table-readers-mem"; private static final String NUMBER_OF_BACKGROUND_ERRORS = "background-errors"; + @SuppressWarnings("deprecation") @Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new Object[][] { {StreamsConfig.AT_LEAST_ONCE}, {StreamsConfig.EXACTLY_ONCE}, - {StreamsConfig.EXACTLY_ONCE_BETA} + {StreamsConfig.EXACTLY_ONCE_V2} }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java index c44899839932d..5ba607528faca 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java @@ -79,11 +79,12 @@ public class StandbyTaskEOSIntegrationTest { private final static int KEY_0 = 0; private final static int KEY_1 = 1; + @SuppressWarnings("deprecation") @Parameterized.Parameters(name = "{0}") public static Collection data() { return asList(new String[][] { {StreamsConfig.EXACTLY_ONCE}, - {StreamsConfig.EXACTLY_ONCE_BETA} + {StreamsConfig.EXACTLY_ONCE_V2} }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionDurabilityIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionDurabilityIntegrationTest.java index 3b5567563b48e..b6e9676abcc8d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionDurabilityIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionDurabilityIntegrationTest.java @@ -85,7 +85,6 @@ @RunWith(Parameterized.class) @Category({IntegrationTest.class}) public class SuppressionDurabilityIntegrationTest { - private static final Logger LOG = LoggerFactory.getLogger(SuppressionDurabilityIntegrationTest.class); public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster( 3, @@ -112,12 +111,13 @@ public static void closeCluster() { private static final LongDeserializer LONG_DESERIALIZER = new LongDeserializer(); private static final int COMMIT_INTERVAL = 100; + @SuppressWarnings("deprecation") @Parameterized.Parameters(name = "{0}") public static Collection data() { return Arrays.asList(new String[][] { {StreamsConfig.AT_LEAST_ONCE}, {StreamsConfig.EXACTLY_ONCE}, - {StreamsConfig.EXACTLY_ONCE_BETA} + {StreamsConfig.EXACTLY_ONCE_V2} }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java index 3541c10cae2c4..54c92dfef4caf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ActiveTaskCreatorTest.java @@ -128,7 +128,7 @@ public void shouldFailOnStreamsProducerPerTaskIfEosDisabled() { () -> activeTaskCreator.streamsProducerForTask(null) ); - assertThat(thrown.getMessage(), is("Producer per thread is used.")); + assertThat(thrown.getMessage(), is("Expected EXACTLY_ONCE to be enabled, but the processing mode was AT_LEAST_ONCE")); } @Test @@ -140,7 +140,7 @@ public void shouldFailOnGetThreadProducerIfEosDisabled() { activeTaskCreator::threadProducer ); - assertThat(thrown.getMessage(), is("Exactly-once beta is not enabled.")); + assertThat(thrown.getMessage(), is("Expected EXACTLY_ONCE_V2 to be enabled, but the processing mode was AT_LEAST_ONCE")); } @Test @@ -163,6 +163,7 @@ public void shouldThrowStreamsExceptionOnErrorCloseThreadProducerIfEosDisabled() // functional test + @SuppressWarnings("deprecation") @Test public void shouldReturnStreamsProducerPerTaskIfEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -170,6 +171,7 @@ public void shouldReturnStreamsProducerPerTaskIfEosAlphaEnabled() { shouldReturnStreamsProducerPerTask(); } + @SuppressWarnings("deprecation") @Test public void shouldConstructProducerMetricsWithEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -177,6 +179,7 @@ public void shouldConstructProducerMetricsWithEosAlphaEnabled() { shouldConstructProducerMetricsPerTask(); } + @SuppressWarnings("deprecation") @Test public void shouldConstructClientIdWithEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -188,6 +191,7 @@ public void shouldConstructClientIdWithEosAlphaEnabled() { assertThat(clientIds, is(mkSet("clientId-StreamThread-0-0_0-producer", "clientId-StreamThread-0-0_1-producer"))); } + @SuppressWarnings("deprecation") @Test public void shouldNoOpCloseThreadProducerIfEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -200,6 +204,7 @@ public void shouldNoOpCloseThreadProducerIfEosAlphaEnabled() { assertThat(mockClientSupplier.producers.get(1).closed(), is(false)); } + @SuppressWarnings("deprecation") @Test public void shouldCloseTaskProducersIfEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -221,6 +226,7 @@ public void shouldCloseTaskProducersIfEosAlphaEnabled() { // error handling + @SuppressWarnings("deprecation") @Test public void shouldFailForUnknownTaskOnStreamsProducerPerTaskIfEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -246,6 +252,7 @@ public void shouldFailForUnknownTaskOnStreamsProducerPerTaskIfEosAlphaEnabled() } } + @SuppressWarnings("deprecation") @Test public void shouldFailOnGetThreadProducerIfEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -258,9 +265,10 @@ public void shouldFailOnGetThreadProducerIfEosAlphaEnabled() { activeTaskCreator::threadProducer ); - assertThat(thrown.getMessage(), is("Exactly-once beta is not enabled.")); + assertThat(thrown.getMessage(), is("Expected EXACTLY_ONCE_V2 to be enabled, but the processing mode was EXACTLY_ONCE_ALPHA")); } + @SuppressWarnings("deprecation") @Test public void shouldThrowStreamsExceptionOnErrorCloseTaskProducerIfEosAlphaEnabled() { properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); @@ -282,13 +290,13 @@ public void shouldThrowStreamsExceptionOnErrorCloseTaskProducerIfEosAlphaEnabled - // eos-beta test + // eos-v2 test // functional test @Test - public void shouldReturnThreadProducerIfEosBetaEnabled() { - properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldReturnThreadProducerIfEosV2Enabled() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); mockClientSupplier.setApplicationIdForProducer("appId"); createTasks(); @@ -300,16 +308,16 @@ public void shouldReturnThreadProducerIfEosBetaEnabled() { } @Test - public void shouldConstructProducerMetricsWithEosBetaEnabled() { - properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldConstructProducerMetricsWithEosV2Enabled() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); mockClientSupplier.setApplicationIdForProducer("appId"); shouldConstructThreadProducerMetric(); } @Test - public void shouldConstructClientIdWithEosBetaEnabled() { - properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldConstructClientIdWithEosV2Enabled() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); mockClientSupplier.setApplicationIdForProducer("appId"); createTasks(); @@ -319,8 +327,8 @@ public void shouldConstructClientIdWithEosBetaEnabled() { } @Test - public void shouldCloseThreadProducerIfEosBetaEnabled() { - properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldCloseThreadProducerIfEosV2Enabled() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); mockClientSupplier.setApplicationIdForProducer("appId"); createTasks(); @@ -330,8 +338,8 @@ public void shouldCloseThreadProducerIfEosBetaEnabled() { } @Test - public void shouldNoOpCloseTaskProducerIfEosBetaEnabled() { - properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldNoOpCloseTaskProducerIfEosV2Enabled() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); mockClientSupplier.setApplicationIdForProducer("appId"); createTasks(); @@ -345,8 +353,8 @@ public void shouldNoOpCloseTaskProducerIfEosBetaEnabled() { // error handling @Test - public void shouldFailOnStreamsProducerPerTaskIfEosBetaEnabled() { - properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldFailOnStreamsProducerPerTaskIfEosV2Enabled() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); mockClientSupplier.setApplicationIdForProducer("appId"); createTasks(); @@ -356,12 +364,12 @@ public void shouldFailOnStreamsProducerPerTaskIfEosBetaEnabled() { () -> activeTaskCreator.streamsProducerForTask(null) ); - assertThat(thrown.getMessage(), is("Producer per thread is used.")); + assertThat(thrown.getMessage(), is("Expected EXACTLY_ONCE to be enabled, but the processing mode was EXACTLY_ONCE_V2")); } @Test - public void shouldThrowStreamsExceptionOnErrorCloseThreadProducerIfEosBetaEnabled() { - properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA); + public void shouldThrowStreamsExceptionOnErrorCloseThreadProducerIfEosV2Enabled() { + properties.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2); mockClientSupplier.setApplicationIdForProducer("appId"); createTasks(); mockClientSupplier.producers.get(0).closeException = new RuntimeException("KABOOM!"); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index ee803992550c0..5a83c6839d3d5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -54,6 +54,8 @@ import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.test.MockClientSupplier; + +import java.util.UUID; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -93,7 +95,7 @@ public class RecordCollectorTest { private final StreamsConfig eosConfig = new StreamsConfig(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234"), - mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE) + mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2) )); private final String topic = "topic"; @@ -111,6 +113,7 @@ public class RecordCollectorTest { private final StringSerializer stringSerializer = new StringSerializer(); private final ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); + private final UUID processId = UUID.randomUUID(); private final StreamPartitioner streamPartitioner = (topic, key, value, numPartitions) -> Integer.parseInt(key) % numPartitions; @@ -126,10 +129,10 @@ public void setup() { clientSupplier.setCluster(cluster); streamsProducer = new StreamsProducer( config, - "threadId", + processId + "-StreamThread-1", clientSupplier, null, - null, + processId, logContext ); mockProducer = clientSupplier.producers.get(0); @@ -775,7 +778,7 @@ public void shouldNotAbortTxnOnEOSCloseDirtyIfNothingSent() { taskId, new StreamsProducer( eosConfig, - "threadId", + "-StreamThread-1", new MockClientSupplier() { @Override public Producer getProducer(final Map config) { @@ -788,7 +791,7 @@ public void abortTransaction() { } }, taskId, - null, + processId, logContext ), productionExceptionHandler, @@ -806,7 +809,7 @@ public void shouldThrowIfTopicIsUnknownOnSendWithPartitioner() { taskId, new StreamsProducer( config, - "threadId", + processId + "-StreamThread-1", new MockClientSupplier() { @Override public Producer getProducer(final Map config) { @@ -845,7 +848,7 @@ public void shouldNotCloseInternalProducerForEOS() { taskId, new StreamsProducer( eosConfig, - "threadId", + processId + "-StreamThread-1", new MockClientSupplier() { @Override public Producer getProducer(final Map config) { @@ -853,7 +856,7 @@ public Producer getProducer(final Map config) { } }, taskId, - null, + processId, logContext ), productionExceptionHandler, @@ -877,7 +880,7 @@ public void shouldNotCloseInternalProducerForNonEOS() { private StreamsProducer getExceptionalStreamsProducerOnSend(final Exception exception) { return new StreamsProducer( config, - "threadId", + processId + "-StreamThread-1", new MockClientSupplier() { @Override public Producer getProducer(final Map config) { @@ -899,7 +902,7 @@ public synchronized Future send(final ProducerRecord getProducer(final Map config) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 4a0e3ac1bea95..60a95d4834427 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -457,6 +457,7 @@ public void shouldCloseStateManagerOnTaskCreated() { assertEquals(Task.State.CLOSED, task.state()); } + @SuppressWarnings("deprecation") @Test public void shouldDeleteStateDirOnTaskCreatedAndEosAlphaUncleanClose() { stateManager.close(); @@ -488,7 +489,7 @@ public void shouldDeleteStateDirOnTaskCreatedAndEosAlphaUncleanClose() { } @Test - public void shouldDeleteStateDirOnTaskCreatedAndEosBetaUncleanClose() { + public void shouldDeleteStateDirOnTaskCreatedAndEosV2UncleanClose() { stateManager.close(); EasyMock.expectLastCall(); @@ -501,7 +502,7 @@ public void shouldDeleteStateDirOnTaskCreatedAndEosBetaUncleanClose() { config = new StreamsConfig(mkProperties(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, applicationId), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"), - mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA) + mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2) ))); task = createStandbyTask(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 9bac9cc137383..5eeaaa24524f1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -321,7 +321,7 @@ public void shouldAttemptToDeleteStateDirectoryWhenCloseDirtyAndEosEnabled() thr ctrl.checkOrder(true); ctrl.replay(); - task = createStatefulTask(createConfig(StreamsConfig.EXACTLY_ONCE, "100"), true, stateManager); + task = createStatefulTask(createConfig(StreamsConfig.EXACTLY_ONCE_V2, "100"), true, stateManager); task.suspend(); task.closeDirty(); task = null; @@ -525,6 +525,7 @@ public void shouldProcessRecordsAfterPrepareCommitWhenEosDisabled() { assertFalse(task.process(time.milliseconds())); } + @SuppressWarnings("deprecation") @Test public void shouldNotProcessRecordsAfterPrepareCommitWhenEosAlphaEnabled() { task = createSingleSourceStateless(createConfig(StreamsConfig.EXACTLY_ONCE, "0"), StreamsConfig.METRICS_LATEST); @@ -548,8 +549,8 @@ public void shouldNotProcessRecordsAfterPrepareCommitWhenEosAlphaEnabled() { } @Test - public void shouldNotProcessRecordsAfterPrepareCommitWhenEosBetaEnabled() { - task = createSingleSourceStateless(createConfig(StreamsConfig.EXACTLY_ONCE_BETA, "0"), StreamsConfig.METRICS_LATEST); + public void shouldNotProcessRecordsAfterPrepareCommitWhenEosV2Enabled() { + task = createSingleSourceStateless(createConfig(StreamsConfig.EXACTLY_ONCE_V2, "0"), StreamsConfig.METRICS_LATEST); assertFalse(task.process(time.milliseconds())); @@ -736,7 +737,7 @@ public void shouldThrowOnTimeoutExceptionAndBufferRecordForRetryIfEosDisabled() @Test public void shouldThrowTaskCorruptedExceptionOnTimeoutExceptionIfEosEnabled() { - createTimeoutTask(StreamsConfig.EXACTLY_ONCE); + createTimeoutTask(StreamsConfig.EXACTLY_ONCE_V2); task.addRecords(partition1, singletonList(getConsumerRecordWithOffsetAsTimestamp(0, 0L))); @@ -1512,7 +1513,7 @@ public void shouldNotCheckpointOffsetsOnCommitIfEosIsEnabled() { EasyMock.expect(recordCollector.offsets()).andReturn(emptyMap()).anyTimes(); EasyMock.replay(stateManager, recordCollector); - task = createStatefulTask(createConfig(StreamsConfig.EXACTLY_ONCE, "100"), true); + task = createStatefulTask(createConfig(StreamsConfig.EXACTLY_ONCE_V2, "100"), true); task.initializeIfNeeded(); task.completeRestoration(noOpResetter -> { }); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 80fb2c90b996a..bbb1c60639767 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -193,7 +193,7 @@ private Properties configProps(final boolean enableEoS) { mkEntry(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"), mkEntry(StreamsConfig.DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()), mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()), - mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE : StreamsConfig.AT_LEAST_ONCE) + mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, enableEoS ? StreamsConfig.EXACTLY_ONCE_V2 : StreamsConfig.AT_LEAST_ONCE) )); } @@ -257,7 +257,7 @@ public void onChange(final Thread thread, final ThreadStateTransitionValidator oldState) { ++numChanges; if (this.newState != null) { - if (this.newState != oldState) { + if (!this.newState.equals(oldState)) { throw new RuntimeException("State mismatch " + oldState + " different from " + this.newState); } } @@ -992,11 +992,14 @@ public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEo assertSame(clientSupplier.restoreConsumer, thread.restoreConsumer()); } + @SuppressWarnings("deprecation") @Test - public void shouldInjectProducerPerTaskUsingClientSupplierOnCreateIfEosEnable() { + public void shouldInjectProducerPerTaskUsingClientSupplierOnCreateIfEosAlphaEnabled() { internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); - final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(configProps(true)), true); + final Properties props = configProps(true); + props.put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(props), true); thread.setState(StreamThread.State.STARTING); thread.rebalanceListener().onPartitionsRevoked(Collections.emptyList()); @@ -1027,6 +1030,42 @@ public void shouldInjectProducerPerTaskUsingClientSupplierOnCreateIfEosEnable() assertSame(clientSupplier.restoreConsumer, thread.restoreConsumer()); } + @Test + public void shouldInjectProducerPerThreadUsingClientSupplierOnCreateIfEosV2Enabled() { + internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); + + final Properties props = configProps(true); + final StreamThread thread = createStreamThread(CLIENT_ID, new StreamsConfig(props), true); + + thread.setState(StreamThread.State.STARTING); + thread.rebalanceListener().onPartitionsRevoked(Collections.emptyList()); + + final Map> activeTasks = new HashMap<>(); + final List assignedPartitions = new ArrayList<>(); + + // assign single partition + assignedPartitions.add(t1p1); + assignedPartitions.add(t1p2); + activeTasks.put(task1, Collections.singleton(t1p1)); + activeTasks.put(task2, Collections.singleton(t1p2)); + + thread.taskManager().handleAssignment(activeTasks, emptyMap()); + + final MockConsumer mockConsumer = (MockConsumer) thread.mainConsumer(); + mockConsumer.assign(assignedPartitions); + final Map beginOffsets = new HashMap<>(); + beginOffsets.put(t1p1, 0L); + beginOffsets.put(t1p2, 0L); + mockConsumer.updateBeginningOffsets(beginOffsets); + thread.rebalanceListener().onPartitionsAssigned(new HashSet<>(assignedPartitions)); + + thread.runOnce(); + + assertThat(clientSupplier.producers.size(), is(1)); + assertSame(clientSupplier.consumer, thread.mainConsumer()); + assertSame(clientSupplier.restoreConsumer, thread.restoreConsumer()); + } + @Test public void shouldOnlyCompleteShutdownAfterRebalanceNotInProgress() throws InterruptedException { internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java index 2bfc43e528234..f4dec89829464 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsProducerTest.java @@ -80,6 +80,7 @@ public class StreamsProducerTest { mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234")) ); + @SuppressWarnings("deprecation") private final StreamsConfig eosAlphaConfig = new StreamsConfig(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234"), @@ -89,7 +90,7 @@ public class StreamsProducerTest { private final StreamsConfig eosBetaConfig = new StreamsConfig(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "appId"), mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy:1234"), - mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_BETA)) + mkEntry(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2)) ); final Producer mockedProducer = mock(Producer.class); @@ -311,7 +312,7 @@ public void shouldFailOnResetProducerForAtLeastOnce() { () -> nonEosStreamsProducer.resetProducer() ); - assertThat(thrown.getMessage(), is("Exactly-once beta is not enabled [test]")); + assertThat(thrown.getMessage(), is("Expected eos-v2 to be enabled, but the processing mode was AT_LEAST_ONCE")); } @Test @@ -321,7 +322,7 @@ public void shouldFailOnResetProducerForExactlyOnceAlpha() { () -> eosAlphaStreamsProducer.resetProducer() ); - assertThat(thrown.getMessage(), is("Exactly-once beta is not enabled [test]")); + assertThat(thrown.getMessage(), is("Expected eos-v2 to be enabled, but the processing mode was EXACTLY_ONCE_ALPHA")); } @@ -330,7 +331,7 @@ public void shouldFailOnResetProducerForExactlyOnceAlpha() { // functional tests @Test - public void shouldNotSetTransactionIdIfEosDisable() { + public void shouldNotSetTransactionIdIfEosDisabled() { final StreamsConfig mockConfig = mock(StreamsConfig.class); expect(mockConfig.getProducerConfigs("threadId-producer")).andReturn(mock(Map.class)); expect(mockConfig.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)).andReturn(StreamsConfig.AT_LEAST_ONCE).anyTimes(); @@ -441,6 +442,7 @@ public void shouldEnableEosIfEosBetaEnabled() { assertThat(eosBetaStreamsProducer.eosEnabled(), is(true)); } + @SuppressWarnings("deprecation") @Test public void shouldSetTransactionIdUsingTaskIdIfEosAlphaEnabled() { final Map mockMap = mock(Map.class); @@ -467,7 +469,7 @@ public void shouldSetTransactionIdUsingTaskIdIfEosAlphaEnabled() { } @Test - public void shouldSetTransactionIdUsingProcessIdIfEosBetaEnable() { + public void shouldSetTransactionIdUsingProcessIdIfEosV2Enabled() { final UUID processId = UUID.randomUUID(); final Map mockMap = mock(Map.class); @@ -477,7 +479,7 @@ public void shouldSetTransactionIdUsingProcessIdIfEosBetaEnable() { final StreamsConfig mockConfig = mock(StreamsConfig.class); expect(mockConfig.getProducerConfigs("threadId-StreamThread-0-producer")).andReturn(mockMap); expect(mockConfig.getString(StreamsConfig.APPLICATION_ID_CONFIG)).andReturn("appId"); - expect(mockConfig.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)).andReturn(StreamsConfig.EXACTLY_ONCE_BETA).anyTimes(); + expect(mockConfig.getString(StreamsConfig.PROCESSING_GUARANTEE_CONFIG)).andReturn(StreamsConfig.EXACTLY_ONCE_V2).anyTimes(); replay(mockMap, mockConfig); @@ -682,7 +684,7 @@ public void shouldFailIfProcessIdNullForEosBeta() { logContext) ); - assertThat(thrown.getMessage(), is("processId cannot be null for exactly-once beta")); + assertThat(thrown.getMessage(), is("processId cannot be null for exactly-once v2")); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 555f21c7f5658..07fc378e18514 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -572,12 +572,12 @@ public void shouldCloseActiveTasksWhenHandlingLostTasks() throws Exception { } @Test - public void shouldReInitializeThreadProducerOnHandleLostAllIfEosBetaEnabled() { + public void shouldReInitializeThreadProducerOnHandleLostAllIfEosV2Enabled() { activeTaskCreator.reInitializeThreadProducer(); expectLastCall(); replay(activeTaskCreator); - setUpTaskManager(ProcessingMode.EXACTLY_ONCE_BETA); + setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2); taskManager.handleLostAll(); @@ -938,7 +938,7 @@ public void markChangelogAsCorrupted(final Collection partitions @Test public void shouldCloseAndReviveUncorruptedTasksWhenTimeoutExceptionThrownFromCommitDuringHandleCorruptedWithEOS() { - setUpTaskManager(ProcessingMode.EXACTLY_ONCE_BETA); + setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2); final StreamsProducer producer = mock(StreamsProducer.class); expect(activeTaskCreator.threadProducer()).andStubReturn(producer); final ProcessorStateManager stateManager = EasyMock.createMock(ProcessorStateManager.class); @@ -1076,7 +1076,7 @@ public void markChangelogAsCorrupted(final Collection partitions @Test public void shouldCloseAndReviveUncorruptedTasksWhenTimeoutExceptionThrownFromCommitDuringRevocationWithEOS() { - setUpTaskManager(ProcessingMode.EXACTLY_ONCE_BETA); + setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2); final StreamsProducer producer = mock(StreamsProducer.class); expect(activeTaskCreator.threadProducer()).andStubReturn(producer); final ProcessorStateManager stateManager = EasyMock.createMock(ProcessorStateManager.class); @@ -1352,9 +1352,9 @@ public void shouldSuspendActiveTasksDuringRevocation() { } @Test - public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEosBeta() { + public void shouldCommitAllActiveTasksThatNeedCommittingOnHandleRevocationWithEosV2() { final StreamsProducer producer = mock(StreamsProducer.class); - setUpTaskManager(ProcessingMode.EXACTLY_ONCE_BETA); + setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2); final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true); final Map offsets00 = singletonMap(t1p0, new OffsetAndMetadata(0L, null)); @@ -2218,7 +2218,7 @@ public void shouldCommitViaProducerIfEosAlphaEnabled() { } @Test - public void shouldCommitViaProducerIfEosBetaEnabled() { + public void shouldCommitViaProducerIfEosV2Enabled() { final StreamsProducer producer = mock(StreamsProducer.class); expect(activeTaskCreator.threadProducer()).andReturn(producer); @@ -2231,7 +2231,7 @@ public void shouldCommitViaProducerIfEosBetaEnabled() { producer.commitTransaction(allOffsets, new ConsumerGroupMetadata("appId")); expectLastCall(); - shouldCommitViaProducerIfEosEnabled(StreamThread.ProcessingMode.EXACTLY_ONCE_BETA, producer, offsetsT01, offsetsT02); + shouldCommitViaProducerIfEosEnabled(StreamThread.ProcessingMode.EXACTLY_ONCE_V2, producer, offsetsT01, offsetsT02); } private void shouldCommitViaProducerIfEosEnabled(final StreamThread.ProcessingMode processingMode, @@ -3056,8 +3056,8 @@ public void shouldNotFailForTimeoutExceptionOnCommitWithEosAlpha() { } @Test - public void shouldThrowTaskCorruptedExceptionForTimeoutExceptionOnCommitWithEosBeta() { - setUpTaskManager(ProcessingMode.EXACTLY_ONCE_BETA); + public void shouldThrowTaskCorruptedExceptionForTimeoutExceptionOnCommitWithEosV2() { + setUpTaskManager(ProcessingMode.EXACTLY_ONCE_V2); final StreamsProducer producer = mock(StreamsProducer.class); expect(activeTaskCreator.threadProducer()) diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java index cd0425122be26..90a6b94b511c3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java @@ -104,7 +104,7 @@ public static void main(final String[] args) throws IOException { System.out.println("start Kafka Streams"); streams.start(); - final boolean eosEnabled = processingMode.startsWith(StreamsConfig.EXACTLY_ONCE); + final boolean eosEnabled = processingMode.startsWith("exactly_once"); System.out.println("send data"); final Properties producerProperties = new Properties(); diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsEosTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsEosTest.java index aa195d09c2853..5ad0641b60fb2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsEosTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsEosTest.java @@ -29,6 +29,7 @@ public class StreamsEosTest { * args ::= kafka propFileName command * command := "run" | "process" | "verify" */ + @SuppressWarnings("deprecation") public static void main(final String[] args) throws IOException { if (args.length < 2) { System.err.println("StreamsEosTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); @@ -49,9 +50,11 @@ public static void main(final String[] args) throws IOException { if ("process".equals(command) || "process-complex".equals(command)) { if (!StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE_BETA.equals(processingGuarantee)) { + !StreamsConfig.EXACTLY_ONCE_BETA.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE_V2.equals(processingGuarantee)) { - System.err.println("processingGuarantee must be either " + StreamsConfig.EXACTLY_ONCE + " or " + StreamsConfig.EXACTLY_ONCE_BETA); + System.err.println("processingGuarantee must be either " + StreamsConfig.EXACTLY_ONCE + " or " + + StreamsConfig.EXACTLY_ONCE_BETA + " or " + StreamsConfig.EXACTLY_ONCE_V2); Exit.exit(1); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java index 1b425c5073930..e26e0af7cbf02 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -38,6 +38,7 @@ public class StreamsSmokeTest { * * @param args */ + @SuppressWarnings("deprecation") public static void main(final String[] args) throws IOException { if (args.length < 2) { System.err.println("StreamsSmokeTest are expecting two parameters: propFile, command; but only see " + args.length + " parameter"); @@ -60,10 +61,14 @@ public static void main(final String[] args) throws IOException { if ("process".equals(command)) { if (!StreamsConfig.AT_LEAST_ONCE.equals(processingGuarantee) && !StreamsConfig.EXACTLY_ONCE.equals(processingGuarantee) && - !StreamsConfig.EXACTLY_ONCE_BETA.equals(processingGuarantee)) { + !StreamsConfig.EXACTLY_ONCE_BETA.equals(processingGuarantee) && + !StreamsConfig.EXACTLY_ONCE_V2.equals(processingGuarantee)) { - System.err.println("processingGuarantee must be either " + StreamsConfig.AT_LEAST_ONCE + ", " + - StreamsConfig.EXACTLY_ONCE + ", or " + StreamsConfig.EXACTLY_ONCE_BETA); + System.err.println("processingGuarantee must be either " + + StreamsConfig.AT_LEAST_ONCE + ", " + + StreamsConfig.EXACTLY_ONCE + ", or " + + StreamsConfig.EXACTLY_ONCE_BETA + ", or " + + StreamsConfig.EXACTLY_ONCE_V2); Exit.exit(1); } diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index 118bdbbd41df9..ae4ca99651a6a 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -114,7 +114,7 @@ import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.AT_LEAST_ONCE; import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_ALPHA; -import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_BETA; +import static org.apache.kafka.streams.processor.internals.StreamThread.ProcessingMode.EXACTLY_ONCE_V2; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; /** @@ -467,6 +467,7 @@ private void setupGlobalTask(final Time mockWallClockTime, } } + @SuppressWarnings("deprecation") private void setupTask(final StreamsConfig streamsConfig, final StreamsMetricsImpl streamsMetrics, final ThreadCache cache) { @@ -610,7 +611,7 @@ private void completeAllProcessableWork() { } private void commit(final Map offsets) { - if (processingMode == EXACTLY_ONCE_ALPHA || processingMode == EXACTLY_ONCE_BETA) { + if (processingMode == EXACTLY_ONCE_ALPHA || processingMode == EXACTLY_ONCE_V2) { testDriverProducer.commitTransaction(offsets, new ConsumerGroupMetadata("dummy-app-id")); } else { consumer.commitSync(offsets); @@ -1313,7 +1314,7 @@ private static class TestDriverProducer extends StreamsProducer { public TestDriverProducer(final StreamsConfig config, final KafkaClientSupplier clientSupplier, final LogContext logContext) { - super(config, "TopologyTestDriver-Thread", clientSupplier, new TaskId(0, 0), UUID.randomUUID(), logContext); + super(config, "TopologyTestDriver-StreamThread-1", clientSupplier, new TaskId(0, 0), UUID.randomUUID(), logContext); } @Override diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverEosTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverEosTest.java index 8e4664e5c3a1c..a2c21398ac3f8 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverEosTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverEosTest.java @@ -21,6 +21,6 @@ public class TopologyTestDriverEosTest extends TopologyTestDriverTest { TopologyTestDriverEosTest() { - super(Collections.singletonMap(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE)); + super(Collections.singletonMap(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE_V2)); } } diff --git a/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java b/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java index cbdfa4df1a27f..230402ec91299 100644 --- a/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java +++ b/tools/src/main/java/org/apache/kafka/tools/TransactionalMessageCopier.java @@ -22,6 +22,7 @@ import net.sourceforge.argparse4j.inf.ArgumentParser; import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -353,11 +354,8 @@ public void onPartitionsAssigned(Collection partitions) { long messagesSentWithinCurrentTxn = records.count(); - if (useGroupMetadata) { - producer.sendOffsetsToTransaction(consumerPositions(consumer), consumer.groupMetadata()); - } else { - producer.sendOffsetsToTransaction(consumerPositions(consumer), consumerGroup); - } + ConsumerGroupMetadata groupMetadata = useGroupMetadata ? consumer.groupMetadata() : new ConsumerGroupMetadata(consumerGroup); + producer.sendOffsetsToTransaction(consumerPositions(consumer), groupMetadata); if (enableRandomAborts && random.nextInt() % 3 == 0) { throw new KafkaException("Aborting transaction"); From bf359f8e2924ee03b34a6f7e7eaf80bef55f9d98 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Sergio=20Pe=C3=B1a?= Date: Wed, 28 Apr 2021 19:57:28 -0500 Subject: [PATCH 097/155] KAFKA-10847: Fix spurious results on left/outer stream-stream joins (#10462) Fixes the issue with https://issues.apache.org/jira/browse/KAFKA-10847. To fix the above problem, the left/outer stream-stream join processor uses a buffer to hold non-joined records for some time until the window closes, so they are not processed if a join is found during the join window time. If the window of a record closes and a join was not found, then this should be emitted and processed by the consequent topology processor. A new time-ordered window store is used to temporary hold records that do not have a join and keep the records keys ordered by time. The KStreamStreamJoin has a reference to this new store . For every non-joined record seen, the processor writes it to this new state store without processing it. When a joined record is seen, the processor deletes the joined record from the new state store to prevent further processing. Records that were never joined at the end of the window + grace period are emitted to the next topology processor. I use the stream time to check for the expiry time for determinism results . The KStreamStreamJoin checks for expired records and emit them every time a new record is processed in the join processor. The new state store is shared with the left and right join nodes. The new store needs to serialize the record keys using a combined key of . This key combination helps to delete the records from the other join if a joined record is found. Two new serdes are created for this, KeyAndJoinSideSerde which serializes a boolean value that specifies the side where the key is found, and ValueOrOtherValueSerde that serializes either V1 or V2 based on where the key was found. Reviewers: Matthias J. Sax , Guozhang Wang --- checkstyle/suppressions.xml | 4 +- .../kstream/internals/KStreamImpl.java | 2 + .../kstream/internals/KStreamImplJoin.java | 116 ++- .../kstream/internals/KStreamKStreamJoin.java | 113 ++- .../internals/graph/StreamStreamJoinNode.java | 18 +- .../state/internals/KeyAndJoinSide.java | 81 ++ .../internals/KeyAndJoinSideDeserializer.java | 66 ++ .../state/internals/KeyAndJoinSideSerde.java | 29 + .../internals/KeyAndJoinSideSerializer.java | 69 ++ .../state/internals/LeftOrRightValue.java | 113 +++ .../LeftOrRightValueDeserializer.java | 74 ++ .../internals/LeftOrRightValueSerde.java | 33 + .../internals/LeftOrRightValueSerializer.java | 88 ++ .../TimeOrderedWindowStoreBuilder.java | 4 - .../kafka/streams/StreamsBuilderTest.java | 13 +- .../apache/kafka/streams/TopologyTest.java | 259 ++++++ .../StreamStreamJoinIntegrationTest.java | 8 +- .../KStreamImplValueJoinerWithKeyTest.java | 6 +- .../internals/KStreamKStreamJoinTest.java | 13 +- .../internals/KStreamKStreamLeftJoinTest.java | 429 ++++++++- .../KStreamKStreamOuterJoinTest.java | 873 ++++++++++++++++++ .../KeyAndJoinSideSerializerTest.java | 72 ++ .../LeftOrRightValueSerializerTest.java | 78 ++ 23 files changed, 2500 insertions(+), 61 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSide.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideDeserializer.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerde.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializer.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueDeserializer.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerde.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializer.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializerTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializerTest.java diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index a0955305d0bb2..1c64a6f7b298c 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -167,7 +167,7 @@ files="StreamsMetricsImpl.java"/> + files="(KafkaStreams|StreamsPartitionAssignor|StreamThread|TaskManager|GlobalStateManagerImpl|KStreamImplJoin).java"/> @@ -211,7 +211,7 @@ files="Murmur3Test.java"/> + files="(KStreamSlidingWindowAggregateTest|KStreamKStreamLeftJoinTest|KStreamKStreamOuterJoinTest).java"/> diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index c013bd63a916f..7ac688108970d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -93,6 +93,8 @@ public class KStreamImpl extends AbstractStream implements KStream KStream join(final KStream lhs, final ProcessorGraphNode otherWindowedStreamsNode = new ProcessorGraphNode<>(otherWindowStreamProcessorName, otherWindowStreamProcessorParams); builder.addGraphNode(otherGraphNode, otherWindowedStreamsNode); + Optional, LeftOrRightValue>>> outerJoinWindowStore = Optional.empty(); + if (leftOuter) { + final String outerJoinSuffix = rightOuter ? "-outer-shared-join" : "-left-shared-join"; + + // Get the suffix index of the joinThisGeneratedName to build the outer join store name. + final String outerJoinStoreGeneratedName = KStreamImpl.OUTERSHARED_NAME + + joinThisGeneratedName.substring( + rightOuter + ? KStreamImpl.OUTERTHIS_NAME.length() + : KStreamImpl.JOINTHIS_NAME.length()); + + final String outerJoinStoreName = userProvidedBaseStoreName == null ? outerJoinStoreGeneratedName : userProvidedBaseStoreName + outerJoinSuffix; + + outerJoinWindowStore = Optional.of(sharedOuterJoinWindowStoreBuilder(outerJoinStoreName, windows, streamJoinedInternal)); + } + + // Time shared between joins to keep track of the maximum stream time + final MaxObservedStreamTime maxObservedStreamTime = new MaxObservedStreamTime(); + final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>( + true, otherWindowStore.name(), windows.beforeMs, windows.afterMs, + windows.gracePeriodMs(), joiner, - leftOuter + leftOuter, + outerJoinWindowStore.map(StoreBuilder::name), + maxObservedStreamTime ); final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>( + false, thisWindowStore.name(), windows.afterMs, windows.beforeMs, + windows.gracePeriodMs(), AbstractStream.reverseJoinerWithKey(joiner), - rightOuter + rightOuter, + outerJoinWindowStore.map(StoreBuilder::name), + maxObservedStreamTime ); final PassThrough joinMerge = new PassThrough<>(); @@ -149,6 +200,7 @@ public KStream join(final KStream lhs, .withOtherWindowStoreBuilder(otherWindowStore) .withThisWindowedStreamProcessorParameters(thisWindowStreamProcessorParams) .withOtherWindowedStreamProcessorParameters(otherWindowStreamProcessorParams) + .withOuterJoinWindowStoreBuilder(outerJoinWindowStore) .withValueJoiner(joiner) .withNodeName(joinMergeName); @@ -211,6 +263,66 @@ private static StoreBuilder> joinWindowStoreBuilder(fin return builder; } + @SuppressWarnings("unchecked") + private static StoreBuilder, LeftOrRightValue>> sharedOuterJoinWindowStoreBuilder(final String storeName, + final JoinWindows windows, + final StreamJoinedInternal streamJoinedInternal) { + final StoreBuilder, LeftOrRightValue>> builder = new TimeOrderedWindowStoreBuilder, LeftOrRightValue>( + persistentTimeOrderedWindowStore( + storeName + "-store", + Duration.ofMillis(windows.size() + windows.gracePeriodMs()), + Duration.ofMillis(windows.size()) + ), + new KeyAndJoinSideSerde<>(streamJoinedInternal.keySerde()), + new LeftOrRightValueSerde(streamJoinedInternal.valueSerde(), streamJoinedInternal.otherValueSerde()), + Time.SYSTEM + ); + if (streamJoinedInternal.loggingEnabled()) { + builder.withLoggingEnabled(streamJoinedInternal.logConfig()); + } else { + builder.withLoggingDisabled(); + } + + return builder; + } + + // This method has same code as Store.persistentWindowStore(). But TimeOrderedWindowStore is + // a non-public API, so we need to keep duplicate code until it becomes public. + private static WindowBytesStoreSupplier persistentTimeOrderedWindowStore(final String storeName, + final Duration retentionPeriod, + final Duration windowSize) { + Objects.requireNonNull(storeName, "name cannot be null"); + final String rpMsgPrefix = prepareMillisCheckFailMsgPrefix(retentionPeriod, "retentionPeriod"); + final long retentionMs = validateMillisecondDuration(retentionPeriod, rpMsgPrefix); + final String wsMsgPrefix = prepareMillisCheckFailMsgPrefix(windowSize, "windowSize"); + final long windowSizeMs = validateMillisecondDuration(windowSize, wsMsgPrefix); + + final long segmentInterval = Math.max(retentionMs / 2, 60_000L); + + if (retentionMs < 0L) { + throw new IllegalArgumentException("retentionPeriod cannot be negative"); + } + if (windowSizeMs < 0L) { + throw new IllegalArgumentException("windowSize cannot be negative"); + } + if (segmentInterval < 1L) { + throw new IllegalArgumentException("segmentInterval cannot be zero or negative"); + } + if (windowSizeMs > retentionMs) { + throw new IllegalArgumentException("The retention period of the window store " + + storeName + " must be no smaller than its window size. Got size=[" + + windowSizeMs + "], retention=[" + retentionMs + "]"); + } + + return new RocksDbWindowBytesStoreSupplier( + storeName, + retentionMs, + segmentInterval, + windowSizeMs, + true, + RocksDbWindowBytesStoreSupplier.WindowStoreTypes.TIME_ORDERED_WINDOW_STORE); + } + private static StoreBuilder> joinWindowStoreBuilderFromSupplier(final WindowBytesStoreSupplier storeSupplier, final Serde keySerde, final Serde valueSerde) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 53d328f093638..a27586774b138 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -19,17 +19,24 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; +import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; +import org.apache.kafka.streams.state.internals.KeyAndJoinSide; +import org.apache.kafka.streams.state.internals.LeftOrRightValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Optional; +import java.util.function.Predicate; + import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; class KStreamKStreamJoin implements ProcessorSupplier { @@ -38,20 +45,33 @@ class KStreamKStreamJoin implements ProcessorSupplier { private final String otherWindowName; private final long joinBeforeMs; private final long joinAfterMs; + private final long joinGraceMs; private final ValueJoinerWithKey joiner; private final boolean outer; + private final Optional outerJoinWindowName; + private final boolean isLeftSide; + + private final KStreamImplJoin.MaxObservedStreamTime maxObservedStreamTime; - KStreamKStreamJoin(final String otherWindowName, + KStreamKStreamJoin(final boolean isLeftSide, + final String otherWindowName, final long joinBeforeMs, final long joinAfterMs, + final long joinGraceMs, final ValueJoinerWithKey joiner, - final boolean outer) { + final boolean outer, + final Optional outerJoinWindowName, + final KStreamImplJoin.MaxObservedStreamTime maxObservedStreamTime) { + this.isLeftSide = isLeftSide; this.otherWindowName = otherWindowName; this.joinBeforeMs = joinBeforeMs; this.joinAfterMs = joinAfterMs; + this.joinGraceMs = joinGraceMs; this.joiner = joiner; this.outer = outer; + this.outerJoinWindowName = outerJoinWindowName; + this.maxObservedStreamTime = maxObservedStreamTime; } @Override @@ -60,10 +80,13 @@ public Processor get() { } private class KStreamKStreamJoinProcessor extends AbstractProcessor { + private final Predicate>> recordWindowHasClosed = + windowedKey -> windowedKey.window().start() + joinAfterMs + joinGraceMs < maxObservedStreamTime.get(); - private WindowStore otherWindow; + private WindowStore otherWindowStore; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; + private Optional, LeftOrRightValue>> outerJoinWindowStore = Optional.empty(); @SuppressWarnings("unchecked") @Override @@ -71,10 +94,10 @@ public void init(final ProcessorContext context) { super.init(context); metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); - otherWindow = (WindowStore) context.getStateStore(otherWindowName); + otherWindowStore = context.getStateStore(otherWindowName); + outerJoinWindowStore = outerJoinWindowName.map(name -> context.getStateStore(name)); } - @Override public void process(final K key, final V1 value) { // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record @@ -98,18 +121,92 @@ key, value, context().topic(), context().partition(), context().offset() final long timeFrom = Math.max(0L, inputRecordTimestamp - joinBeforeMs); final long timeTo = Math.max(0L, inputRecordTimestamp + joinAfterMs); - try (final WindowStoreIterator iter = otherWindow.fetch(key, timeFrom, timeTo)) { + maxObservedStreamTime.advance(inputRecordTimestamp); + + // Emit all non-joined records which window has closed + if (inputRecordTimestamp == maxObservedStreamTime.get()) { + outerJoinWindowStore.ifPresent(store -> emitNonJoinedOuterRecords(store)); + } + + try (final WindowStoreIterator iter = otherWindowStore.fetch(key, timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; final KeyValue otherRecord = iter.next(); + final long otherRecordTimestamp = otherRecord.key; + + outerJoinWindowStore.ifPresent(store -> { + // Delete the joined record from the non-joined outer window store + store.put(KeyAndJoinSide.make(!isLeftSide, key), null, otherRecordTimestamp); + }); + context().forward( key, joiner.apply(key, value, otherRecord.value), - To.all().withTimestamp(Math.max(inputRecordTimestamp, otherRecord.key))); + To.all().withTimestamp(Math.max(inputRecordTimestamp, otherRecordTimestamp))); } if (needOuterJoin) { - context().forward(key, joiner.apply(key, value, null)); + // The maxStreamTime contains the max time observed in both sides of the join. + // Having access to the time observed in the other join side fixes the following + // problem: + // + // Say we have a window size of 5 seconds + // 1. A non-joined record wth time T10 is seen in the left-topic (maxLeftStreamTime: 10) + // The record is not processed yet, and is added to the outer-join store + // 2. A non-joined record with time T2 is seen in the right-topic (maxRightStreamTime: 2) + // The record is not processed yet, and is added to the outer-join store + // 3. A joined record with time T11 is seen in the left-topic (maxLeftStreamTime: 11) + // It is time to look at the expired records. T10 and T2 should be emitted, but + // because T2 was late, then it is not fetched by the window store, so it is not processed + // + // See KStreamKStreamLeftJoinTest.testLowerWindowBound() tests + // + // This condition below allows us to process the out-of-order records without the need + // to hold it in the temporary outer store + if (!outerJoinWindowStore.isPresent() || timeTo < maxObservedStreamTime.get()) { + context().forward(key, joiner.apply(key, value, null)); + } else { + outerJoinWindowStore.ifPresent(store -> store.put( + KeyAndJoinSide.make(isLeftSide, key), + LeftOrRightValue.make(isLeftSide, value), + inputRecordTimestamp)); + } + } + } + } + + @SuppressWarnings("unchecked") + private void emitNonJoinedOuterRecords(final WindowStore, LeftOrRightValue> store) { + try (final KeyValueIterator>, LeftOrRightValue> it = store.all()) { + while (it.hasNext()) { + final KeyValue>, LeftOrRightValue> record = it.next(); + + final Windowed> windowedKey = record.key; + final LeftOrRightValue value = record.value; + + // Skip next records if window has not closed + if (windowedKey.window().start() + joinAfterMs + joinGraceMs >= maxObservedStreamTime.get()) { + break; + } + + final K key = windowedKey.key().getKey(); + final long time = windowedKey.window().start(); + + final R nullJoinedValue; + if (isLeftSide) { + nullJoinedValue = joiner.apply(key, + (V1) value.getLeftValue(), + (V2) value.getRightValue()); + } else { + nullJoinedValue = joiner.apply(key, + (V1) value.getRightValue(), + (V2) value.getLeftValue()); + } + + context().forward(key, nullJoinedValue, To.all().withTimestamp(time)); + + // Delete the key from the outer window store now it is emitted + store.put(record.key.key(), null, record.key.window().start()); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java index c69f7433a62e4..eeba0c8211a4c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java @@ -22,6 +22,10 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.internals.KeyAndJoinSide; +import org.apache.kafka.streams.state.internals.LeftOrRightValue; + +import java.util.Optional; /** * Too much information to generalize, so Stream-Stream joins are represented by a specific node. @@ -32,6 +36,7 @@ public class StreamStreamJoinNode extends BaseJoinProcessorNode otherWindowedStreamProcessorParameters; private final StoreBuilder> thisWindowStoreBuilder; private final StoreBuilder> otherWindowStoreBuilder; + private final Optional, LeftOrRightValue>>> outerJoinWindowStoreBuilder; private final Joined joined; @@ -44,6 +49,7 @@ private StreamStreamJoinNode(final String nodeName, final ProcessorParameters otherWindowedStreamProcessorParameters, final StoreBuilder> thisWindowStoreBuilder, final StoreBuilder> otherWindowStoreBuilder, + final Optional, LeftOrRightValue>>> outerJoinWindowStoreBuilder, final Joined joined) { super(nodeName, @@ -59,7 +65,7 @@ private StreamStreamJoinNode(final String nodeName, this.joined = joined; this.thisWindowedStreamProcessorParameters = thisWindowedStreamProcessorParameters; this.otherWindowedStreamProcessorParameters = otherWindowedStreamProcessorParameters; - + this.outerJoinWindowStoreBuilder = outerJoinWindowStoreBuilder; } @@ -70,6 +76,7 @@ public String toString() { ", otherWindowedStreamProcessorParameters=" + otherWindowedStreamProcessorParameters + ", thisWindowStoreBuilder=" + thisWindowStoreBuilder + ", otherWindowStoreBuilder=" + otherWindowStoreBuilder + + ", outerJoinWindowStoreBuilder=" + outerJoinWindowStoreBuilder + ", joined=" + joined + "} " + super.toString(); } @@ -87,6 +94,8 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { topologyBuilder.addProcessor(mergeProcessorParameters().processorName(), mergeProcessorParameters().processorSupplier(), thisProcessorName, otherProcessorName); topologyBuilder.addStateStore(thisWindowStoreBuilder, thisWindowedStreamProcessorName, otherProcessorName); topologyBuilder.addStateStore(otherWindowStoreBuilder, otherWindowedStreamProcessorName, thisProcessorName); + + outerJoinWindowStoreBuilder.ifPresent(builder -> topologyBuilder.addStateStore(builder, thisProcessorName, otherProcessorName)); } public static StreamStreamJoinNodeBuilder streamStreamJoinNodeBuilder() { @@ -104,6 +113,7 @@ public static final class StreamStreamJoinNodeBuilder { private ProcessorParameters otherWindowedStreamProcessorParameters; private StoreBuilder> thisWindowStoreBuilder; private StoreBuilder> otherWindowStoreBuilder; + private Optional, LeftOrRightValue>>> outerJoinWindowStoreBuilder; private Joined joined; @@ -157,6 +167,11 @@ public StreamStreamJoinNodeBuilder withOtherWindowStoreBuilder(fi return this; } + public StreamStreamJoinNodeBuilder withOuterJoinWindowStoreBuilder(final Optional, LeftOrRightValue>>> outerJoinWindowStoreBuilder) { + this.outerJoinWindowStoreBuilder = outerJoinWindowStoreBuilder; + return this; + } + public StreamStreamJoinNodeBuilder withJoined(final Joined joined) { this.joined = joined; return this; @@ -173,6 +188,7 @@ public StreamStreamJoinNode build() { otherWindowedStreamProcessorParameters, thisWindowStoreBuilder, otherWindowStoreBuilder, + outerJoinWindowStoreBuilder, joined); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSide.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSide.java new file mode 100644 index 0000000000000..c3a7b7aa10ede --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSide.java @@ -0,0 +1,81 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.streams.KeyValue; + +import java.util.Objects; + +/** + * Combines a key from a {@link KeyValue} with a boolean value referencing if the key is + * part of the left join (true) or right join (false). This class is only useful when a state + * store needs to be shared between left and right processors, and each processor needs to + * access the key of the other processor. + */ +public class KeyAndJoinSide { + private final K key; + private final boolean leftSide; + + private KeyAndJoinSide(final boolean leftSide, final K key) { + this.key = Objects.requireNonNull(key, "key cannot be null"); + this.leftSide = leftSide; + } + + /** + * Create a new {@link KeyAndJoinSide} instance if the provide {@code key} is not {@code null}. + * + * @param leftSide True if the key is part of the left join side; False if it is from the right join side + * @param key the key + * @param the type of the key + * @return a new {@link KeyAndJoinSide} instance if the provide {@code key} is not {@code null} + */ + public static KeyAndJoinSide make(final boolean leftSide, final K key) { + return new KeyAndJoinSide<>(leftSide, key); + } + + public boolean isLeftSide() { + return leftSide; + } + + public K getKey() { + return key; + } + + @Override + public String toString() { + final String joinSide = leftSide ? "left" : "right"; + return "<" + joinSide + "," + key + ">"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final KeyAndJoinSide that = (KeyAndJoinSide) o; + return leftSide == that.leftSide && + Objects.equals(key, that.key); + } + + @Override + public int hashCode() { + return Objects.hash(leftSide, key); + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideDeserializer.java new file mode 100644 index 0000000000000..b79712dec45b8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideDeserializer.java @@ -0,0 +1,66 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer; + +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer; + +public class KeyAndJoinSideDeserializer implements WrappingNullableDeserializer, K, Void> { + private Deserializer keyDeserializer; + + KeyAndJoinSideDeserializer(final Deserializer keyDeserializer) { + this.keyDeserializer = keyDeserializer; + } + + @Override + public void setIfUnset(final Deserializer defaultKeyDeserializer, final Deserializer defaultValueDeserializer) { + if (keyDeserializer == null) { + keyDeserializer = Objects.requireNonNull(defaultKeyDeserializer, "defaultKeyDeserializer cannot be null"); + } + + initNullableDeserializer(keyDeserializer, defaultKeyDeserializer, defaultValueDeserializer); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + keyDeserializer.configure(configs, isKey); + } + + @Override + public KeyAndJoinSide deserialize(final String topic, final byte[] data) { + final boolean bool = data[0] == 1 ? true : false; + final K key = keyDeserializer.deserialize(topic, rawKey(data)); + + return KeyAndJoinSide.make(bool, key); + } + + private byte[] rawKey(final byte[] data) { + final byte[] rawKey = new byte[data.length - 1]; + System.arraycopy(data, 1, rawKey, 0, rawKey.length); + return rawKey; + } + + @Override + public void close() { + keyDeserializer.close(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerde.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerde.java new file mode 100644 index 0000000000000..6ed0d50048b4c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerde.java @@ -0,0 +1,29 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.kstream.internals.WrappingNullableSerde; + +public class KeyAndJoinSideSerde extends WrappingNullableSerde, K, Void> { + public KeyAndJoinSideSerde(final Serde keySerde) { + super( + new KeyAndJoinSideSerializer<>(keySerde != null ? keySerde.serializer() : null), + new KeyAndJoinSideDeserializer<>(keySerde != null ? keySerde.deserializer() : null) + ); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializer.java new file mode 100644 index 0000000000000..d6f25c2b1befd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializer.java @@ -0,0 +1,69 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableSerializer; + +/** + * Serializes a {@link KeyAndJoinSide}. The serialized bytes starts with a byte that references + * to the join side of the key followed by the key in bytes. + */ +public class KeyAndJoinSideSerializer implements WrappingNullableSerializer, K, Void> { + private Serializer keySerializer; + + KeyAndJoinSideSerializer(final Serializer keySerializer) { + this.keySerializer = keySerializer; + } + + @Override + public void setIfUnset(final Serializer defaultKeySerializer, final Serializer defaultValueSerializer) { + if (keySerializer == null) { + keySerializer = Objects.requireNonNull(defaultKeySerializer, "defaultKeySerializer cannot be null"); + } + + initNullableSerializer(keySerializer, defaultKeySerializer, defaultValueSerializer); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + keySerializer.configure(configs, isKey); + } + + @Override + public byte[] serialize(final String topic, final KeyAndJoinSide data) { + final byte boolByte = (byte) (data.isLeftSide() ? 1 : 0); + final byte[] keyBytes = keySerializer.serialize(topic, data.getKey()); + + return ByteBuffer + .allocate(keyBytes.length + 1) + .put(boolByte) + .put(keyBytes) + .array(); + } + + @Override + public void close() { + keySerializer.close(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java new file mode 100644 index 0000000000000..cbdba0796f1bc --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValue.java @@ -0,0 +1,113 @@ +/* + * 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.streams.state.internals; + +import java.util.Objects; + +/** + * This class is used in combination of {@link KeyAndJoinSide}. The {@link KeyAndJoinSide} class + * combines a key with a boolean value that specifies if the key is found in the left side of a + * join or on the right side. This {@link LeftOrRightValue} object contains either the V1 value, + * which is found in the left topic, or V2 value if it is found in the right topic. + */ +public class LeftOrRightValue { + private final V1 leftValue; + private final V2 rightValue; + + private LeftOrRightValue(final V1 leftValue, final V2 rightValue) { + if (leftValue != null && rightValue != null) { + throw new IllegalArgumentException("Only one value cannot be null"); + } else if (leftValue == null && rightValue == null) { + throw new NullPointerException("Only one value can be null"); + } + + this.leftValue = leftValue; + this.rightValue = rightValue; + } + + /** + * Create a new {@link LeftOrRightValue} instance with the V1 value as {@code leftValue} and + * V2 value as null. + * + * @param leftValue the left V1 value + * @param the type of the value + * @return a new {@link LeftOrRightValue} instance + */ + public static LeftOrRightValue makeLeftValue(final V1 leftValue) { + return new LeftOrRightValue<>(leftValue, null); + } + + /** + * Create a new {@link LeftOrRightValue} instance with the V2 value as {@code rightValue} and + * V1 value as null. + * + * @param rightValue the right V2 value + * @param the type of the value + * @return a new {@link LeftOrRightValue} instance + */ + public static LeftOrRightValue makeRightValue(final V2 rightValue) { + return new LeftOrRightValue<>(null, rightValue); + } + + /** + * Create a new {@link LeftOrRightValue} instance with the V value as {@code leftValue} if + * {@code isLeftSide} is True; otherwise {@code rightValue} if {@code isLeftSide} is False. + * + * @param value the V value (either V1 or V2 type) + * @param the type of the value + * @return a new {@link LeftOrRightValue} instance + */ + public static LeftOrRightValue make(final boolean isLeftSide, final V value) { + Objects.requireNonNull(value, "value is null"); + return isLeftSide + ? LeftOrRightValue.makeLeftValue(value) + : LeftOrRightValue.makeRightValue(value); + } + + public V1 getLeftValue() { + return leftValue; + } + + public V2 getRightValue() { + return rightValue; + } + + @Override + public String toString() { + return "<" + + ((leftValue != null) ? "left," + leftValue : "right," + rightValue) + + ">"; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final LeftOrRightValue that = (LeftOrRightValue) o; + return Objects.equals(leftValue, that.leftValue) && + Objects.equals(rightValue, that.rightValue); + } + + @Override + public int hashCode() { + return Objects.hash(leftValue, rightValue); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueDeserializer.java new file mode 100644 index 0000000000000..1d1efec676c2b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueDeserializer.java @@ -0,0 +1,74 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.kstream.internals.WrappingNullableDeserializer; + +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableDeserializer; + +public class LeftOrRightValueDeserializer implements WrappingNullableDeserializer, Void, Object> { + public Deserializer leftDeserializer; + public Deserializer rightDeserializer; + + public LeftOrRightValueDeserializer(final Deserializer leftDeserializer, final Deserializer rightDeserializer) { + this.leftDeserializer = leftDeserializer; + this.rightDeserializer = rightDeserializer; + } + + @SuppressWarnings("unchecked") + @Override + public void setIfUnset(final Deserializer defaultKeyDeserializer, final Deserializer defaultValueDeserializer) { + if (leftDeserializer == null) { + leftDeserializer = (Deserializer) Objects.requireNonNull(defaultValueDeserializer, "defaultValueDeserializer cannot be null"); + } + + if (rightDeserializer == null) { + rightDeserializer = (Deserializer) Objects.requireNonNull(defaultValueDeserializer, "defaultValueDeserializer cannot be null"); + } + + initNullableDeserializer(leftDeserializer, defaultKeyDeserializer, defaultValueDeserializer); + initNullableDeserializer(rightDeserializer, defaultKeyDeserializer, defaultValueDeserializer); + } + + @Override + public void configure(final Map configs, + final boolean isKey) { + leftDeserializer.configure(configs, isKey); + rightDeserializer.configure(configs, isKey); + } + + @Override + public LeftOrRightValue deserialize(final String topic, final byte[] data) { + if (data == null || data.length == 0) { + return null; + } + + return (data[0] == 1) + ? LeftOrRightValue.makeLeftValue(leftDeserializer.deserialize(topic, rawValue(data))) + : LeftOrRightValue.makeRightValue(rightDeserializer.deserialize(topic, rawValue(data))); + } + + private byte[] rawValue(final byte[] data) { + final byte[] rawValue = new byte[data.length - 1]; + System.arraycopy(data, 1, rawValue, 0, rawValue.length); + return rawValue; + } +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerde.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerde.java new file mode 100644 index 0000000000000..cc2d068326e6b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerde.java @@ -0,0 +1,33 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.kstream.internals.WrappingNullableSerde; + +public class LeftOrRightValueSerde extends WrappingNullableSerde, Void, Object> { + public LeftOrRightValueSerde(final Serde leftValueSerde, final Serde rightValueSerde) { + super( + new LeftOrRightValueSerializer<>( + leftValueSerde != null ? leftValueSerde.serializer() : null, + rightValueSerde != null ? rightValueSerde.serializer() : null), + new LeftOrRightValueDeserializer<>( + leftValueSerde != null ? leftValueSerde.deserializer() : null, + rightValueSerde != null ? rightValueSerde.deserializer() : null) + ); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializer.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializer.java new file mode 100644 index 0000000000000..837fd701e4a55 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializer.java @@ -0,0 +1,88 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.internals.WrappingNullableSerializer; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Objects; + +import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.initNullableSerializer; + +/** + * Serializes a {@link LeftOrRightValue}. The serialized bytes starts with a byte that references + * to whether the value is V1 or V2. + */ +public class LeftOrRightValueSerializer implements WrappingNullableSerializer, Void, Object> { + private Serializer leftSerializer; + private Serializer rightSerializer; + + public LeftOrRightValueSerializer(final Serializer leftSerializer, final Serializer rightSerializer) { + this.leftSerializer = leftSerializer; + this.rightSerializer = rightSerializer; + } + + @SuppressWarnings("unchecked") + @Override + public void setIfUnset(final Serializer defaultKeySerializer, final Serializer defaultValueSerializer) { + if (leftSerializer == null) { + leftSerializer = (Serializer) Objects.requireNonNull(defaultValueSerializer, "defaultValueSerializer cannot be null"); + } + + if (rightSerializer == null) { + rightSerializer = (Serializer) Objects.requireNonNull(defaultValueSerializer, "defaultValueSerializer cannot be null"); + } + + initNullableSerializer(leftSerializer, defaultKeySerializer, defaultValueSerializer); + initNullableSerializer(rightSerializer, defaultKeySerializer, defaultValueSerializer); + } + + @Override + public void configure(final Map configs, final boolean isKey) { + leftSerializer.configure(configs, isKey); + rightSerializer.configure(configs, isKey); + } + + @Override + public byte[] serialize(final String topic, final LeftOrRightValue data) { + if (data == null) { + return null; + } + + final byte[] rawValue = (data.getLeftValue() != null) + ? leftSerializer.serialize(topic, data.getLeftValue()) + : rightSerializer.serialize(topic, data.getRightValue()); + + if (rawValue == null) { + return null; + } + + return ByteBuffer + .allocate(1 + rawValue.length) + .put((byte) (data.getLeftValue() != null ? 1 : 0)) + .put(rawValue) + .array(); + } + + @Override + public void close() { + leftSerializer.close(); + rightSerializer.close(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java index 78c6774edf2dd..bce22eec8afeb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/TimeOrderedWindowStoreBuilder.java @@ -77,8 +77,4 @@ private WindowStore maybeWrapLogging(final WindowStore stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + stream1.join( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [KSTREAM-JOINTHIS-0000000004-store])\n" + + " --> KSTREAM-JOINTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [KSTREAM-JOINOTHER-0000000005-store])\n" + + " --> KSTREAM-JOINOTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-JOINOTHER-0000000005 (stores: [KSTREAM-JOINTHIS-0000000004-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-JOINTHIS-0000000004 (stores: [KSTREAM-JOINOTHER-0000000005-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-JOINTHIS-0000000004, KSTREAM-JOINOTHER-0000000005\n\n", + describe.toString()); + } + + @Test + public void streamStreamJoinTopologyWithCustomStoresNames() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + stream1.join( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + .withStoreName("custom-name")); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [custom-name-this-join-store])\n" + + " --> KSTREAM-JOINTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [custom-name-other-join-store])\n" + + " --> KSTREAM-JOINOTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-JOINOTHER-0000000005 (stores: [custom-name-this-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-JOINTHIS-0000000004 (stores: [custom-name-other-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-JOINTHIS-0000000004, KSTREAM-JOINOTHER-0000000005\n\n", + describe.toString()); + } + + @Test + public void streamStreamLeftJoinTopologyWithDefaultStoresNames() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [KSTREAM-JOINTHIS-0000000004-store])\n" + + " --> KSTREAM-JOINTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [KSTREAM-OUTEROTHER-0000000005-store])\n" + + " --> KSTREAM-OUTEROTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-JOINTHIS-0000000004 (stores: [KSTREAM-OUTEROTHER-0000000005-store, KSTREAM-OUTERSHARED-0000000004-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-OUTEROTHER-0000000005 (stores: [KSTREAM-JOINTHIS-0000000004-store, KSTREAM-OUTERSHARED-0000000004-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-JOINTHIS-0000000004, KSTREAM-OUTEROTHER-0000000005\n\n", + describe.toString()); + } + + @Test + public void streamStreamLeftJoinTopologyWithCustomStoresNames() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + .withStoreName("custom-name")); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [custom-name-this-join-store])\n" + + " --> KSTREAM-JOINTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [custom-name-outer-other-join-store])\n" + + " --> KSTREAM-OUTEROTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-JOINTHIS-0000000004 (stores: [custom-name-outer-other-join-store, custom-name-left-shared-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-OUTEROTHER-0000000005 (stores: [custom-name-this-join-store, custom-name-left-shared-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-JOINTHIS-0000000004, KSTREAM-OUTEROTHER-0000000005\n\n", + describe.toString()); + } + + @Test + public void streamStreamOuterJoinTopologyWithDefaultStoresNames() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [KSTREAM-OUTERTHIS-0000000004-store])\n" + + " --> KSTREAM-OUTERTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [KSTREAM-OUTEROTHER-0000000005-store])\n" + + " --> KSTREAM-OUTEROTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-OUTEROTHER-0000000005 (stores: [KSTREAM-OUTERTHIS-0000000004-store, KSTREAM-OUTERSHARED-0000000004-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-OUTERTHIS-0000000004 (stores: [KSTREAM-OUTEROTHER-0000000005-store, KSTREAM-OUTERSHARED-0000000004-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-OUTERTHIS-0000000004, KSTREAM-OUTEROTHER-0000000005\n\n", + describe.toString()); + } + + @Test + public void streamStreamOuterJoinTopologyWithCustomStoresNames() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + .withStoreName("custom-name")); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [custom-name-outer-this-join-store])\n" + + " --> KSTREAM-OUTERTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [custom-name-outer-other-join-store])\n" + + " --> KSTREAM-OUTEROTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-OUTEROTHER-0000000005 (stores: [custom-name-outer-this-join-store, custom-name-outer-shared-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-OUTERTHIS-0000000004 (stores: [custom-name-outer-other-join-store, custom-name-outer-shared-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-OUTERTHIS-0000000004, KSTREAM-OUTEROTHER-0000000005\n\n", + describe.toString()); + } + @Test public void topologyWithDynamicRoutingShouldDescribeExtractorClass() { final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java index 67db9bde266a1..c5a4fef976b52 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java @@ -152,7 +152,7 @@ public void testLeft() { final List>> expectedResult = Arrays.asList( null, null, - Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-null", null, 3L)), + null, Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-a", null, 4L)), Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "B-a", null, 5L)), Arrays.asList( @@ -193,7 +193,7 @@ public void testLeftRepartitioned() { final List>> expectedResult = Arrays.asList( null, null, - Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-null", null, 3L)), + null, Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-a", null, 4L)), Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "B-a", null, 5L)), Arrays.asList( @@ -237,7 +237,7 @@ public void testOuter() { final List>> expectedResult = Arrays.asList( null, null, - Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-null", null, 3L)), + null, Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-a", null, 4L)), Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "B-a", null, 5L)), Arrays.asList( @@ -278,7 +278,7 @@ public void testOuterRepartitioned() { final List>> expectedResult = Arrays.asList( null, null, - Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-null", null, 3L)), + null, Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "A-a", null, 4L)), Collections.singletonList(new TestRecord<>(ANY_UNIQUE_KEY, "B-a", null, 5L)), Arrays.asList( diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplValueJoinerWithKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplValueJoinerWithKeyTest.java index 0d93cf09d6623..000faadfdc953 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplValueJoinerWithKeyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplValueJoinerWithKeyTest.java @@ -107,8 +107,7 @@ public void shouldIncludeKeyInStreamLeftJoinResults() { ).to(outputTopic, Produced.with(Serdes.String(), Serdes.String())); // Left KV A, 3, Right KV A, 5 // TTD pipes records to left stream first, then right - // with TTD there's no caching, so join emits immediately with "A, 3, null" then "A, 3, 5" - final List> expectedResults = Arrays.asList(KeyValue.pair("A", "A:3"), KeyValue.pair("A", "A:5")); + final List> expectedResults = Arrays.asList(KeyValue.pair("A", "A:5")); runJoinTopology(builder, expectedResults, false, @@ -127,8 +126,7 @@ public void shouldIncludeKeyInStreamOuterJoinResults() { // Left KV A, 3, Right KV A, 5 // TTD pipes records to left stream first, then right - // with TTD there's no caching, so join emits immediately with "A, 3, null" then "A, 3, 5" - final List> expectedResults = Arrays.asList(KeyValue.pair("A", "A:3"), KeyValue.pair("A", "A:5")); + final List> expectedResults = Arrays.asList(KeyValue.pair("A", "A:5")); runJoinTopology(builder, expectedResults, false, diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index 78340d7aea5fd..ca26393bb33a0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -518,7 +518,7 @@ public void testOuterJoin() { driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final MockProcessor processor = supplier.theCapturedProcessor(); - // push two items to the primary stream; the other window is empty; this should produce two items + // push two items to the primary stream; the other window is empty; this should not produce items yet // w1 = {} // w2 = {} // --> w1 = { 0:A0, 1:A1 } @@ -526,8 +526,7 @@ public void testOuterJoin() { for (int i = 0; i < 2; i++) { inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i]); } - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+null", 0), - new KeyValueTimestamp<>(1, "A1+null", 0)); + processor.checkAndClearProcessResult(); // push two items to the other stream; this should produce two items // w1 = { 0:A0, 1:A1 } @@ -540,7 +539,7 @@ public void testOuterJoin() { processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+a0", 0), new KeyValueTimestamp<>(1, "A1+a1", 0)); - // push all four items to the primary stream; this should produce four items + // push all four items to the primary stream; this should produce two items // w1 = { 0:A0, 1:A1 } // w2 = { 0:a0, 1:a1 } // --> w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2, 3:B3 } @@ -549,9 +548,7 @@ public void testOuterJoin() { inputTopic1.pipeInput(expectedKey, "B" + expectedKey); } processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "B0+a0", 0), - new KeyValueTimestamp<>(1, "B1+a1", 0), - new KeyValueTimestamp<>(2, "B2+null", 0), - new KeyValueTimestamp<>(3, "B3+null", 0)); + new KeyValueTimestamp<>(1, "B1+a1", 0)); // push all items to the other stream; this should produce six items // w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2, 3:B3 } @@ -1144,7 +1141,7 @@ public void testAsymmetricWindowingAfter() { joined = stream1.join( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(0)).after(ofMillis(100)), + JoinWindows.of(ofMillis(0)).after(ofMillis(100)).grace(ofMillis(0)), StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index fafda4ade89f2..b7f7df5117a96 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -53,6 +53,257 @@ public class KStreamKStreamLeftJoinTest { private final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + @Test + public void testLeftJoinDuplicates() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(10)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // verifies non-joined duplicates are emitted when window has closed + inputTopic1.pipeInput(0, "A0", 0); + inputTopic1.pipeInput(0, "A0-0", 0); + inputTopic2.pipeInput(1, "a0", 111); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(0, "A0-0+null", 0)); + + // verifies joined duplicates are emitted + inputTopic1.pipeInput(2, "A2", 200); + inputTopic1.pipeInput(2, "A2-0", 200); + inputTopic2.pipeInput(2, "a2", 201); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "A2+a2", 201), + new KeyValueTimestamp<>(2, "A2-0+a2", 201)); + + // this record should expired non-joined records, but because A2 and A2-0 are joined and + // emitted already, then they won't be emitted again + inputTopic2.pipeInput(3, "a3", 315); + + processor.checkAndClearProcessResult(); + } + } + + @Test + public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic1.pipeInput(0, "A0", windowStart + 1); + inputTopic1.pipeInput(1, "A1", windowStart + 2); + inputTopic1.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic2.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "A1+a1", windowStart + 3)); + + // Dummy record in left topic will emit expired non-joined records from the left topic + inputTopic1.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", windowStart + 1), + new KeyValueTimestamp<>(0, "A0-0+null", windowStart + 3)); + + // Flush internal non-joined state store by joining the dummy record + inputTopic2.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 401)); + } + } + + @Test + public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic1.pipeInput(0, "A0", windowStart + 1); + inputTopic1.pipeInput(1, "A1", windowStart + 2); + inputTopic1.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic2.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "A1+a1", windowStart + 3)); + + // Dummy record in right topic will emit expired non-joined records from the left topic + inputTopic2.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", windowStart + 1), + new KeyValueTimestamp<>(0, "A0-0+null", windowStart + 3)); + + // Flush internal non-joined state store by joining the dummy record + inputTopic1.pipeInput(2, "dummy", windowStart + 402); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 402)); + } + } + + @Test + public void testRightNonJoinedRecordsAreNeverEmittedByTheLeftProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic2.pipeInput(0, "A0", windowStart + 1); + inputTopic2.pipeInput(1, "A1", windowStart + 2); + inputTopic2.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic1.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "a1+A1", windowStart + 3)); + + // Dummy record in left topic will not emit records + inputTopic1.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult(); + + // Process the dummy joined record + inputTopic2.pipeInput(2, "dummy", windowStart + 402); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 402)); + } + } + + @Test + public void testRightNonJoinedRecordsAreNeverEmittedByTheRightProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic2.pipeInput(0, "A0", windowStart + 1); + inputTopic2.pipeInput(1, "A1", windowStart + 2); + inputTopic2.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic1.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "a1+A1", windowStart + 3)); + + // Dummy record in right topic will not emit records + inputTopic2.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult(); + + // Process the dummy joined record + inputTopic1.pipeInput(2, "dummy", windowStart + 402); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 402)); + } + } + @Test public void testLeftJoin() { final StreamsBuilder builder = new StreamsBuilder(); @@ -94,8 +345,7 @@ public void testLeftJoin() { for (int i = 0; i < 2; i++) { inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i]); } - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+null", 0), - new KeyValueTimestamp<>(1, "A1+null", 0)); + processor.checkAndClearProcessResult(); // push two items to the other stream; this should produce two items // w1 = { 0:A0, 1:A1 } // w2 {} @@ -106,7 +356,7 @@ public void testLeftJoin() { } processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+a0", 0), new KeyValueTimestamp<>(1, "A1+a1", 0)); - // push three items to the primary stream; this should produce four items + // push three items to the primary stream; this should produce two joined items // w1 = { 0:A0, 1:A1 } // w2 = { 0:a0, 1:a1 } // --> w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2 } @@ -115,8 +365,7 @@ public void testLeftJoin() { inputTopic1.pipeInput(expectedKeys[i], "B" + expectedKeys[i]); } processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "B0+a0", 0), - new KeyValueTimestamp<>(1, "B1+a1", 0), - new KeyValueTimestamp<>(2, "B2+null", 0)); + new KeyValueTimestamp<>(1, "B1+a1", 0)); // push all items to the other stream; this should produce five items // w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2 } // w2 = { 0:a0, 1:a1 } @@ -130,7 +379,7 @@ public void testLeftJoin() { new KeyValueTimestamp<>(1, "A1+b1", 0), new KeyValueTimestamp<>(1, "B1+b1", 0), new KeyValueTimestamp<>(2, "B2+b2", 0)); - // push all four items to the primary stream; this should produce six items + // push all four items to the primary stream; this should produce six joined items // w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2 } // w2 = { 0:a0, 1:a1, 0:b0, 1:b1, 2:b2, 3:b3 } // --> w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2, 0:C0, 1:C1, 2:C2, 3:C3 } @@ -144,6 +393,127 @@ public void testLeftJoin() { new KeyValueTimestamp<>(1, "C1+b1", 0), new KeyValueTimestamp<>(2, "C2+b2", 0), new KeyValueTimestamp<>(3, "C3+b3", 0)); + + // push a dummy record that should expire non-joined items; it should not produce any items because + // all of them are joined + inputTopic1.pipeInput(0, "dummy", 1000); + processor.checkAndClearProcessResult(); + } + } + + @Test + public void testOrdering() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // push two items to the primary stream; the other window is empty; this should not produce any item yet + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = {} + inputTopic1.pipeInput(0, "A0", 0); + inputTopic1.pipeInput(1, "A1", 100); + processor.checkAndClearProcessResult(); + + // push one item to the other window that has a join; this should produce non-joined records with a closed window first, then + // the joined records + // by the time they were produced before + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = { } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 100) } + inputTopic2.pipeInput(1, "a1", 110); + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(1, "A1+a1", 110)); + } + } + + @Test + public void testGracePeriod() { + final StreamsBuilder builder = new StreamsBuilder(); + final int[] expectedKeys = new int[] {0, 1, 2, 3}; + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(10)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + final Collection> copartitionGroups = + TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups(); + + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // push two items to the primary stream; the other window is empty; this should not produce items because window has not closed + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = {} + long time = 0L; + for (int i = 0; i < 2; i++) { + inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time); + } + processor.checkAndClearProcessResult(); + + // push two items to the other stream with a window time after the previous window ended (not closed); this should not produce + // joined records because the window has ended, but not closed. + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = { } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 101), 1:a1 (ts: 101) } + time += 101L; + for (final int expectedKey : expectedKeys) { + inputTopic2.pipeInput(expectedKey, "a" + expectedKey, time); + } + processor.checkAndClearProcessResult(); + + // push a dummy item to the other stream after the window is closed; this should only produced the expired non-joined records, but + // not the joined records because the window has closed + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = { 0:a0 (ts: 101), 1:a1 (ts: 101) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 101), 1:a1 (ts: 101), + // 0:dummy (ts: 211)} + time += 1100L; + inputTopic2.pipeInput(0, "dummy", time); + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(1, "A1+null", 0)); } } @@ -162,7 +532,7 @@ public void testWindowing() { joined = stream1.leftJoin( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(100)), + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); joined.process(supplier); @@ -180,7 +550,7 @@ public void testWindowing() { final MockProcessor processor = supplier.theCapturedProcessor(); final long time = 0L; - // push two items to the primary stream; the other window is empty; this should produce two left-join items + // push two items to the primary stream; the other window is empty; this should not produce any items // w1 = {} // w2 = {} // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } @@ -188,8 +558,7 @@ public void testWindowing() { for (int i = 0; i < 2; i++) { inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time); } - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+null", 0), - new KeyValueTimestamp<>(1, "A1+null", 0)); + processor.checkAndClearProcessResult(); // push four items to the other stream; this should produce two full-join items // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } // w2 = {} @@ -243,7 +612,7 @@ private void testUpperWindowBound(final int[] expectedKeys, new KeyValueTimestamp<>(1, "B1+b1", 1100), new KeyValueTimestamp<>(2, "B2+b2", 1100), new KeyValueTimestamp<>(3, "B3+b3", 1100)); - // push four items with increased timestamp to the primary stream; this should produce one left-join and three full-join items + // push four items with increased timestamp to the primary stream; this should produce one left-join and three full-join items (non-joined item is not produced yet) // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100) } // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), @@ -257,11 +626,10 @@ private void testUpperWindowBound(final int[] expectedKeys, for (final int expectedKey : expectedKeys) { inputTopic1.pipeInput(expectedKey, "C" + expectedKey, time); } - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "C0+null", 1101), - new KeyValueTimestamp<>(1, "C1+b1", 1101), + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(1, "C1+b1", 1101), new KeyValueTimestamp<>(2, "C2+b2", 1101), new KeyValueTimestamp<>(3, "C3+b3", 1101)); - // push four items with increased timestamp to the primary stream; this should produce two left-join and two full-join items + // push four items with increased timestamp to the primary stream; this should produce two left-join and two full-join items (non-joined items are not produced yet) // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101) } @@ -277,11 +645,9 @@ private void testUpperWindowBound(final int[] expectedKeys, for (final int expectedKey : expectedKeys) { inputTopic1.pipeInput(expectedKey, "D" + expectedKey, time); } - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "D0+null", 1102), - new KeyValueTimestamp<>(1, "D1+null", 1102), - new KeyValueTimestamp<>(2, "D2+b2", 1102), + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(2, "D2+b2", 1102), new KeyValueTimestamp<>(3, "D3+b3", 1102)); - // push four items with increased timestamp to the primary stream; this should produce three left-join and one full-join items + // push four items with increased timestamp to the primary stream; this should produce one full-join items (three non-joined left-join are not produced yet) // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), @@ -299,11 +665,9 @@ private void testUpperWindowBound(final int[] expectedKeys, for (final int expectedKey : expectedKeys) { inputTopic1.pipeInput(expectedKey, "E" + expectedKey, time); } - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "E0+null", 1103), - new KeyValueTimestamp<>(1, "E1+null", 1103), - new KeyValueTimestamp<>(2, "E2+null", 1103), - new KeyValueTimestamp<>(3, "E3+b3", 1103)); - // push four items with increased timestamp to the primary stream; this should produce four left-join and no full-join items + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(3, "E3+b3", 1103)); + + // push four items with increased timestamp to the primary stream; this should produce no full-join items (four non-joined left-join are not produced yet) // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), @@ -323,7 +687,19 @@ private void testUpperWindowBound(final int[] expectedKeys, for (final int expectedKey : expectedKeys) { inputTopic1.pipeInput(expectedKey, "F" + expectedKey, time); } - processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "F0+null", 1104), + processor.checkAndClearProcessResult(); + + // push a dummy record to produce all left-join non-joined items + time += 301L; + inputTopic1.pipeInput(0, "dummy", time); + + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "C0+null", 1101), + new KeyValueTimestamp<>(0, "D0+null", 1102), + new KeyValueTimestamp<>(1, "D1+null", 1102), + new KeyValueTimestamp<>(0, "E0+null", 1103), + new KeyValueTimestamp<>(1, "E1+null", 1103), + new KeyValueTimestamp<>(2, "E2+null", 1103), + new KeyValueTimestamp<>(0, "F0+null", 1104), new KeyValueTimestamp<>(1, "F1+null", 1104), new KeyValueTimestamp<>(2, "F2+null", 1104), new KeyValueTimestamp<>(3, "F3+null", 1104)); @@ -480,5 +856,10 @@ private void testLowerWindowBound(final int[] expectedKeys, processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "K0+b0", 1000), new KeyValueTimestamp<>(1, "K1+b1", 1001), new KeyValueTimestamp<>(2, "K2+b2", 1002), new KeyValueTimestamp<>(3, "K3+b3", 1003)); + + // push a dummy record that should expire non-joined items; it should produce only the dummy+null record because + // all previous late records were emitted immediately + inputTopic1.pipeInput(0, "dummy", time + 300); + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "dummy+null", 1203)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java new file mode 100644 index 0000000000000..36cf5877e4c19 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -0,0 +1,873 @@ +/* + * 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.streams.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KeyValueTimestamp; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.TestInputTopic; +import org.apache.kafka.streams.TopologyTestDriver; +import org.apache.kafka.streams.TopologyWrapper; +import org.apache.kafka.streams.kstream.Consumed; +import org.apache.kafka.streams.kstream.JoinWindows; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.StreamJoined; +import org.apache.kafka.test.MockProcessor; +import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.StreamsTestUtils; +import org.junit.Test; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Properties; +import java.util.Set; + +import static java.time.Duration.ofMillis; +import static org.junit.Assert.assertEquals; + +public class KStreamKStreamOuterJoinTest { + private final static KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0]; + + private final String topic1 = "topic1"; + private final String topic2 = "topic2"; + private final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); + private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + + @Test + public void testOuterJoinDuplicates() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(10)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // verifies non-joined duplicates are emitted when window has closed + inputTopic1.pipeInput(0, "A0", 0); + inputTopic1.pipeInput(0, "A0-0", 0); + inputTopic2.pipeInput(1, "a1", 0); + inputTopic2.pipeInput(1, "a1-0", 0); + inputTopic2.pipeInput(1, "a0", 111); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "null+a1", 0), + new KeyValueTimestamp<>(1, "null+a1-0", 0), + new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(0, "A0-0+null", 0)); + + // verifies joined duplicates are emitted + inputTopic1.pipeInput(2, "A2", 200); + inputTopic1.pipeInput(2, "A2-0", 200); + inputTopic2.pipeInput(2, "a2", 201); + inputTopic2.pipeInput(2, "a2-0", 201); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "A2+a2", 201), + new KeyValueTimestamp<>(2, "A2-0+a2", 201), + new KeyValueTimestamp<>(2, "A2+a2-0", 201), + new KeyValueTimestamp<>(2, "A2-0+a2-0", 201)); + + // this record should expired non-joined records; only null+a0 will be emitted because + // it did not have a join + inputTopic2.pipeInput(3, "a3", 315); + + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(1, "null+a0", 111)); + } + } + + @Test + public void testLeftExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic1.pipeInput(0, "A0", windowStart + 1); + inputTopic1.pipeInput(1, "A1", windowStart + 2); + inputTopic1.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic2.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "A1+a1", windowStart + 3)); + + // Dummy record in left topic will emit expired non-joined records from the left topic + inputTopic1.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", windowStart + 1), + new KeyValueTimestamp<>(0, "A0-0+null", windowStart + 3)); + + // Flush internal non-joined state store by joining the dummy record + inputTopic2.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 401)); + } + } + + @Test + public void testLeftExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic1.pipeInput(0, "A0", windowStart + 1); + inputTopic1.pipeInput(1, "A1", windowStart + 2); + inputTopic1.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic2.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "A1+a1", windowStart + 3)); + + // Dummy record in right topic will emit expired non-joined records from the left topic + inputTopic2.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", windowStart + 1), + new KeyValueTimestamp<>(0, "A0-0+null", windowStart + 3)); + + // Flush internal non-joined state store by joining the dummy record + inputTopic1.pipeInput(2, "dummy", windowStart + 402); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 402)); + } + } + + @Test + public void testRightExpiredNonJoinedRecordsAreEmittedByTheLeftProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic2.pipeInput(0, "A0", windowStart + 1); + inputTopic2.pipeInput(1, "A1", windowStart + 2); + inputTopic2.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic1.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "a1+A1", windowStart + 3)); + + // Dummy record in left topic will emit expired non-joined records from the right topic + inputTopic1.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "null+A0", windowStart + 1), + new KeyValueTimestamp<>(0, "null+A0-0", windowStart + 3)); + + // Process the dummy joined record + inputTopic2.pipeInput(2, "dummy", windowStart + 402); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 402)); + } + } + + @Test + public void testRightExpiredNonJoinedRecordsAreEmittedByTheRightProcessor() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + final long windowStart = 0; + + // No joins detected; No null-joins emitted + inputTopic2.pipeInput(0, "A0", windowStart + 1); + inputTopic2.pipeInput(1, "A1", windowStart + 2); + inputTopic2.pipeInput(0, "A0-0", windowStart + 3); + processor.checkAndClearProcessResult(); + + // Join detected; No null-joins emitted + inputTopic1.pipeInput(1, "a1", windowStart + 3); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(1, "a1+A1", windowStart + 3)); + + // Dummy record in right topic will emit expired non-joined records from the right topic + inputTopic2.pipeInput(2, "dummy", windowStart + 401); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "null+A0", windowStart + 1), + new KeyValueTimestamp<>(0, "null+A0-0", windowStart + 3)); + + // Process the dummy joined record + inputTopic1.pipeInput(2, "dummy", windowStart + 402); + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(2, "dummy+dummy", windowStart + 402)); + } + } + + @Test + public void testOrdering() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // push two items to the primary stream; the other window is empty; this should not produce any item yet + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = {} + inputTopic1.pipeInput(0, "A0", 0); + inputTopic1.pipeInput(1, "A1", 100); + processor.checkAndClearProcessResult(); + + // push one item to the other window that has a join; this should produce non-joined records with a closed window first, then + // the joined records + // by the time they were produced before + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = { } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 100) } + inputTopic2.pipeInput(1, "a1", 110); + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(1, "A1+a1", 110)); + } + } + + @Test + public void testGracePeriod() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(10)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + final Collection> copartitionGroups = + TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups(); + + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // push one item to the primary stream; and one item in other stream; this should not produce items because there are no joins + // and window has not ended + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0) } + // --> w2 = { 1:a1 (ts: 0) } + inputTopic1.pipeInput(0, "A0", 0); + inputTopic2.pipeInput(1, "a1", 0); + processor.checkAndClearProcessResult(); + + // push one item on each stream with a window time after the previous window ended (not closed); this should not produce + // joined records because the window has ended, but will not produce non-joined records because the window has not closed. + // w1 = { 0:A0 (ts: 0) } + // w2 = { 1:a1 (ts: 0) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 101), 1:a1 (ts: 101) } + inputTopic2.pipeInput(0, "a0", 101); + inputTopic1.pipeInput(1, "A1", 101); + processor.checkAndClearProcessResult(); + + // push a dummy item to the any stream after the window is closed; this should produced all expired non-joined records because + // the window has closed + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = { 0:a0 (ts: 101), 1:a1 (ts: 101) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 101), 1:a1 (ts: 101), 0:dummy (ts: 112) } + inputTopic2.pipeInput(0, "dummy", 112); + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(1, "null+a1", 0), + new KeyValueTimestamp<>(0, "A0+null", 0)); + } + } + + @Test + public void testOuterJoin() { + final StreamsBuilder builder = new StreamsBuilder(); + + final int[] expectedKeys = new int[] {0, 1, 2, 3}; + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + final Collection> copartitionGroups = + TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups(); + + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // push two items to the primary stream; the other window is empty; this should not + // produce any items because window has not expired + // w1 {} + // w2 {} + // --> w1 = { 0:A0, 1:A1 } + // --> w2 = {} + for (int i = 0; i < 2; i++) { + inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i]); + } + processor.checkAndClearProcessResult(); + + // push two items to the other stream; this should produce two full-joined items + // w1 = { 0:A0, 1:A1 } + // w2 {} + // --> w1 = { 0:A0, 1:A1 } + // --> w2 = { 0:a0, 1:a1 } + for (int i = 0; i < 2; i++) { + inputTopic2.pipeInput(expectedKeys[i], "a" + expectedKeys[i]); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+a0", 0), + new KeyValueTimestamp<>(1, "A1+a1", 0)); + + // push three items to the primary stream; this should produce two full-joined items + // w1 = { 0:A0, 1:A1 } + // w2 = { 0:a0, 1:a1 } + // --> w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2 } + // --> w2 = { 0:a0, 1:a1 } + for (int i = 0; i < 3; i++) { + inputTopic1.pipeInput(expectedKeys[i], "B" + expectedKeys[i]); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "B0+a0", 0), + new KeyValueTimestamp<>(1, "B1+a1", 0)); + + // push all items to the other stream; this should produce five full-joined items + // w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2 } + // w2 = { 0:a0, 1:a1 } + // --> w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2 } + // --> w2 = { 0:a0, 1:a1, 0:b0, 1:b1, 2:b2, 3:b3 } + for (final int expectedKey : expectedKeys) { + inputTopic2.pipeInput(expectedKey, "b" + expectedKey); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+b0", 0), + new KeyValueTimestamp<>(0, "B0+b0", 0), + new KeyValueTimestamp<>(1, "A1+b1", 0), + new KeyValueTimestamp<>(1, "B1+b1", 0), + new KeyValueTimestamp<>(2, "B2+b2", 0)); + + // push all four items to the primary stream; this should produce six full-joined items + // w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2 } + // w2 = { 0:a0, 1:a1, 0:b0, 1:b1, 2:b2, 3:b3 } + // --> w1 = { 0:A0, 1:A1, 0:B0, 1:B1, 2:B2, 0:C0, 1:C1, 2:C2, 3:C3 } + // --> w2 = { 0:a0, 1:a1, 0:b0, 1:b1, 2:b2, 3:b3 } + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "C" + expectedKey); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "C0+a0", 0), + new KeyValueTimestamp<>(0, "C0+b0", 0), + new KeyValueTimestamp<>(1, "C1+a1", 0), + new KeyValueTimestamp<>(1, "C1+b1", 0), + new KeyValueTimestamp<>(2, "C2+b2", 0), + new KeyValueTimestamp<>(3, "C3+b3", 0)); + + // push a dummy record that should expire non-joined items; it should not produce any items because + // all of them are joined + inputTopic1.pipeInput(0, "dummy", 400); + processor.checkAndClearProcessResult(); + } + } + + @Test + public void testWindowing() { + final StreamsBuilder builder = new StreamsBuilder(); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + final Collection> copartitionGroups = + TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups(); + + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + final long time = 0L; + + // push two items to the primary stream; the other window is empty; this should not produce items because window has not closed + // w1 = {} + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = {} + for (int i = 0; i < 2; i++) { + inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i], time); + } + processor.checkAndClearProcessResult(); + // push four items to the other stream; this should produce two full-join items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = {} + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0) } + for (final int expectedKey : expectedKeys) { + inputTopic2.pipeInput(expectedKey, "a" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+a0", 0), + new KeyValueTimestamp<>(1, "A1+a1", 0)); + testUpperWindowBound(expectedKeys, driver, processor); + testLowerWindowBound(expectedKeys, driver, processor); + } + } + + private void testUpperWindowBound(final int[] expectedKeys, + final TopologyTestDriver driver, + final MockProcessor processor) { + long time; + + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + + // push four items with larger and increasing timestamp (out of window) to the other stream; this should produced 2 expired non-joined records + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time = 1000L; + for (int i = 0; i < expectedKeys.length; i++) { + inputTopic2.pipeInput(expectedKeys[i], "b" + expectedKeys[i], time + i); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(2, "null+a2", 0), + new KeyValueTimestamp<>(3, "null+a3", 0)); + + // push four items with larger timestamp to the primary stream; this should produce four full-join items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time = 1000L + 100L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "B" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "B0+b0", 1100), + new KeyValueTimestamp<>(1, "B1+b1", 1100), + new KeyValueTimestamp<>(2, "B2+b2", 1100), + new KeyValueTimestamp<>(3, "B3+b3", 1100)); + // push four items with increased timestamp to the primary stream; this should produce three full-join items (non-joined item is not produced yet) + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "C" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(1, "C1+b1", 1101), + new KeyValueTimestamp<>(2, "C2+b2", 1101), + new KeyValueTimestamp<>(3, "C3+b3", 1101)); + // push four items with increased timestamp to the primary stream; this should produce two full-join items (non-joined items are not produced yet) + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "D" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(2, "D2+b2", 1102), + new KeyValueTimestamp<>(3, "D3+b3", 1102)); + // push four items with increased timestamp to the primary stream; this should produce one full-join items (three non-joined left-join are not produced yet) + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "E" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(3, "E3+b3", 1103)); + // push four items with increased timestamp to the primary stream; this should produce no full-join items (four non-joined left-join are not produced yet) + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "F" + expectedKey, time); + } + processor.checkAndClearProcessResult(); + + // push a dummy record to produce all left-join non-joined items + time += 301L; + inputTopic1.pipeInput(0, "dummy", time); + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "C0+null", 1101), + new KeyValueTimestamp<>(0, "D0+null", 1102), + new KeyValueTimestamp<>(1, "D1+null", 1102), + new KeyValueTimestamp<>(0, "E0+null", 1103), + new KeyValueTimestamp<>(1, "E1+null", 1103), + new KeyValueTimestamp<>(2, "E2+null", 1103), + new KeyValueTimestamp<>(0, "F0+null", 1104), + new KeyValueTimestamp<>(1, "F1+null", 1104), + new KeyValueTimestamp<>(2, "F2+null", 1104), + new KeyValueTimestamp<>(3, "F3+null", 1104)); + } + + private void testLowerWindowBound(final int[] expectedKeys, + final TopologyTestDriver driver, + final MockProcessor processor) { + long time; + final TestInputTopic inputTopic1 = driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer()); + // push four items with smaller timestamp (before the window) to the primary stream; this should produce four left-join and no full-join items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time = 1000L - 100L - 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "G" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "G0+null", 899), + new KeyValueTimestamp<>(1, "G1+null", 899), new KeyValueTimestamp<>(2, "G2+null", 899), + new KeyValueTimestamp<>(3, "G3+null", 899)); + // push four items with increase timestamp to the primary stream; this should produce three left-join and one full-join items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899), + // 0:H0 (ts: 900), 1:H1 (ts: 900), 2:H2 (ts: 900), 3:H3 (ts: 900) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "H" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "H0+b0", 1000), + new KeyValueTimestamp<>(1, "H1+null", 900), new KeyValueTimestamp<>(2, "H2+null", 900), + new KeyValueTimestamp<>(3, "H3+null", 900)); + // push four items with increase timestamp to the primary stream; this should produce two left-join and two full-join items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899), + // 0:H0 (ts: 900), 1:H1 (ts: 900), 2:H2 (ts: 900), 3:H3 (ts: 900) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899), + // 0:H0 (ts: 900), 1:H1 (ts: 900), 2:H2 (ts: 900), 3:H3 (ts: 900), + // 0:I0 (ts: 901), 1:I1 (ts: 901), 2:I2 (ts: 901), 3:I3 (ts: 901) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "I" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "I0+b0", 1000), + new KeyValueTimestamp<>(1, "I1+b1", 1001), new KeyValueTimestamp<>(2, "I2+null", 901), + new KeyValueTimestamp<>(3, "I3+null", 901)); + // push four items with increase timestamp to the primary stream; this should produce one left-join and three full-join items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899), + // 0:H0 (ts: 900), 1:H1 (ts: 900), 2:H2 (ts: 900), 3:H3 (ts: 900), + // 0:I0 (ts: 901), 1:I1 (ts: 901), 2:I2 (ts: 901), 3:I3 (ts: 901) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899), + // 0:H0 (ts: 900), 1:H1 (ts: 900), 2:H2 (ts: 900), 3:H3 (ts: 900), + // 0:I0 (ts: 901), 1:I1 (ts: 901), 2:I2 (ts: 901), 3:I3 (ts: 901), + // 0:J0 (ts: 902), 1:J1 (ts: 902), 2:J2 (ts: 902), 3:J3 (ts: 902) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "J" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "J0+b0", 1000), + new KeyValueTimestamp<>(1, "J1+b1", 1001), new KeyValueTimestamp<>(2, "J2+b2", 1002), + new KeyValueTimestamp<>(3, "J3+null", 902)); + // push four items with increase timestamp to the primary stream; this should produce one left-join and three full-join items + // w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899), + // 0:H0 (ts: 900), 1:H1 (ts: 900), 2:H2 (ts: 900), 3:H3 (ts: 900), + // 0:I0 (ts: 901), 1:I1 (ts: 901), 2:I2 (ts: 901), 3:I3 (ts: 901), + // 0:J0 (ts: 902), 1:J1 (ts: 902), 2:J2 (ts: 902), 3:J3 (ts: 902) } + // w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + // --> w1 = { 0:A0 (ts: 0), 1:A1 (ts: 0), + // 0:B0 (ts: 1100), 1:B1 (ts: 1100), 2:B2 (ts: 1100), 3:B3 (ts: 1100), + // 0:C0 (ts: 1101), 1:C1 (ts: 1101), 2:C2 (ts: 1101), 3:C3 (ts: 1101), + // 0:D0 (ts: 1102), 1:D1 (ts: 1102), 2:D2 (ts: 1102), 3:D3 (ts: 1102), + // 0:E0 (ts: 1103), 1:E1 (ts: 1103), 2:E2 (ts: 1103), 3:E3 (ts: 1103), + // 0:F0 (ts: 1104), 1:F1 (ts: 1104), 2:F2 (ts: 1104), 3:F3 (ts: 1104), + // 0:G0 (ts: 899), 1:G1 (ts: 899), 2:G2 (ts: 899), 3:G3 (ts: 899), + // 0:H0 (ts: 900), 1:H1 (ts: 900), 2:H2 (ts: 900), 3:H3 (ts: 900), + // 0:I0 (ts: 901), 1:I1 (ts: 901), 2:I2 (ts: 901), 3:I3 (ts: 901), + // 0:J0 (ts: 902), 1:J1 (ts: 902), 2:J2 (ts: 902), 3:J3 (ts: 902), + // 0:K0 (ts: 903), 1:K1 (ts: 903), 2:K2 (ts: 903), 3:K3 (ts: 903) } + // --> w2 = { 0:a0 (ts: 0), 1:a1 (ts: 0), 2:a2 (ts: 0), 3:a3 (ts: 0), + // 0:b0 (ts: 1000), 1:b1 (ts: 1001), 2:b2 (ts: 1002), 3:b3 (ts: 1003) } + time += 1L; + for (final int expectedKey : expectedKeys) { + inputTopic1.pipeInput(expectedKey, "K" + expectedKey, time); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "K0+b0", 1000), + new KeyValueTimestamp<>(1, "K1+b1", 1001), new KeyValueTimestamp<>(2, "K2+b2", 1002), + new KeyValueTimestamp<>(3, "K3+b3", 1003)); + + // push a dummy record to verify there are no expired records to produce + // dummy window is behind the max. stream time seen (1205 used in testUpperWindowBound) + inputTopic1.pipeInput(0, "dummy", time + 200); + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "dummy+null", 1103)); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializerTest.java new file mode 100644 index 0000000000000..f3ad1562ee72f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyAndJoinSideSerializerTest.java @@ -0,0 +1,72 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serdes; +import org.junit.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.assertThrows; + +public class KeyAndJoinSideSerializerTest { + private static final String TOPIC = "some-topic"; + + private static final KeyAndJoinSideSerde STRING_SERDE = + new KeyAndJoinSideSerde<>(Serdes.String()); + + @Test + public void shouldSerializeKeyWithJoinSideAsTrue() { + final String value = "some-string"; + + final KeyAndJoinSide keyAndJoinSide = KeyAndJoinSide.make(true, value); + + final byte[] serialized = + STRING_SERDE.serializer().serialize(TOPIC, keyAndJoinSide); + + assertThat(serialized, is(notNullValue())); + + final KeyAndJoinSide deserialized = + STRING_SERDE.deserializer().deserialize(TOPIC, serialized); + + assertThat(deserialized, is(keyAndJoinSide)); + } + + @Test + public void shouldSerializeKeyWithJoinSideAsFalse() { + final String value = "some-string"; + + final KeyAndJoinSide keyAndJoinSide = KeyAndJoinSide.make(false, value); + + final byte[] serialized = + STRING_SERDE.serializer().serialize(TOPIC, keyAndJoinSide); + + assertThat(serialized, is(notNullValue())); + + final KeyAndJoinSide deserialized = + STRING_SERDE.deserializer().deserialize(TOPIC, serialized); + + assertThat(deserialized, is(keyAndJoinSide)); + } + + @Test + public void shouldThrowIfSerializeNullData() { + assertThrows(NullPointerException.class, + () -> STRING_SERDE.serializer().serialize(TOPIC, KeyAndJoinSide.make(true, null))); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializerTest.java new file mode 100644 index 0000000000000..f1ebda6ed44ea --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/LeftOrRightValueSerializerTest.java @@ -0,0 +1,78 @@ +/* + * 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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serdes; +import org.junit.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.assertThrows; + +public class LeftOrRightValueSerializerTest { + private static final String TOPIC = "some-topic"; + + private static final LeftOrRightValueSerde STRING_OR_INTEGER_SERDE = + new LeftOrRightValueSerde<>(Serdes.String(), Serdes.Integer()); + + @Test + public void shouldSerializeStringValue() { + final String value = "some-string"; + + final LeftOrRightValue leftOrRightValue = LeftOrRightValue.makeLeftValue(value); + + final byte[] serialized = + STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, leftOrRightValue); + + assertThat(serialized, is(notNullValue())); + + final LeftOrRightValue deserialized = + STRING_OR_INTEGER_SERDE.deserializer().deserialize(TOPIC, serialized); + + assertThat(deserialized, is(leftOrRightValue)); + } + + @Test + public void shouldSerializeIntegerValue() { + final int value = 5; + + final LeftOrRightValue leftOrRightValue = LeftOrRightValue.makeRightValue(value); + + final byte[] serialized = + STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, leftOrRightValue); + + assertThat(serialized, is(notNullValue())); + + final LeftOrRightValue deserialized = + STRING_OR_INTEGER_SERDE.deserializer().deserialize(TOPIC, serialized); + + assertThat(deserialized, is(leftOrRightValue)); + } + + @Test + public void shouldThrowIfSerializeValueAsNull() { + assertThrows(NullPointerException.class, + () -> STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, LeftOrRightValue.makeLeftValue(null))); + } + + @Test + public void shouldThrowIfSerializeOtherValueAsNull() { + assertThrows(NullPointerException.class, + () -> STRING_OR_INTEGER_SERDE.serializer().serialize(TOPIC, LeftOrRightValue.makeRightValue(null))); + } +} From f9de25f046452b2a6d916e6bca41e31d49bbdecf Mon Sep 17 00:00:00 2001 From: dengziming Date: Thu, 29 Apr 2021 11:39:55 +0800 Subject: [PATCH 098/155] MINOR: rename wrong topic id variable name and description (#10598) Reviewers: Luke Chen , Chia-Ping Tsai --- .../main/scala/kafka/tools/ConsoleConsumer.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 1269ff5b51b75..a40c0416cabf8 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -187,7 +187,7 @@ object ConsoleConsumer extends Logging { } class ConsumerConfig(args: Array[String]) extends CommandDefaultOptions(args) { - val topicIdOpt = parser.accepts("topic", "The topic id to consume on.") + val topicOpt = parser.accepts("topic", "The topic to consume on.") .withRequiredArg .describedAs("topic") .ofType(classOf[String]) @@ -200,7 +200,7 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("partition") .ofType(classOf[java.lang.Integer]) - val offsetOpt = parser.accepts("offset", "The offset id to consume from (a non-negative number), or 'earliest' which means from beginning, or 'latest' which means from end") + val offsetOpt = parser.accepts("offset", "The offset to consume from (a non-negative number), or 'earliest' which means from beginning, or 'latest' which means from end") .withRequiredArg .describedAs("consume offset") .ofType(classOf[String]) @@ -209,7 +209,7 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("consumer_prop") .ofType(classOf[String]) - val consumerConfigOpt = parser.accepts("consumer.config", s"Consumer config properties file. Note that ${consumerPropertyOpt} takes precedence over this config.") + val consumerConfigOpt = parser.accepts("consumer.config", s"Consumer config properties file. Note that $consumerPropertyOpt takes precedence over this config.") .withRequiredArg .describedAs("config file") .ofType(classOf[String]) @@ -306,23 +306,23 @@ object ConsoleConsumer extends Logging { val valueDeserializer = options.valueOf(valueDeserializerOpt) val formatter: MessageFormatter = messageFormatterClass.getDeclaredConstructor().newInstance().asInstanceOf[MessageFormatter] - if (keyDeserializer != null && !keyDeserializer.isEmpty) { + if (keyDeserializer != null && keyDeserializer.nonEmpty) { formatterArgs.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializer) } - if (valueDeserializer != null && !valueDeserializer.isEmpty) { + if (valueDeserializer != null && valueDeserializer.nonEmpty) { formatterArgs.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer) } formatter.configure(formatterArgs.asScala.asJava) - val topicOrFilterOpt = List(topicIdOpt, whitelistOpt).filter(options.has) + val topicOrFilterOpt = List(topicOpt, whitelistOpt).filter(options.has) if (topicOrFilterOpt.size != 1) CommandLineUtils.printUsageAndDie(parser, "Exactly one of whitelist/topic is required.") - topicArg = options.valueOf(topicIdOpt) + topicArg = options.valueOf(topicOpt) whitelistArg = options.valueOf(whitelistOpt) if (partitionArg.isDefined) { - if (!options.has(topicIdOpt)) + if (!options.has(topicOpt)) CommandLineUtils.printUsageAndDie(parser, "The topic is required when partition is specified.") if (fromBeginning && options.has(offsetOpt)) CommandLineUtils.printUsageAndDie(parser, "Options from-beginning and offset cannot be specified together.") From eaa14a8a688c8a6c23eb8d081dbee9f53fd8dc03 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Thu, 29 Apr 2021 14:32:50 +0100 Subject: [PATCH 099/155] KAFKA-12730; Avoid duplicate logout if Kerberos login fails (#10611) From Java 9 onwards, LoginContext#logout() throws an NPE if invoked multiple times due to https://bugs.openjdk.java.net/browse/JDK-8173069. KerberosLogin currently attempts logout followed by login in a background refresh thread. If login fails we retry the same sequence. As a result, a single login failure prevents subsequent re-login. And clients will never be able to authenticate successfully after the first failure, until the process is restarted. The commit checks if logout is necessary before invoking LoginContext#logout(). Also adds a test for this case. Reviewers: Manikumar Reddy --- .../security/kerberos/KerberosLogin.java | 14 +++- .../server/GssapiAuthenticationTest.scala | 64 +++++++++++++++---- 2 files changed, 63 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java index 9626da846df38..3f0a46ed952e6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java @@ -362,16 +362,24 @@ protected void reLogin() throws LoginException { lastLogin = currentElapsedTime(); //clear up the kerberos state. But the tokens are not cleared! As per //the Java kerberos login module code, only the kerberos credentials - //are cleared - logout(); + //are cleared. If previous logout succeeded but login failed, we shouldn't + //logout again since duplicate logout causes NPE from Java 9 onwards. + if (subject != null && !subject.getPrincipals().isEmpty()) { + logout(); + } //login and also update the subject field of this instance to //have the new credentials (pass it to the LoginContext constructor) loginContext = new LoginContext(contextName(), subject, null, configuration()); log.info("Initiating re-login for {}", principal); - loginContext.login(); + login(loginContext); } } + // Visibility to override for testing + protected void login(LoginContext loginContext) throws LoginException { + loginContext.login(); + } + // Visibility to override for testing protected void logout() throws LoginException { loginContext.logout(); diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index 90454bbb3814d..5c755073429af 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -23,6 +23,7 @@ import java.time.Duration import java.util.{Collections, Properties} import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} +import javax.security.auth.login.LoginContext import kafka.api.{Both, IntegrationTestHarness, SaslSetup} import kafka.utils.TestUtils import org.apache.kafka.clients.CommonClientConfigs @@ -101,6 +102,29 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { assertTrue(successfulAuths > successfulAuthsPerThread * numThreads, "Too few authentications: " + successfulAuths) } + /** + * Verifies that if login fails, subsequent re-login without failures works and clients + * are able to connect after the second re-login. Verifies that logout is performed only once + * since duplicate logouts without successful login results in NPE from Java 9 onwards. + */ + @Test + def testLoginFailure(): Unit = { + val selector = createSelectorWithRelogin() + try { + val login = TestableKerberosLogin.instance + assertNotNull(login) + login.loginException = Some(new RuntimeException("Test exception to fail login")) + executor.submit(() => login.reLogin(), 0) + executor.submit(() => login.reLogin(), 0) + + verifyRelogin(selector, login) + assertEquals(2, login.loginAttempts) + assertEquals(1, login.logoutAttempts) + } finally { + selector.close() + } + } + /** * Verifies that there are no authentication failures during Kerberos re-login. If authentication * is performed when credentials are unavailable between logout and login, we handle it as a @@ -113,23 +137,26 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { val login = TestableKerberosLogin.instance assertNotNull(login) executor.submit(() => login.reLogin(), 0) - - val node1 = "1" - selector.connect(node1, serverAddr, 1024, 1024) - login.logoutResumeLatch.countDown() - login.logoutCompleteLatch.await(15, TimeUnit.SECONDS) - assertFalse(pollUntilReadyOrDisconnected(selector, node1), "Authenticated during re-login") - - login.reLoginResumeLatch.countDown() - login.reLoginCompleteLatch.await(15, TimeUnit.SECONDS) - val node2 = "2" - selector.connect(node2, serverAddr, 1024, 1024) - assertTrue(pollUntilReadyOrDisconnected(selector, node2), "Authenticated failed after re-login") + verifyRelogin(selector, login) } finally { selector.close() } } + private def verifyRelogin(selector: Selector, login: TestableKerberosLogin): Unit = { + val node1 = "1" + selector.connect(node1, serverAddr, 1024, 1024) + login.logoutResumeLatch.countDown() + login.logoutCompleteLatch.await(15, TimeUnit.SECONDS) + assertFalse(pollUntilReadyOrDisconnected(selector, node1), "Authenticated during re-login") + + login.reLoginResumeLatch.countDown() + login.reLoginCompleteLatch.await(15, TimeUnit.SECONDS) + val node2 = "2" + selector.connect(node2, serverAddr, 1024, 1024) + assertTrue(pollUntilReadyOrDisconnected(selector, node2), "Authenticated failed after re-login") + } + /** * Tests that Kerberos error `Server not found in Kerberos database (7)` is handled * as a fatal authentication failure. @@ -256,6 +283,9 @@ class TestableKerberosLogin extends KerberosLogin { val logoutCompleteLatch = new CountDownLatch(1) val reLoginResumeLatch = new CountDownLatch(1) val reLoginCompleteLatch = new CountDownLatch(1) + @volatile var loginException: Option[RuntimeException] = None + @volatile var loginAttempts = 0 + @volatile var logoutAttempts = 0 assertNull(TestableKerberosLogin.instance) TestableKerberosLogin.instance = this @@ -265,7 +295,17 @@ class TestableKerberosLogin extends KerberosLogin { reLoginCompleteLatch.countDown() } + override protected def login(loginContext: LoginContext): Unit = { + loginAttempts += 1 + loginException.foreach { e => + loginException = None + throw e + } + super.login(loginContext) + } + override protected def logout(): Unit = { + logoutAttempts += 1 logoutResumeLatch.await(15, TimeUnit.SECONDS) super.logout() logoutCompleteLatch.countDown() From a855f6ac37149d6908499c68df46671a2754d21a Mon Sep 17 00:00:00 2001 From: Ryan <35785891+dielhennr@users.noreply.github.com> Date: Thu, 29 Apr 2021 09:25:21 -0700 Subject: [PATCH 100/155] KAFKA-12265; Move the BatchAccumulator in KafkaRaftClient to LeaderState (#10480) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The KafkaRaftClient has a field for the BatchAccumulator that is only used and set when it is the leader. In other cases, leader specific information was stored in LeaderState. In a recent change EpochState, which LeaderState implements, was changed to be a Closable. QuorumState makes sure to always close the previous state before transitioning to the next state. This redesign was used to move the BatchAccumulator to the LeaderState and simplify some of the handling in KafkaRaftClient. Reviewers: José Armando García Sancio , Jason Gustafson --- .../kafka/common/record/MemoryRecords.java | 5 +- .../common/record/MemoryRecordsTest.java | 2 + .../apache/kafka/raft/KafkaRaftClient.java | 107 +++++--------- .../org/apache/kafka/raft/LeaderState.java | 41 +++++- .../org/apache/kafka/raft/QuorumState.java | 15 +- .../raft/internals/BatchAccumulator.java | 62 ++++++++- .../kafka/raft/internals/BatchBuilder.java | 2 +- .../kafka/raft/KafkaRaftClientTest.java | 20 ++- .../apache/kafka/raft/LeaderStateTest.java | 63 ++++++--- .../org/apache/kafka/raft/MockLogTest.java | 3 +- .../apache/kafka/raft/QuorumStateTest.java | 39 +++--- .../raft/internals/BatchAccumulatorTest.java | 130 +++++++++++++++++- .../raft/internals/KafkaRaftMetricsTest.java | 5 +- 13 files changed, 356 insertions(+), 138 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index ae844bff0330f..ed2eb1496a452 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -641,12 +641,9 @@ public static MemoryRecords withLeaderChangeMessage( long initialOffset, long timestamp, int leaderEpoch, + ByteBuffer buffer, LeaderChangeMessage leaderChangeMessage ) { - // To avoid calling message toStruct multiple times, we supply a fixed message size - // for leader change, as it happens rare and the buffer could still grow if not sufficient in - // certain edge cases. - ByteBuffer buffer = ByteBuffer.allocate(256); writeLeaderChangeMessage(buffer, initialOffset, timestamp, leaderEpoch, leaderChangeMessage); buffer.flip(); return MemoryRecords.readableRecords(buffer); diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index 32945223c554a..16a51e6e3dd17 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -494,10 +494,12 @@ public void testBuildLeaderChangeMessage() { .setLeaderId(leaderId) .setVoters(Collections.singletonList( new Voter().setVoterId(voterId))); + ByteBuffer buffer = ByteBuffer.allocate(256); MemoryRecords records = MemoryRecords.withLeaderChangeMessage( initialOffset, System.currentTimeMillis(), leaderEpoch, + buffer, leaderChangeMessage ); diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index a3dbbdd3a66fd..d8f46392c2640 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -32,8 +32,6 @@ import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchSnapshotRequestData; import org.apache.kafka.common.message.FetchSnapshotResponseData; -import org.apache.kafka.common.message.LeaderChangeMessage; -import org.apache.kafka.common.message.LeaderChangeMessage.Voter; import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.metrics.Metrics; @@ -166,8 +164,6 @@ public class KafkaRaftClient implements RaftClient { private final List listenerContexts = new ArrayList<>(); private final ConcurrentLinkedQueue> pendingListeners = new ConcurrentLinkedQueue<>(); - private volatile BatchAccumulator accumulator; - /** * Create a new instance. * @@ -272,7 +268,7 @@ private void updateFollowerHighWatermark( } private void updateLeaderEndOffsetAndTimestamp( - LeaderState state, + LeaderState state, long currentTimeMs ) { final LogOffsetMetadata endOffsetMetadata = log.endOffset(); @@ -285,7 +281,7 @@ private void updateLeaderEndOffsetAndTimestamp( } private void onUpdateLeaderHighWatermark( - LeaderState state, + LeaderState state, long currentTimeMs ) { state.highWatermark().ifPresent(highWatermark -> { @@ -340,7 +336,7 @@ private void maybeFireHandleCommit(long baseOffset, int epoch, List records) } } - private void maybeFireHandleClaim(LeaderState state) { + private void maybeFireHandleClaim(LeaderState state) { int leaderEpoch = state.epoch(); long epochStartOffset = state.epochStartOffset(); for (ListenerContext listenerContext : listenerContexts) { @@ -395,24 +391,12 @@ private void resetConnections() { requestManager.resetAll(); } - private void onBecomeLeader(long currentTimeMs) { - LeaderState state = quorum.leaderStateOrThrow(); - - log.initializeLeaderEpoch(quorum.epoch()); - - // The high watermark can only be advanced once we have written a record - // from the new leader's epoch. Hence we write a control message immediately - // to ensure there is no delay committing pending data. - appendLeaderChangeMessage(state, log.endOffset().offset, currentTimeMs); - updateLeaderEndOffsetAndTimestamp(state, currentTimeMs); - - resetConnections(); + private void onBecomeLeader(long currentTimeMs) throws IOException { + long endOffset = log.endOffset().offset; - kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs); - - accumulator = new BatchAccumulator<>( + BatchAccumulator accumulator = new BatchAccumulator<>( quorum.epoch(), - log.endOffset().offset, + endOffset, raftConfig.appendLingerMs(), MAX_BATCH_SIZE_BYTES, memoryPool, @@ -420,38 +404,21 @@ private void onBecomeLeader(long currentTimeMs) { CompressionType.NONE, serde ); - } - - private static List convertToVoters(Set voterIds) { - return voterIds.stream() - .map(follower -> new Voter().setVoterId(follower)) - .collect(Collectors.toList()); - } - - private void appendLeaderChangeMessage(LeaderState state, long baseOffset, long currentTimeMs) { - List voters = convertToVoters(state.followers()); - List grantingVoters = convertToVoters(state.grantingVoters()); - // Adding the leader to the voters as any voter always votes for itself. - voters.add(new Voter().setVoterId(state.election().leaderId())); + LeaderState state = quorum.transitionToLeader(endOffset, accumulator); - LeaderChangeMessage leaderChangeMessage = new LeaderChangeMessage() - .setLeaderId(state.election().leaderId()) - .setVoters(voters) - .setGrantingVoters(grantingVoters); + log.initializeLeaderEpoch(quorum.epoch()); - MemoryRecords records = MemoryRecords.withLeaderChangeMessage( - baseOffset, - currentTimeMs, - quorum.epoch(), - leaderChangeMessage - ); + // The high watermark can only be advanced once we have written a record + // from the new leader's epoch. Hence we write a control message immediately + // to ensure there is no delay committing pending data. + state.appendLeaderChangeMessage(currentTimeMs); - appendAsLeader(records); - flushLeaderLog(state, currentTimeMs); + resetConnections(); + kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs); } - private void flushLeaderLog(LeaderState state, long currentTimeMs) { + private void flushLeaderLog(LeaderState state, long currentTimeMs) { // We update the end offset before flushing so that parked fetches can return sooner updateLeaderEndOffsetAndTimestamp(state, currentTimeMs); log.flush(); @@ -459,8 +426,6 @@ private void flushLeaderLog(LeaderState state, long currentTimeMs) { private boolean maybeTransitionToLeader(CandidateState state, long currentTimeMs) throws IOException { if (state.isVoteGranted()) { - long endOffset = log.endOffset().offset; - quorum.transitionToLeader(endOffset); onBecomeLeader(currentTimeMs); return true; } else { @@ -480,11 +445,6 @@ private void maybeResignLeadership() { if (quorum.isLeader()) { fireHandleResign(quorum.epoch()); } - - if (accumulator != null) { - accumulator.close(); - accumulator = null; - } } private void transitionToCandidate(long currentTimeMs) throws IOException { @@ -761,7 +721,7 @@ private boolean handleBeginQuorumEpochResponse( return handled.get(); } else if (partitionError == Errors.NONE) { if (quorum.isLeader()) { - LeaderState state = quorum.leaderStateOrThrow(); + LeaderState state = quorum.leaderStateOrThrow(); state.addAcknowledgementFrom(remoteNodeId); } else { logger.debug("Ignoring BeginQuorumEpoch response {} since " + @@ -1025,7 +985,7 @@ private FetchResponseData tryCompleteFetchRequest( long fetchOffset = request.fetchOffset(); int lastFetchedEpoch = request.lastFetchedEpoch(); - LeaderState state = quorum.leaderStateOrThrow(); + LeaderState state = quorum.leaderStateOrThrow(); ValidOffsetAndEpoch validOffsetAndEpoch = log.validateOffsetAndEpoch(fetchOffset, lastFetchedEpoch); final Records records; @@ -1180,7 +1140,7 @@ private DescribeQuorumResponseData handleDescribeQuorumRequest( return DescribeQuorumRequest.getTopLevelErrorResponse(Errors.INVALID_REQUEST); } - LeaderState leaderState = quorum.leaderStateOrThrow(); + LeaderState leaderState = quorum.leaderStateOrThrow(); return DescribeQuorumResponse.singletonResponse(log.topicPartition(), leaderState.localId(), leaderState.epoch(), @@ -1847,7 +1807,7 @@ public boolean isShuttingDown() { } private void appendBatch( - LeaderState state, + LeaderState state, BatchAccumulator.CompletedBatch batch, long appendTimeMs ) { @@ -1859,15 +1819,14 @@ private void appendBatch( offsetAndEpoch.offset + 1, Integer.MAX_VALUE); future.whenComplete((commitTimeMs, exception) -> { - int numRecords = batch.records.size(); if (exception != null) { - logger.debug("Failed to commit {} records at {}", numRecords, offsetAndEpoch, exception); + logger.debug("Failed to commit {} records at {}", batch.numRecords, offsetAndEpoch, exception); } else { long elapsedTime = Math.max(0, commitTimeMs - appendTimeMs); - double elapsedTimePerRecord = (double) elapsedTime / numRecords; + double elapsedTimePerRecord = (double) elapsedTime / batch.numRecords; kafkaRaftMetrics.updateCommitLatency(elapsedTimePerRecord, appendTimeMs); - logger.debug("Completed commit of {} records at {}", numRecords, offsetAndEpoch); - maybeFireHandleCommit(batch.baseOffset, epoch, batch.records); + logger.debug("Completed commit of {} records at {}", batch.numRecords, offsetAndEpoch); + batch.records.ifPresent(records -> maybeFireHandleCommit(batch.baseOffset, epoch, records)); } }); } finally { @@ -1876,12 +1835,12 @@ private void appendBatch( } private long maybeAppendBatches( - LeaderState state, + LeaderState state, long currentTimeMs ) { - long timeUnitFlush = accumulator.timeUntilDrain(currentTimeMs); - if (timeUnitFlush <= 0) { - List> batches = accumulator.drain(); + long timeUntilDrain = state.accumulator().timeUntilDrain(currentTimeMs); + if (timeUntilDrain <= 0) { + List> batches = state.accumulator().drain(); Iterator> iterator = batches.iterator(); try { @@ -1897,7 +1856,7 @@ private long maybeAppendBatches( } } } - return timeUnitFlush; + return timeUntilDrain; } private long pollResigned(long currentTimeMs) throws IOException { @@ -1925,7 +1884,7 @@ private long pollResigned(long currentTimeMs) throws IOException { } private long pollLeader(long currentTimeMs) { - LeaderState state = quorum.leaderStateOrThrow(); + LeaderState state = quorum.leaderStateOrThrow(); maybeFireHandleClaim(state); GracefulShutdown shutdown = this.shutdown.get(); @@ -2223,8 +2182,10 @@ public Long scheduleAtomicAppend(int epoch, List records) { } private Long append(int epoch, List records, boolean isAtomic) { - BatchAccumulator accumulator = this.accumulator; - if (accumulator == null) { + BatchAccumulator accumulator; + try { + accumulator = quorum.leaderStateOrThrow().accumulator(); + } catch (IllegalStateException ise) { return Long.MAX_VALUE; } diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 762769b12c51f..93d40679874e2 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -17,13 +17,17 @@ package org.apache.kafka.raft; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.raft.internals.BatchAccumulator; import org.slf4j.Logger; +import org.apache.kafka.common.message.LeaderChangeMessage; +import org.apache.kafka.common.message.LeaderChangeMessage.Voter; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; import java.util.Set; @@ -35,7 +39,7 @@ * More specifically, the set of unacknowledged voters are targets for BeginQuorumEpoch requests from the leader until * they acknowledge the leader. */ -public class LeaderState implements EpochState { +public class LeaderState implements EpochState { static final long OBSERVER_SESSION_TIMEOUT_MS = 300_000L; private final int localId; @@ -48,12 +52,15 @@ public class LeaderState implements EpochState { private final Set grantingVoters = new HashSet<>(); private final Logger log; + private final BatchAccumulator accumulator; + protected LeaderState( int localId, int epoch, long epochStartOffset, Set voters, Set grantingVoters, + BatchAccumulator accumulator, LogContext logContext ) { this.localId = localId; @@ -67,6 +74,30 @@ protected LeaderState( } this.grantingVoters.addAll(grantingVoters); this.log = logContext.logger(LeaderState.class); + this.accumulator = Objects.requireNonNull(accumulator, "accumulator must be non-null"); + } + + public BatchAccumulator accumulator() { + return this.accumulator; + } + + private static List convertToVoters(Set voterIds) { + return voterIds.stream() + .map(follower -> new Voter().setVoterId(follower)) + .collect(Collectors.toList()); + } + + public void appendLeaderChangeMessage(long currentTimeMs) { + List voters = convertToVoters(voterStates.keySet()); + List grantingVoters = convertToVoters(this.grantingVoters()); + + LeaderChangeMessage leaderChangeMessage = new LeaderChangeMessage() + .setLeaderId(this.election().leaderId()) + .setVoters(voters) + .setGrantingVoters(grantingVoters); + + accumulator.appendLeaderChangeMessage(leaderChangeMessage, currentTimeMs); + accumulator.forceDrain(); } @Override @@ -84,10 +115,6 @@ public int epoch() { return epoch; } - public Set followers() { - return voterStates.keySet().stream().filter(id -> id != localId).collect(Collectors.toSet()); - } - public Set grantingVoters() { return this.grantingVoters; } @@ -331,6 +358,8 @@ public String name() { } @Override - public void close() {} + public void close() { + accumulator.close(); + } } diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 86f8c187d8f35..7aaa708be05f3 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.raft.internals.BatchAccumulator; import org.slf4j.Logger; import java.io.IOException; @@ -422,7 +423,7 @@ public void transitionToCandidate() throws IOException { )); } - public void transitionToLeader(long epochStartOffset) throws IOException { + public LeaderState transitionToLeader(long epochStartOffset, BatchAccumulator accumulator) throws IOException { if (isObserver()) { throw new IllegalStateException("Cannot transition to Leader since the local broker.id=" + localId + " is not one of the voters " + voters); @@ -445,14 +446,17 @@ public void transitionToLeader(long epochStartOffset) throws IOException { // could address this problem by decoupling the local high watermark, but // we typically expect the state machine to be caught up anyway. - transitionTo(new LeaderState( + LeaderState state = new LeaderState<>( localIdOrThrow(), epoch(), epochStartOffset, voters, candidateState.grantingVoters(), + accumulator, logContext - )); + ); + transitionTo(state); + return state; } private void transitionTo(EpochState state) throws IOException { @@ -493,9 +497,10 @@ public UnattachedState unattachedStateOrThrow() { throw new IllegalStateException("Expected to be Unattached, but current state is " + state); } - public LeaderState leaderStateOrThrow() { + @SuppressWarnings("unchecked") + public LeaderState leaderStateOrThrow() { if (isLeader()) - return (LeaderState) state; + return (LeaderState) state; throw new IllegalStateException("Expected to be Leader, but current state is " + state); } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index 07d1015c9da92..84f744fc7826a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -24,12 +24,14 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.common.message.LeaderChangeMessage; import java.io.Closeable; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.OptionalInt; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicLong; @@ -202,6 +204,45 @@ private void completeCurrentBatch() { currentBatch = null; } + public void appendLeaderChangeMessage(LeaderChangeMessage leaderChangeMessage, long currentTimeMs) { + appendLock.lock(); + try { + forceDrain(); + ByteBuffer buffer = memoryPool.tryAllocate(256); + if (buffer != null) { + MemoryRecords data = MemoryRecords.withLeaderChangeMessage( + this.nextOffset, + currentTimeMs, + this.epoch, + buffer, + leaderChangeMessage + ); + completed.add(new CompletedBatch<>( + nextOffset, + 1, + data, + memoryPool, + buffer + )); + nextOffset += 1; + } else { + throw new IllegalStateException("Could not allocate buffer for the leader change record."); + } + } finally { + appendLock.unlock(); + } + } + + public void forceDrain() { + appendLock.lock(); + try { + drainStatus = DrainStatus.STARTED; + maybeCompleteDrain(); + } finally { + appendLock.unlock(); + } + } + private void maybeCompleteDrain() { if (drainStatus == DrainStatus.STARTED) { if (currentBatch != null && currentBatch.nonEmpty()) { @@ -339,7 +380,8 @@ public void close() { public static class CompletedBatch { public final long baseOffset; - public final List records; + public final int numRecords; + public final Optional> records; public final MemoryRecords data; private final MemoryPool pool; // Buffer that was allocated by the MemoryPool (pool). This may not be the buffer used in @@ -354,7 +396,23 @@ private CompletedBatch( ByteBuffer initialBuffer ) { this.baseOffset = baseOffset; - this.records = records; + this.records = Optional.of(records); + this.numRecords = records.size(); + this.data = data; + this.pool = pool; + this.initialBuffer = initialBuffer; + } + + private CompletedBatch( + long baseOffset, + int numRecords, + MemoryRecords data, + MemoryPool pool, + ByteBuffer initialBuffer + ) { + this.baseOffset = baseOffset; + this.records = Optional.empty(); + this.numRecords = numRecords; this.data = data; this.pool = pool; this.initialBuffer = initialBuffer; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java index c953b6a6371c7..93dc6546eb73a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java @@ -96,7 +96,7 @@ public BatchBuilder( } /** - * Append a record to this patch. The caller must first verify there is room for the batch + * Append a record to this batch. The caller must first verify there is room for the batch * using {@link #bytesNeeded(Collection, ObjectSerializationCache)}. * * @param record the record to append diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 55d4e16dc86e7..b2092aa8e17d3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -80,6 +80,7 @@ public void testInitializeAsLeaderFromStateStoreSingleMemberQuorum() throws Exce .withElectedLeader(initialEpoch, localId) .build(); + context.pollUntil(() -> context.log.endOffset().offset == 1L); assertEquals(1L, context.log.endOffset().offset); assertEquals(initialEpoch + 1, context.log.lastFetchedEpoch()); assertEquals(new LeaderAndEpoch(OptionalInt.of(localId), initialEpoch + 1), @@ -392,7 +393,7 @@ public void testInitializeAsCandidateAndBecomeLeader() throws Exception { context.deliverResponse(correlationId, otherNodeId, context.voteResponse(true, Optional.empty(), 1)); // Become leader after receiving the vote - context.client.poll(); + context.pollUntil(() -> context.log.endOffset().offset == 1L); context.assertElectedLeader(1, localId); long electionTimestamp = context.time.milliseconds(); @@ -410,7 +411,7 @@ public void testInitializeAsCandidateAndBecomeLeader() throws Exception { Record record = batch.iterator().next(); assertEquals(electionTimestamp, record.timestamp()); - RaftClientTestContext.verifyLeaderChangeMessage(localId, Arrays.asList(otherNodeId, localId), + RaftClientTestContext.verifyLeaderChangeMessage(localId, Arrays.asList(localId, otherNodeId), Arrays.asList(otherNodeId, localId), record.key(), record.value()); } @@ -432,7 +433,7 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree() throws Excep context.deliverResponse(correlationId, firstNodeId, context.voteResponse(true, Optional.empty(), 1)); // Become leader after receiving the vote - context.client.poll(); + context.pollUntil(() -> context.log.endOffset().offset == 1L); context.assertElectedLeader(1, localId); long electionTimestamp = context.time.milliseconds(); @@ -450,7 +451,7 @@ public void testInitializeAsCandidateAndBecomeLeaderQuorumOfThree() throws Excep Record record = batch.iterator().next(); assertEquals(electionTimestamp, record.timestamp()); - RaftClientTestContext.verifyLeaderChangeMessage(localId, Arrays.asList(firstNodeId, secondNodeId, localId), + RaftClientTestContext.verifyLeaderChangeMessage(localId, Arrays.asList(localId, firstNodeId, secondNodeId), Arrays.asList(firstNodeId, localId), record.key(), record.value()); } @@ -586,8 +587,11 @@ public void testAccumulatorClearedAfterBecomingFollower() throws Exception { MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); + ByteBuffer leaderBuffer = ByteBuffer.allocate(256); Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); + Mockito.when(memoryPool.tryAllocate(256)) + .thenReturn(leaderBuffer); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) @@ -615,8 +619,11 @@ public void testAccumulatorClearedAfterBecomingVoted() throws Exception { MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); + ByteBuffer leaderBuffer = ByteBuffer.allocate(256); Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); + Mockito.when(memoryPool.tryAllocate(256)) + .thenReturn(leaderBuffer); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) @@ -645,8 +652,11 @@ public void testAccumulatorClearedAfterBecomingUnattached() throws Exception { MemoryPool memoryPool = Mockito.mock(MemoryPool.class); ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES); + ByteBuffer leaderBuffer = ByteBuffer.allocate(256); Mockito.when(memoryPool.tryAllocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES)) .thenReturn(buffer); + Mockito.when(memoryPool.tryAllocate(256)) + .thenReturn(leaderBuffer); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withAppendLingerMs(lingerMs) @@ -1983,6 +1993,7 @@ public void testLeaderAppendSingleMemberQuorum() throws Exception { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters).build(); long now = context.time.milliseconds(); + context.pollUntil(() -> context.log.endOffset().offset == 1L); context.assertElectedLeader(1, localId); // We still write the leader change message @@ -2069,6 +2080,7 @@ public void testMetrics() throws Exception { int epoch = 1; RaftClientTestContext context = new RaftClientTestContext.Builder(localId, Collections.singleton(localId)) .build(); + context.pollUntil(() -> context.log.endOffset().offset == 1L); assertNotNull(getMetric(context.metrics, "current-state")); assertNotNull(getMetric(context.metrics, "current-leader")); diff --git a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java index 29ff800c4afbe..5f9989d55e4c3 100644 --- a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java @@ -19,9 +19,12 @@ 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.raft.internals.BatchAccumulator; + import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.Mockito; import java.util.Arrays; import java.util.Collections; @@ -44,25 +47,41 @@ public class LeaderStateTest { private final int epoch = 5; private final LogContext logContext = new LogContext(); - private LeaderState newLeaderState( + private final BatchAccumulator accumulator = Mockito.mock(BatchAccumulator.class); + + private LeaderState newLeaderState( Set voters, long epochStartOffset ) { - return new LeaderState( + return new LeaderState<>( localId, epoch, epochStartOffset, voters, voters, + accumulator, logContext ); } + @Test + public void testRequireNonNullAccumulator() { + assertThrows(NullPointerException.class, () -> new LeaderState<>( + localId, + epoch, + 0, + Collections.emptySet(), + Collections.emptySet(), + null, + logContext + )); + } + @Test public void testFollowerAcknowledgement() { int node1 = 1; int node2 = 2; - LeaderState state = newLeaderState(mkSet(localId, node1, node2), 0L); + LeaderState state = newLeaderState(mkSet(localId, node1, node2), 0L); assertEquals(mkSet(node1, node2), state.nonAcknowledgingVoters()); state.addAcknowledgementFrom(node1); assertEquals(singleton(node2), state.nonAcknowledgingVoters()); @@ -73,13 +92,13 @@ public void testFollowerAcknowledgement() { @Test public void testNonFollowerAcknowledgement() { int nonVoterId = 1; - LeaderState state = newLeaderState(singleton(localId), 0L); + LeaderState state = newLeaderState(singleton(localId), 0L); assertThrows(IllegalArgumentException.class, () -> state.addAcknowledgementFrom(nonVoterId)); } @Test public void testUpdateHighWatermarkQuorumSizeOne() { - LeaderState state = newLeaderState(singleton(localId), 15L); + LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); assertFalse(state.updateLocalState(0, new LogOffsetMetadata(15L))); assertEquals(emptySet(), state.nonAcknowledgingVoters()); @@ -92,7 +111,7 @@ public void testUpdateHighWatermarkQuorumSizeOne() { @Test public void testNonMonotonicLocalEndOffsetUpdate() { - LeaderState state = newLeaderState(singleton(localId), 15L); + LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); assertTrue(state.updateLocalState(0, new LogOffsetMetadata(16L))); assertEquals(Optional.of(new LogOffsetMetadata(16L)), state.highWatermark()); @@ -102,7 +121,7 @@ public void testNonMonotonicLocalEndOffsetUpdate() { @Test public void testIdempotentEndOffsetUpdate() { - LeaderState state = newLeaderState(singleton(localId), 15L); + LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); assertTrue(state.updateLocalState(0, new LogOffsetMetadata(16L))); assertFalse(state.updateLocalState(0, new LogOffsetMetadata(16L))); @@ -111,7 +130,7 @@ public void testIdempotentEndOffsetUpdate() { @Test public void testUpdateHighWatermarkMetadata() { - LeaderState state = newLeaderState(singleton(localId), 15L); + LeaderState state = newLeaderState(singleton(localId), 15L); assertEquals(Optional.empty(), state.highWatermark()); LogOffsetMetadata initialHw = new LogOffsetMetadata(16L, Optional.of(new MockOffsetMetadata("bar"))); @@ -126,7 +145,7 @@ public void testUpdateHighWatermarkMetadata() { @Test public void testUpdateHighWatermarkQuorumSizeTwo() { int otherNodeId = 1; - LeaderState state = newLeaderState(mkSet(localId, otherNodeId), 10L); + LeaderState state = newLeaderState(mkSet(localId, otherNodeId), 10L); assertFalse(state.updateLocalState(0, new LogOffsetMetadata(13L))); assertEquals(singleton(otherNodeId), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); @@ -143,7 +162,7 @@ public void testUpdateHighWatermarkQuorumSizeTwo() { public void testUpdateHighWatermarkQuorumSizeThree() { int node1 = 1; int node2 = 2; - LeaderState state = newLeaderState(mkSet(localId, node1, node2), 10L); + LeaderState state = newLeaderState(mkSet(localId, node1, node2), 10L); assertFalse(state.updateLocalState(0, new LogOffsetMetadata(15L))); assertEquals(mkSet(node1, node2), state.nonAcknowledgingVoters()); assertEquals(Optional.empty(), state.highWatermark()); @@ -167,7 +186,7 @@ public void testUpdateHighWatermarkQuorumSizeThree() { public void testNonMonotonicHighWatermarkUpdate() { MockTime time = new MockTime(); int node1 = 1; - LeaderState state = newLeaderState(mkSet(localId, node1), 0L); + LeaderState state = newLeaderState(mkSet(localId, node1), 0L); state.updateLocalState(time.milliseconds(), new LogOffsetMetadata(10L)); state.updateReplicaState(node1, time.milliseconds(), new LogOffsetMetadata(10L)); assertEquals(Optional.of(new LogOffsetMetadata(10L)), state.highWatermark()); @@ -186,7 +205,7 @@ public void testGetNonLeaderFollowersByFetchOffsetDescending() { long leaderStartOffset = 10L; long leaderEndOffset = 15L; - LeaderState state = setUpLeaderAndFollowers(node1, node2, leaderStartOffset, leaderEndOffset); + LeaderState state = setUpLeaderAndFollowers(node1, node2, leaderStartOffset, leaderEndOffset); // Leader should not be included; the follower with larger offset should be prioritized. assertEquals(Arrays.asList(node2, node1), state.nonLeaderVotersByDescendingFetchOffset()); @@ -199,7 +218,7 @@ public void testGetVoterStates() { long leaderStartOffset = 10L; long leaderEndOffset = 15L; - LeaderState state = setUpLeaderAndFollowers(node1, node2, leaderStartOffset, leaderEndOffset); + LeaderState state = setUpLeaderAndFollowers(node1, node2, leaderStartOffset, leaderEndOffset); assertEquals(mkMap( mkEntry(localId, leaderEndOffset), @@ -208,11 +227,11 @@ public void testGetVoterStates() { ), state.getVoterEndOffsets()); } - private LeaderState setUpLeaderAndFollowers(int follower1, - int follower2, - long leaderStartOffset, - long leaderEndOffset) { - LeaderState state = newLeaderState(mkSet(localId, follower1, follower2), leaderStartOffset); + private LeaderState setUpLeaderAndFollowers(int follower1, + int follower2, + long leaderStartOffset, + long leaderEndOffset) { + LeaderState state = newLeaderState(mkSet(localId, follower1, follower2), leaderStartOffset); state.updateLocalState(0, new LogOffsetMetadata(leaderEndOffset)); assertEquals(Optional.empty(), state.highWatermark()); state.updateReplicaState(follower1, 0, new LogOffsetMetadata(leaderStartOffset)); @@ -225,7 +244,7 @@ public void testGetObserverStatesWithObserver() { int observerId = 10; long epochStartOffset = 10L; - LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); + LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); long timestamp = 20L; assertFalse(state.updateReplicaState(observerId, timestamp, new LogOffsetMetadata(epochStartOffset))); @@ -237,7 +256,7 @@ public void testNoOpForNegativeRemoteNodeId() { int observerId = -1; long epochStartOffset = 10L; - LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); + LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); assertFalse(state.updateReplicaState(observerId, 0, new LogOffsetMetadata(epochStartOffset))); assertEquals(Collections.emptyMap(), state.getObserverStates(10)); @@ -248,7 +267,7 @@ public void testObserverStateExpiration() { MockTime time = new MockTime(); int observerId = 10; long epochStartOffset = 10L; - LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); + LeaderState state = newLeaderState(mkSet(localId), epochStartOffset); state.updateReplicaState(observerId, time.milliseconds(), new LogOffsetMetadata(epochStartOffset)); assertEquals(singleton(observerId), state.getObserverStates(time.milliseconds()).keySet()); @@ -260,7 +279,7 @@ public void testObserverStateExpiration() { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testGrantVote(boolean isLogUpToDate) { - LeaderState state = newLeaderState(Utils.mkSet(1, 2, 3), 1); + LeaderState state = newLeaderState(Utils.mkSet(1, 2, 3), 1); assertFalse(state.canGrantVote(1, isLogUpToDate)); assertFalse(state.canGrantVote(2, isLogUpToDate)); diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java index f219dea729a02..d2af3408f362b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -235,8 +235,9 @@ public void testAppendControlRecord() { final long initialOffset = 0; final int currentEpoch = 3; LeaderChangeMessage messageData = new LeaderChangeMessage().setLeaderId(0); + ByteBuffer buffer = ByteBuffer.allocate(256); log.appendAsLeader( - MemoryRecords.withLeaderChangeMessage(initialOffset, 0L, 2, messageData), + MemoryRecords.withLeaderChangeMessage(initialOffset, 0L, 2, buffer, messageData), currentEpoch ); diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java index deda71a61fe08..71ebfac0e26bf 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumStateTest.java @@ -19,6 +19,7 @@ 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.raft.internals.BatchAccumulator; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -45,6 +46,8 @@ public class QuorumStateTest { private final int fetchTimeoutMs = 10000; private final Random random = Mockito.spy(new Random(1)); + private BatchAccumulator accumulator = Mockito.mock(BatchAccumulator.class); + private QuorumState buildQuorumState(Set voters) { return buildQuorumState(OptionalInt.of(localId), voters); } @@ -269,7 +272,7 @@ public void testCandidateToLeader() throws IOException { assertTrue(state.isCandidate()); assertEquals(1, state.epoch()); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); LeaderState leaderState = state.leaderStateOrThrow(); assertTrue(state.isLeader()); assertEquals(1, leaderState.epoch()); @@ -284,10 +287,10 @@ public void testCandidateToLeaderWithoutGrantedVote() throws IOException { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); assertFalse(state.candidateStateOrThrow().isVoteGranted()); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L)); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); assertTrue(state.candidateStateOrThrow().isVoteGranted()); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); } @@ -359,11 +362,11 @@ public void testLeaderToLeader() throws IOException { QuorumState state = initializeEmptyState(voters); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); assertEquals(1, state.epoch()); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L)); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); assertTrue(state.isLeader()); assertEquals(1, state.epoch()); } @@ -376,7 +379,7 @@ public void testLeaderToResigned() throws IOException { QuorumState state = initializeEmptyState(voters); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); assertEquals(1, state.epoch()); @@ -397,7 +400,7 @@ public void testLeaderToCandidate() throws IOException { QuorumState state = initializeEmptyState(voters); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); assertTrue(state.isLeader()); assertEquals(1, state.epoch()); @@ -415,7 +418,7 @@ public void testLeaderToFollower() throws IOException { state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); state.transitionToFollower(5, otherNodeId); assertEquals(5, state.epoch()); @@ -431,7 +434,7 @@ public void testLeaderToUnattached() throws IOException { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); state.transitionToUnattached(5); assertEquals(5, state.epoch()); assertEquals(OptionalInt.empty(), state.leaderId()); @@ -446,7 +449,7 @@ public void testLeaderToVoted() throws IOException { state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); state.transitionToVoted(5, otherNodeId); assertEquals(5, state.epoch()); @@ -465,7 +468,7 @@ public void testLeaderToAnyStateLowerEpoch() throws IOException { state.transitionToUnattached(5); state.transitionToCandidate(); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); assertThrows(IllegalStateException.class, () -> state.transitionToUnattached(4)); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(4, otherNodeId)); assertThrows(IllegalStateException.class, () -> state.transitionToFollower(4, otherNodeId)); @@ -492,7 +495,7 @@ public void testUnattachedToLeaderOrResigned() throws IOException { QuorumState state = initializeEmptyState(voters); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isUnattached()); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L)); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -626,7 +629,7 @@ public void testVotedToInvalidLeaderOrResigned() throws IOException { QuorumState state = initializeEmptyState(voters); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToVoted(5, node1); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0)); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -780,7 +783,7 @@ public void testFollowerToLeaderOrResigned() throws IOException { QuorumState state = initializeEmptyState(voters); state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); state.transitionToFollower(8, node2); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0)); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToResigned(Collections.emptyList())); } @@ -900,7 +903,7 @@ public void testObserverCannotBecomeCandidateOrLeaderOrVoted() throws IOExceptio state.initialize(new OffsetAndEpoch(0L, logEndEpoch)); assertTrue(state.isObserver()); assertThrows(IllegalStateException.class, state::transitionToCandidate); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L)); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(5, otherNodeId)); } @@ -982,7 +985,7 @@ public void testHasRemoteLeader() throws IOException { assertFalse(state.hasRemoteLeader()); state.candidateStateOrThrow().recordGrantedVote(otherNodeId); - state.transitionToLeader(0L); + state.transitionToLeader(0L, accumulator); assertFalse(state.hasRemoteLeader()); state.transitionToUnattached(state.epoch() + 1); @@ -1022,7 +1025,7 @@ public void testHighWatermarkRetained() throws IOException { candidateState.recordGrantedVote(otherNodeId); assertTrue(candidateState.isVoteGranted()); - state.transitionToLeader(10L); + state.transitionToLeader(10L, accumulator); assertEquals(Optional.empty(), state.highWatermark()); } @@ -1036,7 +1039,7 @@ public void testInitializeWithEmptyLocalId() throws IOException { assertThrows(IllegalStateException.class, state::transitionToCandidate); assertThrows(IllegalStateException.class, () -> state.transitionToVoted(1, 1)); - assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L)); + assertThrows(IllegalStateException.class, () -> state.transitionToLeader(0L, accumulator)); state.transitionToFollower(1, 1); assertTrue(state.isFollower()); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java index b32168ec3101b..aacdeaa43c76c 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/BatchAccumulatorTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.raft.internals; import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.Writable; import org.apache.kafka.common.record.AbstractRecords; @@ -65,6 +66,133 @@ private BatchAccumulator buildAccumulator( ); } + @Test + public void testLeaderChangeMessageWritten() { + int leaderEpoch = 17; + long baseOffset = 0; + int lingerMs = 50; + int maxBatchSize = 512; + + ByteBuffer buffer = ByteBuffer.allocate(256); + Mockito.when(memoryPool.tryAllocate(256)) + .thenReturn(buffer); + + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); + + acc.appendLeaderChangeMessage(new LeaderChangeMessage(), time.milliseconds()); + assertTrue(acc.needsDrain(time.milliseconds())); + + List> batches = acc.drain(); + assertEquals(1, batches.size()); + + BatchAccumulator.CompletedBatch batch = batches.get(0); + batch.release(); + Mockito.verify(memoryPool).release(buffer); + } + + @Test + public void testForceDrain() { + asList(APPEND, APPEND_ATOMIC).forEach(appender -> { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; + + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(ByteBuffer.allocate(maxBatchSize)); + + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); + + List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); + + // Append records + assertEquals(baseOffset, appender.call(acc, leaderEpoch, records.subList(0, 1))); + assertEquals(baseOffset + 2, appender.call(acc, leaderEpoch, records.subList(1, 3))); + assertEquals(baseOffset + 5, appender.call(acc, leaderEpoch, records.subList(3, 6))); + assertEquals(baseOffset + 7, appender.call(acc, leaderEpoch, records.subList(6, 8))); + assertEquals(baseOffset + 8, appender.call(acc, leaderEpoch, records.subList(8, 9))); + + assertFalse(acc.needsDrain(time.milliseconds())); + acc.forceDrain(); + assertTrue(acc.needsDrain(time.milliseconds())); + assertEquals(0, acc.timeUntilDrain(time.milliseconds())); + + // Drain completed batches + List> batches = acc.drain(); + + assertEquals(1, batches.size()); + assertFalse(acc.needsDrain(time.milliseconds())); + assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); + + BatchAccumulator.CompletedBatch batch = batches.get(0); + assertEquals(records, batch.records.get()); + assertEquals(baseOffset, batch.baseOffset); + }); + } + + @Test + public void testForceDrainBeforeAppendLeaderChangeMessage() { + asList(APPEND, APPEND_ATOMIC).forEach(appender -> { + int leaderEpoch = 17; + long baseOffset = 157; + int lingerMs = 50; + int maxBatchSize = 512; + + Mockito.when(memoryPool.tryAllocate(maxBatchSize)) + .thenReturn(ByteBuffer.allocate(maxBatchSize)); + Mockito.when(memoryPool.tryAllocate(256)) + .thenReturn(ByteBuffer.allocate(256)); + + BatchAccumulator acc = buildAccumulator( + leaderEpoch, + baseOffset, + lingerMs, + maxBatchSize + ); + + List records = asList("a", "b", "c", "d", "e", "f", "g", "h", "i"); + + // Append records + assertEquals(baseOffset, appender.call(acc, leaderEpoch, records.subList(0, 1))); + assertEquals(baseOffset + 2, appender.call(acc, leaderEpoch, records.subList(1, 3))); + assertEquals(baseOffset + 5, appender.call(acc, leaderEpoch, records.subList(3, 6))); + assertEquals(baseOffset + 7, appender.call(acc, leaderEpoch, records.subList(6, 8))); + assertEquals(baseOffset + 8, appender.call(acc, leaderEpoch, records.subList(8, 9))); + + assertFalse(acc.needsDrain(time.milliseconds())); + + // Append a leader change message + acc.appendLeaderChangeMessage(new LeaderChangeMessage(), time.milliseconds()); + + assertTrue(acc.needsDrain(time.milliseconds())); + + // Test that drain status is FINISHED + assertEquals(0, acc.timeUntilDrain(time.milliseconds())); + + // Drain completed batches + List> batches = acc.drain(); + + // Should have 2 batches, one consisting of `records` and one `leaderChangeMessage` + assertEquals(2, batches.size()); + assertFalse(acc.needsDrain(time.milliseconds())); + assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); + + BatchAccumulator.CompletedBatch batch = batches.get(0); + assertEquals(records, batch.records.get()); + assertEquals(baseOffset, batch.baseOffset); + }); + } + @Test public void testLingerIgnoredIfAccumulatorEmpty() { int leaderEpoch = 17; @@ -202,7 +330,7 @@ public void testSingleBatchAccumulation() { assertEquals(Long.MAX_VALUE - time.milliseconds(), acc.timeUntilDrain(time.milliseconds())); BatchAccumulator.CompletedBatch batch = batches.get(0); - assertEquals(records, batch.records); + assertEquals(records, batch.records.get()); assertEquals(baseOffset, batch.baseOffset); }); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java index 3122873c2483b..0d64eac1cc2e2 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/KafkaRaftMetricsTest.java @@ -29,6 +29,7 @@ import org.apache.kafka.raft.QuorumState; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.io.IOException; import java.util.Collections; @@ -50,6 +51,8 @@ public class KafkaRaftMetricsTest { private final Random random = new Random(1); private KafkaRaftMetrics raftMetrics; + private BatchAccumulator accumulator = Mockito.mock(BatchAccumulator.class); + @AfterEach public void tearDown() { if (raftMetrics != null) { @@ -92,7 +95,7 @@ public void shouldRecordVoterQuorumState() throws IOException { assertEquals((double) -1L, getMetric(metrics, "high-watermark").metricValue()); state.candidateStateOrThrow().recordGrantedVote(1); - state.transitionToLeader(2L); + state.transitionToLeader(2L, accumulator); assertEquals("leader", getMetric(metrics, "current-state").metricValue()); assertEquals((double) localId, getMetric(metrics, "current-leader").metricValue()); assertEquals((double) localId, getMetric(metrics, "current-vote").metricValue()); From 8d38189eddd66d8ae53749f15bfff557f102a936 Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Thu, 29 Apr 2021 11:20:30 -0700 Subject: [PATCH 101/155] MINOR: clean up some replication code (#10564) Centralize leader and ISR changes in generateLeaderAndIsrUpdates. Consolidate handleNodeDeactivated and handleNodeActivated into this function. Rename BrokersToIsrs#noLeaderIterator to BrokersToIsrs#partitionsWithNoLeader. Create BrokersToIsrs#partitionsLedByBroker, BrokersToIsrs#partitionsWithBrokerInIsr In ReplicationControlManagerTest, createTestTopic should be a member function of ReplicationControlTestContext. It should invoke ReplicationControlTestContext#replay so that records are applied to all parts of the test context. Reviewers: Jun Rao , Chia-Ping Tsai --- .../kafka/controller/BrokersToIsrs.java | 10 +- .../ConfigurationControlManager.java | 4 + .../controller/ReplicationControlManager.java | 300 +++++++++--------- .../kafka/controller/BrokersToIsrsTest.java | 4 +- .../controller/QuorumControllerTest.java | 2 +- .../ReplicationControlManagerTest.java | 97 ++++-- 6 files changed, 241 insertions(+), 176 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java index 9d54c20380825..d8e0319f94f7e 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java @@ -311,10 +311,18 @@ PartitionsOnReplicaIterator iterator(int brokerId, boolean leadersOnly) { return new PartitionsOnReplicaIterator(topicMap, leadersOnly); } - PartitionsOnReplicaIterator noLeaderIterator() { + PartitionsOnReplicaIterator partitionsWithNoLeader() { return iterator(NO_LEADER, true); } + PartitionsOnReplicaIterator partitionsLedByBroker(int brokerId) { + return iterator(brokerId, true); + } + + PartitionsOnReplicaIterator partitionsWithBrokerInIsr(int brokerId) { + return iterator(brokerId, false); + } + boolean hasLeaderships(int brokerId) { return iterator(brokerId, true).hasNext(); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index 3e9e9e1d593c8..b53926e21fb62 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -374,6 +374,10 @@ void deleteTopicConfigs(String name) { configData.remove(new ConfigResource(Type.TOPIC, name)); } + boolean uncleanLeaderElectionEnabledForTopic(String name) { + return false; // TODO: support configuring unclean leader election. + } + class ConfigurationControlIterator implements Iterator> { private final long epoch; private final Iterator>> iterator; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index f169d1fb4aafc..ea94a0052d34b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -72,9 +72,11 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.function.Function; import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.ListIterator; import java.util.Map; import java.util.Map.Entry; import java.util.NoSuchElementException; @@ -172,47 +174,54 @@ String diff(PartitionControlInfo prev) { StringBuilder builder = new StringBuilder(); String prefix = ""; if (!Arrays.equals(replicas, prev.replicas)) { - builder.append(prefix).append("oldReplicas=").append(Arrays.toString(prev.replicas)); + builder.append(prefix).append("replicas: "). + append(Arrays.toString(prev.replicas)). + append(" -> ").append(Arrays.toString(replicas)); prefix = ", "; - builder.append(prefix).append("newReplicas=").append(Arrays.toString(replicas)); } if (!Arrays.equals(isr, prev.isr)) { - builder.append(prefix).append("oldIsr=").append(Arrays.toString(prev.isr)); + builder.append(prefix).append("isr: "). + append(Arrays.toString(prev.isr)). + append(" -> ").append(Arrays.toString(isr)); prefix = ", "; - builder.append(prefix).append("newIsr=").append(Arrays.toString(isr)); } if (!Arrays.equals(removingReplicas, prev.removingReplicas)) { - builder.append(prefix).append("oldRemovingReplicas="). - append(Arrays.toString(prev.removingReplicas)); + builder.append(prefix).append("removingReplicas: "). + append(Arrays.toString(prev.removingReplicas)). + append(" -> ").append(Arrays.toString(removingReplicas)); prefix = ", "; - builder.append(prefix).append("newRemovingReplicas="). - append(Arrays.toString(removingReplicas)); } if (!Arrays.equals(addingReplicas, prev.addingReplicas)) { - builder.append(prefix).append("oldAddingReplicas="). - append(Arrays.toString(prev.addingReplicas)); + builder.append(prefix).append("addingReplicas: "). + append(Arrays.toString(prev.addingReplicas)). + append(" -> ").append(Arrays.toString(addingReplicas)); prefix = ", "; - builder.append(prefix).append("newAddingReplicas="). - append(Arrays.toString(addingReplicas)); } if (leader != prev.leader) { - builder.append(prefix).append("oldLeader=").append(prev.leader); + builder.append(prefix).append("leader: "). + append(prev.leader).append(" -> ").append(leader); prefix = ", "; - builder.append(prefix).append("newLeader=").append(leader); } if (leaderEpoch != prev.leaderEpoch) { - builder.append(prefix).append("oldLeaderEpoch=").append(prev.leaderEpoch); + builder.append(prefix).append("leaderEpoch: "). + append(prev.leaderEpoch).append(" -> ").append(leaderEpoch); prefix = ", "; - builder.append(prefix).append("newLeaderEpoch=").append(leaderEpoch); } if (partitionEpoch != prev.partitionEpoch) { - builder.append(prefix).append("oldPartitionEpoch=").append(prev.partitionEpoch); - prefix = ", "; - builder.append(prefix).append("newPartitionEpoch=").append(partitionEpoch); + builder.append(prefix).append("partitionEpoch: "). + append(prev.partitionEpoch).append(" -> ").append(partitionEpoch); } return builder.toString(); } + void maybeLogPartitionChange(Logger log, String description, PartitionControlInfo prev) { + if (!electionWasClean(leader, prev.isr)) { + log.info("UNCLEAN partition change for {}: {}", description, diff(prev)); + } else if (log.isDebugEnabled()) { + log.debug("partition change for {}: {}", description, diff(prev)); + } + } + boolean hasLeader() { return leader != NO_LEADER; } @@ -231,7 +240,13 @@ public int hashCode() { public boolean equals(Object o) { if (!(o instanceof PartitionControlInfo)) return false; PartitionControlInfo other = (PartitionControlInfo) o; - return diff(other).isEmpty(); + return Arrays.equals(replicas, other.replicas) && + Arrays.equals(isr, other.isr) && + Arrays.equals(removingReplicas, other.removingReplicas) && + Arrays.equals(addingReplicas, other.addingReplicas) && + leader == other.leader && + leaderEpoch == other.leaderEpoch && + partitionEpoch == other.partitionEpoch; } @Override @@ -310,7 +325,7 @@ public void replay(TopicRecord record) { topicsByName.put(record.name(), record.topicId()); topics.put(record.topicId(), new TopicControlInfo(record.name(), snapshotRegistry, record.topicId())); - log.info("Created topic {} with ID {}.", record.name(), record.topicId()); + log.info("Created topic {} with topic ID {}.", record.name(), record.topicId()); } public void replay(PartitionRecord record) { @@ -321,22 +336,18 @@ public void replay(PartitionRecord record) { } PartitionControlInfo newPartInfo = new PartitionControlInfo(record); PartitionControlInfo prevPartInfo = topicInfo.parts.get(record.partitionId()); + String description = topicInfo.name + "-" + record.partitionId() + + " with topic ID " + record.topicId(); if (prevPartInfo == null) { - log.info("Created partition {}:{} with {}.", record.topicId(), - record.partitionId(), newPartInfo.toString()); + log.info("Created partition {} and {}.", description, newPartInfo); topicInfo.parts.put(record.partitionId(), newPartInfo); brokersToIsrs.update(record.topicId(), record.partitionId(), null, newPartInfo.isr, NO_LEADER, newPartInfo.leader); - } else { - String diff = newPartInfo.diff(prevPartInfo); - if (!diff.isEmpty()) { - log.info("Modified partition {}:{}: {}.", record.topicId(), - record.partitionId(), diff); - topicInfo.parts.put(record.partitionId(), newPartInfo); - brokersToIsrs.update(record.topicId(), record.partitionId(), - prevPartInfo.isr, newPartInfo.isr, prevPartInfo.leader, - newPartInfo.leader); - } + } else if (!newPartInfo.equals(prevPartInfo)) { + newPartInfo.maybeLogPartitionChange(log, description, prevPartInfo); + topicInfo.parts.put(record.partitionId(), newPartInfo); + brokersToIsrs.update(record.topicId(), record.partitionId(), prevPartInfo.isr, + newPartInfo.isr, prevPartInfo.leader, newPartInfo.leader); } } @@ -356,7 +367,9 @@ public void replay(PartitionChangeRecord record) { brokersToIsrs.update(record.topicId(), record.partitionId(), prevPartitionInfo.isr, newPartitionInfo.isr, prevPartitionInfo.leader, newPartitionInfo.leader); - log.debug("Applied ISR change record: {}", record.toString()); + String topicPart = topicInfo.name + "-" + record.partitionId() + " with topic ID " + + record.topicId(); + newPartitionInfo.maybeLogPartitionChange(log, topicPart, prevPartitionInfo); } public void replay(RemoveTopicRecord record) { @@ -723,7 +736,8 @@ void handleBrokerFenced(int brokerId, List records) { if (brokerRegistration == null) { throw new RuntimeException("Can't find broker registration for broker " + brokerId); } - handleNodeDeactivated(brokerId, records); + generateLeaderAndIsrUpdates("handleBrokerFenced", brokerId, NO_LEADER, records, + brokersToIsrs.partitionsWithBrokerInIsr(brokerId)); records.add(new ApiMessageAndVersion(new FenceBrokerRecord(). setId(brokerId).setEpoch(brokerRegistration.epoch()), (short) 0)); } @@ -740,60 +754,12 @@ void handleBrokerFenced(int brokerId, List records) { */ void handleBrokerUnregistered(int brokerId, long brokerEpoch, List records) { - handleNodeDeactivated(brokerId, records); + generateLeaderAndIsrUpdates("handleBrokerUnregistered", brokerId, NO_LEADER, records, + brokersToIsrs.partitionsWithBrokerInIsr(brokerId)); records.add(new ApiMessageAndVersion(new UnregisterBrokerRecord(). setBrokerId(brokerId).setBrokerEpoch(brokerEpoch), (short) 0)); } - /** - * Handle a broker being deactivated. This means we remove it from any ISR that has - * more than one element. We do not remove the broker from ISRs where it is the only - * member since this would preclude clean leader election in the future. - * It is removed as the leader for all partitions it leads. - * - * @param brokerId The broker id. - * @param records The record list to append to. - */ - void handleNodeDeactivated(int brokerId, List records) { - Iterator iterator = brokersToIsrs.iterator(brokerId, false); - while (iterator.hasNext()) { - TopicIdPartition topicIdPartition = iterator.next(); - TopicControlInfo topic = topics.get(topicIdPartition.topicId()); - if (topic == null) { - throw new RuntimeException("Topic ID " + topicIdPartition.topicId() + " existed in " + - "isrMembers, but not in the topics map."); - } - PartitionControlInfo partition = topic.parts.get(topicIdPartition.partitionId()); - if (partition == null) { - throw new RuntimeException("Partition " + topicIdPartition + - " existed in isrMembers, but not in the partitions map."); - } - PartitionChangeRecord record = new PartitionChangeRecord(). - setPartitionId(topicIdPartition.partitionId()). - setTopicId(topic.id); - int[] newIsr = Replicas.copyWithout(partition.isr, brokerId); - if (newIsr.length == 0) { - // We don't want to shrink the ISR to size 0. So, leave the node in the ISR. - if (record.leader() != NO_LEADER) { - // The partition is now leaderless, so set its leader to -1. - record.setLeader(-1); - records.add(new ApiMessageAndVersion(record, (short) 0)); - } - } else { - record.setIsr(Replicas.toList(newIsr)); - if (partition.leader == brokerId) { - // The fenced node will no longer be the leader. - int newLeader = bestLeader(partition.replicas, newIsr, false); - record.setLeader(newLeader); - } else { - // Bump the partition leader epoch. - record.setLeader(partition.leader); - } - records.add(new ApiMessageAndVersion(record, (short) 0)); - } - } - } - /** * Generate the appropriate records to handle a broker becoming unfenced. * @@ -808,43 +774,12 @@ void handleNodeDeactivated(int brokerId, List records) { void handleBrokerUnfenced(int brokerId, long brokerEpoch, List records) { records.add(new ApiMessageAndVersion(new UnfenceBrokerRecord(). setId(brokerId).setEpoch(brokerEpoch), (short) 0)); - handleNodeActivated(brokerId, records); - } - - /** - * Handle a broker being activated. This means we check if it can become the leader - * for any partition that currently has no leader (aka offline partition). - * - * @param brokerId The broker id. - * @param records The record list to append to. - */ - void handleNodeActivated(int brokerId, List records) { - Iterator iterator = brokersToIsrs.noLeaderIterator(); - while (iterator.hasNext()) { - TopicIdPartition topicIdPartition = iterator.next(); - TopicControlInfo topic = topics.get(topicIdPartition.topicId()); - if (topic == null) { - throw new RuntimeException("Topic ID " + topicIdPartition.topicId() + " existed in " + - "isrMembers, but not in the topics map."); - } - PartitionControlInfo partition = topic.parts.get(topicIdPartition.partitionId()); - if (partition == null) { - throw new RuntimeException("Partition " + topicIdPartition + - " existed in isrMembers, but not in the partitions map."); - } - // TODO: if this partition is configured for unclean leader election, - // check the replica set rather than the ISR. - if (Replicas.contains(partition.isr, brokerId)) { - records.add(new ApiMessageAndVersion(new PartitionChangeRecord(). - setPartitionId(topicIdPartition.partitionId()). - setTopicId(topic.id). - setLeader(brokerId), (short) 0)); - } - } + generateLeaderAndIsrUpdates("handleBrokerUnfenced", NO_LEADER, brokerId, records, + brokersToIsrs.partitionsWithNoLeader()); } ControllerResult electLeaders(ElectLeadersRequestData request) { - boolean unclean = electionIsUnclean(request.electionType()); + boolean uncleanOk = electionTypeIsUnclean(request.electionType()); List records = new ArrayList<>(); ElectLeadersResponseData response = new ElectLeadersResponseData(); for (TopicPartitions topic : request.topicPartitions()) { @@ -852,7 +787,7 @@ ControllerResult electLeaders(ElectLeadersRequestData new ReplicaElectionResult().setTopic(topic.topic()); response.replicaElectionResults().add(topicResults); for (int partitionId : topic.partitions()) { - ApiError error = electLeader(topic.topic(), partitionId, unclean, records); + ApiError error = electLeader(topic.topic(), partitionId, uncleanOk, records); topicResults.partitionResult().add(new PartitionResult(). setPartitionId(partitionId). setErrorCode(error.error().code()). @@ -862,7 +797,7 @@ ControllerResult electLeaders(ElectLeadersRequestData return ControllerResult.of(records, response); } - static boolean electionIsUnclean(byte electionType) { + static boolean electionTypeIsUnclean(byte electionType) { ElectionType type; try { type = ElectionType.valueOf(electionType); @@ -872,7 +807,7 @@ static boolean electionIsUnclean(byte electionType) { return type == ElectionType.UNCLEAN; } - ApiError electLeader(String topic, int partitionId, boolean unclean, + ApiError electLeader(String topic, int partitionId, boolean uncleanOk, List records) { Uuid topicId = topicsByName.get(topic); if (topicId == null) { @@ -889,7 +824,8 @@ ApiError electLeader(String topic, int partitionId, boolean unclean, return new ApiError(UNKNOWN_TOPIC_OR_PARTITION, "No such partition as " + topic + "-" + partitionId); } - int newLeader = bestLeader(partitionInfo.replicas, partitionInfo.isr, unclean); + int newLeader = bestLeader(partitionInfo.replicas, partitionInfo.isr, uncleanOk, + r -> clusterControl.unfenced(r)); if (newLeader == NO_LEADER) { // If we can't find any leader for the partition, return an error. return new ApiError(Errors.LEADER_NOT_AVAILABLE, @@ -907,13 +843,13 @@ ApiError electLeader(String topic, int partitionId, boolean unclean, } PartitionChangeRecord record = new PartitionChangeRecord(). setPartitionId(partitionId). - setTopicId(topicId); - if (unclean && !Replicas.contains(partitionInfo.isr, newLeader)) { - // If the election was unclean, we may have to forcibly add the replica to - // the ISR. This can result in data loss! + setTopicId(topicId). + setLeader(newLeader); + if (!electionWasClean(newLeader, partitionInfo.isr)) { + // If the election was unclean, we have to forcibly set the ISR to just the + // new leader. This can result in data loss! record.setIsr(Collections.singletonList(newLeader)); } - record.setLeader(newLeader); records.add(new ApiMessageAndVersion(record, (short) 0)); return ApiError.NONE; } @@ -936,10 +872,8 @@ ControllerResult processBrokerHeartbeat( handleBrokerUnfenced(brokerId, brokerEpoch, records); break; case CONTROLLED_SHUTDOWN: - // Note: we always bump the leader epoch of each partition that the - // shutting down broker is in here. This prevents the broker from - // getting re-added to the ISR later. - handleNodeDeactivated(brokerId, records); + generateLeaderAndIsrUpdates("enterControlledShutdown[" + brokerId + "]", + brokerId, NO_LEADER, records, brokersToIsrs.partitionsWithBrokerInIsr(brokerId)); break; case SHUTDOWN_NOW: handleBrokerFenced(brokerId, records); @@ -957,22 +891,27 @@ ControllerResult processBrokerHeartbeat( return ControllerResult.of(records, reply); } - int bestLeader(int[] replicas, int[] isr, boolean unclean) { + static boolean isGoodLeader(int[] isr, int leader) { + return Replicas.contains(isr, leader); + } + + static int bestLeader(int[] replicas, int[] isr, boolean uncleanOk, + Function isAcceptableLeader) { + int bestUnclean = NO_LEADER; for (int i = 0; i < replicas.length; i++) { int replica = replicas[i]; - if (Replicas.contains(isr, replica)) { - return replica; - } - } - if (unclean) { - for (int i = 0; i < replicas.length; i++) { - int replica = replicas[i]; - if (clusterControl.unfenced(replica)) { + if (isAcceptableLeader.apply(replica)) { + if (bestUnclean == NO_LEADER) bestUnclean = replica; + if (Replicas.contains(isr, replica)) { return replica; } } } - return NO_LEADER; + return uncleanOk ? bestUnclean : NO_LEADER; + } + + static boolean electionWasClean(int newLeader, int[] isr) { + return newLeader == NO_LEADER || Replicas.contains(isr, newLeader); } public ControllerResult unregisterBroker(int brokerId) { @@ -1119,6 +1058,83 @@ void validateManualPartitionAssignment(List assignment, } } + /** + * Iterate over a sequence of partitions and generate ISR changes and/or leader + * changes if necessary. + * + * @param context A human-readable context string used in log4j logging. + * @param brokerToRemove NO_LEADER if no broker is being removed; the ID of the + * broker to remove from the ISR and leadership, otherwise. + * @param brokerToAdd NO_LEADER if no broker is being added; the ID of the + * broker which is now eligible to be a leader, otherwise. + * @param records A list of records which we will append to. + * @param iterator The iterator containing the partitions to examine. + */ + void generateLeaderAndIsrUpdates(String context, + int brokerToRemove, + int brokerToAdd, + List records, + Iterator iterator) { + int oldSize = records.size(); + Function isAcceptableLeader = + r -> (r != brokerToRemove) && (r == brokerToAdd || clusterControl.unfenced(r)); + while (iterator.hasNext()) { + TopicIdPartition topicIdPart = iterator.next(); + TopicControlInfo topic = topics.get(topicIdPart.topicId()); + if (topic == null) { + throw new RuntimeException("Topic ID " + topicIdPart.topicId() + + " existed in isrMembers, but not in the topics map."); + } + PartitionControlInfo partition = topic.parts.get(topicIdPart.partitionId()); + if (partition == null) { + throw new RuntimeException("Partition " + topicIdPart + + " existed in isrMembers, but not in the partitions map."); + } + int[] newIsr = Replicas.copyWithout(partition.isr, brokerToRemove); + int newLeader; + if (isGoodLeader(newIsr, partition.leader)) { + // If the current leader is good, don't change. + newLeader = partition.leader; + } else { + // Choose a new leader. + boolean uncleanOk = configurationControl.uncleanLeaderElectionEnabledForTopic(topic.name); + newLeader = bestLeader(partition.replicas, newIsr, uncleanOk, isAcceptableLeader); + } + if (!electionWasClean(newLeader, newIsr)) { + // After an unclean leader election, the ISR is reset to just the new leader. + newIsr = new int[] {newLeader}; + } else if (newIsr.length == 0) { + // We never want to shrink the ISR to size 0. + newIsr = partition.isr; + } + PartitionChangeRecord record = new PartitionChangeRecord(). + setPartitionId(topicIdPart.partitionId()). + setTopicId(topic.id); + if (newLeader != partition.leader) record.setLeader(newLeader); + if (!Arrays.equals(newIsr, partition.isr)) record.setIsr(Replicas.toList(newIsr)); + if (record.leader() != NO_LEADER_CHANGE || record.isr() != null) { + records.add(new ApiMessageAndVersion(record, (short) 0)); + } + } + if (records.size() != oldSize) { + if (log.isDebugEnabled()) { + StringBuilder bld = new StringBuilder(); + String prefix = ""; + for (ListIterator iter = records.listIterator(oldSize); + iter.hasNext(); ) { + ApiMessageAndVersion apiMessageAndVersion = iter.next(); + PartitionChangeRecord record = (PartitionChangeRecord) apiMessageAndVersion.message(); + bld.append(prefix).append(topics.get(record.topicId()).name).append("-"). + append(record.partitionId()); + prefix = ", "; + } + log.debug("{}: changing partition(s): {}", context, bld.toString()); + } else if (log.isInfoEnabled()) { + log.info("{}: changing {} partition(s)", context, records.size() - oldSize); + } + } + } + class ReplicationControlIterator implements Iterator> { private final long epoch; private final Iterator iterator; diff --git a/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java index 525bf1ee3ae80..6510ee5d4b9eb 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java @@ -101,9 +101,9 @@ public void testNoLeader() { assertEquals(toSet(new TopicIdPartition(UUIDS[0], 2)), toSet(brokersToIsrs.iterator(3, true))); assertEquals(toSet(), toSet(brokersToIsrs.iterator(2, true))); - assertEquals(toSet(), toSet(brokersToIsrs.noLeaderIterator())); + assertEquals(toSet(), toSet(brokersToIsrs.partitionsWithNoLeader())); brokersToIsrs.update(UUIDS[0], 2, new int[]{1, 2, 3}, new int[]{1, 2, 3}, 3, -1); assertEquals(toSet(new TopicIdPartition(UUIDS[0], 2)), - toSet(brokersToIsrs.noLeaderIterator())); + toSet(brokersToIsrs.partitionsWithNoLeader())); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index d0588a0d27038..8c64cec8bd712 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -185,7 +185,7 @@ public void testUnregisterBroker() throws Throwable { topicPartitionFuture = active.appendReadEvent( "debugGetPartition", () -> { Iterator iterator = active. - replicationControl().brokersToIsrs().noLeaderIterator(); + replicationControl().brokersToIsrs().partitionsWithNoLeader(); assertTrue(iterator.hasNext()); return iterator.next(); }); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index e524581f32e36..2a456be830bb9 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection; import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; +import org.apache.kafka.common.metadata.PartitionChangeRecord; import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.TopicRecord; @@ -105,6 +106,27 @@ void replay(List records) throws Exception { ReplicationControlTestContext() { clusterControl.activate(); } + + CreatableTopicResult createTestTopic(String name, int[][] replicas) throws Exception { + assertFalse(replicas.length == 0); + CreateTopicsRequestData request = new CreateTopicsRequestData(); + CreatableTopic topic = new CreatableTopic().setName(name); + topic.setNumPartitions(-1).setReplicationFactor((short) -1); + for (int i = 0; i < replicas.length; i++) { + topic.assignments().add(new CreatableReplicaAssignment(). + setPartitionIndex(i).setBrokerIds(Replicas.toList(replicas[i]))); + } + request.topics().add(topic); + ControllerResult result = + replicationControl.createTopics(request); + CreatableTopicResult topicResult = result.response().topics().find(name); + assertNotNull(topicResult); + assertEquals((short) 0, topicResult.errorCode()); + assertEquals(replicas.length, topicResult.numPartitions()); + assertEquals(replicas[0].length, topicResult.replicationFactor()); + replay(result.records()); + return topicResult; + } } private static void registerBroker(int brokerId, ReplicationControlTestContext ctx) { @@ -125,7 +147,7 @@ private static void unfenceBroker(int brokerId, setBrokerId(brokerId).setBrokerEpoch(brokerId + 100).setCurrentMetadataOffset(1). setWantFence(false).setWantShutDown(false), 0); assertEquals(new BrokerHeartbeatReply(true, false, false, false), result.response()); - ControllerTestUtils.replayAll(ctx.clusterControl, result.records()); + ctx.replay(result.records()); } @Test @@ -157,7 +179,7 @@ public void testCreateTopics() throws Exception { setErrorMessage(null).setErrorCode((short) 0). setTopicId(result2.response().topics().find("foo").topicId())); assertEquals(expectedResponse2, result2.response()); - ControllerTestUtils.replayAll(replicationControl, result2.records()); + ctx.replay(result2.records()); assertEquals(new PartitionControlInfo(new int[] {2, 0, 1}, new int[] {2, 0, 1}, null, null, 2, 0, 0), replicationControl.getPartition( @@ -197,29 +219,6 @@ public void testValidateNewTopicNames() { assertEquals(expectedTopicErrors, topicErrors); } - private static CreatableTopicResult createTestTopic( - ReplicationControlManager replicationControl, String name, - int[][] replicas) throws Exception { - assertFalse(replicas.length == 0); - CreateTopicsRequestData request = new CreateTopicsRequestData(); - CreatableTopic topic = new CreatableTopic().setName(name); - topic.setNumPartitions(-1).setReplicationFactor((short) -1); - for (int i = 0; i < replicas.length; i++) { - topic.assignments().add(new CreatableReplicaAssignment(). - setPartitionIndex(i).setBrokerIds(Replicas.toList(replicas[i]))); - } - request.topics().add(topic); - ControllerResult result = - replicationControl.createTopics(request); - CreatableTopicResult topicResult = result.response().topics().find(name); - assertNotNull(topicResult); - assertEquals((short) 0, topicResult.errorCode()); - assertEquals(replicas.length, topicResult.numPartitions()); - assertEquals(replicas[0].length, topicResult.replicationFactor()); - ControllerTestUtils.replayAll(replicationControl, result.records()); - return topicResult; - } - @Test public void testRemoveLeaderships() throws Exception { ReplicationControlTestContext ctx = new ReplicationControlTestContext(); @@ -228,7 +227,7 @@ public void testRemoveLeaderships() throws Exception { registerBroker(i, ctx); unfenceBroker(i, ctx); } - CreatableTopicResult result = createTestTopic(replicationControl, "foo", + CreatableTopicResult result = ctx.createTestTopic("foo", new int[][] { new int[] {0, 1, 2}, new int[] {1, 2, 3}, @@ -241,8 +240,8 @@ public void testRemoveLeaderships() throws Exception { assertEquals(expectedPartitions, ControllerTestUtils. iteratorToSet(replicationControl.brokersToIsrs().iterator(0, true))); List records = new ArrayList<>(); - replicationControl.handleNodeDeactivated(0, records); - ControllerTestUtils.replayAll(replicationControl, records); + replicationControl.handleBrokerFenced(0, records); + ctx.replay(records); assertEquals(Collections.emptySet(), ControllerTestUtils. iteratorToSet(replicationControl.brokersToIsrs().iterator(0, true))); } @@ -255,7 +254,7 @@ public void testShrinkAndExpandIsr() throws Exception { registerBroker(i, ctx); unfenceBroker(i, ctx); } - CreatableTopicResult createTopicResult = createTestTopic(replicationControl, "foo", + CreatableTopicResult createTopicResult = ctx.createTestTopic("foo", new int[][] {new int[] {0, 1, 2}}); TopicIdPartition topicIdPartition = new TopicIdPartition(createTopicResult.topicId(), 0); @@ -287,7 +286,7 @@ public void testInvalidAlterIsrRequests() throws Exception { registerBroker(i, ctx); unfenceBroker(i, ctx); } - CreatableTopicResult createTopicResult = createTestTopic(replicationControl, "foo", + CreatableTopicResult createTopicResult = ctx.createTestTopic("foo", new int[][] {new int[] {0, 1, 2}}); TopicIdPartition topicIdPartition = new TopicIdPartition(createTopicResult.topicId(), 0); @@ -652,4 +651,42 @@ public void testValidateBadManualPartitionAssignments() throws Exception { ctx.replicationControl.validateManualPartitionAssignment(Arrays.asList(1, 2), OptionalInt.of(3))).getMessage()); } + + @Test + public void testElectionWasClean() { + assertTrue(ReplicationControlManager.electionWasClean(1, new int[] {1, 2})); + assertFalse(ReplicationControlManager.electionWasClean(1, new int[] {0, 2})); + assertFalse(ReplicationControlManager.electionWasClean(1, new int[] {})); + assertTrue(ReplicationControlManager.electionWasClean(3, new int[] {1, 2, 3, 4, 5, 6})); + } + + @Test + public void testPartitionControlInfoMergeAndDiff() { + PartitionControlInfo a = new PartitionControlInfo( + new int[]{1, 2, 3}, new int[]{1, 2}, null, null, 1, 0, 0); + PartitionControlInfo b = new PartitionControlInfo( + new int[]{1, 2, 3}, new int[]{3}, null, null, 3, 1, 1); + PartitionControlInfo c = new PartitionControlInfo( + new int[]{1, 2, 3}, new int[]{1}, null, null, 1, 0, 1); + assertEquals(b, a.merge(new PartitionChangeRecord(). + setLeader(3).setIsr(Arrays.asList(3)))); + assertEquals("isr: [1, 2] -> [3], leader: 1 -> 3, leaderEpoch: 0 -> 1, partitionEpoch: 0 -> 1", + b.diff(a)); + assertEquals("isr: [1, 2] -> [1], partitionEpoch: 0 -> 1", + c.diff(a)); + } + + @Test + public void testBestLeader() { + assertEquals(2, ReplicationControlManager.bestLeader( + new int[]{1, 2, 3, 4}, new int[]{4, 2, 3}, false, __ -> true)); + assertEquals(3, ReplicationControlManager.bestLeader( + new int[]{3, 2, 1, 4}, new int[]{4, 2, 3}, false, __ -> true)); + assertEquals(4, ReplicationControlManager.bestLeader( + new int[]{3, 2, 1, 4}, new int[]{4, 2, 3}, false, r -> r == 4)); + assertEquals(-1, ReplicationControlManager.bestLeader( + new int[]{3, 4, 5}, new int[]{1, 2}, false, r -> r == 4)); + assertEquals(4, ReplicationControlManager.bestLeader( + new int[]{3, 4, 5}, new int[]{1, 2}, true, r -> r == 4)); + } } From 9dbf2226cdae4a30e44340a552613446c5053d87 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 29 Apr 2021 12:30:48 -0700 Subject: [PATCH 102/155] MINOR: clean up some remaining locking stuff in StateDirectory (#10608) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Minor followup to #10342 that I noticed while working on the NamedTopology stuff. Cleans up a few things: We no longer need locking for the global state directory either, since it's contained within the top-level state directory lock. Definitely less critical than the task directory locking, since it's less vulnerable to IOExceptions given that it's just locked and unlocked once during the application lifetime, but nice to have nonetheless Clears out misc. usages of the LOCK_FILE_NAME that no longer apply. This has the awesome side effect of finally being able to actually delete obsolete task directories, whereas previously we had to leave behind the empty directory due to a ridiculous Windows bug (though I'm sure they would claim "it's not a bug it's a feature" 😉 ) Lazily delete old-and-now-unused lock files in the StateDirectory#taskDirIsEmpty method to clean up the state directory for applications that upgraded from an older version that still used task locking Reviewers: Walker Carlson --- .../internals/GlobalStateManagerImpl.java | 71 +++++---------- .../internals/GlobalStreamThread.java | 6 -- .../processor/internals/StateDirectory.java | 86 ++++++------------- .../internals/GlobalStateManagerImplTest.java | 79 ----------------- .../internals/StateDirectoryTest.java | 63 ++------------ 5 files changed, 56 insertions(+), 249 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index f10c707da4022..54ead671e49ba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateRestoreCallback; @@ -67,7 +66,6 @@ public class GlobalStateManagerImpl implements GlobalStateManager { private final Time time; private final Consumer globalConsumer; private final File baseDir; - private final StateDirectory stateDirectory; private final Set globalStoreNames = new HashSet<>(); private final FixedOrderMap> globalStores = new FixedOrderMap<>(); private final StateRestoreListener stateRestoreListener; @@ -103,7 +101,6 @@ public GlobalStateManagerImpl(final LogContext logContext, log = logContext.logger(GlobalStateManagerImpl.class); this.globalConsumer = globalConsumer; - this.stateDirectory = stateDirectory; this.stateRestoreListener = stateRestoreListener; final Map consumerProps = config.getGlobalConsumerConfigs("dummy"); @@ -124,22 +121,9 @@ public void setGlobalProcessorContext(final InternalProcessorContext globalProce @Override public Set initialize() { - try { - if (!stateDirectory.lockGlobalState()) { - throw new LockException(String.format("Failed to lock the global state directory: %s", baseDir)); - } - } catch (final IOException e) { - throw new LockException(String.format("Failed to lock the global state directory: %s", baseDir), e); - } - try { checkpointFileCache.putAll(checkpointFile.read()); } catch (final IOException e) { - try { - stateDirectory.unlockGlobalState(); - } catch (final IOException e1) { - log.error("Failed to unlock the global state directory", e); - } throw new StreamsException("Failed to read checkpoints for global state globalStores", e); } @@ -161,11 +145,6 @@ public Set initialize() { tp.topic(), checkpointFile.toString() ); - try { - stateDirectory.unlockGlobalState(); - } catch (final IOException e) { - log.error("Failed to unlock the global state directory", e); - } throw new StreamsException("Encountered a topic-partition not associated with any global state store"); } }); @@ -394,35 +373,31 @@ public void flush() { } @Override - public void close() throws IOException { - try { - if (globalStores.isEmpty()) { - return; - } - final StringBuilder closeFailed = new StringBuilder(); - for (final Map.Entry> entry : globalStores.entrySet()) { - if (entry.getValue().isPresent()) { - log.debug("Closing global storage engine {}", entry.getKey()); - try { - entry.getValue().get().close(); - } catch (final RuntimeException e) { - log.error("Failed to close global state store {}", entry.getKey(), e); - closeFailed.append("Failed to close global state store:") - .append(entry.getKey()) - .append(". Reason: ") - .append(e) - .append("\n"); - } - globalStores.put(entry.getKey(), Optional.empty()); - } else { - log.info("Skipping to close non-initialized store {}", entry.getKey()); + public void close() { + if (globalStores.isEmpty()) { + return; + } + final StringBuilder closeFailed = new StringBuilder(); + for (final Map.Entry> entry : globalStores.entrySet()) { + if (entry.getValue().isPresent()) { + log.debug("Closing global storage engine {}", entry.getKey()); + try { + entry.getValue().get().close(); + } catch (final RuntimeException e) { + log.error("Failed to close global state store {}", entry.getKey(), e); + closeFailed.append("Failed to close global state store:") + .append(entry.getKey()) + .append(". Reason: ") + .append(e) + .append("\n"); } + globalStores.put(entry.getKey(), Optional.empty()); + } else { + log.info("Skipping to close non-initialized store {}", entry.getKey()); } - if (closeFailed.length() > 0) { - throw new ProcessorStateException("Exceptions caught during close of 1 or more global state globalStores\n" + closeFailed); - } - } finally { - stateDirectory.unlockGlobalState(); + } + if (closeFailed.length() > 0) { + throw new ProcessorStateException("Exceptions caught during close of 1 or more global state globalStores\n" + closeFailed); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java index 4d416adac9c4c..2f8c0ed51b0b5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java @@ -27,7 +27,6 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; @@ -411,11 +410,6 @@ private StateConsumer initialize() { } return stateConsumer; - } catch (final LockException fatalException) { - final String errorMsg = "Could not lock global state directory. This could happen if multiple KafkaStreams " + - "instances are running on the same host using the same state directory."; - log.error(errorMsg, fatalException); - startupException = new StreamsException(errorMsg, fatalException); } catch (final StreamsException fatalException) { startupException = fatalException; } catch (final Exception fatalException) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index 6baf04acf541e..1af2d8ea4af2c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -34,13 +34,11 @@ import java.nio.channels.FileChannel; import java.nio.channels.FileLock; import java.nio.channels.OverlappingFileLockException; -import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.Files; import java.nio.file.StandardOpenOption; import java.nio.file.attribute.PosixFilePermission; import java.nio.file.attribute.PosixFilePermissions; -import java.util.Collections; import java.util.HashMap; import java.util.Set; import java.util.UUID; @@ -70,6 +68,7 @@ static class StateDirectoryProcessFile { @JsonProperty private final UUID processId; + // required by jackson -- do not remove, your IDE may be warning that this is unused but it's lying to you public StateDirectoryProcessFile() { this.processId = null; } @@ -90,9 +89,6 @@ public StateDirectoryProcessFile() { private FileChannel stateDirLockChannel; private FileLock stateDirLock; - private FileChannel globalStateChannel; - private FileLock globalStateLock; - /** * Ensures that the state base directory as well as the application's sub-directory are created. * @@ -243,16 +239,34 @@ File checkpointFileFor(final TaskId taskId) { boolean directoryForTaskIsEmpty(final TaskId taskId) { final File taskDir = getOrCreateDirectoryForTask(taskId); - return taskDirEmpty(taskDir); + return taskDirIsEmpty(taskDir); } - private boolean taskDirEmpty(final File taskDir) { + private boolean taskDirIsEmpty(final File taskDir) { final File[] storeDirs = taskDir.listFiles(pathname -> - !pathname.getName().equals(LOCK_FILE_NAME) && !pathname.getName().equals(CHECKPOINT_FILE_NAME)); + boolean taskDirEmpty = true; + // if the task is stateless, storeDirs would be null - return storeDirs == null || storeDirs.length == 0; + if (storeDirs != null && storeDirs.length > 0) { + for (final File file : storeDirs) { + // We removed the task directory locking but some upgrading applications may still have old lock files on disk, + // we just lazily delete those in this method since it's the only thing that would be affected by these + if (file.getName().endsWith(LOCK_FILE_NAME)) { + if (!file.delete()) { + // If we hit an error deleting this just ignore it and move on, we'll retry again at some point + log.warn("Error encountered deleting lock file in {}", taskDir); + } + } else { + // If it's not a lock file then the directory is not empty, + // but finish up the loop in case there's a lock file left to delete + log.trace("TaskDir {} was not empty, found {}", taskDir, file); + taskDirEmpty = false; + } + } + } + return taskDirEmpty; } /** @@ -304,51 +318,6 @@ synchronized boolean lock(final TaskId taskId) { } } - synchronized boolean lockGlobalState() throws IOException { - if (!hasPersistentStores) { - return true; - } - - if (globalStateLock != null) { - log.trace("{} Found cached state dir lock for the global task", logPrefix()); - return true; - } - - final File lockFile = new File(globalStateDir(), LOCK_FILE_NAME); - final FileChannel channel; - try { - channel = FileChannel.open(lockFile.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE); - } catch (final NoSuchFileException e) { - // FileChannel.open(..) could throw NoSuchFileException when there is another thread - // concurrently deleting the parent directory (i.e. the directory of the taskId) of the lock - // file, in this case we will return immediately indicating locking failed. - return false; - } - final FileLock fileLock = tryLock(channel); - if (fileLock == null) { - channel.close(); - return false; - } - globalStateChannel = channel; - globalStateLock = fileLock; - - log.debug("{} Acquired global state dir lock", logPrefix()); - - return true; - } - - synchronized void unlockGlobalState() throws IOException { - if (globalStateLock == null) { - return; - } - globalStateLock.release(); - globalStateChannel.close(); - globalStateLock = null; - globalStateChannel = null; - - log.debug("{} Released global state dir lock", logPrefix()); - } - /** * Unlock the state directory for the given {@link TaskId}. */ @@ -377,9 +346,6 @@ public void close() { if (!lockedTasksToOwner.isEmpty()) { log.error("Some task directories still locked while closing state, this indicates unclean shutdown: {}", lockedTasksToOwner); } - if (globalStateLock != null) { - log.error("Global state lock is present while closing the state, this indicates unclean shutdown"); - } } } @@ -431,7 +397,7 @@ private void cleanRemovedTasksCalledByCleanerThread(final long cleanupDelayMs) { if (now > lastModifiedMs + cleanupDelayMs) { log.info("{} Deleting obsolete state directory {} for task {} as {}ms has elapsed (cleanup delay is {}ms).", logPrefix(), dirName, id, now - lastModifiedMs, cleanupDelayMs); - Utils.delete(taskDir, Collections.singletonList(new File(taskDir, LOCK_FILE_NAME))); + Utils.delete(taskDir); } } } catch (final IOException exception) { @@ -457,7 +423,7 @@ private void cleanRemovedTasksCalledByUser() throws Exception { if (lock(id)) { log.info("{} Deleting state directory {} for task {} as user calling cleanup.", logPrefix(), dirName, id); - Utils.delete(taskDir, Collections.singletonList(new File(taskDir, LOCK_FILE_NAME))); + Utils.delete(taskDir); } else { log.warn("{} Could not get lock for state directory {} for task {} as user calling cleanup.", logPrefix(), dirName, id); @@ -506,7 +472,7 @@ File[] listNonEmptyTaskDirectories() { if (!pathname.isDirectory() || !TASK_DIR_PATH_NAME.matcher(pathname.getName()).matches()) { return false; } else { - return !taskDirEmpty(pathname); + return !taskDirIsEmpty(pathname); } }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java index 584cc9456ee5a..ee5ca80efdeb0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.errors.LockException; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.StateRestoreCallback; @@ -42,7 +41,6 @@ import org.apache.kafka.test.MockStateRestoreListener; import org.apache.kafka.test.NoOpReadOnlyStore; import org.apache.kafka.test.TestUtils; -import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -152,28 +150,6 @@ public void before() { checkpointFile = new File(stateManager.baseDir(), StateManagerUtil.CHECKPOINT_FILE_NAME); } - @After - public void after() throws IOException { - stateDirectory.unlockGlobalState(); - } - - @Test - public void shouldLockGlobalStateDirectory() { - stateManager.initialize(); - assertTrue(new File(stateDirectory.globalStateDir(), ".lock").exists()); - } - - @Test - public void shouldThrowLockExceptionIfCantGetLock() throws IOException { - final StateDirectory stateDir = new StateDirectory(streamsConfig, time, true); - try { - stateDir.lockGlobalState(); - assertThrows(LockException.class, stateManager::initialize); - } finally { - stateDir.unlockGlobalState(); - } - } - @Test public void shouldReadCheckpointOffsets() throws IOException { final Map expected = writeCheckpoint(); @@ -451,19 +427,6 @@ public void shouldThrowIllegalArgumentExceptionIfCallbackIsNull() { } } - @Test - public void shouldUnlockGlobalStateDirectoryOnClose() throws IOException { - stateManager.initialize(); - stateManager.close(); - final StateDirectory stateDir = new StateDirectory(streamsConfig, new MockTime(), true); - try { - // should be able to get the lock now as it should've been released in close - assertTrue(stateDir.lockGlobalState()); - } finally { - stateDir.unlockGlobalState(); - } - } - @Test public void shouldNotCloseStoresIfCloseAlreadyCalled() throws IOException { stateManager.initialize(); @@ -507,23 +470,6 @@ public void close() { assertFalse(store2.isOpen()); } - @Test - public void shouldReleaseLockIfExceptionWhenLoadingCheckpoints() throws IOException { - writeCorruptCheckpoint(); - try { - stateManager.initialize(); - } catch (final StreamsException e) { - // expected - } - final StateDirectory stateDir = new StateDirectory(streamsConfig, new MockTime(), true); - try { - // should be able to get the lock now as it should've been released - assertTrue(stateDir.lockGlobalState()); - } finally { - stateDir.unlockGlobalState(); - } - } - @Test public void shouldCheckpointOffsets() throws IOException { final Map offsets = Collections.singletonMap(t1, 25L); @@ -604,31 +550,6 @@ private Map readOffsetsCheckpoint() throws IOException { return offsetCheckpoint.read(); } - @Test - public void shouldThrowLockExceptionIfIOExceptionCaughtWhenTryingToLockStateDir() { - stateManager = new GlobalStateManagerImpl( - new LogContext("mock"), - time, - topology, - consumer, - new StateDirectory(streamsConfig, time, true) { - @Override - public boolean lockGlobalState() throws IOException { - throw new IOException("KABOOM!"); - } - }, - stateRestoreListener, - streamsConfig - ); - - try { - stateManager.initialize(); - fail("Should have thrown LockException"); - } catch (final LockException e) { - // pass - } - } - @Test public void shouldNotRetryWhenEndOffsetsThrowsTimeoutExceptionAndTaskTimeoutIsZero() { final AtomicInteger numberOfCalls = new AtomicInteger(0); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index b4e7644e55c02..503d6c4ef5e05 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -39,7 +39,6 @@ import java.io.File; import java.io.IOException; import java.nio.channels.FileChannel; -import java.nio.channels.OverlappingFileLockException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; @@ -285,7 +284,7 @@ public void shouldCleanUpTaskStateDirectoriesThatAreNotCurrentlyLocked() { files = Arrays.stream( Objects.requireNonNull(directory.listAllTaskDirectories())).collect(Collectors.toSet()); - assertEquals(mkSet(dir0, dir1, dir2), files); + assertEquals(mkSet(dir0, dir1), files); files = Arrays.stream( Objects.requireNonNull(directory.listNonEmptyTaskDirectories())).collect(Collectors.toSet()); @@ -309,13 +308,13 @@ public void shouldCleanupStateDirectoriesWhenLastModifiedIsLessThanNowMinusClean time.sleep(cleanupDelayMs + 1000); directory.cleanRemovedTasks(cleanupDelayMs); - assertTrue(dir.exists()); - assertEquals(1, directory.listAllTaskDirectories().length); + assertFalse(dir.exists()); + assertEquals(0, directory.listAllTaskDirectories().length); assertEquals(0, directory.listNonEmptyTaskDirectories().length); } @Test - public void shouldCleanupObsoleteStateDirectoriesOnlyOnce() { + public void shouldCleanupObsoleteTaskDirectoriesAndDeleteTheDirectoryItself() { final File dir = directory.getOrCreateDirectoryForTask(new TaskId(2, 0)); assertTrue(new File(dir, "store").mkdir()); assertEquals(1, directory.listAllTaskDirectories().length); @@ -324,26 +323,14 @@ public void shouldCleanupObsoleteStateDirectoriesOnlyOnce() { try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateDirectory.class)) { time.sleep(5000); directory.cleanRemovedTasks(0); - assertTrue(dir.exists()); - assertEquals(1, directory.listAllTaskDirectories().length); + assertFalse(dir.exists()); + assertEquals(0, directory.listAllTaskDirectories().length); assertEquals(0, directory.listNonEmptyTaskDirectories().length); assertThat( appender.getMessages(), hasItem(containsString("Deleting obsolete state directory")) ); } - - try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StateDirectory.class)) { - time.sleep(5000); - directory.cleanRemovedTasks(0); - assertTrue(dir.exists()); - assertEquals(1, directory.listAllTaskDirectories().length); - assertEquals(0, directory.listNonEmptyTaskDirectories().length); - assertThat( - appender.getMessages(), - not(hasItem(containsString("Deleting obsolete state directory"))) - ); - } } @Test @@ -428,37 +415,6 @@ public void shouldCreateDirectoriesIfParentDoesntExist() { assertTrue(taskDir.exists()); } - @Test - public void shouldLockGlobalStateDirectory() throws IOException { - try ( - final FileChannel channel = FileChannel.open( - new File(directory.globalStateDir(), LOCK_FILE_NAME).toPath(), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE) - ) { - directory.lockGlobalState(); - assertThrows(OverlappingFileLockException.class, channel::lock); - } finally { - directory.unlockGlobalState(); - } - } - - @Test - public void shouldUnlockGlobalStateDirectory() throws IOException { - directory.lockGlobalState(); - directory.unlockGlobalState(); - - try ( - final FileChannel channel = FileChannel.open( - new File(directory.globalStateDir(), LOCK_FILE_NAME).toPath(), - StandardOpenOption.CREATE, - StandardOpenOption.WRITE) - ) { - // should lock without any exceptions - channel.lock(); - } - } - @Test public void shouldNotLockStateDirLockedByAnotherThread() throws Exception { final TaskId taskId = new TaskId(0, 0); @@ -548,12 +504,6 @@ public void shouldLockTaskStateDirectoryWhenDirectoryCreationDisabled() throws I assertTrue(directory.lock(taskId)); } - @Test - public void shouldLockGlobalStateDirectoryWhenDirectoryCreationDisabled() throws IOException { - initializeStateDirectory(false); - assertTrue(directory.lockGlobalState()); - } - @Test public void shouldNotFailWhenCreatingTaskDirectoryInParallel() throws Exception { final TaskId taskId = new TaskId(0, 0); @@ -685,6 +635,7 @@ private static class FutureStateDirectoryProcessFile { @JsonProperty private final String newField; + // required by jackson -- do not remove, your IDE may be warning that this is unused but it's lying to you public FutureStateDirectoryProcessFile() { this.processId = null; this.newField = null; From e454becb33faac0851b7d145452992f183cbff76 Mon Sep 17 00:00:00 2001 From: Valery Kokorev <44339288+Nathan22177@users.noreply.github.com> Date: Fri, 30 Apr 2021 05:47:26 +0300 Subject: [PATCH 103/155] KAFKA-12396: added null check for state stores key (#10548) Reviewers: Bruno Cadonna , Matthias J. Sax --- .../internals/InMemoryKeyValueStore.java | 3 - .../state/internals/MeteredKeyValueStore.java | 13 ++++- .../state/internals/MeteredSessionStore.java | 6 ++ .../state/internals/MeteredWindowStore.java | 10 ++++ .../streams/state/internals/RocksDBStore.java | 3 - .../internals/InMemoryKeyValueStoreTest.java | 6 ++ .../internals/MeteredKeyValueStoreTest.java | 51 +++++++++++++++++ .../internals/MeteredSessionStoreTest.java | 55 +++++++++++++++++++ .../internals/MeteredWindowStoreTest.java | 36 ++++++++++++ 9 files changed, 176 insertions(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java index a832e75b10ce3..59ed97abc9c89 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java @@ -32,7 +32,6 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; -import java.util.Objects; public class InMemoryKeyValueStore implements KeyValueStore { @@ -107,8 +106,6 @@ public void putAll(final List> entries) { @Override public , P> KeyValueIterator prefixScan(final P prefix, final PS prefixKeySerializer) { - Objects.requireNonNull(prefix, "prefix cannot be null"); - Objects.requireNonNull(prefixKeySerializer, "prefixKeySerializer cannot be null"); final Bytes from = Bytes.wrap(prefixKeySerializer.serialize(null, prefix)); final Bytes to = Bytes.increment(from); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 51e6dd02e99d8..18c44e8a49faa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -38,6 +38,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.Objects; import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; @@ -185,6 +186,7 @@ public boolean setFlushListener(final CacheFlushListener listener, @Override public V get(final K key) { + Objects.requireNonNull(key, "key cannot be null"); try { return maybeMeasureLatency(() -> outerValue(wrapped().get(keyBytes(key))), time, getSensor); } catch (final ProcessorStateException e) { @@ -196,6 +198,7 @@ public V get(final K key) { @Override public void put(final K key, final V value) { + Objects.requireNonNull(key, "key cannot be null"); try { maybeMeasureLatency(() -> wrapped().put(keyBytes(key), serdes.rawValue(value)), time, putSensor); maybeRecordE2ELatency(); @@ -208,6 +211,7 @@ public void put(final K key, @Override public V putIfAbsent(final K key, final V value) { + Objects.requireNonNull(key, "key cannot be null"); final V currentValue = maybeMeasureLatency( () -> outerValue(wrapped().putIfAbsent(keyBytes(key), serdes.rawValue(value))), time, @@ -219,11 +223,13 @@ public V putIfAbsent(final K key, @Override public void putAll(final List> entries) { + entries.forEach(entry -> Objects.requireNonNull(entry.key, "key cannot be null")); maybeMeasureLatency(() -> wrapped().putAll(innerEntries(entries)), time, putAllSensor); } @Override public V delete(final K key) { + Objects.requireNonNull(key, "key cannot be null"); try { return maybeMeasureLatency(() -> outerValue(wrapped().delete(keyBytes(key))), time, deleteSensor); } catch (final ProcessorStateException e) { @@ -234,13 +240,16 @@ public V delete(final K key) { @Override public , P> KeyValueIterator prefixScan(final P prefix, final PS prefixKeySerializer) { - + Objects.requireNonNull(prefix, "key cannot be null"); + Objects.requireNonNull(prefixKeySerializer, "prefixKeySerializer cannot be null"); return new MeteredKeyValueIterator(wrapped().prefixScan(prefix, prefixKeySerializer), prefixScanSensor); } @Override public KeyValueIterator range(final K from, final K to) { + Objects.requireNonNull(from, "keyFrom cannot be null"); + Objects.requireNonNull(to, "keyTo cannot be null"); return new MeteredKeyValueIterator( wrapped().range(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to))), rangeSensor @@ -250,6 +259,8 @@ public KeyValueIterator range(final K from, @Override public KeyValueIterator reverseRange(final K from, final K to) { + Objects.requireNonNull(from, "keyFrom cannot be null"); + Objects.requireNonNull(to, "keyTo cannot be null"); return new MeteredKeyValueIterator( wrapped().reverseRange(Bytes.wrap(serdes.rawKey(from)), Bytes.wrap(serdes.rawKey(to))), rangeSensor diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 0419192e7ec11..7a31b170ea4e6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -159,6 +159,9 @@ public boolean setFlushListener(final CacheFlushListener, V> listene public void put(final Windowed sessionKey, final V aggregate) { Objects.requireNonNull(sessionKey, "sessionKey can't be null"); + Objects.requireNonNull(sessionKey.key(), "sessionKey.key() can't be null"); + Objects.requireNonNull(sessionKey.window(), "sessionKey.window() can't be null"); + try { maybeMeasureLatency( () -> { @@ -178,6 +181,9 @@ public void put(final Windowed sessionKey, @Override public void remove(final Windowed sessionKey) { Objects.requireNonNull(sessionKey, "sessionKey can't be null"); + Objects.requireNonNull(sessionKey.key(), "sessionKey.key() can't be null"); + Objects.requireNonNull(sessionKey.window(), "sessionKey.window() can't be null"); + try { maybeMeasureLatency( () -> { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index e7516d29a5068..041026b1a8b2b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -36,6 +36,8 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.internals.metrics.StateStoreMetrics; +import java.util.Objects; + import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerde; import static org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency; @@ -161,6 +163,7 @@ public boolean setFlushListener(final CacheFlushListener, V> listene public void put(final K key, final V value, final long windowStartTimestamp) { + Objects.requireNonNull(key, "key cannot be null"); try { maybeMeasureLatency( () -> wrapped().put(keyBytes(key), serdes.rawValue(value), windowStartTimestamp), @@ -177,6 +180,7 @@ public void put(final K key, @Override public V fetch(final K key, final long timestamp) { + Objects.requireNonNull(key, "key cannot be null"); return maybeMeasureLatency( () -> { final byte[] result = wrapped().fetch(keyBytes(key), timestamp); @@ -195,6 +199,7 @@ public V fetch(final K key, public WindowStoreIterator fetch(final K key, final long timeFrom, final long timeTo) { + Objects.requireNonNull(key, "key cannot be null"); return new MeteredWindowStoreIterator<>( wrapped().fetch(keyBytes(key), timeFrom, timeTo), fetchSensor, @@ -208,6 +213,7 @@ public WindowStoreIterator fetch(final K key, public WindowStoreIterator backwardFetch(final K key, final long timeFrom, final long timeTo) { + Objects.requireNonNull(key, "key cannot be null"); return new MeteredWindowStoreIterator<>( wrapped().backwardFetch(keyBytes(key), timeFrom, timeTo), fetchSensor, @@ -223,6 +229,8 @@ public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, final long timeFrom, final long timeTo) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); return new MeteredWindowedKeyValueIterator<>( wrapped().fetch(keyBytes(keyFrom), keyBytes(keyTo), timeFrom, timeTo), fetchSensor, @@ -236,6 +244,8 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, final K keyTo, final long timeFrom, final long timeTo) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); return new MeteredWindowedKeyValueIterator<>( wrapped().backwardFetch(keyBytes(keyFrom), keyBytes(keyTo), timeFrom, timeTo), fetchSensor, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 1abe2a5d925e2..5596bbf76ae73 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -307,9 +307,6 @@ public void putAll(final List> entries) { @Override public , P> KeyValueIterator prefixScan(final P prefix, final PS prefixKeySerializer) { - Objects.requireNonNull(prefix, "prefix cannot be null"); - Objects.requireNonNull(prefixKeySerializer, "prefixKeySerializer cannot be null"); - validateStoreOpen(); final Bytes prefixBytes = Bytes.wrap(prefixKeySerializer.serialize(null, prefix)); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java index b4e3e1f3c2918..831e6841edbf0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java @@ -40,6 +40,7 @@ import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; public class InMemoryKeyValueStoreTest extends AbstractKeyValueStoreTest { @@ -224,4 +225,9 @@ public void shouldReturnNoKeys() { } assertThat(numberOfKeysReturned, is(0)); } + + @Test + public void shouldThrowNullPointerIfPrefixKeySerializerIsNull() { + assertThrows(NullPointerException.class, () -> byteStore.prefixScan("bb", null)); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java index 88b72df468c3d..095ff87f6f77f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStoreTest.java @@ -435,6 +435,57 @@ public void shouldRemoveMetricsEvenIfWrappedStoreThrowsOnClose() { verify(inner); } + @Test + public void shouldThrowNullPointerOnGetIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> metered.get(null)); + } + + @Test + public void shouldThrowNullPointerOnPutIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> metered.put(null, VALUE)); + } + + @Test + public void shouldThrowNullPointerOnPutIfAbsentIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> metered.putIfAbsent(null, VALUE)); + } + + @Test + public void shouldThrowNullPointerOnDeleteIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> metered.delete(null)); + } + + @Test + public void shouldThrowNullPointerOnPutAllIfAnyKeyIsNull() { + assertThrows(NullPointerException.class, () -> metered.putAll(Collections.singletonList(KeyValue.pair(null, VALUE)))); + } + + @Test + public void shouldThrowNullPointerOnPrefixScanIfPrefixIsNull() { + final StringSerializer stringSerializer = new StringSerializer(); + assertThrows(NullPointerException.class, () -> metered.prefixScan(null, stringSerializer)); + } + + @Test + public void shouldThrowNullPointerOnRangeIfFromIsNull() { + assertThrows(NullPointerException.class, () -> metered.range(null, "to")); + } + + @Test + public void shouldThrowNullPointerOnRangeIfToIsNull() { + assertThrows(NullPointerException.class, () -> metered.range("from", null)); + } + + @Test + public void shouldThrowNullPointerOnReverseRangeIfFromIsNull() { + assertThrows(NullPointerException.class, () -> metered.reverseRange(null, "to")); + } + + @Test + public void shouldThrowNullPointerOnReverseRangeIfToIsNull() { + assertThrows(NullPointerException.class, () -> metered.reverseRange("from", null)); + } + @Test public void shouldGetRecordsWithPrefixKey() { final StringSerializer stringSerializer = new StringSerializer(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java index 7e277488c4686..07ffed3b31056 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSessionStoreTest.java @@ -472,11 +472,36 @@ public void shouldThrowNullPointerOnRemoveIfKeyIsNull() { assertThrows(NullPointerException.class, () -> store.remove(null)); } + @Test + public void shouldThrowNullPointerOnPutIfWrappedKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.put(new Windowed<>(null, new SessionWindow(0, 0)), "a")); + } + + @Test + public void shouldThrowNullPointerOnRemoveIfWrappedKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.remove(new Windowed<>(null, new SessionWindow(0, 0)))); + } + + @Test + public void shouldThrowNullPointerOnPutIfWindowIsNull() { + assertThrows(NullPointerException.class, () -> store.put(new Windowed<>(KEY, null), "a")); + } + + @Test + public void shouldThrowNullPointerOnRemoveIfWindowIsNull() { + assertThrows(NullPointerException.class, () -> store.remove(new Windowed<>(KEY, null))); + } + @Test public void shouldThrowNullPointerOnFetchIfKeyIsNull() { assertThrows(NullPointerException.class, () -> store.fetch(null)); } + @Test + public void shouldThrowNullPointerOnFetchSessionIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.fetchSession(null, 0, Long.MAX_VALUE)); + } + @Test public void shouldThrowNullPointerOnFetchRangeIfFromIsNull() { assertThrows(NullPointerException.class, () -> store.fetch(null, "to")); @@ -487,6 +512,21 @@ public void shouldThrowNullPointerOnFetchRangeIfToIsNull() { assertThrows(NullPointerException.class, () -> store.fetch("from", null)); } + @Test + public void shouldThrowNullPointerOnBackwardFetchIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFetch(null)); + } + + @Test + public void shouldThrowNullPointerOnBackwardFetchIfFromIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFetch(null, "to")); + } + + @Test + public void shouldThrowNullPointerOnBackwardFetchIfToIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFetch("from", null)); + } + @Test public void shouldThrowNullPointerOnFindSessionsIfKeyIsNull() { assertThrows(NullPointerException.class, () -> store.findSessions(null, 0, 0)); @@ -502,6 +542,21 @@ public void shouldThrowNullPointerOnFindSessionsRangeIfToIsNull() { assertThrows(NullPointerException.class, () -> store.findSessions("a", null, 0, 0)); } + @Test + public void shouldThrowNullPointerOnBackwardFindSessionsIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFindSessions(null, 0, 0)); + } + + @Test + public void shouldThrowNullPointerOnBackwardFindSessionsRangeIfFromIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFindSessions(null, "a", 0, 0)); + } + + @Test + public void shouldThrowNullPointerOnBackwardFindSessionsRangeIfToIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFindSessions("a", null, 0, 0)); + } + private interface CachedSessionStore extends SessionStore, CachedStateStore { } @SuppressWarnings("unchecked") diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java index cb3a268cace15..538c8d3173a69 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredWindowStoreTest.java @@ -468,6 +468,42 @@ public void shouldRemoveMetricsEvenIfWrappedStoreThrowsOnClose() { verify(innerStoreMock); } + @Test + public void shouldThrowNullPointerOnPutIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.put(null, "a", 1L)); + } + + @Test + public void shouldThrowNullPointerOnFetchIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.fetch(null, 0L, 1L)); + } + + @Test + public void shouldThrowNullPointerOnBackwardFetchIfKeyIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFetch(null, 0L, 1L)); + } + + @Test + public void shouldThrowNullPointerOnFetchRangeIfFromIsNull() { + assertThrows(NullPointerException.class, () -> store.fetch(null, "to", 0L, 1L)); + } + + @Test + public void shouldThrowNullPointerOnFetchRangeIfToIsNull() { + assertThrows(NullPointerException.class, () -> store.fetch("from", null, 0L, 1L)); + } + + + @Test + public void shouldThrowNullPointerOnbackwardFetchRangeIfFromIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFetch(null, "to", 0L, 1L)); + } + + @Test + public void shouldThrowNullPointerOnbackwardFetchRangeIfToIsNull() { + assertThrows(NullPointerException.class, () -> store.backwardFetch("from", null, 0L, 1L)); + } + private List storeMetrics() { return metrics.metrics() .keySet() From 0a1ef2e9824df1f5d6f32a4be2c1cd0c57bfb398 Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Sat, 1 May 2021 04:50:37 +0900 Subject: [PATCH 104/155] MINOR: Group KafkaController, ReplicaManager metrics in documentation (#7891) Some minor edits to the docs Reviewers: Anna Sophie Blee-Goldman --- docs/ops.html | 92 +++++++++++++++++++++++++-------------------------- 1 file changed, 46 insertions(+), 46 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index 8c157b89d60d7..0f7713b38c83e 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -74,7 +74,7 @@

      auto.leader.rebalance.enable=true You can also set this to false, but you will then need to manually restore leadership to the restored replicas by running the command: @@ -464,7 +464,7 @@
      Safe usage of throttled replication
      kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)
      -

      The lag should constantly decrease during replication. If the metric does not decrease the administrator should +

      The lag should constantly decrease during replication. If the metric does not decrease the administrator should increase the throttle throughput as described above.

      @@ -1317,7 +1317,7 @@

      OS It is unlikely to require much OS-level tuning, but there are three potentially important OS-level configurations: @@ -1518,31 +1518,11 @@

      Common monitoring metrics for producer/consumer/connect/streams

      - The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections. + The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections. @@ -1962,7 +1962,7 @@

      Common monitoring me

      - + @@ -1970,7 +1970,7 @@

      Common monitoring me

      Common Per-broker metrics for producer/consumer/connect/streams

      - The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections. + The following metrics are available on producer/consumer/connector/streams instances. For specific metrics, please see following sections.

      successful-authentication-no-reauth-totalTotal connections that were successfully authenticated by older, pre-2.2.0 SASL clients that do not support re-authentication. May only be non-zero Total connections that were successfully authenticated by older, pre-2.2.0 SASL clients that do not support re-authentication. May only be non-zero. kafka.[producer|consumer|connect]:type=[producer|consumer|connect]-metrics,client-id=([-.\w]+)
      @@ -2509,7 +2509,7 @@
      Date: Fri, 30 Apr 2021 22:46:00 -0700 Subject: [PATCH 105/155] KAFKA-12648: basic skeleton API for NamedTopology (#10615) Just the API for NamedTopology. Reviewers: Guozhang Wang , Walker Carlson --- .../apache/kafka/streams/StreamsBuilder.java | 16 ++- .../org/apache/kafka/streams/Topology.java | 2 +- .../kafka/streams/processor/TaskId.java | 135 +++++++++++++++--- .../internals/InternalTopologyBuilder.java | 5 + .../processor/internals/StateDirectory.java | 4 +- .../processor/internals/TaskManager.java | 2 +- .../internals/assignment/AssignmentInfo.java | 9 +- .../StreamsAssignmentProtocolVersions.java | 6 +- .../KafkaStreamsNamedTopologyWrapper.java | 42 ++++++ .../namedtopology/NamedTopology.java | 48 +++++++ .../NamedTopologyStreamsBuilder.java | 44 ++++++ ...ilabilityStreamsPartitionAssignorTest.java | 9 +- .../assignment/AssignmentTestUtils.java | 5 + .../LegacySubscriptionInfoSerde.java | 33 ++--- .../assignment/StickyTaskAssignorTest.java | 22 ++- .../streams/tests/StreamsUpgradeTest.java | 4 +- 16 files changed, 329 insertions(+), 57 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index ea7dd297434b0..560b8c89c60be 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -67,12 +67,22 @@ public class StreamsBuilder { /** The actual topology that is constructed by this StreamsBuilder. */ - private final Topology topology = new Topology(); + protected final Topology topology; /** The topology's internal builder. */ - final InternalTopologyBuilder internalTopologyBuilder = topology.internalTopologyBuilder; + protected final InternalTopologyBuilder internalTopologyBuilder; - private final InternalStreamsBuilder internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); + protected final InternalStreamsBuilder internalStreamsBuilder; + + public StreamsBuilder() { + topology = getNewTopology(); + internalTopologyBuilder = topology.internalTopologyBuilder; + internalStreamsBuilder = new InternalStreamsBuilder(internalTopologyBuilder); + } + + protected Topology getNewTopology() { + return new Topology(); + } /** * Create a {@link KStream} from the specified topic. diff --git a/streams/src/main/java/org/apache/kafka/streams/Topology.java b/streams/src/main/java/org/apache/kafka/streams/Topology.java index 5986b6b02776c..f7cb8d679cb3a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -54,7 +54,7 @@ */ public class Topology { - final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder(); + protected final InternalTopologyBuilder internalTopologyBuilder = new InternalTopologyBuilder(); /** * Sets the {@code auto.offset.reset} configuration when diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java index 9d1b82cbab3b2..ff6a0c7e260e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java @@ -22,30 +22,63 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Objects; +import java.util.Optional; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.MIN_NAMED_TOPOLOGY_VERSION; /** * The task ID representation composed as topic group ID plus the assigned partition ID. */ public class TaskId implements Comparable { + private static final Logger LOG = LoggerFactory.getLogger(TaskId.class); + /** The ID of the topic group. */ public final int topicGroupId; /** The ID of the partition. */ public final int partition; + /** The namedTopology that this task belongs to, or null if it does not belong to one */ + private final String namedTopology; public TaskId(final int topicGroupId, final int partition) { + this(topicGroupId, partition, null); + } + + public TaskId(final int topicGroupId, final int partition, final String namedTopology) { this.topicGroupId = topicGroupId; this.partition = partition; + if (namedTopology != null && namedTopology.length() == 0) { + LOG.warn("Empty string passed in for task's namedTopology, since NamedTopology name cannot be empty, we " + + "assume this task does not belong to a NamedTopology and downgrade this to null"); + this.namedTopology = null; + } else { + this.namedTopology = namedTopology; + } + } + + public Optional namedTopology() { + return namedTopology == null ? Optional.empty() : Optional.of(namedTopology); } + @Override public String toString() { + return namedTopology != null ? namedTopology + "_" + topicGroupId + "_" + partition : topicGroupId + "_" + partition; + } + + public String toTaskDirString() { return topicGroupId + "_" + partition; } /** - * @throws TaskIdFormatException if the taskIdStr is not a valid {@link TaskId} + * Parse the task directory name (of the form topicGroupId_partition) and construct the TaskId with the + * optional namedTopology (may be null) + * + * @throws TaskIdFormatException if the taskIdStr is not a valid {@link TaskId} */ - public static TaskId parse(final String taskIdStr) { + public static TaskId parseTaskDirectoryName(final String taskIdStr, final String namedTopology) { final int index = taskIdStr.indexOf('_'); if (index <= 0 || index + 1 >= taskIdStr.length()) { throw new TaskIdFormatException(taskIdStr); @@ -55,7 +88,7 @@ public static TaskId parse(final String taskIdStr) { final int topicGroupId = Integer.parseInt(taskIdStr.substring(0, index)); final int partition = Integer.parseInt(taskIdStr.substring(index + 1)); - return new TaskId(topicGroupId, partition); + return new TaskId(topicGroupId, partition, namedTopology); } catch (final Exception e) { throw new TaskIdFormatException(taskIdStr); } @@ -64,25 +97,78 @@ public static TaskId parse(final String taskIdStr) { /** * @throws IOException if cannot write to output stream */ - public void writeTo(final DataOutputStream out) throws IOException { + public void writeTo(final DataOutputStream out, final int version) throws IOException { out.writeInt(topicGroupId); out.writeInt(partition); + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + if (namedTopology != null) { + out.writeInt(namedTopology.length()); + out.writeChars(namedTopology); + } else { + out.writeInt(0); + } + } } /** * @throws IOException if cannot read from input stream */ - public static TaskId readFrom(final DataInputStream in) throws IOException { - return new TaskId(in.readInt(), in.readInt()); + public static TaskId readFrom(final DataInputStream in, final int version) throws IOException { + final int topicGroupId = in.readInt(); + final int partition = in.readInt(); + final String namedTopology; + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + final int numNamedTopologyChars = in.readInt(); + final StringBuilder namedTopologyBuilder = new StringBuilder(); + for (int i = 0; i < numNamedTopologyChars; ++i) { + namedTopologyBuilder.append(in.readChar()); + } + namedTopology = namedTopologyBuilder.toString(); + } else { + namedTopology = null; + } + return new TaskId(topicGroupId, partition, getNamedTopologyOrElseNull(namedTopology)); } - public void writeTo(final ByteBuffer buf) { + public void writeTo(final ByteBuffer buf, final int version) { buf.putInt(topicGroupId); buf.putInt(partition); + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + if (namedTopology != null) { + buf.putInt(namedTopology.length()); + for (final char c : namedTopology.toCharArray()) { + buf.putChar(c); + } + } else { + buf.putInt(0); + } + } } - public static TaskId readFrom(final ByteBuffer buf) { - return new TaskId(buf.getInt(), buf.getInt()); + public static TaskId readFrom(final ByteBuffer buf, final int version) { + final int topicGroupId = buf.getInt(); + final int partition = buf.getInt(); + final String namedTopology; + if (version >= MIN_NAMED_TOPOLOGY_VERSION) { + final int numNamedTopologyChars = buf.getInt(); + final StringBuilder namedTopologyBuilder = new StringBuilder(); + for (int i = 0; i < numNamedTopologyChars; ++i) { + namedTopologyBuilder.append(buf.getChar()); + } + namedTopology = namedTopologyBuilder.toString(); + } else { + namedTopology = null; + } + return new TaskId(topicGroupId, partition, getNamedTopologyOrElseNull(namedTopology)); + } + + /** + * @return the namedTopology name, or null if the passed in namedTopology is null or the empty string + */ + private static String getNamedTopologyOrElseNull(final String namedTopology) { + return (namedTopology == null || namedTopology.length() == 0) ? + null : + namedTopology; } @Override @@ -90,24 +176,39 @@ public boolean equals(final Object o) { if (this == o) { return true; } + if (o == null || getClass() != o.getClass()) { + return false; + } + final TaskId taskId = (TaskId) o; - if (o instanceof TaskId) { - final TaskId other = (TaskId) o; - return other.topicGroupId == this.topicGroupId && other.partition == this.partition; - } else { + if (topicGroupId != taskId.topicGroupId || partition != taskId.partition) { return false; } + + if (namedTopology != null && taskId.namedTopology != null) { + return namedTopology.equals(taskId.namedTopology); + } else { + return namedTopology == null && taskId.namedTopology == null; + } } @Override public int hashCode() { - final long n = ((long) topicGroupId << 32) | (long) partition; - return (int) (n % 0xFFFFFFFFL); + return Objects.hash(topicGroupId, partition, namedTopology); } @Override public int compareTo(final TaskId other) { - final int compare = Integer.compare(this.topicGroupId, other.topicGroupId); - return compare != 0 ? compare : Integer.compare(this.partition, other.partition); + if (namedTopology != null && other.namedTopology != null) { + final int comparingNamedTopologies = namedTopology.compareTo(other.namedTopology); + if (comparingNamedTopologies != 0) { + return comparingNamedTopologies; + } + } else if (namedTopology != null || other.namedTopology != null) { + LOG.error("Tried to compare this = {} with other = {}, but only one had a valid named topology", this, other); + throw new IllegalStateException("Can't compare a TaskId with a namedTopology to one without"); + } + final int comparingTopicGroupId = Integer.compare(this.topicGroupId, other.topicGroupId); + return comparingTopicGroupId != 0 ? comparingTopicGroupId : Integer.compare(this.partition, other.partition); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index bb18f92ef23d2..3e2f99dea20cf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -2013,6 +2013,11 @@ private void updateSubscribedTopics(final Set topics, final String logPr setRegexMatchedTopicToStateStore(); } + + public synchronized List fullSourceTopicNames() { + return maybeDecorateInternalSourceTopics(sourceTopicNames); + } + // following functions are for test only public synchronized Set sourceTopicNames() { return sourceTopicNames; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index 1af2d8ea4af2c..7f01264bbdf38 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -388,7 +388,7 @@ public synchronized void cleanRemovedTasks(final long cleanupDelayMs) { private void cleanRemovedTasksCalledByCleanerThread(final long cleanupDelayMs) { for (final File taskDir : listNonEmptyTaskDirectories()) { final String dirName = taskDir.getName(); - final TaskId id = TaskId.parse(dirName); + final TaskId id = TaskId.parseTaskDirectoryName(dirName, null); if (!lockedTasksToOwner.containsKey(id)) { try { if (lock(id)) { @@ -417,7 +417,7 @@ private void cleanRemovedTasksCalledByUser() throws Exception { final AtomicReference firstException = new AtomicReference<>(); for (final File taskDir : listAllTaskDirectories()) { final String dirName = taskDir.getName(); - final TaskId id = TaskId.parse(dirName); + final TaskId id = TaskId.parseTaskDirectoryName(dirName, null); if (!lockedTasksToOwner.containsKey(id)) { try { if (lock(id)) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index ffa0f563d4c2e..0de51e56efe81 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -683,7 +683,7 @@ private void tryToLockAllNonEmptyTaskDirectories() { for (final File dir : stateDirectory.listNonEmptyTaskDirectories()) { try { - final TaskId id = TaskId.parse(dir.getName()); + final TaskId id = TaskId.parseTaskDirectoryName(dir.getName(), null); if (stateDirectory.lock(id)) { lockedTaskDirectories.add(id); if (!tasks.owned(id)) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index f58f5daabfd4b..4fea12b4544fe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -21,6 +21,7 @@ import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.HostInfo; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -206,14 +207,14 @@ private void encodeActiveAndStandbyTaskAssignment(final DataOutputStream out) th // encode active tasks out.writeInt(activeTasks.size()); for (final TaskId id : activeTasks) { - id.writeTo(out); + id.writeTo(out, usedVersion); } // encode standby tasks out.writeInt(standbyTasks.size()); for (final Map.Entry> entry : standbyTasks.entrySet()) { final TaskId id = entry.getKey(); - id.writeTo(out); + id.writeTo(out, usedVersion); final Set partitions = entry.getValue(); writeTopicPartitions(out, partitions); @@ -382,7 +383,7 @@ private static void decodeActiveTasks(final AssignmentInfo assignmentInfo, final int count = in.readInt(); assignmentInfo.activeTasks = new ArrayList<>(count); for (int i = 0; i < count; i++) { - assignmentInfo.activeTasks.add(TaskId.readFrom(in)); + assignmentInfo.activeTasks.add(TaskId.readFrom(in, assignmentInfo.usedVersion)); } } @@ -391,7 +392,7 @@ private static void decodeStandbyTasks(final AssignmentInfo assignmentInfo, final int count = in.readInt(); assignmentInfo.standbyTasks = new HashMap<>(count); for (int i = 0; i < count; i++) { - final TaskId id = TaskId.readFrom(in); + final TaskId id = TaskId.readFrom(in, assignmentInfo.usedVersion); assignmentInfo.standbyTasks.put(id, readTopicPartitions(in)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java index 01a59c307e8d0..0273999ed4074 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java @@ -19,9 +19,11 @@ public final class StreamsAssignmentProtocolVersions { public static final int UNKNOWN = -1; public static final int EARLIEST_PROBEABLE_VERSION = 3; + public static final int MIN_NAMED_TOPOLOGY_VERSION = 10; public static final int LATEST_SUPPORTED_VERSION = 9; - //When changing the versions update this test: streams_upgrade_test.py::StreamsUpgradeTest.test_version_probing_upgrade - //Add add a unit test in SubscriptionInfoTest + // When changing the versions: + // 1) Update the version_probing_message and end_of_upgrade_message in streams_upgrade_test.py::StreamsUpgradeTest.test_version_probing_upgrade + // 2) Add a unit test in SubscriptionInfoTest and/or AssignmentInfoTest private StreamsAssignmentProtocolVersions() {} } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java new file mode 100644 index 0000000000000..e11a78fbf34e8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/KafkaStreamsNamedTopologyWrapper.java @@ -0,0 +1,42 @@ +/* + * 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.streams.processor.internals.namedtopology; + +import org.apache.kafka.streams.KafkaClientSupplier; +import org.apache.kafka.streams.KafkaStreams; + +import java.util.Properties; + +public class KafkaStreamsNamedTopologyWrapper extends KafkaStreams { + + //TODO It should be possible to start up streams with no NamedTopology (or regular Topology) at all, in the meantime we can just pass in an empty NamedTopology + public KafkaStreamsNamedTopologyWrapper(final NamedTopology topology, final Properties props, final KafkaClientSupplier clientSupplier) { + super(topology, props, clientSupplier); + } + + public NamedTopology getTopologyByName(final String name) { + throw new UnsupportedOperationException(); + } + + public void addNamedTopology(final NamedTopology topology) { + throw new UnsupportedOperationException(); + } + + public void removeNamedTopology(final NamedTopology topology) { + throw new UnsupportedOperationException(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java new file mode 100644 index 0000000000000..18e89f77d11d9 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopology.java @@ -0,0 +1,48 @@ +/* + * 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.streams.processor.internals.namedtopology; + +import org.apache.kafka.streams.Topology; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +public class NamedTopology extends Topology { + + private final Logger log = LoggerFactory.getLogger(NamedTopology.class); + private String name; + + + + void setTopologyName(final String newTopologyName) { + if (name != null) { + log.error("Unable to set topologyName = {} because the name is already set to {}", newTopologyName, name); + throw new IllegalStateException("Tried to set topologyName but the name was already set"); + } + name = newTopologyName; + } + + public String name() { + return name; + } + + + public List sourceTopics() { + return super.internalTopologyBuilder.fullSourceTopicNames(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java new file mode 100644 index 0000000000000..291ece7c49a2a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTopologyStreamsBuilder.java @@ -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. + */ +package org.apache.kafka.streams.processor.internals.namedtopology; + +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.Topology; + +import java.util.Properties; + +public class NamedTopologyStreamsBuilder extends StreamsBuilder { + + final String topologyName; + + public NamedTopologyStreamsBuilder(final String topologyName) { + super(); + this.topologyName = topologyName; + } + + public synchronized NamedTopology buildNamedTopology(final Properties props) { + super.build(props); + final NamedTopology namedTopology = (NamedTopology) super.topology; + namedTopology.setTopologyName(topologyName); + return namedTopology; + } + + @Override + public Topology getNewTopology() { + return new NamedTopology(); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java index 44aef92e5fbba..d020cb37f1367 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/HighAvailabilityStreamsPartitionAssignorTest.java @@ -47,6 +47,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -230,7 +231,9 @@ public void shouldReturnAllActiveTasksToPreviousOwnerRegardlessOfBalanceAndTrigg final List newConsumerActiveTasks = newConsumerUserData.activeTasks(); // The tasks were returned to their prior owner - assertThat(firstConsumerActiveTasks, equalTo(new ArrayList<>(allTasks))); + final ArrayList sortedExpectedTasks = new ArrayList<>(allTasks); + Collections.sort(sortedExpectedTasks); + assertThat(firstConsumerActiveTasks, equalTo(sortedExpectedTasks)); assertThat(newConsumerActiveTasks, empty()); // There is a rebalance scheduled @@ -281,7 +284,9 @@ public void shouldScheduleProbingRebalanceOnThisClientIfWarmupTasksRequired() { final List newConsumerActiveTasks = AssignmentInfo.decode(assignments.get(newConsumer).userData()).activeTasks(); - assertThat(firstConsumerActiveTasks, equalTo(new ArrayList<>(allTasks))); + final ArrayList sortedExpectedTasks = new ArrayList<>(allTasks); + Collections.sort(sortedExpectedTasks); + assertThat(firstConsumerActiveTasks, equalTo(sortedExpectedTasks)); assertThat(newConsumerActiveTasks, empty()); assertThat(referenceContainer.assignmentErrorCode.get(), equalTo(AssignorError.NONE.code())); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java index 66d7e6fbe0229..58567529346b7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java @@ -85,6 +85,11 @@ public final class AssignmentTestUtils { public static final TaskId TASK_2_2 = new TaskId(2, 2); public static final TaskId TASK_2_3 = new TaskId(2, 3); + public static final TaskId NAMED_TASK_0_0 = new TaskId(0, 0, "topology0"); + public static final TaskId NAMED_TASK_0_1 = new TaskId(0, 1, "topology0"); + public static final TaskId NAMED_TASK_1_0 = new TaskId(1, 0, "topology1"); + public static final TaskId NAMED_TASK_1_1 = new TaskId(1, 1, "topology1"); + public static final Set EMPTY_TASKS = emptySet(); public static final Map EMPTY_CHANGELOG_END_OFFSETS = new HashMap<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java index e8976ee53ea48..cdd651f239b47 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java @@ -111,8 +111,8 @@ public ByteBuffer encode() { buf.putInt(usedVersion); // used version buf.putInt(LATEST_SUPPORTED_VERSION); // supported version encodeClientUUID(buf, processId()); - encodeTasks(buf, prevTasks); - encodeTasks(buf, standbyTasks); + encodeTasks(buf, prevTasks, usedVersion); + encodeTasks(buf, standbyTasks, usedVersion); encodeUserEndPoint(buf, endPointBytes); buf.rewind(); @@ -131,8 +131,8 @@ public ByteBuffer encode() { buf.putInt(2); // version encodeClientUUID(buf, processId()); - encodeTasks(buf, prevTasks); - encodeTasks(buf, standbyTasks); + encodeTasks(buf, prevTasks, usedVersion); + encodeTasks(buf, standbyTasks, usedVersion); encodeUserEndPoint(buf, endPointBytes); buf.rewind(); @@ -148,8 +148,8 @@ public ByteBuffer encode() { buf1.putInt(1); // version encodeClientUUID(buf1, processId()); - encodeTasks(buf1, prevTasks); - encodeTasks(buf1, standbyTasks); + encodeTasks(buf1, prevTasks, usedVersion); + encodeTasks(buf1, standbyTasks, usedVersion); buf1.rewind(); return buf1; } else { @@ -164,10 +164,11 @@ public static void encodeClientUUID(final ByteBuffer buf, final UUID processId) } public static void encodeTasks(final ByteBuffer buf, - final Collection taskIds) { + final Collection taskIds, + final int version) { buf.putInt(taskIds.size()); for (final TaskId id : taskIds) { - id.writeTo(buf); + id.writeTo(buf, version); } } @@ -199,20 +200,20 @@ public static LegacySubscriptionInfoSerde decode(final ByteBuffer data) { if (usedVersion > 2 && usedVersion < 7) { final int latestSupportedVersion = data.getInt(); final UUID processId = decodeProcessId(data); - final Set prevTasks = decodeTasks(data); - final Set standbyTasks = decodeTasks(data); + final Set prevTasks = decodeTasks(data, usedVersion); + final Set standbyTasks = decodeTasks(data, usedVersion); final String userEndPoint = decodeUserEndpoint(data); return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, processId, prevTasks, standbyTasks, userEndPoint); } else if (usedVersion == 2) { final UUID processId = decodeProcessId(data); - final Set prevTasks = decodeTasks(data); - final Set standbyTasks = decodeTasks(data); + final Set prevTasks = decodeTasks(data, usedVersion); + final Set standbyTasks = decodeTasks(data, usedVersion); final String userEndPoint = decodeUserEndpoint(data); return new LegacySubscriptionInfoSerde(2, UNKNOWN, processId, prevTasks, standbyTasks, userEndPoint); } else if (usedVersion == 1) { final UUID processId = decodeProcessId(data); - final Set prevTasks = decodeTasks(data); - final Set standbyTasks = decodeTasks(data); + final Set prevTasks = decodeTasks(data, usedVersion); + final Set standbyTasks = decodeTasks(data, usedVersion); return new LegacySubscriptionInfoSerde(1, UNKNOWN, processId, prevTasks, standbyTasks, null); } else { final int latestSupportedVersion = data.getInt(); @@ -228,11 +229,11 @@ private static String decodeUserEndpoint(final ByteBuffer data) { return new String(userEndpointBytes, StandardCharsets.UTF_8); } - private static Set decodeTasks(final ByteBuffer data) { + private static Set decodeTasks(final ByteBuffer data, final int version) { final Set prevTasks = new HashSet<>(); final int numPrevTasks = data.getInt(); for (int i = 0; i < numPrevTasks; i++) { - prevTasks.add(TaskId.readFrom(data)); + prevTasks.add(TaskId.readFrom(data, version)); } return prevTasks; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java index 4ade333aaacf5..47ca31922301e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/StickyTaskAssignorTest.java @@ -166,12 +166,18 @@ public void shouldAssignTasksEvenlyWithUnequalTopicGroupSizes() { assertThat(assign(TASK_1_0, TASK_0_0, TASK_0_1, TASK_0_2, TASK_0_3, TASK_0_4, TASK_0_5), is(false)); - final Set expectedClientITasks = new HashSet<>(asList(TASK_0_0, TASK_0_1, TASK_1_0, TASK_0_5)); - final Set expectedClientIITasks = new HashSet<>(asList(TASK_0_2, TASK_0_3, TASK_0_4)); + final Set allTasks = new HashSet<>(asList(TASK_0_0, TASK_0_1, TASK_1_0, TASK_0_5, TASK_0_2, TASK_0_3, TASK_0_4)); + final Set client1Tasks = clients.get(UUID_1).activeTasks(); + final Set client2Tasks = clients.get(UUID_2).activeTasks(); - - assertThat(clients.get(UUID_1).activeTasks(), equalTo(expectedClientITasks)); - assertThat(clients.get(UUID_2).activeTasks(), equalTo(expectedClientIITasks)); + // one client should get 3 tasks and the other should have 4 + assertThat( + (client1Tasks.size() == 3 && client2Tasks.size() == 4) || + (client1Tasks.size() == 4 && client2Tasks.size() == 3), + is(true)); + allTasks.removeAll(client1Tasks); + // client2 should have all the remaining tasks not assigned to client 1 + assertThat(client2Tasks, equalTo(allTasks)); } @Test @@ -646,14 +652,16 @@ public void shouldAssignTasksToNewClientWithoutFlippingAssignmentBetweenExisting final boolean probingRebalanceNeeded = assign(TASK_0_0, TASK_0_1, TASK_0_2, TASK_0_3, TASK_0_4, TASK_0_5, TASK_0_6); assertThat(probingRebalanceNeeded, is(false)); + + // it's possible for either client 1 or 2 to get three tasks since they both had three previously assigned assertThat(c1.activeTasks(), not(hasItem(TASK_0_3))); assertThat(c1.activeTasks(), not(hasItem(TASK_0_4))); assertThat(c1.activeTasks(), not(hasItem(TASK_0_5))); - assertThat(c1.activeTaskCount(), equalTo(3)); + assertThat(c1.activeTaskCount(), greaterThanOrEqualTo(2)); assertThat(c2.activeTasks(), not(hasItems(TASK_0_0))); assertThat(c2.activeTasks(), not(hasItems(TASK_0_1))); assertThat(c2.activeTasks(), not(hasItems(TASK_0_2))); - assertThat(c2.activeTaskCount(), equalTo(2)); + assertThat(c2.activeTaskCount(), greaterThanOrEqualTo(2)); assertThat(newClient.activeTaskCount(), equalTo(2)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java index dcc45e115fa15..82b6ccce7591f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -330,8 +330,8 @@ private ByteBuffer encode() { buf.putInt(version); // used version buf.putInt(version); // supported version LegacySubscriptionInfoSerde.encodeClientUUID(buf, processId); - LegacySubscriptionInfoSerde.encodeTasks(buf, activeTasks); - LegacySubscriptionInfoSerde.encodeTasks(buf, standbyTasks); + LegacySubscriptionInfoSerde.encodeTasks(buf, activeTasks, version); + LegacySubscriptionInfoSerde.encodeTasks(buf, standbyTasks, version); LegacySubscriptionInfoSerde.encodeUserEndPoint(buf, endPointBytes); buf.rewind(); From 1f4207c7c187b201fa3fc47f5752b7ee3df681bd Mon Sep 17 00:00:00 2001 From: Ron Dagostino Date: Sat, 1 May 2021 11:22:46 -0400 Subject: [PATCH 106/155] MINOR: system test spelling/pydoc/dead code fixes (#10604) Reviewers: Kamal Chandraprakash , Chia-Ping Tsai --- tests/kafkatest/services/kafka/kafka.py | 46 ++++++++++--------- .../tests/connect/connect_distributed_test.py | 2 +- .../tests/core/delegation_token_test.py | 2 +- .../tests/core/fetch_from_follower_test.py | 2 +- .../tests/core/log_dir_failure_test.py | 2 +- .../tests/core/reassign_partitions_test.py | 2 +- .../streams_broker_compatibility_test.py | 2 +- .../tests/tools/log_compaction_test.py | 2 +- 8 files changed, 31 insertions(+), 29 deletions(-) diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index fcf390056c270..37a6d67bb5b2d 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -187,7 +187,7 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI interbroker_security_protocol=SecurityConfig.PLAINTEXT, client_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, authorizer_class_name=None, topics=None, version=DEV_BRANCH, jmx_object_names=None, - jmx_attributes=None, zk_connect_timeout=18000, zk_session_timeout=18000, server_prop_overides=None, zk_chroot=None, + jmx_attributes=None, zk_connect_timeout=18000, zk_session_timeout=18000, server_prop_overrides=None, zk_chroot=None, zk_client_secure=False, listener_security_config=ListenerSecurityConfig(), per_node_server_prop_overrides=None, extra_kafka_opts="", tls_version=None, @@ -199,29 +199,33 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI :param int num_nodes: the number of nodes in the service. There are 4 possibilities: 1) Zookeeper quorum: The number of brokers is defined by this parameter. + The broker.id values will be 1..num_nodes. 2) Co-located Raft quorum: The number of nodes having a broker role is defined by this parameter. + The node.id values will be 1..num_nodes The number of nodes having a controller role will by default be 1, 3, or 5 depending on num_nodes (1 if num_nodes < 3, otherwise 3 if num_nodes < 5, otherwise 5). This calculation can be overridden via controller_num_nodes_override, which must be between 1 and num_nodes, inclusive, when non-zero. Here are some possibilities: num_nodes = 1: - node 0: broker.roles=broker+controller + broker having node.id=1: broker.roles=broker+controller num_nodes = 2: - node 0: broker.roles=broker+controller - node 1: broker.roles=broker + broker having node.id=1: broker.roles=broker+controller + broker having node.id=2: broker.roles=broker num_nodes = 3: - node 0: broker.roles=broker+controller - node 1: broker.roles=broker+controller - node 2: broker.roles=broker+controller + broker having node.id=1: broker.roles=broker+controller + broker having node.id=2: broker.roles=broker+controller + broker having node.id=3: broker.roles=broker+controller num_nodes = 3, controller_num_nodes_override = 1 - node 0: broker.roles=broker+controller - node 1: broker.roles=broker - node 2: broker.roles=broker + broker having node.id=1: broker.roles=broker+controller + broker having node.id=2: broker.roles=broker + broker having node.id=3: broker.roles=broker 3) Remote Raft quorum when instantiating the broker service: The number of nodes, all of which will have broker.roles=broker, is defined by this parameter. + The node.id values will be 1..num_nodes 4) Remote Raft quorum when instantiating the controller service: The number of nodes, all of which will have broker.roles=controller, is defined by this parameter. + The node.id values will be 3001..(3000 + num_nodes) The value passed in is determined by the broker service when that is instantiated, and it uses the same algorithm as described above: 1, 3, or 5 unless controller_num_nodes_override is provided. :param ZookeeperService zk: @@ -237,13 +241,14 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI :param jmx_attributes: :param int zk_connect_timeout: :param int zk_session_timeout: - :param dict server_prop_overides: overrides for kafka.properties file + :param list[list] server_prop_overrides: overrides for kafka.properties file + e.g: [["config1", "true"], ["config2", "1000"]] :param str zk_chroot: :param bool zk_client_secure: connect to Zookeeper over secure client port (TLS) when True :param ListenerSecurityConfig listener_security_config: listener config to use - :param dict per_node_server_prop_overrides: overrides for kafka.properties file keyed by 0-based node number + :param dict per_node_server_prop_overrides: overrides for kafka.properties file keyed by 1-based node number + e.g: {1: [["config1", "true"], ["config2", "1000"]], 2: [["config1", "false"], ["config2", "0"]]} :param str extra_kafka_opts: jvm args to add to KAFKA_OPTS variable - :param str tls_version: TLS version to use :param KafkaService remote_kafka: process.roles=controller for this cluster when not None; ignored when using ZooKeeper :param int controller_num_nodes_override: the number of nodes to use in the cluster, instead of 5, 3, or 1 based on num_nodes, if positive, not using ZooKeeper, and remote_kafka is not None; ignored otherwise @@ -314,10 +319,10 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.minikdc = None self.authorizer_class_name = authorizer_class_name self.zk_set_acl = False - if server_prop_overides is None: - self.server_prop_overides = [] + if server_prop_overrides is None: + self.server_prop_overrides = [] else: - self.server_prop_overides = server_prop_overides + self.server_prop_overrides = server_prop_overrides if per_node_server_prop_overrides is None: self.per_node_server_prop_overrides = {} else: @@ -560,12 +565,9 @@ def start(self, add_principals=""): if self.quorum_info.using_zk: self.logger.info("Waiting for brokers to register at ZK") - retries = 30 expected_broker_ids = set(self.nodes) - wait_until(lambda: {node for node in self.nodes if self.is_registered(node)} == expected_broker_ids, 30, 1) - - if retries == 0: - raise RuntimeError("Kafka servers didn't register at ZK within 30 seconds") + wait_until(lambda: {node for node in self.nodes if self.is_registered(node)} == expected_broker_ids, + timeout_sec=30, backoff_sec=1, err_msg="Kafka servers didn't register at ZK within 30 seconds") # Create topics if necessary if self.topics is not None: @@ -634,7 +636,7 @@ def prop_file(self, node): else: override_configs[config_property.ZOOKEEPER_SSL_CLIENT_ENABLE] = 'false' - for prop in self.server_prop_overides: + for prop in self.server_prop_overrides: override_configs[prop[0]] = prop[1] for prop in self.per_node_server_prop_overrides.get(self.idx(node), []): diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index c661dfb285b9d..5e0006ec16a69 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -84,7 +84,7 @@ def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT, timestamp_t self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk, security_protocol=security_protocol, interbroker_security_protocol=security_protocol, topics=self.topics, version=broker_version, - server_prop_overides=[["auto.create.topics.enable", str(auto_create_topics)]]) + server_prop_overrides=[["auto.create.topics.enable", str(auto_create_topics)]]) if timestamp_type is not None: for node in self.kafka.nodes: node.config[config_property.MESSAGE_TIMESTAMP_TYPE] = timestamp_type diff --git a/tests/kafkatest/tests/core/delegation_token_test.py b/tests/kafkatest/tests/core/delegation_token_test.py index 5fe8d126210ba..7b508bc378645 100644 --- a/tests/kafkatest/tests/core/delegation_token_test.py +++ b/tests/kafkatest/tests/core/delegation_token_test.py @@ -37,7 +37,7 @@ def __init__(self, test_context): self.zk = ZookeeperService(test_context, num_nodes=1) self.kafka = KafkaService(self.test_context, num_nodes=1, zk=self.zk, zk_chroot="/kafka", topics={self.topic: {"partitions": 1, "replication-factor": 1}}, - server_prop_overides=[ + server_prop_overrides=[ [config_property.DELEGATION_TOKEN_MAX_LIFETIME_MS, "604800000"], [config_property.DELEGATION_TOKEN_EXPIRY_TIME_MS, "86400000"], [config_property.DELEGATION_TOKEN_SECRET_KEY, "test12345"], diff --git a/tests/kafkatest/tests/core/fetch_from_follower_test.py b/tests/kafkatest/tests/core/fetch_from_follower_test.py index fab5cfa6269ae..f720de11464a3 100644 --- a/tests/kafkatest/tests/core/fetch_from_follower_test.py +++ b/tests/kafkatest/tests/core/fetch_from_follower_test.py @@ -47,7 +47,7 @@ def __init__(self, test_context): "replication-factor": 3, "configs": {"min.insync.replicas": 1}}, }, - server_prop_overides=[ + server_prop_overrides=[ ["replica.selector.class", self.RACK_AWARE_REPLICA_SELECTOR] ], per_node_server_prop_overrides={ diff --git a/tests/kafkatest/tests/core/log_dir_failure_test.py b/tests/kafkatest/tests/core/log_dir_failure_test.py index 108d003f4c019..31a20fde1cced 100644 --- a/tests/kafkatest/tests/core/log_dir_failure_test.py +++ b/tests/kafkatest/tests/core/log_dir_failure_test.py @@ -72,7 +72,7 @@ def __init__(self, test_context): }, # Set log.roll.ms to 3 seconds so that broker will detect disk error sooner when it creates log segment # Otherwise broker will still be able to read/write the log file even if the log directory is inaccessible. - server_prop_overides=[ + server_prop_overrides=[ [config_property.OFFSETS_TOPIC_NUM_PARTITIONS, "1"], [config_property.LOG_FLUSH_INTERVAL_MESSAGE, "5"], [config_property.REPLICA_HIGHWATERMARK_CHECKPOINT_INTERVAL_MS, "60000"], diff --git a/tests/kafkatest/tests/core/reassign_partitions_test.py b/tests/kafkatest/tests/core/reassign_partitions_test.py index f36f77473557c..f783479e18224 100644 --- a/tests/kafkatest/tests/core/reassign_partitions_test.py +++ b/tests/kafkatest/tests/core/reassign_partitions_test.py @@ -47,7 +47,7 @@ def __init__(self, test_context): # reassignment for upto one replica per partition, which is not # desirable for this test in particular. self.kafka = KafkaService(test_context, num_nodes=4, zk=self.zk, - server_prop_overides=[ + server_prop_overrides=[ [config_property.LOG_ROLL_TIME_MS, "5000"], [config_property.LOG_RETENTION_CHECK_INTERVAL_MS, "5000"] ], diff --git a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py index 69fc3500ef380..3bca7062d8b71 100644 --- a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py @@ -48,7 +48,7 @@ def __init__(self, test_context): self.input: {'partitions': 1, 'replication-factor': 1}, self.output: {'partitions': 1, 'replication-factor': 1} }, - server_prop_overides=[ + server_prop_overrides=[ ["transaction.state.log.replication.factor", "1"], ["transaction.state.log.min.isr", "1"] ]) diff --git a/tests/kafkatest/tests/tools/log_compaction_test.py b/tests/kafkatest/tests/tools/log_compaction_test.py index a91a976550a9f..96b6d0bedb0ad 100644 --- a/tests/kafkatest/tests/tools/log_compaction_test.py +++ b/tests/kafkatest/tests/tools/log_compaction_test.py @@ -49,7 +49,7 @@ def start_kafka(self, security_protocol, interbroker_security_protocol): zk = self.zk, security_protocol=security_protocol, interbroker_security_protocol=interbroker_security_protocol, - server_prop_overides=[ + server_prop_overrides=[ [config_property.LOG_SEGMENT_BYTES, LogCompactionTest.LOG_SEGMENT_BYTES], ], controller_num_nodes_override=self.num_zk) From 3ec6317ee67de0774a6c8c2b48faf0b38b8643d9 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 1 May 2021 08:31:41 -0700 Subject: [PATCH 107/155] KAFKA-12683: Remove deprecated UsePreviousTimeOnInvalidTimestamp (#10557) Reviewers: Anna Sophie Blee-Goldman --- docs/streams/upgrade-guide.html | 7 +- .../UsePreviousTimeOnInvalidTimestamp.java | 71 ------------------- ...UsePreviousTimeOnInvalidTimestampTest.java | 56 --------------- 3 files changed, 4 insertions(+), 130 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index f7221ed28821b..02f258c1d176e 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -113,6 +113,8 @@

      Streams API We removed the following deprecated APIs:

      The following dependencies were removed from Kafka Streams: diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java b/streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java deleted file mode 100644 index 507a7bfcd2205..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestamp.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.streams.processor; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streams.errors.StreamsException; - -/** - * Retrieves embedded metadata timestamps from Kafka messages. - * If a record has a negative (invalid) timestamp, a new timestamp will be inferred from the current stream-time. - *

      - * Embedded metadata timestamp was introduced in "KIP-32: Add timestamps to Kafka message" for the new - * 0.10+ Kafka message format. - *

      - * Here, "embedded metadata" refers to the fact that compatible Kafka producer clients automatically and - * transparently embed such timestamps into message metadata they send to Kafka, which can then be retrieved - * via this timestamp extractor. - *

      - * If the embedded metadata timestamp represents CreateTime (cf. Kafka broker setting - * {@code message.timestamp.type} and Kafka topic setting {@code log.message.timestamp.type}), - * this extractor effectively provides event-time semantics. - * If LogAppendTime is used as broker/topic setting to define the embedded metadata timestamps, - * using this extractor effectively provides ingestion-time semantics. - *

      - * If you need processing-time semantics, use {@link WallclockTimestampExtractor}. - * @deprecated since 2.5. Use {@link UsePartitionTimeOnInvalidTimestamp} instead - * @see FailOnInvalidTimestamp - * @see LogAndSkipOnInvalidTimestamp - * @see WallclockTimestampExtractor - */ -@SuppressWarnings("deprecation") -@Deprecated -public class UsePreviousTimeOnInvalidTimestamp extends ExtractRecordMetadataTimestamp { - - /** - * Returns the current stream-time as new timestamp for the record. - * - * @param record a data record - * @param recordTimestamp the timestamp extractor from the record - * @param partitionTime the highest extracted valid timestamp of the current record's partition˙ (could be -1 if unknown) - * @return the provided highest extracted valid timestamp as new timestamp for the record - * @throws StreamsException if highest extracted valid timestamp is unknown - */ - @Override - public long onInvalidTimestamp(final ConsumerRecord record, - final long recordTimestamp, - final long partitionTime) - throws StreamsException { - if (partitionTime < 0) { - throw new StreamsException("Could not infer new timestamp for input record " + record - + " because partition time is unknown."); - } - return partitionTime; - } - - -} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java deleted file mode 100644 index 280a93950bdab..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/processor/UsePreviousTimeOnInvalidTimestampTest.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.streams.processor; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streams.errors.StreamsException; -import org.junit.Test; - -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.fail; - -@SuppressWarnings("deprecation") -public class UsePreviousTimeOnInvalidTimestampTest extends TimestampExtractorTest { - @Test - public void extractMetadataTimestamp() { - testExtractMetadataTimestamp(new UsePreviousTimeOnInvalidTimestamp()); - } - - @Test - public void usePreviousTimeOnInvalidTimestamp() { - final long previousTime = 42; - - final TimestampExtractor extractor = new UsePreviousTimeOnInvalidTimestamp(); - final long timestamp = extractor.extract( - new ConsumerRecord<>("anyTopic", 0, 0, null, null), - previousTime - ); - - assertThat(timestamp, is(previousTime)); - } - - @Test - public void shouldThrowStreamsException() { - final TimestampExtractor extractor = new UsePreviousTimeOnInvalidTimestamp(); - final ConsumerRecord record = new ConsumerRecord<>("anyTopic", 0, 0, null, null); - try { - extractor.extract(record, -1); - fail("should have thrown StreamsException"); - } catch (final StreamsException expected) { } - } -} From bee3cf7d98645af5460f3483acf81cf1c60de08e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sat, 1 May 2021 08:32:14 -0700 Subject: [PATCH 108/155] MINOR: Remove unused Utils.delete (#10622) Anna Sophie Blee-Goldman --- .../org/apache/kafka/common/utils/Utils.java | 26 ++----------------- .../internals/StateDirectoryTest.java | 4 +-- 2 files changed, 4 insertions(+), 26 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 2a6affe2eb5a4..b56ad155b1f21 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -810,18 +810,6 @@ public static Properties mkObjectProperties(final Map properties * @param rootFile The root file at which to begin deleting */ public static void delete(final File rootFile) throws IOException { - delete(rootFile, Collections.emptyList()); - } - - /** - * Recursively delete the subfiles (if any exist) of the passed in root file that are not included - * in the list to keep - * - * @param rootFile The root file at which to begin deleting - * @param filesToKeep The subfiles to keep (note that if a subfile is to be kept, so are all its parent - * files in its pat)h; if empty we would also delete the root file - */ - public static void delete(final File rootFile, final List filesToKeep) throws IOException { if (rootFile == null) return; Files.walkFileTree(rootFile.toPath(), new SimpleFileVisitor() { @@ -835,9 +823,7 @@ public FileVisitResult visitFileFailed(Path path, IOException exc) throws IOExce @Override public FileVisitResult visitFile(Path path, BasicFileAttributes attrs) throws IOException { - if (!filesToKeep.contains(path.toFile())) { - Files.delete(path); - } + Files.delete(path); return FileVisitResult.CONTINUE; } @@ -848,15 +834,7 @@ public FileVisitResult postVisitDirectory(Path path, IOException exc) throws IOE throw exc; } - if (rootFile.toPath().equals(path)) { - // only delete the parent directory if there's nothing to keep - if (filesToKeep.isEmpty()) { - Files.delete(path); - } - } else if (!filesToKeep.contains(path.toFile())) { - Files.delete(path); - } - + Files.delete(path); return FileVisitResult.CONTINUE; } }); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index 503d6c4ef5e05..b17169c64f143 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -389,9 +389,9 @@ public void shouldOnlyListNonEmptyTaskDirectories() throws IOException { assertEquals(mkSet(taskDir1), Arrays.stream( directory.listNonEmptyTaskDirectories()).collect(Collectors.toSet())); - Utils.delete(taskDir1, Collections.singletonList(new File(taskDir1, LOCK_FILE_NAME))); + Utils.delete(taskDir1); - assertEquals(mkSet(taskDir1, taskDir2), Arrays.stream( + assertEquals(mkSet(taskDir2), Arrays.stream( directory.listAllTaskDirectories()).collect(Collectors.toSet())); assertEquals(Collections.emptySet(), Arrays.stream( directory.listNonEmptyTaskDirectories()).collect(Collectors.toSet())); From 6203bf8b94c7c340671c1729f4a8e4fcc302605e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jos=C3=A9=20Armando=20Garc=C3=ADa=20Sancio?= Date: Sat, 1 May 2021 10:05:45 -0700 Subject: [PATCH 109/155] KAFKA-12154; Raft Snapshot Loading API (#10085) Implement Raft Snapshot loading API. 1. Adds a new method `handleSnapshot` to `raft.Listener` which is called whenever the `RaftClient` determines that the `Listener` needs to load a new snapshot before reading the log. This happens when the `Listener`'s next offset is less than the log start offset also known as the earliest snapshot. 2. Adds a new type `SnapshotReader` which provides a `Iterator>` interface and de-serializes records in the `RawSnapshotReader` into `T`s 3. Adds a new type `RecordsIterator` that implements an `Iterator>` by scanning a `Records` object and deserializes the batches and records into `Batch`. This type is used by both `SnapshotReader` and `RecordsBatchReader` internally to implement the `Iterator` interface that they expose. 4. Changes the `MockLog` implementation to read one or two batches at a time. The previous implementation always read from the given offset to the high-watermark. This made it impossible to test interesting snapshot loading scenarios. 5. Removed `throws IOException` from some methods. Some of types were inconsistently throwing `IOException` in some cases and throwing `RuntimeException(..., new IOException(...))` in others. This PR improves the consistent by wrapping `IOException` in `RuntimeException` in a few more places and replacing `Closeable` with `AutoCloseable`. 6. Updated the Kafka Raft simulation test to take into account snapshot. `ReplicatedCounter` was updated to generate snapshot after 10 records get committed. This means that the `ConsistentCommittedData` validation was extended to take snapshots into account. Also added a new invariant to ensure that the log start offset is consistently set with the earliest snapshot. Reviewers: dengziming , David Arthur , Jason Gustafson --- checkstyle/import-control.xml | 1 + .../scala/kafka/tools/TestRaftServer.scala | 15 +- .../java/org/apache/kafka/raft/Batch.java | 133 ++++++++++ .../org/apache/kafka/raft/BatchReader.java | 57 +---- .../org/apache/kafka/raft/FollowerState.java | 2 + .../apache/kafka/raft/KafkaRaftClient.java | 138 ++++++---- .../org/apache/kafka/raft/LeaderState.java | 26 +- .../org/apache/kafka/raft/RaftClient.java | 28 +- .../apache/kafka/raft/ReplicatedCounter.java | 99 ++++++-- .../org/apache/kafka/raft/ReplicatedLog.java | 11 +- .../raft/internals/MemoryBatchReader.java | 1 + .../raft/internals/RecordsBatchReader.java | 189 ++++---------- .../kafka/raft/internals/RecordsIterator.java | 239 ++++++++++++++++++ .../kafka/raft/metadata/MetaLogRaftShim.java | 9 +- .../kafka/snapshot/FileRawSnapshotReader.java | 13 +- .../kafka/snapshot/FileRawSnapshotWriter.java | 109 +++++--- .../kafka/snapshot/RawSnapshotReader.java | 27 +- .../kafka/snapshot/RawSnapshotWriter.java | 23 +- .../apache/kafka/snapshot/SnapshotReader.java | 81 ++++++ .../apache/kafka/snapshot/SnapshotWriter.java | 19 +- .../raft/KafkaRaftClientSnapshotTest.java | 187 +++++++++++++- .../kafka/raft/KafkaRaftClientTest.java | 67 +++-- .../java/org/apache/kafka/raft/MockLog.java | 110 ++++---- .../org/apache/kafka/raft/MockLogTest.java | 141 ++++++----- .../kafka/raft/RaftClientTestContext.java | 102 ++++++-- .../kafka/raft/RaftEventSimulationTest.java | 104 +++++++- .../raft/internals/MemoryBatchReaderTest.java | 20 +- .../internals/RecordsBatchReaderTest.java | 57 +---- .../raft/internals/RecordsIteratorTest.java | 188 ++++++++++++++ .../kafka/snapshot/FileRawSnapshotTest.java | 62 +++-- ...est.java => SnapshotWriterReaderTest.java} | 44 +++- .../kafka/shell/MetadataNodeManager.java | 18 +- 32 files changed, 1676 insertions(+), 644 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/Batch.java create mode 100644 raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java create mode 100644 raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java create mode 100644 raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java rename raft/src/test/java/org/apache/kafka/snapshot/{SnapshotWriterTest.java => SnapshotWriterReaderTest.java} (77%) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index e7e2e4dca414f..8506bba5c509f 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -286,6 +286,7 @@ + diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index c0c1c65df349c..3d432159ce4c9 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -35,8 +35,8 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{TopicPartition, Uuid, protocol} -import org.apache.kafka.raft.BatchReader.Batch -import org.apache.kafka.raft.{BatchReader, RaftClient, RaftConfig, RecordSerde} +import org.apache.kafka.raft.{Batch, BatchReader, RaftClient, RaftConfig, RecordSerde} +import org.apache.kafka.snapshot.SnapshotReader import scala.jdk.CollectionConverters._ @@ -150,6 +150,7 @@ class TestRaftServer( case class HandleClaim(epoch: Int) extends RaftEvent case object HandleResign extends RaftEvent case class HandleCommit(reader: BatchReader[Array[Byte]]) extends RaftEvent + case class HandleSnapshot(reader: SnapshotReader[Array[Byte]]) extends RaftEvent case object Shutdown extends RaftEvent private val eventQueue = new LinkedBlockingDeque[RaftEvent]() @@ -175,6 +176,10 @@ class TestRaftServer( eventQueue.offer(HandleCommit(reader)) } + override def handleSnapshot(reader: SnapshotReader[Array[Byte]]): Unit = { + eventQueue.offer(HandleSnapshot(reader)) + } + override def initiateShutdown(): Boolean = { val initiated = super.initiateShutdown() eventQueue.offer(Shutdown) @@ -226,7 +231,11 @@ class TestRaftServer( reader.close() } - case _ => + case HandleSnapshot(reader) => + // Ignore snapshots; only interested in records appended by this leader + reader.close() + + case Shutdown => // Ignore shutdown command } } diff --git a/raft/src/main/java/org/apache/kafka/raft/Batch.java b/raft/src/main/java/org/apache/kafka/raft/Batch.java new file mode 100644 index 0000000000000..daa1e05c07a12 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/Batch.java @@ -0,0 +1,133 @@ +/* + * 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.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; + +/** + * A batch of records. + * + * This type contains a list of records `T` along with the information associated with those records. + */ +public final class Batch implements Iterable { + private final long baseOffset; + private final int epoch; + private final long lastOffset; + private final List records; + + private Batch(long baseOffset, int epoch, long lastOffset, List records) { + this.baseOffset = baseOffset; + this.epoch = epoch; + this.lastOffset = lastOffset; + this.records = records; + } + + /** + * The offset of the last record in the batch. + */ + public long lastOffset() { + return lastOffset; + } + + /** + * The offset of the first record in the batch. + */ + public long baseOffset() { + return baseOffset; + } + + /** + * The list of records in the batch. + */ + public List records() { + return records; + } + + /** + * The epoch of the leader that appended the record batch. + */ + public int epoch() { + return epoch; + } + + @Override + public Iterator iterator() { + return records.iterator(); + } + + @Override + public String toString() { + return "Batch(" + + "baseOffset=" + baseOffset + + ", epoch=" + epoch + + ", lastOffset=" + lastOffset + + ", records=" + records + + ')'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Batch batch = (Batch) o; + return baseOffset == batch.baseOffset && + epoch == batch.epoch && + Objects.equals(records, batch.records); + } + + @Override + public int hashCode() { + return Objects.hash(baseOffset, epoch, records); + } + + /** + * Create a batch without any records. + * + * Internally this is used to propagate offset information for control batches which do not decode to the type T. + * + * @param baseOffset offset of the batch + * @param epoch epoch of the leader that created this batch + * @param lastOffset offset of the last record of this batch + */ + public static Batch empty(long baseOffset, int epoch, long lastOffset) { + return new Batch<>(baseOffset, epoch, lastOffset, Collections.emptyList()); + } + + /** + * Create a batch with the given base offset, epoch and records. + * + * @param baseOffset offset of the first record in the batch + * @param epoch epoch of the leader that created this batch + * @param records the list of records in this batch + */ + public static Batch of(long baseOffset, int epoch, List records) { + if (records.isEmpty()) { + throw new IllegalArgumentException( + String.format( + "Batch must contain at least one record; baseOffset = %s; epoch = %s", + baseOffset, + epoch + ) + ); + } + + return new Batch<>(baseOffset, epoch, baseOffset + records.size() - 1, records); + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java index e5f9e38612a0d..6469af2095f57 100644 --- a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java @@ -17,8 +17,6 @@ package org.apache.kafka.raft; import java.util.Iterator; -import java.util.List; -import java.util.Objects; import java.util.OptionalLong; /** @@ -32,7 +30,7 @@ * * @param record type (see {@link org.apache.kafka.raft.RecordSerde}) */ -public interface BatchReader extends Iterator>, AutoCloseable { +public interface BatchReader extends Iterator>, AutoCloseable { /** * Get the base offset of the readable batches. Note that this value is a constant @@ -59,57 +57,4 @@ public interface BatchReader extends Iterator>, AutoClos */ @Override void close(); - - class Batch { - private final long baseOffset; - private final int epoch; - private final List records; - - public Batch(long baseOffset, int epoch, List records) { - this.baseOffset = baseOffset; - this.epoch = epoch; - this.records = records; - } - - public long lastOffset() { - return baseOffset + records.size() - 1; - } - - public long baseOffset() { - return baseOffset; - } - - public List records() { - return records; - } - - public int epoch() { - return epoch; - } - - @Override - public String toString() { - return "Batch(" + - "baseOffset=" + baseOffset + - ", epoch=" + epoch + - ", records=" + records + - ')'; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Batch batch = (Batch) o; - return baseOffset == batch.baseOffset && - epoch == batch.epoch && - Objects.equals(records, batch.records); - } - - @Override - public int hashCode() { - return Objects.hash(baseOffset, epoch, records); - } - } - } diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 8bfad3abc71af..0c277fb661c13 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -159,6 +159,8 @@ public String toString() { ", epoch=" + epoch + ", leaderId=" + leaderId + ", voters=" + voters + + ", highWatermark=" + highWatermark + + ", fetchingSnapshot=" + fetchingSnapshot + ')'; } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index d8f46392c2640..1c759514cd6c9 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -70,6 +70,7 @@ import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; import org.slf4j.Logger; @@ -135,7 +136,6 @@ * than the leader's log start offset. This API is similar to the Fetch API since the snapshot is stored * as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records * are not necessarily offset-aligned. - * */ public class KafkaRaftClient implements RaftClient { private static final int RETRY_BACKOFF_BASE_MS = 100; @@ -307,8 +307,19 @@ private void updateListenersProgress(long highWatermark) { private void updateListenersProgress(List listenerContexts, long highWatermark) { for (ListenerContext listenerContext : listenerContexts) { listenerContext.nextExpectedOffset().ifPresent(nextExpectedOffset -> { - if (nextExpectedOffset < log.startOffset()) { - listenerContext.fireHandleSnapshot(log.startOffset()); + if (nextExpectedOffset < log.startOffset() && nextExpectedOffset < highWatermark) { + SnapshotReader snapshot = latestSnapshot().orElseThrow(() -> { + return new IllegalStateException( + String.format( + "Snapshot expected since next offset of %s is %s, log start offset is %s and high-watermark is %s", + listenerContext.listener.getClass().getTypeName(), + nextExpectedOffset, + log.startOffset(), + highWatermark + ) + ); + }); + listenerContext.fireHandleSnapshot(snapshot); } }); @@ -322,6 +333,16 @@ private void updateListenersProgress(List listenerContexts, lon } } + private Optional> latestSnapshot() { + return log.latestSnapshotId().flatMap(snapshotId -> + log + .readSnapshot(snapshotId) + .map(reader -> + SnapshotReader.of(reader, serde, BufferSupplier.create(), MAX_BATCH_SIZE_BYTES) + ) + ); + } + private void maybeFireHandleCommit(long baseOffset, int epoch, List records) { for (ListenerContext listenerContext : listenerContexts) { OptionalLong nextExpectedOffsetOpt = listenerContext.nextExpectedOffset(); @@ -351,24 +372,28 @@ private void fireHandleResign(int epoch) { } @Override - public void initialize() throws IOException { - quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch())); + public void initialize() { + try { + quorum.initialize(new OffsetAndEpoch(log.endOffset().offset, log.lastFetchedEpoch())); - long currentTimeMs = time.milliseconds(); - if (quorum.isLeader()) { - throw new IllegalStateException("Voter cannot initialize as a Leader"); - } else if (quorum.isCandidate()) { - onBecomeCandidate(currentTimeMs); - } else if (quorum.isFollower()) { - onBecomeFollower(currentTimeMs); - } + long currentTimeMs = time.milliseconds(); + if (quorum.isLeader()) { + throw new IllegalStateException("Voter cannot initialize as a Leader"); + } else if (quorum.isCandidate()) { + onBecomeCandidate(currentTimeMs); + } else if (quorum.isFollower()) { + onBecomeFollower(currentTimeMs); + } - // When there is only a single voter, become candidate immediately - if (quorum.isVoter() - && quorum.remoteVoters().isEmpty() - && !quorum.isLeader() - && !quorum.isCandidate()) { - transitionToCandidate(currentTimeMs); + // When there is only a single voter, become candidate immediately + if (quorum.isVoter() + && quorum.remoteVoters().isEmpty() + && !quorum.isCandidate()) { + + transitionToCandidate(currentTimeMs); + } + } catch (IOException e) { + throw new RuntimeException(e); } } @@ -1091,6 +1116,7 @@ private boolean handleFetchResponse( if (records.sizeInBytes() > 0) { appendAsFollower(records); } + OptionalLong highWatermark = partitionResponse.highWatermark() < 0 ? OptionalLong.empty() : OptionalLong.of(partitionResponse.highWatermark()); updateFollowerHighWatermark(state, highWatermark); @@ -1220,7 +1246,8 @@ private FetchSnapshotResponseData handleFetchSnapshotRequest( } try (RawSnapshotReader snapshot = snapshotOpt.get()) { - if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshot.sizeInBytes()) { + long snapshotSize = snapshot.sizeInBytes(); + if (partitionSnapshot.position() < 0 || partitionSnapshot.position() >= snapshotSize) { return FetchSnapshotResponse.singleton( log.topicPartition(), responsePartitionSnapshot -> addQuorumLeader(responsePartitionSnapshot) @@ -1228,20 +1255,25 @@ private FetchSnapshotResponseData handleFetchSnapshotRequest( ); } + if (partitionSnapshot.position() > Integer.MAX_VALUE) { + throw new IllegalStateException( + String.format( + "Trying to fetch a snapshot with size (%s) and a position (%s) larger than %s", + snapshotSize, + partitionSnapshot.position(), + Integer.MAX_VALUE + ) + ); + } + int maxSnapshotSize; try { - maxSnapshotSize = Math.toIntExact(snapshot.sizeInBytes()); + maxSnapshotSize = Math.toIntExact(snapshotSize); } catch (ArithmeticException e) { maxSnapshotSize = Integer.MAX_VALUE; } - if (partitionSnapshot.position() > Integer.MAX_VALUE) { - throw new IllegalStateException(String.format("Trying to fetch a snapshot with position: %d lager than Int.MaxValue", partitionSnapshot.position())); - } - - UnalignedRecords records = snapshot.read(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize)); - - long snapshotSize = snapshot.sizeInBytes(); + UnalignedRecords records = snapshot.slice(partitionSnapshot.position(), Math.min(data.maxBytes(), maxSnapshotSize)); return FetchSnapshotResponse.singleton( log.topicPartition(), @@ -1346,10 +1378,15 @@ private boolean handleFetchSnapshotResponse( ); } - if (!(partitionSnapshot.unalignedRecords() instanceof MemoryRecords)) { + final UnalignedMemoryRecords records; + if (partitionSnapshot.unalignedRecords() instanceof MemoryRecords) { + records = new UnalignedMemoryRecords(((MemoryRecords) partitionSnapshot.unalignedRecords()).buffer()); + } else if (partitionSnapshot.unalignedRecords() instanceof UnalignedMemoryRecords) { + records = (UnalignedMemoryRecords) partitionSnapshot.unalignedRecords(); + } else { throw new IllegalStateException(String.format("Received unexpected fetch snapshot response: %s", partitionSnapshot)); } - snapshot.append(new UnalignedMemoryRecords(((MemoryRecords) partitionSnapshot.unalignedRecords()).buffer())); + snapshot.append(records); if (snapshot.sizeInBytes() == partitionSnapshot.size()) { // Finished fetching the snapshot. @@ -2062,7 +2099,7 @@ private long pollUnattachedAsObserver(UnattachedState state, long currentTimeMs) } private long pollCurrentState(long currentTimeMs) throws IOException { - maybeUpdateOldestSnapshotId(); + maybeDeleteBeforeSnapshot(); if (quorum.isLeader()) { return pollLeader(currentTimeMs); @@ -2126,8 +2163,14 @@ private boolean maybeCompleteShutdown(long currentTimeMs) { return false; } - private void maybeUpdateOldestSnapshotId() { - log.latestSnapshotId().ifPresent(log::deleteBeforeSnapshot); + private void maybeDeleteBeforeSnapshot() { + log.latestSnapshotId().ifPresent(snapshotId -> { + quorum.highWatermark().ifPresent(highWatermark -> { + if (highWatermark.offset >= snapshotId.offset) { + log.deleteBeforeSnapshot(snapshotId); + } + }); + }); } private void wakeup() { @@ -2218,7 +2261,7 @@ public CompletableFuture shutdown(int timeoutMs) { } @Override - public SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) throws IOException { + public SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) { return new SnapshotWriter<>( log.createSnapshot(snapshotId), MAX_BATCH_SIZE_BYTES, @@ -2326,14 +2369,16 @@ public synchronized OptionalLong nextExpectedOffset() { } /** - * This API is used when the Listener needs to be notified of a new Snapshot. This happens - * when the context last acked end offset is less that then log start offset. + * This API is used when the Listener needs to be notified of a new snapshot. This happens + * when the context's next offset is less than the log start offset. */ - public void fireHandleSnapshot(long logStartOffset) { + public void fireHandleSnapshot(SnapshotReader reader) { synchronized (this) { - nextOffset = logStartOffset; + nextOffset = reader.snapshotId().offset; lastSent = null; } + + listener.handleSnapshot(reader); } /** @@ -2343,10 +2388,16 @@ public void fireHandleSnapshot(long logStartOffset) { * data in memory, we let the state machine read the records from disk. */ public void fireHandleCommit(long baseOffset, Records records) { - BufferSupplier bufferSupplier = BufferSupplier.create(); - RecordsBatchReader reader = new RecordsBatchReader<>(baseOffset, records, - serde, bufferSupplier, this); - fireHandleCommit(reader); + fireHandleCommit( + RecordsBatchReader.of( + baseOffset, + records, + serde, + BufferSupplier.create(), + MAX_BATCH_SIZE_BYTES, + this + ) + ); } /** @@ -2357,7 +2408,7 @@ public void fireHandleCommit(long baseOffset, Records records) { * followers. */ public void fireHandleCommit(long baseOffset, int epoch, List records) { - BatchReader.Batch batch = new BatchReader.Batch<>(baseOffset, epoch, records); + Batch batch = Batch.of(baseOffset, epoch, records); MemoryBatchReader reader = new MemoryBatchReader<>(Collections.singletonList(batch), this); fireHandleCommit(reader); } @@ -2386,6 +2437,7 @@ void fireHandleResign(int epoch) { public synchronized void onClose(BatchReader reader) { OptionalLong lastOffset = reader.lastOffset(); + if (lastOffset.isPresent()) { nextOffset = lastOffset.getAsLong() + 1; } diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index 93d40679874e2..4af1821ddaa70 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -327,12 +327,13 @@ else if (!that.endOffset.isPresent()) @Override public String toString() { - return "ReplicaState(" + - "nodeId=" + nodeId + - ", endOffset=" + endOffset + - ", lastFetchTimestamp=" + lastFetchTimestamp + - ", hasAcknowledgedLeader=" + hasAcknowledgedLeader + - ')'; + return String.format( + "ReplicaState(nodeId=%s, endOffset=%s, lastFetchTimestamp=%s, hasAcknowledgedLeader=%s)", + nodeId, + endOffset, + lastFetchTimestamp, + hasAcknowledgedLeader + ); } } @@ -345,11 +346,14 @@ public boolean canGrantVote(int candidateId, boolean isLogUpToDate) { @Override public String toString() { - return "Leader(" + - "localId=" + localId + - ", epoch=" + epoch + - ", epochStartOffset=" + epochStartOffset + - ')'; + return String.format( + "Leader(localId=%s, epoch=%s, epochStartOffset=%s, highWatermark=%s, voterStates=%s)", + localId, + epoch, + epochStartOffset, + highWatermark, + voterStates + ); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index 74488b450ede1..7293f796eb772 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -16,19 +16,18 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; -import java.io.Closeable; -import java.io.IOException; import java.util.List; import java.util.concurrent.CompletableFuture; -public interface RaftClient extends Closeable { +public interface RaftClient extends AutoCloseable { interface Listener { /** * Callback which is invoked for all records committed to the log. - * It is the responsibility of the caller to invoke {@link BatchReader#close()} + * It is the responsibility of this implementation to invoke {@link BatchReader#close()} * after consuming the reader. * * Note that there is not a one-to-one correspondence between writes through @@ -44,6 +43,18 @@ interface Listener { */ void handleCommit(BatchReader reader); + /** + * Callback which is invoked when the Listener needs to load a snapshot. + * It is the responsibility of this implementation to invoke {@link SnapshotReader#close()} + * after consuming the reader. + * + * When handling this call, the implementation must assume that all previous calls + * to {@link #handleCommit} contain invalid data. + * + * @param reader snapshot reader instance which must be iterated and closed + */ + void handleSnapshot(SnapshotReader reader); + /** * Invoked after this node has become a leader. This is only called after * all commits up to the start of the leader's epoch have been sent to @@ -66,12 +77,9 @@ default void handleResign(int epoch) {} } /** - * Initialize the client. - * This should only be called once on startup. - * - * @throws IOException For any IO errors during initialization + * Initialize the client. This should only be called once on startup. */ - void initialize() throws IOException; + void initialize(); /** * Register a listener to get commit/leader notifications. @@ -157,5 +165,5 @@ default void handleResign(int epoch) {} * @param snapshotId the end offset and epoch that identifies the snapshot * @return a writable snapshot */ - SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) throws IOException; + SnapshotWriter createSnapshot(OffsetAndEpoch snapshotId); } diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java index 3db4d736a53f7..634ce37392f17 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedCounter.java @@ -18,9 +18,11 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.snapshot.SnapshotReader; +import org.apache.kafka.snapshot.SnapshotWriter; import org.slf4j.Logger; -import java.util.Optional; +import java.util.OptionalInt; import static java.util.Collections.singletonList; @@ -28,10 +30,12 @@ public class ReplicatedCounter implements RaftClient.Listener { private final int nodeId; private final Logger log; private final RaftClient client; + private final int snapshotDelayInRecords = 10; - private int committed; - private int uncommitted; - private Optional claimedEpoch; + private int committed = 0; + private int uncommitted = 0; + private OptionalInt claimedEpoch = OptionalInt.empty(); + private long lastSnapshotEndOffset = 0; public ReplicatedCounter( int nodeId, @@ -40,11 +44,7 @@ public ReplicatedCounter( ) { this.nodeId = nodeId; this.client = client; - this.log = logContext.logger(ReplicatedCounter.class); - - this.committed = 0; - this.uncommitted = 0; - this.claimedEpoch = Optional.empty(); + log = logContext.logger(ReplicatedCounter.class); } public synchronized boolean isWritable() { @@ -56,7 +56,7 @@ public synchronized void increment() { throw new KafkaException("Counter is not currently writable"); } - int epoch = claimedEpoch.get(); + int epoch = claimedEpoch.getAsInt(); uncommitted += 1; Long offset = client.scheduleAppend(epoch, singletonList(uncommitted)); if (offset != null) { @@ -68,20 +68,72 @@ public synchronized void increment() { @Override public synchronized void handleCommit(BatchReader reader) { try { - int initialValue = this.committed; + int initialCommitted = committed; + long nextReadOffset = 0; + int readEpoch = 0; + while (reader.hasNext()) { - BatchReader.Batch batch = reader.next(); - log.debug("Handle commit of batch with records {} at base offset {}", - batch.records(), batch.baseOffset()); - for (Integer value : batch.records()) { - if (value != this.committed + 1) { - throw new AssertionError("Expected next committed value to be " + - (this.committed + 1) + ", but instead found " + value + " on node " + nodeId); + Batch batch = reader.next(); + log.debug( + "Handle commit of batch with records {} at base offset {}", + batch.records(), + batch.baseOffset() + ); + for (Integer nextCommitted: batch.records()) { + if (nextCommitted != committed + 1) { + throw new AssertionError( + String.format( + "Expected next committed value to be %s, but instead found %s on node %s", + committed + 1, + nextCommitted, + nodeId + ) + ); } - this.committed = value; + committed = nextCommitted; + } + + nextReadOffset = batch.lastOffset() + 1; + readEpoch = batch.epoch(); + } + log.debug("Counter incremented from {} to {}", initialCommitted, committed); + + if (lastSnapshotEndOffset + snapshotDelayInRecords < nextReadOffset) { + log.debug("Generating new snapshot at {} since next commit offset is {}", lastSnapshotEndOffset, nextReadOffset); + try (SnapshotWriter snapshot = client.createSnapshot(new OffsetAndEpoch(nextReadOffset, readEpoch))) { + snapshot.append(singletonList(committed)); + snapshot.freeze(); + lastSnapshotEndOffset = nextReadOffset; } } - log.debug("Counter incremented from {} to {}", initialValue, committed); + } finally { + reader.close(); + } + } + + @Override + public synchronized void handleSnapshot(SnapshotReader reader) { + try { + log.debug("Loading snapshot {}", reader.snapshotId()); + while (reader.hasNext()) { + Batch batch = reader.next(); + if (batch.records().size() != 1) { + throw new AssertionError( + String.format( + "Expected the snapshot at %s to only contain one record %s", + reader.snapshotId(), + batch.records() + ) + ); + } + + for (Integer value : batch) { + log.debug("Setting value: {}", value); + committed = value; + uncommitted = value; + } + } + log.debug("Finished loading snapshot. Set value: {}", committed); } finally { reader.close(); } @@ -91,15 +143,14 @@ public synchronized void handleCommit(BatchReader reader) { public synchronized void handleClaim(int epoch) { log.debug("Counter uncommitted value initialized to {} after claiming leadership in epoch {}", committed, epoch); - this.uncommitted = committed; - this.claimedEpoch = Optional.of(epoch); + uncommitted = committed; + claimedEpoch = OptionalInt.of(epoch); } @Override public synchronized void handleResign(int epoch) { log.debug("Counter uncommitted value reset after resigning leadership"); this.uncommitted = -1; - this.claimedEpoch = Optional.empty(); + this.claimedEpoch = OptionalInt.empty(); } - } diff --git a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java index 198a2c630a79f..6a6b74b10acbf 100644 --- a/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java +++ b/raft/src/main/java/org/apache/kafka/raft/ReplicatedLog.java @@ -22,11 +22,9 @@ import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; -import java.io.Closeable; -import java.io.IOException; import java.util.Optional; -public interface ReplicatedLog extends Closeable { +public interface ReplicatedLog extends AutoCloseable { /** * Write a set of records to the local leader log. These messages will either @@ -231,12 +229,13 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { /** * Create a writable snapshot for the given snapshot id. * - * See {@link RawSnapshotWriter} for details on how to use this object. + * See {@link RawSnapshotWriter} for details on how to use this object. The caller of + * this method is responsible for invoking {@link RawSnapshotWriter#close()}. * * @param snapshotId the end offset and epoch that identifies the snapshot * @return a writable snapshot */ - RawSnapshotWriter createSnapshot(OffsetAndEpoch snapshotId) throws IOException; + RawSnapshotWriter createSnapshot(OffsetAndEpoch snapshotId); /** * Opens a readable snapshot for the given snapshot id. @@ -249,7 +248,7 @@ default long truncateToEndOffset(OffsetAndEpoch endOffset) { * @return an Optional with a readable snapshot, if the snapshot exists, otherwise * returns an empty Optional */ - Optional readSnapshot(OffsetAndEpoch snapshotId) throws IOException; + Optional readSnapshot(OffsetAndEpoch snapshotId); /** * Returns the latest snapshot id if one exists. diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java index 2ff2b8cdda7b6..07ad1bbcb4533 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/MemoryBatchReader.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft.internals; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import java.util.Iterator; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index 0817138e3ff0a..a55815d0b5e5e 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -16,136 +16,59 @@ */ package org.apache.kafka.raft.internals; -import org.apache.kafka.common.protocol.DataInputStreamReadable; -import org.apache.kafka.common.protocol.Readable; -import org.apache.kafka.common.utils.BufferSupplier; -import org.apache.kafka.common.record.DefaultRecordBatch; -import org.apache.kafka.common.record.FileRecords; -import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RecordSerde; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.OptionalLong; -public class RecordsBatchReader implements BatchReader { +public final class RecordsBatchReader implements BatchReader { private final long baseOffset; - private final Records records; - private final RecordSerde serde; - private final BufferSupplier bufferSupplier; + private final RecordsIterator iterator; private final CloseListener> closeListener; - private Iterator batchIterator; private long lastReturnedOffset; - private Batch nextBatch; + + private Optional> nextBatch = Optional.empty(); private boolean isClosed = false; - private ByteBuffer allocatedBuffer = null; - public RecordsBatchReader( + private RecordsBatchReader( long baseOffset, - Records records, - RecordSerde serde, - BufferSupplier bufferSupplier, + RecordsIterator iterator, CloseListener> closeListener ) { this.baseOffset = baseOffset; - this.records = records; - this.serde = serde; - this.bufferSupplier = bufferSupplier; + this.iterator = iterator; this.closeListener = closeListener; this.lastReturnedOffset = baseOffset; } - private void materializeIterator() throws IOException { - if (records instanceof MemoryRecords) { - batchIterator = ((MemoryRecords) records).batchIterator(); - } else if (records instanceof FileRecords) { - this.allocatedBuffer = bufferSupplier.get(records.sizeInBytes()); - ((FileRecords) records).readInto(allocatedBuffer, 0); - MemoryRecords memRecords = MemoryRecords.readableRecords(allocatedBuffer); - batchIterator = memRecords.batchIterator(); - } else { - throw new IllegalStateException("Unexpected Records type " + records.getClass()); - } - } - - private void findNextDataBatch() { - if (batchIterator == null) { - try { - materializeIterator(); - } catch (IOException e) { - throw new RuntimeException("Failed to read records into memory", e); - } - } - - while (batchIterator.hasNext()) { - MutableRecordBatch nextBatch = batchIterator.next(); - if (!(nextBatch instanceof DefaultRecordBatch)) { - throw new IllegalStateException(); - } - - DefaultRecordBatch batch = (DefaultRecordBatch) nextBatch; - if (!batch.isControlBatch()) { - this.nextBatch = readBatch(batch); - return; - } else { - this.lastReturnedOffset = batch.lastOffset(); - } - } - } - - private Batch readBatch(DefaultRecordBatch batch) { - Integer numRecords = batch.countOrNull(); - if (numRecords == null) { - throw new IllegalStateException(); - } - - List records = new ArrayList<>(numRecords); - try (DataInputStreamReadable input = new DataInputStreamReadable( - batch.recordInputStream(bufferSupplier))) { - for (int i = 0; i < numRecords; i++) { - T record = readRecord(input); - records.add(record); - } - return new Batch<>( - batch.baseOffset(), - batch.partitionLeaderEpoch(), - records - ); - } - } - @Override public boolean hasNext() { - if (nextBatch != null) { - return true; - } else { - findNextDataBatch(); - return nextBatch != null; + ensureOpen(); + + if (!nextBatch.isPresent()) { + nextBatch = nextBatch(); } + + return nextBatch.isPresent(); } @Override public Batch next() { - if (nextBatch != null) { - Batch res = nextBatch; - nextBatch = null; - lastReturnedOffset = res.lastOffset(); - return res; - } else { - findNextDataBatch(); - if (nextBatch == null) { - throw new NoSuchElementException(); - } - return next(); + if (!hasNext()) { + throw new NoSuchElementException("Records batch reader doesn't have any more elements"); } + + Batch batch = nextBatch.get(); + nextBatch = Optional.empty(); + + lastReturnedOffset = batch.lastOffset(); + return batch; } @Override @@ -163,48 +86,46 @@ public OptionalLong lastOffset() { @Override public void close() { - isClosed = true; + if (!isClosed) { + isClosed = true; - if (allocatedBuffer != null) { - bufferSupplier.release(allocatedBuffer); + iterator.close(); + closeListener.onClose(this); } - - closeListener.onClose(this); } - public T readRecord(Readable input) { - // Read size of body in bytes - input.readVarint(); - - // Read unused attributes - input.readByte(); - - long timestampDelta = input.readVarlong(); - if (timestampDelta != 0) { - throw new IllegalArgumentException(); - } - - // Read offset delta - input.readVarint(); - - int keySize = input.readVarint(); - if (keySize != -1) { - throw new IllegalArgumentException("Unexpected key size " + keySize); - } + public static RecordsBatchReader of( + long baseOffset, + Records records, + RecordSerde serde, + BufferSupplier bufferSupplier, + int maxBatchSize, + CloseListener> closeListener + ) { + return new RecordsBatchReader<>( + baseOffset, + new RecordsIterator<>(records, serde, bufferSupplier, maxBatchSize), + closeListener + ); + } - int valueSize = input.readVarint(); - if (valueSize < 0) { - throw new IllegalArgumentException(); + private void ensureOpen() { + if (isClosed) { + throw new IllegalStateException("Records batch reader was closed"); } + } - T record = serde.read(input, valueSize); + private Optional> nextBatch() { + while (iterator.hasNext()) { + Batch batch = iterator.next(); - int numHeaders = input.readVarint(); - if (numHeaders != 0) { - throw new IllegalArgumentException(); + if (batch.records().isEmpty()) { + lastReturnedOffset = batch.lastOffset(); + } else { + return Optional.of(batch); + } } - return record; + return Optional.empty(); } - } diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java new file mode 100644 index 0000000000000..46155b57ee4f6 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java @@ -0,0 +1,239 @@ +/* + * 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.internals; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Optional; +import org.apache.kafka.common.protocol.DataInputStreamReadable; +import org.apache.kafka.common.protocol.Readable; +import org.apache.kafka.common.record.DefaultRecordBatch; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MutableRecordBatch; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.Batch; +import org.apache.kafka.raft.RecordSerde; + +public final class RecordsIterator implements Iterator>, AutoCloseable { + private final Records records; + private final RecordSerde serde; + private final BufferSupplier bufferSupplier; + private final int batchSize; + + private Iterator nextBatches = Collections.emptyIterator(); + private Optional> nextBatch = Optional.empty(); + // Buffer used as the backing store for nextBatches if needed + private Optional allocatedBuffer = Optional.empty(); + // Number of bytes from records read up to now + private int bytesRead = 0; + private boolean isClosed = false; + + public RecordsIterator( + Records records, + RecordSerde serde, + BufferSupplier bufferSupplier, + int batchSize + ) { + this.records = records; + this.serde = serde; + this.bufferSupplier = bufferSupplier; + this.batchSize = Math.max(batchSize, Records.HEADER_SIZE_UP_TO_MAGIC); + } + + @Override + public boolean hasNext() { + ensureOpen(); + + if (!nextBatch.isPresent()) { + nextBatch = nextBatch(); + } + + return nextBatch.isPresent(); + } + + @Override + public Batch next() { + if (!hasNext()) { + throw new NoSuchElementException("Batch iterator doesn't have any more elements"); + } + + Batch batch = nextBatch.get(); + nextBatch = Optional.empty(); + + return batch; + } + + @Override + public void close() { + isClosed = true; + allocatedBuffer.ifPresent(bufferSupplier::release); + allocatedBuffer = Optional.empty(); + } + + private void ensureOpen() { + if (isClosed) { + throw new IllegalStateException("Serde record batch itererator was closed"); + } + } + + private MemoryRecords readFileRecords(FileRecords fileRecords, ByteBuffer buffer) { + int start = buffer.position(); + try { + fileRecords.readInto(buffer, bytesRead); + } catch (IOException e) { + throw new RuntimeException("Failed to read records into memory", e); + } + + bytesRead += buffer.limit() - start; + return MemoryRecords.readableRecords(buffer.slice()); + } + + private MemoryRecords createMemoryRecords(FileRecords fileRecords) { + final ByteBuffer buffer; + if (allocatedBuffer.isPresent()) { + buffer = allocatedBuffer.get(); + buffer.compact(); + } else { + buffer = bufferSupplier.get(Math.min(batchSize, records.sizeInBytes())); + allocatedBuffer = Optional.of(buffer); + } + + MemoryRecords memoryRecords = readFileRecords(fileRecords, buffer); + + // firstBatchSize() is always non-null because the minimum buffer is HEADER_SIZE_UP_TO_MAGIC. + if (memoryRecords.firstBatchSize() <= buffer.remaining()) { + return memoryRecords; + } else { + // Not enough bytes read; create a bigger buffer + ByteBuffer newBuffer = bufferSupplier.get(memoryRecords.firstBatchSize()); + allocatedBuffer = Optional.of(newBuffer); + + newBuffer.put(buffer); + bufferSupplier.release(buffer); + + return readFileRecords(fileRecords, newBuffer); + } + } + + private Iterator nextBatches() { + int recordSize = records.sizeInBytes(); + if (bytesRead < recordSize) { + final MemoryRecords memoryRecords; + if (records instanceof MemoryRecords) { + bytesRead = recordSize; + memoryRecords = (MemoryRecords) records; + } else if (records instanceof FileRecords) { + memoryRecords = createMemoryRecords((FileRecords) records); + } else { + throw new IllegalStateException(String.format("Unexpected Records type %s", records.getClass())); + } + + return memoryRecords.batchIterator(); + } + + return Collections.emptyIterator(); + } + + private Optional> nextBatch() { + if (!nextBatches.hasNext()) { + nextBatches = nextBatches(); + } + + if (nextBatches.hasNext()) { + MutableRecordBatch nextBatch = nextBatches.next(); + + // Update the buffer position to reflect the read batch + allocatedBuffer.ifPresent(buffer -> buffer.position(buffer.position() + nextBatch.sizeInBytes())); + + if (!(nextBatch instanceof DefaultRecordBatch)) { + throw new IllegalStateException( + String.format("DefaultRecordBatch expected by record type was %s", nextBatch.getClass()) + ); + } + + return Optional.of(readBatch((DefaultRecordBatch) nextBatch)); + } + + return Optional.empty(); + } + + private Batch readBatch(DefaultRecordBatch batch) { + final Batch result; + if (batch.isControlBatch()) { + result = Batch.empty(batch.baseOffset(), batch.partitionLeaderEpoch(), batch.lastOffset()); + } else { + Integer numRecords = batch.countOrNull(); + if (numRecords == null) { + throw new IllegalStateException("Expected a record count for the records batch"); + } + + List records = new ArrayList<>(numRecords); + try (DataInputStreamReadable input = new DataInputStreamReadable(batch.recordInputStream(bufferSupplier))) { + for (int i = 0; i < numRecords; i++) { + T record = readRecord(input); + records.add(record); + } + } + + result = Batch.of(batch.baseOffset(), batch.partitionLeaderEpoch(), records); + } + + return result; + } + + private T readRecord(Readable input) { + // Read size of body in bytes + input.readVarint(); + + // Read unused attributes + input.readByte(); + + long timestampDelta = input.readVarlong(); + if (timestampDelta != 0) { + throw new IllegalArgumentException(); + } + + // Read offset delta + input.readVarint(); + + int keySize = input.readVarint(); + if (keySize != -1) { + throw new IllegalArgumentException("Unexpected key size " + keySize); + } + + int valueSize = input.readVarint(); + if (valueSize < 0) { + throw new IllegalArgumentException(); + } + + T record = serde.read(input, valueSize); + + int numHeaders = input.readVarint(); + if (numHeaders != 0) { + throw new IllegalArgumentException(); + } + + return record; + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java index 1ca63f1b9c3cd..0bcf2c67055b1 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java @@ -21,9 +21,11 @@ import org.apache.kafka.metalog.MetaLogLeader; import org.apache.kafka.metalog.MetaLogListener; import org.apache.kafka.metalog.MetaLogManager; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.snapshot.SnapshotReader; import java.util.List; import java.util.stream.Collectors; @@ -113,7 +115,7 @@ public void handleCommit(BatchReader reader) { // not a leader. We want to move this IO to the state machine so that // it does not block Raft replication while (reader.hasNext()) { - BatchReader.Batch batch = reader.next(); + Batch batch = reader.next(); List records = batch.records().stream() .map(ApiMessageAndVersion::message) .collect(Collectors.toList()); @@ -124,6 +126,11 @@ public void handleCommit(BatchReader reader) { } } + @Override + public void handleSnapshot(SnapshotReader reader) { + reader.close(); + } + @Override public void handleClaim(int epoch) { listener.handleNewLeader(new MetaLogLeader(nodeId, epoch)); diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java index d0218c79cc427..820230ee64636 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotReader.java @@ -17,14 +17,12 @@ package org.apache.kafka.snapshot; import org.apache.kafka.common.record.FileRecords; -import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedRecords; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.OffsetAndEpoch; import java.io.IOException; import java.nio.file.Path; -import java.util.Iterator; public final class FileRawSnapshotReader implements RawSnapshotReader { private final FileRecords fileRecords; @@ -46,12 +44,13 @@ public long sizeInBytes() { } @Override - public Iterator iterator() { - return Utils.covariantCast(fileRecords.batchIterator()); + public UnalignedRecords slice(long position, int size) { + return fileRecords.sliceUnaligned(Math.toIntExact(position), size); } - public UnalignedRecords read(long position, int size) { - return fileRecords.sliceUnaligned(Math.toIntExact(position), size); + @Override + public Records records() { + return fileRecords; } @Override diff --git a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java index 6d0c17c4e051d..e259aefaba962 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/FileRawSnapshotWriter.java @@ -54,28 +54,32 @@ public OffsetAndEpoch snapshotId() { } @Override - public long sizeInBytes() throws IOException { - return channel.size(); + public long sizeInBytes() { + try { + return channel.size(); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override - public void append(UnalignedMemoryRecords records) throws IOException { - if (frozen) { - throw new IllegalStateException( - String.format("Append is not supported. Snapshot is already frozen: id = %s; temp path = %s", snapshotId, tempSnapshotPath) - ); + public void append(UnalignedMemoryRecords records) { + try { + checkIfFrozen("Append"); + Utils.writeFully(channel, records.buffer()); + } catch (IOException e) { + throw new RuntimeException(e); } - Utils.writeFully(channel, records.buffer()); } @Override - public void append(MemoryRecords records) throws IOException { - if (frozen) { - throw new IllegalStateException( - String.format("Append is not supported. Snapshot is already frozen: id = %s; temp path = %s", snapshotId, tempSnapshotPath) - ); + public void append(MemoryRecords records) { + try { + checkIfFrozen("Append"); + Utils.writeFully(channel, records.buffer()); + } catch (IOException e) { + throw new RuntimeException(e); } - Utils.writeFully(channel, records.buffer()); } @Override @@ -84,34 +88,57 @@ public boolean isFrozen() { } @Override - public void freeze() throws IOException { - if (frozen) { - throw new IllegalStateException( - String.format("Freeze is not supported. Snapshot is already frozen: id = %s; temp path = %s", snapshotId, tempSnapshotPath) - ); - } + public void freeze() { + try { + checkIfFrozen("Freeze"); - channel.close(); - frozen = true; + channel.close(); + frozen = true; - // Set readonly and ignore the result - if (!tempSnapshotPath.toFile().setReadOnly()) { - throw new IOException(String.format("Unable to set file (%s) as read-only", tempSnapshotPath)); - } + if (!tempSnapshotPath.toFile().setReadOnly()) { + throw new IllegalStateException(String.format("Unable to set file (%s) as read-only", tempSnapshotPath)); + } - Path destination = Snapshots.moveRename(tempSnapshotPath, snapshotId); - Utils.atomicMoveWithFallback(tempSnapshotPath, destination); + Path destination = Snapshots.moveRename(tempSnapshotPath, snapshotId); + Utils.atomicMoveWithFallback(tempSnapshotPath, destination); - replicatedLog.ifPresent(log -> log.onSnapshotFrozen(snapshotId)); + replicatedLog.ifPresent(log -> log.onSnapshotFrozen(snapshotId)); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override - public void close() throws IOException { + public void close() { try { channel.close(); - } finally { // This is a noop if freeze was called before calling close Files.deleteIfExists(tempSnapshotPath); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public String toString() { + return String.format( + "FileRawSnapshotWriter(path=%s, snapshotId=%s, frozen=%s)", + tempSnapshotPath, + snapshotId, + frozen + ); + } + + void checkIfFrozen(String operation) { + if (frozen) { + throw new IllegalStateException( + String.format( + "%s is not supported. Snapshot is already frozen: id = %s; temp path = %s", + operation, + snapshotId, + tempSnapshotPath + ) + ); } } @@ -126,14 +153,18 @@ public static FileRawSnapshotWriter create( Path logDir, OffsetAndEpoch snapshotId, Optional replicatedLog - ) throws IOException { - Path path = Snapshots.createTempFile(logDir, snapshotId); + ) { + try { + Path path = Snapshots.createTempFile(logDir, snapshotId); - return new FileRawSnapshotWriter( - path, - FileChannel.open(path, Utils.mkSet(StandardOpenOption.WRITE, StandardOpenOption.APPEND)), - snapshotId, - replicatedLog - ); + return new FileRawSnapshotWriter( + path, + FileChannel.open(path, Utils.mkSet(StandardOpenOption.WRITE, StandardOpenOption.APPEND)), + snapshotId, + replicatedLog + ); + } catch (IOException e) { + throw new RuntimeException(e); + } } } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java index 6d1ff28950b33..506728d10ae41 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotReader.java @@ -16,17 +16,16 @@ */ package org.apache.kafka.snapshot; -import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.UnalignedRecords; import org.apache.kafka.raft.OffsetAndEpoch; import java.io.Closeable; -import java.io.IOException; /** * Interface for reading snapshots as a sequence of records. */ -public interface RawSnapshotReader extends Closeable, Iterable { +public interface RawSnapshotReader extends Closeable { /** * Returns the end offset and epoch for the snapshot. */ @@ -34,20 +33,22 @@ public interface RawSnapshotReader extends Closeable, Iterable { /** * Returns the number of bytes for the snapshot. - * - * @throws IOException for any IO error while reading the size */ - long sizeInBytes() throws IOException; + long sizeInBytes(); /** - * Reads bytes from position into the given buffer. + * Creates a slize of unaligned records from the position up to a size. * - * It is not guarantee that the given buffer will be filled. + * @param position the starting position of the slice in the snapshot + * @param size the maximum size of the slice + * @return an unaligned slice of records in the snapshot + */ + UnalignedRecords slice(long position, int size); + + /** + * Returns all of the records backing this snapshot reader. * - * @param size size to read from snapshot file - * @param position the starting position in the snapshot to read - * @return the region read from snapshot - * @throws IOException for any IO error while reading the snapshot + * @return all of the records for this snapshot */ - UnalignedRecords read(long position, int size) throws IOException; + Records records(); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java index f8ec58f17fcd2..07d8271e953f6 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/RawSnapshotWriter.java @@ -21,13 +21,10 @@ import org.apache.kafka.common.record.UnalignedMemoryRecords; import org.apache.kafka.raft.OffsetAndEpoch; -import java.io.Closeable; -import java.io.IOException; - /** * Interface for writing snapshot as a sequence of records. */ -public interface RawSnapshotWriter extends Closeable { +public interface RawSnapshotWriter extends AutoCloseable { /** * Returns the end offset and epoch for the snapshot. */ @@ -35,10 +32,8 @@ public interface RawSnapshotWriter extends Closeable { /** * Returns the number of bytes for the snapshot. - * - * @throws IOException for any IO error while reading the size */ - long sizeInBytes() throws IOException; + long sizeInBytes(); /** * Fully appends the memory record set to the snapshot. @@ -47,9 +42,8 @@ public interface RawSnapshotWriter extends Closeable { * snapshot. * * @param records the region to append - * @throws IOException for any IO error during append */ - void append(MemoryRecords records) throws IOException; + void append(MemoryRecords records); /** * Fully appends the memory record set to the snapshot, the difference with {@link RawSnapshotWriter#append(MemoryRecords)} @@ -59,9 +53,8 @@ public interface RawSnapshotWriter extends Closeable { * snapshot. * * @param records the region to append - * @throws IOException for any IO error during append */ - void append(UnalignedMemoryRecords records) throws IOException; + void append(UnalignedMemoryRecords records); /** * Returns true if the snapshot has been frozen, otherwise false is returned. @@ -72,17 +65,13 @@ public interface RawSnapshotWriter extends Closeable { /** * Freezes the snapshot and marking it as immutable. - * - * @throws IOException for any IO error during freezing */ - void freeze() throws IOException; + void freeze(); /** * Closes the snapshot writer. * * If close is called without first calling freeze the snapshot is aborted. - * - * @throws IOException for any IO error during close */ - void close() throws IOException; + void close(); } diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java new file mode 100644 index 0000000000000..af00cdb2286bf --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -0,0 +1,81 @@ +/* + * 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.snapshot; + +import java.util.Iterator; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.Batch; +import org.apache.kafka.raft.OffsetAndEpoch; +import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.raft.internals.RecordsIterator; + +/** + * A type for reading an immutable snapshot. + * + * A snapshot reader can be used to scan through all of the objects T in a snapshot. It + * is assumed that the content of the snapshot represents all of the objects T for the topic + * partition from offset 0 up to but not including the end offset in the snapshot id. + */ +public final class SnapshotReader implements AutoCloseable, Iterator> { + private final OffsetAndEpoch snapshotId; + private final RecordsIterator iterator; + + private SnapshotReader( + OffsetAndEpoch snapshotId, + RecordsIterator iterator + ) { + this.snapshotId = snapshotId; + this.iterator = iterator; + } + + /** + * Returns the end offset and epoch for the snapshot. + */ + public OffsetAndEpoch snapshotId() { + return snapshotId; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Batch next() { + return iterator.next(); + } + + /** + * Closes the snapshot reader. + */ + public void close() { + iterator.close(); + } + + public static SnapshotReader of( + RawSnapshotReader snapshot, + RecordSerde serde, + BufferSupplier bufferSupplier, + int maxBatchSize + ) { + return new SnapshotReader<>( + snapshot.snapshotId(), + new RecordsIterator<>(snapshot.records(), serde, bufferSupplier, maxBatchSize) + ); + } +} diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java index 653859f2e43b8..e9b3c64e0362f 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java @@ -25,12 +25,10 @@ import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchAccumulator.CompletedBatch; -import java.io.Closeable; -import java.io.IOException; import java.util.List; /** - * A type for writing a snapshot fora given end offset and epoch. + * A type for writing a snapshot for a given end offset and epoch. * * A snapshot writer can be used to append objects until freeze is called. When freeze is * called the snapshot is validated and marked as immutable. After freeze is called any @@ -42,7 +40,7 @@ * * @see org.apache.kafka.raft.RaftClient#createSnapshot(OffsetAndEpoch) */ -final public class SnapshotWriter implements Closeable { +final public class SnapshotWriter implements AutoCloseable { final private RawSnapshotWriter snapshot; final private BatchAccumulator accumulator; final private Time time; @@ -102,10 +100,9 @@ public boolean isFrozen() { * The list of record passed are guaranteed to get written together. * * @param records the list of records to append to the snapshot - * @throws IOException for any IO error while appending * @throws IllegalStateException if append is called when isFrozen is true */ - public void append(List records) throws IOException { + public void append(List records) { if (snapshot.isFrozen()) { String message = String.format( "Append not supported. Snapshot is already frozen: id = '%s'.", @@ -124,10 +121,8 @@ public void append(List records) throws IOException { /** * Freezes the snapshot by flushing all pending writes and marking it as immutable. - * - * @throws IOException for any IO error during freezing */ - public void freeze() throws IOException { + public void freeze() { appendBatches(accumulator.drain()); snapshot.freeze(); accumulator.close(); @@ -137,15 +132,13 @@ public void freeze() throws IOException { * Closes the snapshot writer. * * If close is called without first calling freeze the snapshot is aborted. - * - * @throws IOException for any IO error during close */ - public void close() throws IOException { + public void close() { snapshot.close(); accumulator.close(); } - private void appendBatches(List> batches) throws IOException { + private void appendBatches(List> batches) { try { for (CompletedBatch batch : batches) { snapshot.append(batch.data); diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java index 6af2f0d4b7a8f..6fd5147421165 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientSnapshotTest.java @@ -31,8 +31,9 @@ import org.apache.kafka.raft.internals.StringSerde; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.snapshot.SnapshotWriter; -import org.apache.kafka.snapshot.SnapshotWriterTest; +import org.apache.kafka.snapshot.SnapshotWriterReaderTest; import org.junit.jupiter.api.Test; import java.io.IOException; @@ -47,6 +48,166 @@ import static org.junit.jupiter.api.Assertions.assertTrue; final public class KafkaRaftClientSnapshotTest { + @Test + public void testLeaderListernerNotified() throws Exception { + int localId = 0; + int otherNodeId = localId + 1; + Set voters = Utils.mkSet(localId, otherNodeId); + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) + .withEmptySnapshot(snapshotId) + .deleteBeforeSnapshot(snapshotId) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); + + // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); + assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); + } + } + + @Test + public void testFollowerListenerNotified() throws Exception { + int localId = 0; + int leaderId = localId + 1; + Set voters = Utils.mkSet(localId, leaderId); + int epoch = 2; + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) + .withEmptySnapshot(snapshotId) + .deleteBeforeSnapshot(snapshotId) + .withElectedLeader(epoch, leaderId) + .build(); + + // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch); + context.deliverResponse( + fetchRequest.correlationId, + fetchRequest.destinationId(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, localLogEndOffset, Errors.NONE) + ); + + context.pollUntilRequest(); + context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch); + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); + } + } + + @Test + public void testSecondListenerNotified() throws Exception { + int localId = 0; + int leaderId = localId + 1; + Set voters = Utils.mkSet(localId, leaderId); + int epoch = 2; + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) + .withEmptySnapshot(snapshotId) + .withElectedLeader(epoch, leaderId) + .build(); + + // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, localLogEndOffset, snapshotId.epoch); + context.deliverResponse( + fetchRequest.correlationId, + fetchRequest.destinationId(), + context.fetchResponse(epoch, leaderId, MemoryRecords.EMPTY, localLogEndOffset, Errors.NONE) + ); + + context.pollUntilRequest(); + context.assertSentFetchRequest(epoch, localLogEndOffset, snapshotId.epoch); + + RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(); + context.client.register(secondListener); + context.client.poll(); + + // Check that the second listener was notified of the new snapshot + try (SnapshotReader snapshot = secondListener.drainHandledSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); + } + } + + @Test + public void testListenerRenotified() throws Exception { + int localId = 0; + int otherNodeId = localId + 1; + Set voters = Utils.mkSet(localId, otherNodeId); + OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1); + + RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) + .appendToLog(snapshotId.epoch, Arrays.asList("a", "b", "c")) + .appendToLog(snapshotId.epoch, Arrays.asList("d", "e", "f")) + .appendToLog(snapshotId.epoch, Arrays.asList("g", "h", "i")) + .withEmptySnapshot(snapshotId) + .deleteBeforeSnapshot(snapshotId) + .build(); + + context.becomeLeader(); + int epoch = context.currentEpoch(); + + // Stop the listener from reading commit batches + context.listener.updateReadCommit(false); + + // Advance the highWatermark + long localLogEndOffset = context.log.endOffset().offset; + context.deliverRequest(context.fetchRequest(epoch, otherNodeId, localLogEndOffset, epoch, 0)); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(localId)); + assertEquals(localLogEndOffset, context.client.highWatermark().getAsLong()); + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); + } + + // Generate a new snapshot + OffsetAndEpoch secondSnapshot = new OffsetAndEpoch(localLogEndOffset, epoch); + try (SnapshotWriter snapshot = context.client.createSnapshot(secondSnapshot)) { + snapshot.freeze(); + } + context.client.poll(); + + // Resume the listener from reading commit batches + context.listener.updateReadCommit(true); + + context.client.poll(); + // Check that listener was notified of the second snapshot + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { + assertEquals(secondSnapshot, snapshot.snapshotId()); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(), snapshot); + } + } + @Test public void testFetchRequestOffsetLessThanLogStart() throws Exception { int localId = 0; @@ -128,7 +289,7 @@ public void testFetchRequestWithLargerLastFetchedEpoch() throws Exception { } context.client.poll(); - context.client.scheduleAppend(epoch, Arrays.asList("a", "b", "c")); + context.client.scheduleAppend(epoch, Arrays.asList("g", "h", "i")); context.time.sleep(context.appendLingerMs()); context.client.poll(); @@ -412,7 +573,7 @@ public void testFetchSnapshotRequestAsLeader() throws Exception { assertEquals(0, response.position()); assertEquals(snapshot.sizeInBytes(), response.unalignedRecords().sizeInBytes()); - UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.read(0, Math.toIntExact(snapshot.sizeInBytes())); + UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.slice(0, Math.toIntExact(snapshot.sizeInBytes())); assertEquals(memoryRecords.buffer(), ((UnalignedMemoryRecords) response.unalignedRecords()).buffer()); } @@ -456,7 +617,7 @@ public void testPartialFetchSnapshotRequestAsLeader() throws Exception { assertEquals(0, response.position()); assertEquals(snapshot.sizeInBytes() / 2, response.unalignedRecords().sizeInBytes()); - UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.read(0, Math.toIntExact(snapshot.sizeInBytes())); + UnalignedMemoryRecords memoryRecords = (UnalignedMemoryRecords) snapshot.slice(0, Math.toIntExact(snapshot.sizeInBytes())); ByteBuffer snapshotBuffer = memoryRecords.buffer(); ByteBuffer responseBuffer = ByteBuffer.allocate(Math.toIntExact(snapshot.sizeInBytes())); @@ -747,9 +908,16 @@ public void testFetchResponseWithSnapshotId() throws Exception { fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset, snapshotId.epoch); + // Check that the snapshot was written to the log try (RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get()) { assertEquals(memorySnapshot.buffer().remaining(), snapshot.sizeInBytes()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); + } + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } } @@ -844,9 +1012,16 @@ public void testFetchSnapshotResponsePartialData() throws Exception { fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, snapshotId.offset, snapshotId.epoch); + // Check that the snapshot was written to the log try (RawSnapshotReader snapshot = context.log.readSnapshot(snapshotId).get()) { assertEquals(memorySnapshot.buffer().remaining(), snapshot.sizeInBytes()); - SnapshotWriterTest.assertSnapshot(Arrays.asList(records), snapshot); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); + } + + // Check that listener was notified of the new snapshot + try (SnapshotReader snapshot = context.listener.drainHandledSnapshot().get()) { + assertEquals(snapshotId, snapshot.snapshotId()); + SnapshotWriterReaderTest.assertSnapshot(Arrays.asList(records), snapshot); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index b2092aa8e17d3..259957a071125 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -2013,11 +2014,29 @@ public void testLeaderAppendSingleMemberQuorum() throws Exception { // Now try reading it int otherNodeId = 1; - context.deliverRequest(context.fetchRequest(1, otherNodeId, 0L, 0, 500)); - context.pollUntilResponse(); + List batches = new ArrayList<>(2); + boolean appended = true; + + // Continue to fetch until the leader returns an empty response + while (appended) { + long fetchOffset = 0; + int lastFetchedEpoch = 0; + if (!batches.isEmpty()) { + MutableRecordBatch lastBatch = batches.get(batches.size() - 1); + fetchOffset = lastBatch.lastOffset() + 1; + lastFetchedEpoch = lastBatch.partitionLeaderEpoch(); + } + + context.deliverRequest(context.fetchRequest(1, otherNodeId, fetchOffset, lastFetchedEpoch, 0)); + context.pollUntilResponse(); + + MemoryRecords fetchedRecords = context.assertSentFetchPartitionResponse(Errors.NONE, 1, OptionalInt.of(localId)); + List fetchedBatch = Utils.toList(fetchedRecords.batchIterator()); + batches.addAll(fetchedBatch); + + appended = !fetchedBatch.isEmpty(); + } - MemoryRecords fetchedRecords = context.assertSentFetchPartitionResponse(Errors.NONE, 1, OptionalInt.of(localId)); - List batches = Utils.toList(fetchedRecords.batchIterator()); assertEquals(2, batches.size()); MutableRecordBatch leaderChangeBatch = batches.get(0); @@ -2233,6 +2252,7 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse List batch2 = Arrays.asList("4", "5", "6"); List batch3 = Arrays.asList("7", "8", "9"); + List> expectedBatches = Arrays.asList(batch1, batch2, batch3); RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .appendToLog(1, batch1) .appendToLog(1, batch2) @@ -2264,18 +2284,24 @@ public void testHandleClaimCallbackFiresAfterHighWatermarkReachesEpochStartOffse // watermark advances and we can start sending committed data to the // listener. Note that the `LeaderChange` control record is filtered. context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 500)); - context.client.poll(); - assertEquals(OptionalInt.empty(), context.listener.currentClaimedEpoch()); - assertEquals(3, context.listener.numCommittedBatches()); - assertEquals(batch1, context.listener.commitWithBaseOffset(0L)); - assertEquals(batch2, context.listener.commitWithBaseOffset(3L)); - assertEquals(batch3, context.listener.commitWithBaseOffset(6L)); - assertEquals(OptionalLong.of(8L), context.listener.lastCommitOffset()); + context.pollUntil(() -> { + int committedBatches = context.listener.numCommittedBatches(); + long baseOffset = 0; + for (int index = 0; index < committedBatches; index++) { + List expectedBatch = expectedBatches.get(index); + assertEquals(expectedBatch, context.listener.commitWithBaseOffset(baseOffset)); + baseOffset += expectedBatch.size(); + } + + return context.listener.currentClaimedEpoch().isPresent(); + }); - // Now that the listener has caught up to the start of the leader epoch, - // we expect the `handleClaim` callback. - context.client.poll(); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); + // Note that last committed offset is inclusive, hence we subtract 1. + assertEquals( + OptionalLong.of(expectedBatches.stream().mapToInt(List::size).sum() - 1), + context.listener.lastCommitOffset() + ); } @Test @@ -2302,20 +2328,21 @@ public void testLateRegisteredListenerCatchesUp() throws Exception { // Let the initial listener catch up context.deliverRequest(context.fetchRequest(epoch, otherNodeId, 10L, epoch, 0)); - context.client.poll(); + context.pollUntil(() -> OptionalInt.of(epoch).equals(context.listener.currentClaimedEpoch())); assertEquals(OptionalLong.of(10L), context.client.highWatermark()); - context.client.poll(); + assertEquals(OptionalLong.of(8L), context.listener.lastCommitOffset()); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); + // Ensure that the `handleClaim` callback was not fired early + assertEquals(9L, context.listener.claimedEpochStartOffset(epoch)); // Register a second listener and allow it to catch up to the high watermark RaftClientTestContext.MockListener secondListener = new RaftClientTestContext.MockListener(); context.client.register(secondListener); - context.client.poll(); + context.pollUntil(() -> OptionalInt.of(epoch).equals(secondListener.currentClaimedEpoch())); assertEquals(OptionalLong.of(8L), secondListener.lastCommitOffset()); assertEquals(OptionalInt.of(epoch), context.listener.currentClaimedEpoch()); - // Ensure that the `handleClaim` callback was not fired early - assertEquals(9L, context.listener.claimedEpochStartOffset(epoch)); + assertEquals(9L, secondListener.claimedEpochStartOffset(epoch)); } @Test @@ -2406,6 +2433,7 @@ public void testHandleCommitCallbackFiresInVotedState() throws Exception { context.assertVotedCandidate(candidateEpoch, otherNodeId); // Note the offset is 8 because the record at offset 9 is a control record + context.pollUntil(() -> secondListener.lastCommitOffset().equals(OptionalLong.of(8L))); assertEquals(OptionalLong.of(8L), secondListener.lastCommitOffset()); assertEquals(OptionalInt.empty(), secondListener.currentClaimedEpoch()); } @@ -2455,6 +2483,7 @@ public void testHandleCommitCallbackFiresInCandidateState() throws Exception { context.assertVotedCandidate(candidateEpoch, localId); // Note the offset is 8 because the record at offset 9 is a control record + context.pollUntil(() -> secondListener.lastCommitOffset().equals(OptionalLong.of(8L))); assertEquals(OptionalLong.of(8L), secondListener.lastCommitOffset()); assertEquals(OptionalInt.empty(), secondListener.currentClaimedEpoch()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLog.java b/raft/src/test/java/org/apache/kafka/raft/MockLog.java index 3ca50b14d9657..bf03a06ee8f4f 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLog.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLog.java @@ -37,7 +37,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -363,20 +362,26 @@ public LogFetchInfo read(long startOffset, Isolation isolation) { } ByteBuffer buffer = ByteBuffer.allocate(512); - LogEntry firstEntry = null; + int batchCount = 0; + LogOffsetMetadata batchStartOffset = null; for (LogBatch batch : batches) { // Note that start offset is inclusive while max offset is exclusive. We only return // complete batches, so batches which end at an offset larger than the max offset are // filtered, which is effectively the same as having the consumer drop an incomplete // batch returned in a fetch response. - if (batch.lastOffset() >= startOffset) { - if (batch.lastOffset() < maxOffset) { - buffer = batch.writeTo(buffer); + if (batch.lastOffset() >= startOffset && batch.lastOffset() < maxOffset && !batch.entries.isEmpty()) { + buffer = batch.writeTo(buffer); + + if (batchStartOffset == null) { + batchStartOffset = batch.entries.get(0).logOffsetMetadata(); } - if (firstEntry == null && !batch.entries.isEmpty()) { - firstEntry = batch.entries.get(0); + // Read on the mock log should return at most 2 batches. This is a simple solution + // for testing interesting partial read scenarios. + batchCount += 1; + if (batchCount >= 2) { + break; } } } @@ -384,12 +389,12 @@ public LogFetchInfo read(long startOffset, Isolation isolation) { buffer.flip(); Records records = MemoryRecords.readableRecords(buffer); - if (firstEntry == null) { + if (batchStartOffset == null) { throw new RuntimeException("Expected to find at least one entry starting from offset " + startOffset + " but found none"); } - return new LogFetchInfo(records, firstEntry.logOffsetMetadata()); + return new LogFetchInfo(records, batchStartOffset); } @Override @@ -426,48 +431,50 @@ public Optional earliestSnapshotId() { public void onSnapshotFrozen(OffsetAndEpoch snapshotId) {} @Override - public boolean deleteBeforeSnapshot(OffsetAndEpoch logStartSnapshotId) { - if (logStartOffset() > logStartSnapshotId.offset || - highWatermark.offset < logStartSnapshotId.offset) { - + public boolean deleteBeforeSnapshot(OffsetAndEpoch snapshotId) { + if (logStartOffset() > snapshotId.offset) { + throw new OffsetOutOfRangeException( + String.format( + "New log start (%s) is less than the curent log start offset (%s)", + snapshotId, + logStartOffset() + ) + ); + } + if (highWatermark.offset < snapshotId.offset) { throw new OffsetOutOfRangeException( String.format( - "New log start (%s) is less than start offset (%s) or is greater than the high watermark (%s)", - logStartSnapshotId, - logStartOffset(), + "New log start (%s) is greater than the high watermark (%s)", + snapshotId, highWatermark.offset ) ); } boolean updated = false; - Optional snapshotIdOpt = latestSnapshotId(); - if (snapshotIdOpt.isPresent()) { - OffsetAndEpoch snapshotId = snapshotIdOpt.get(); - if (startOffset() < logStartSnapshotId.offset && - highWatermark.offset >= logStartSnapshotId.offset && - snapshotId.offset >= logStartSnapshotId.offset) { + if (snapshots.containsKey(snapshotId)) { + snapshots.headMap(snapshotId, false).clear(); - snapshots.headMap(logStartSnapshotId, false).clear(); + batches.removeIf(entry -> entry.lastOffset() < snapshotId.offset); - batches.removeIf(entry -> entry.lastOffset() < logStartSnapshotId.offset); - - AtomicReference> last = new AtomicReference<>(Optional.empty()); - epochStartOffsets.removeIf(epochStartOffset -> { - if (epochStartOffset.startOffset <= logStartSnapshotId.offset) { - last.set(Optional.of(epochStartOffset)); - return true; - } + AtomicReference> last = new AtomicReference<>(Optional.empty()); + epochStartOffsets.removeIf(epochStartOffset -> { + if (epochStartOffset.startOffset <= snapshotId.offset) { + last.set(Optional.of(epochStartOffset)); + return true; + } - return false; - }); + return false; + }); - last.get().ifPresent(epochStartOffset -> { - epochStartOffsets.add(0, new EpochStartOffset(epochStartOffset.epoch, logStartSnapshotId.offset)); - }); + last.get().ifPresent(epochStartOffset -> { + epochStartOffsets.add( + 0, + new EpochStartOffset(epochStartOffset.epoch, snapshotId.offset) + ); + }); - updated = true; - } + updated = true; } return updated; @@ -530,6 +537,16 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(metadata, offset, record); } + + @Override + public String toString() { + return String.format( + "LogEntry(metadata=%s, offset=%s, record=%s)", + metadata, + offset, + record + ); + } } static class LogBatch { @@ -582,6 +599,11 @@ ByteBuffer writeTo(ByteBuffer buffer) { builder.close(); return builder.buffer(); } + + @Override + public String toString() { + return String.format("LogBatch(entries=%s, epoch=%s, isControlBatch=%s)", entries, epoch, isControlBatch); + } } private static class EpochStartOffset { @@ -676,18 +698,18 @@ public long sizeInBytes() { } @Override - public Iterator iterator() { - return Utils.covariantCast(data.batchIterator()); - } - - @Override - public UnalignedRecords read(long position, int size) { + public UnalignedRecords slice(long position, int size) { ByteBuffer buffer = data.buffer(); buffer.position(Math.toIntExact(position)); buffer.limit(Math.min(buffer.limit(), Math.toIntExact(position + size))); return new UnalignedMemoryRecords(buffer.slice()); } + @Override + public Records records() { + return data; + } + @Override public void close() {} } diff --git a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java index d2af3408f362b..35e13cec65a01 100644 --- a/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/MockLogTest.java @@ -75,49 +75,6 @@ public void testTopicId() { assertEquals(topicId, log.topicId()); } - @Test - public void testAppendAsLeaderHelper() { - int epoch = 2; - SimpleRecord recordOne = new SimpleRecord("one".getBytes()); - appendAsLeader(Collections.singleton(recordOne), epoch); - assertEquals(epoch, log.lastFetchedEpoch()); - assertEquals(0L, log.startOffset()); - assertEquals(1L, log.endOffset().offset); - - Records records = log.read(0, Isolation.UNCOMMITTED).records; - List batches = Utils.toList(records.batches().iterator()); - - RecordBatch batch = batches.get(0); - assertEquals(0, batch.baseOffset()); - assertEquals(0, batch.lastOffset()); - - List fetchedRecords = Utils.toList(batch.iterator()); - assertEquals(1, fetchedRecords.size()); - assertEquals(recordOne, new SimpleRecord(fetchedRecords.get(0))); - assertEquals(0, fetchedRecords.get(0).offset()); - - SimpleRecord recordTwo = new SimpleRecord("two".getBytes()); - SimpleRecord recordThree = new SimpleRecord("three".getBytes()); - appendAsLeader(Arrays.asList(recordTwo, recordThree), epoch); - assertEquals(0L, log.startOffset()); - assertEquals(3L, log.endOffset().offset); - - records = log.read(0, Isolation.UNCOMMITTED).records; - batches = Utils.toList(records.batches().iterator()); - assertEquals(2, batches.size()); - - fetchedRecords = Utils.toList(records.records().iterator()); - assertEquals(3, fetchedRecords.size()); - assertEquals(recordOne, new SimpleRecord(fetchedRecords.get(0))); - assertEquals(0, fetchedRecords.get(0).offset()); - - assertEquals(recordTwo, new SimpleRecord(fetchedRecords.get(1))); - assertEquals(1, fetchedRecords.get(1).offset()); - - assertEquals(recordThree, new SimpleRecord(fetchedRecords.get(2))); - assertEquals(2, fetchedRecords.get(2).offset()); - } - @Test public void testTruncateTo() { int epoch = 2; @@ -174,28 +131,26 @@ public void testAssignEpochStartOffset() { @Test public void testAppendAsLeader() { - SimpleRecord recordFoo = new SimpleRecord("foo".getBytes()); - final int currentEpoch = 3; - final long initialOffset = log.endOffset().offset; + int epoch = 2; + SimpleRecord recordOne = new SimpleRecord("one".getBytes()); + List expectedRecords = new ArrayList<>(); - log.appendAsLeader( - MemoryRecords.withRecords(initialOffset, CompressionType.NONE, recordFoo), - currentEpoch - ); + expectedRecords.add(recordOne); + appendAsLeader(Collections.singleton(recordOne), epoch); - assertEquals(0, log.startOffset()); - assertEquals(1, log.endOffset().offset); - assertEquals(currentEpoch, log.lastFetchedEpoch()); + assertEquals(new OffsetAndEpoch(expectedRecords.size(), epoch), log.endOffsetForEpoch(epoch)); + assertEquals(epoch, log.lastFetchedEpoch()); + validateReadRecords(expectedRecords, log); - Records records = log.read(0, Isolation.UNCOMMITTED).records; - List extractRecords = new ArrayList<>(); - for (Record record : records.records()) { - extractRecords.add(record.value()); - } + SimpleRecord recordTwo = new SimpleRecord("two".getBytes()); + SimpleRecord recordThree = new SimpleRecord("three".getBytes()); + expectedRecords.add(recordTwo); + expectedRecords.add(recordThree); + appendAsLeader(Arrays.asList(recordTwo, recordThree), epoch); - assertEquals(1, extractRecords.size()); - assertEquals(recordFoo.value(), extractRecords.get(0)); - assertEquals(new OffsetAndEpoch(1, currentEpoch), log.endOffsetForEpoch(currentEpoch)); + assertEquals(new OffsetAndEpoch(expectedRecords.size(), epoch), log.endOffsetForEpoch(epoch)); + assertEquals(epoch, log.lastFetchedEpoch()); + validateReadRecords(expectedRecords, log); } @Test @@ -546,9 +501,9 @@ public void testUpdateLogStartOffsetWithMissingSnapshot() { public void testFailToIncreaseLogStartPastHighWatermark() throws IOException { int offset = 10; int epoch = 0; - OffsetAndEpoch snapshotId = new OffsetAndEpoch(2 * offset, 1 + epoch); + OffsetAndEpoch snapshotId = new OffsetAndEpoch(2 * offset, epoch); - appendBatch(offset, epoch); + appendBatch(3 * offset, epoch); log.updateHighWatermark(new LogOffsetMetadata(offset)); try (RawSnapshotWriter snapshot = log.createSnapshot(snapshotId)) { @@ -803,15 +758,32 @@ public void testValidateValidEpochAndOffset() { } private Optional readOffsets(long startOffset, Isolation isolation) { - Records records = log.read(startOffset, isolation).records; + // The current MockLog implementation reads at most one batch + long firstReadOffset = -1L; long lastReadOffset = -1L; - for (Record record : records.records()) { - if (firstReadOffset < 0) - firstReadOffset = record.offset(); - if (record.offset() > lastReadOffset) - lastReadOffset = record.offset(); + + long currentStart = startOffset; + boolean foundRecord = true; + while (foundRecord) { + foundRecord = false; + + Records records = log.read(currentStart, isolation).records; + for (Record record : records.records()) { + foundRecord = true; + + if (firstReadOffset < 0L) { + firstReadOffset = record.offset(); + } + + if (record.offset() > lastReadOffset) { + lastReadOffset = record.offset(); + } + } + + currentStart = lastReadOffset + 1; } + if (firstReadOffset < 0) { return Optional.empty(); } else { @@ -841,6 +813,11 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(startOffset, endOffset); } + + @Override + public String toString() { + return String.format("OffsetRange(startOffset=%s, endOffset=%s)", startOffset, endOffset); + } } private void appendAsLeader(Collection records, int epoch) { @@ -862,4 +839,30 @@ private void appendBatch(int numRecords, int epoch) { appendAsLeader(records, epoch); } + + private static void validateReadRecords(List expectedRecords, MockLog log) { + assertEquals(0L, log.startOffset()); + assertEquals(expectedRecords.size(), log.endOffset().offset); + + int currentOffset = 0; + while (currentOffset < log.endOffset().offset) { + Records records = log.read(currentOffset, Isolation.UNCOMMITTED).records; + List batches = Utils.toList(records.batches().iterator()); + + assertTrue(batches.size() > 0); + for (RecordBatch batch : batches) { + assertTrue(batch.countOrNull() > 0); + assertEquals(currentOffset, batch.baseOffset()); + assertEquals(currentOffset + batch.countOrNull() - 1, batch.lastOffset()); + + for (Record record : batch) { + assertEquals(currentOffset, record.offset()); + assertEquals(expectedRecords.get(currentOffset), new SimpleRecord(record)); + currentOffset += 1; + } + + assertEquals(currentOffset - 1, batch.lastOffset()); + } + } + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 024095af91de5..a70e5b638225b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -21,15 +21,15 @@ import org.apache.kafka.common.memory.MemoryPool; import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData; -import org.apache.kafka.common.message.DescribeQuorumResponseData; 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.FetchSnapshotResponseData; -import org.apache.kafka.common.message.LeaderChangeMessage; 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; @@ -55,6 +55,8 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.BatchBuilder; import org.apache.kafka.raft.internals.StringSerde; +import org.apache.kafka.snapshot.RawSnapshotWriter; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.mockito.Mockito; @@ -78,12 +80,12 @@ import java.util.stream.Collectors; import static org.apache.kafka.raft.RaftUtil.hasValidTopicPartition; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; public final class RaftClientTestContext { - private static final StringSerde STRING_SERDE = new StringSerde(); - + public final RecordSerde serde = Builder.SERDE; final TopicPartition metadataPartition = Builder.METADATA_PARTITION; final int electionBackoffMaxMs = Builder.ELECTION_BACKOFF_MAX_MS; final int fetchMaxWaitMs = Builder.FETCH_MAX_WAIT_MS; @@ -111,6 +113,7 @@ public final class RaftClientTestContext { public static final class Builder { static final int DEFAULT_ELECTION_TIMEOUT_MS = 10000; + private static final RecordSerde SERDE = new StringSerde(); private static final TopicPartition METADATA_PARTITION = new TopicPartition("metadata", 0); private static final int ELECTION_BACKOFF_MAX_MS = 100; private static final int FETCH_MAX_WAIT_MS = 0; @@ -184,6 +187,22 @@ Builder appendToLog(int epoch, List records) { return this; } + Builder withEmptySnapshot(OffsetAndEpoch snapshotId) throws IOException { + try (RawSnapshotWriter snapshot = log.createSnapshot(snapshotId)) { + snapshot.freeze(); + } + return this; + } + + Builder deleteBeforeSnapshot(OffsetAndEpoch snapshotId) throws IOException { + if (snapshotId.offset > log.highWatermark().offset) { + log.updateHighWatermark(new LogOffsetMetadata(snapshotId.offset)); + } + log.deleteBeforeSnapshot(snapshotId); + + return this; + } + Builder withElectionTimeoutMs(int electionTimeoutMs) { this.electionTimeoutMs = electionTimeoutMs; return this; @@ -210,7 +229,7 @@ public RaftClientTestContext build() throws IOException { ELECTION_BACKOFF_MAX_MS, FETCH_TIMEOUT_MS, appendLingerMs); KafkaRaftClient client = new KafkaRaftClient<>( - STRING_SERDE, + SERDE, channel, messageQueue, log, @@ -307,7 +326,7 @@ static MemoryRecords buildBatch( ByteBuffer buffer = ByteBuffer.allocate(512); BatchBuilder builder = new BatchBuilder<>( buffer, - STRING_SERDE, + Builder.SERDE, CompressionType.NONE, baseOffset, timestamp, @@ -1033,9 +1052,12 @@ FetchResponseData divergingFetchResponse( } static class MockListener implements RaftClient.Listener { - private final List> commits = new ArrayList<>(); + private final List> commits = new ArrayList<>(); + private final List> savedBatches = new ArrayList<>(); private final Map claimedEpochStartOffsets = new HashMap<>(); private OptionalInt currentClaimedEpoch = OptionalInt.empty(); + private Optional> snapshot = Optional.empty(); + private boolean readCommit = true; int numCommittedBatches() { return commits.size(); @@ -1045,7 +1067,7 @@ Long claimedEpochStartOffset(int epoch) { return claimedEpochStartOffsets.get(epoch); } - BatchReader.Batch lastCommit() { + Batch lastCommit() { if (commits.isEmpty()) { return null; } else { @@ -1081,6 +1103,42 @@ List commitWithLastOffset(long lastOffset) { .orElse(null); } + Optional> drainHandledSnapshot() { + Optional> temp = snapshot; + snapshot = Optional.empty(); + return temp; + } + + void updateReadCommit(boolean readCommit) { + this.readCommit = readCommit; + + if (readCommit) { + for (BatchReader batch : savedBatches) { + readBatch(batch); + } + + savedBatches.clear(); + } + } + + void readBatch(BatchReader reader) { + try { + while (reader.hasNext()) { + long nextOffset = lastCommitOffset().isPresent() ? + lastCommitOffset().getAsLong() + 1 : 0L; + Batch batch = reader.next(); + // We expect monotonic offsets, but not necessarily sequential + // offsets since control records will be filtered. + assertTrue(batch.baseOffset() >= nextOffset, + "Received non-monotonic commit " + batch + + ". We expected an offset at least as large as " + nextOffset); + commits.add(batch); + } + } finally { + reader.close(); + } + } + @Override public void handleClaim(int epoch) { // We record the next expected offset as the claimed epoch's start @@ -1099,22 +1157,20 @@ public void handleResign(int epoch) { @Override public void handleCommit(BatchReader reader) { - try { - while (reader.hasNext()) { - long nextOffset = lastCommitOffset().isPresent() ? - lastCommitOffset().getAsLong() + 1 : 0L; - BatchReader.Batch batch = reader.next(); - // We expect monotonic offsets, but not necessarily sequential - // offsets since control records will be filtered. - assertTrue(batch.baseOffset() >= nextOffset, - "Received non-monotonic commit " + batch + - ". We expected an offset at least as large as " + nextOffset); - commits.add(batch); - } - } finally { - reader.close(); + if (readCommit) { + readBatch(reader); + } else { + savedBatches.add(reader); } } - } + @Override + public void handleSnapshot(SnapshotReader reader) { + snapshot.ifPresent(snapshot -> assertDoesNotThrow(() -> snapshot.close())); + + commits.clear(); + savedBatches.clear(); + snapshot = Optional.of(reader); + } + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 94b278e7cc4a1..32e701a3854e6 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Writable; import org.apache.kafka.common.protocol.types.Type; +import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -34,9 +35,9 @@ import org.apache.kafka.raft.MockLog.LogBatch; import org.apache.kafka.raft.MockLog.LogEntry; import org.apache.kafka.raft.internals.BatchMemoryPool; +import org.apache.kafka.snapshot.SnapshotReader; import org.junit.jupiter.api.Tag; -import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -53,11 +54,13 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.Collectors; 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.fail; @@ -347,6 +350,7 @@ private EventScheduler schedulerWithDefaultInvariants(Cluster cluster) { scheduler.addInvariant(new MonotonicEpoch(cluster)); scheduler.addInvariant(new MajorityReachedHighWatermark(cluster)); scheduler.addInvariant(new SingleLeader(cluster)); + scheduler.addInvariant(new SnapshotAtLogStart(cluster)); scheduler.addValidation(new ConsistentCommittedData(cluster)); return scheduler; } @@ -717,7 +721,8 @@ void start(int nodeId) { persistentState.store, logContext, time, - random + random, + serde ); node.initialize(); running.put(nodeId, node); @@ -735,6 +740,7 @@ private static class RaftNode { final ReplicatedCounter counter; final Time time; final Random random; + final RecordSerde intSerde; private RaftNode( int nodeId, @@ -745,7 +751,8 @@ private RaftNode( MockQuorumStateStore store, LogContext logContext, Time time, - Random random + Random random, + RecordSerde intSerde ) { this.nodeId = nodeId; this.client = client; @@ -757,15 +764,12 @@ private RaftNode( this.time = time; this.random = random; this.counter = new ReplicatedCounter(nodeId, client, logContext); + this.intSerde = intSerde; } void initialize() { - try { - client.register(this.counter); - client.initialize(); - } catch (IOException e) { - throw new RuntimeException(e); - } + client.register(this.counter); + client.initialize(); } void poll() { @@ -963,6 +967,52 @@ public void verify() { } } + private static class SnapshotAtLogStart implements Invariant { + final Cluster cluster; + + private SnapshotAtLogStart(Cluster cluster) { + this.cluster = cluster; + } + + @Override + public void verify() { + for (Map.Entry nodeEntry : cluster.nodes.entrySet()) { + int nodeId = nodeEntry.getKey(); + ReplicatedLog log = nodeEntry.getValue().log; + log.earliestSnapshotId().ifPresent(earliestSnapshotId -> { + long logStartOffset = log.startOffset(); + ValidOffsetAndEpoch validateOffsetAndEpoch = log.validateOffsetAndEpoch( + earliestSnapshotId.offset, + earliestSnapshotId.epoch + ); + + assertTrue( + logStartOffset <= earliestSnapshotId.offset, + () -> String.format( + "invalid log start offset (%s) and snapshotId offset (%s): nodeId = %s", + logStartOffset, + earliestSnapshotId.offset, + nodeId + ) + ); + assertEquals( + ValidOffsetAndEpoch.valid(earliestSnapshotId), + validateOffsetAndEpoch, + () -> String.format("invalid leader epoch cache: nodeId = %s", nodeId) + ); + + if (logStartOffset > 0) { + assertEquals( + logStartOffset, + earliestSnapshotId.offset, + () -> String.format("mising snapshot at log start offset: nodeId = %s", nodeId) + ); + } + }); + } + } + } + /** * Validating the committed data is expensive, so we do this as a {@link Validation}. We depend * on the following external invariants: @@ -986,14 +1036,44 @@ private int parseSequenceNumber(ByteBuffer value) { return (int) Type.INT32.read(value); } - private void assertCommittedData(int nodeId, KafkaRaftClient manager, MockLog log) { + private void assertCommittedData(RaftNode node) { + final int nodeId = node.nodeId; + final KafkaRaftClient manager = node.client; + final MockLog log = node.log; + OptionalLong highWatermark = manager.highWatermark(); if (!highWatermark.isPresent()) { // We cannot do validation if the current high watermark is unknown return; } - for (LogBatch batch : log.readBatches(0L, highWatermark)) { + AtomicLong startOffset = new AtomicLong(0); + log.earliestSnapshotId().ifPresent(snapshotId -> { + assertTrue(snapshotId.offset <= highWatermark.getAsLong()); + startOffset.set(snapshotId.offset); + + try (SnapshotReader snapshot = + SnapshotReader.of(log.readSnapshot(snapshotId).get(), node.intSerde, BufferSupplier.create(), Integer.MAX_VALUE)) { + // Expect only one batch with only one record + assertTrue(snapshot.hasNext()); + Batch batch = snapshot.next(); + assertFalse(snapshot.hasNext()); + assertEquals(1, batch.records().size()); + + // The snapshotId offset is an "end offset" + long offset = snapshotId.offset - 1; + int sequence = batch.records().get(0); + committedSequenceNumbers.putIfAbsent(offset, sequence); + + assertEquals( + committedSequenceNumbers.get(offset), + sequence, + String.format("Committed sequence at offset %s changed on node %s", offset, nodeId) + ); + } + }); + + for (LogBatch batch : log.readBatches(startOffset.get(), highWatermark)) { if (batch.isControlBatch) { continue; } @@ -1015,7 +1095,7 @@ private void assertCommittedData(int nodeId, KafkaRaftClient manager, M @Override public void validate() { - cluster.forAllRunning(node -> assertCommittedData(node.nodeId, node.client, node.log)); + cluster.forAllRunning(this::assertCommittedData); } } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java index 90cb52714198b..25b843129a9a2 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/MemoryBatchReaderTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft.internals; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -31,17 +32,22 @@ class MemoryBatchReaderTest { @Test public void testIteration() { - BatchReader.Batch batch1 = new BatchReader.Batch<>(0L, 1, - Arrays.asList("a", "b", "c")); - BatchReader.Batch batch2 = new BatchReader.Batch<>(3L, 2, - Arrays.asList("d", "e")); - BatchReader.Batch batch3 = new BatchReader.Batch<>(5L, 2, - Arrays.asList("f", "g", "h", "i")); + Batch batch1 = Batch.of( + 0L, 1, Arrays.asList("a", "b", "c") + ); + Batch batch2 = Batch.of( + 3L, 2, Arrays.asList("d", "e") + ); + Batch batch3 = Batch.of( + 5L, 2, Arrays.asList("f", "g", "h", "i") + ); @SuppressWarnings("unchecked") CloseListener> listener = Mockito.mock(CloseListener.class); MemoryBatchReader reader = new MemoryBatchReader<>( - Arrays.asList(batch1, batch2, batch3), listener); + Arrays.asList(batch1, batch2, batch3), + listener + ); assertEquals(0L, reader.baseOffset()); assertEquals(OptionalLong.of(8L), reader.lastOffset()); diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java index 78ffd51befaf5..e340738965e14 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsBatchReaderTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; -import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -34,7 +34,6 @@ import java.util.NoSuchElementException; import java.util.Set; -import static java.util.Arrays.asList; import static org.apache.kafka.test.TestUtils.tempFile; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -44,7 +43,6 @@ class RecordsBatchReaderTest { private static final int MAX_BATCH_BYTES = 128; - private final MockTime time = new MockTime(); private final StringSerde serde = new StringSerde(); @ParameterizedTest @@ -52,13 +50,9 @@ class RecordsBatchReaderTest { public void testReadFromMemoryRecords(CompressionType compressionType) { long baseOffset = 57; - List> batches = asList( - new BatchReader.Batch<>(baseOffset, 1, asList("a", "b", "c")), - new BatchReader.Batch<>(baseOffset + 3, 2, asList("d", "e")), - new BatchReader.Batch<>(baseOffset + 5, 2, asList("f")) - ); + List> batches = RecordsIteratorTest.createBatches(baseOffset); + MemoryRecords memRecords = RecordsIteratorTest.buildRecords(compressionType, batches); - MemoryRecords memRecords = buildRecords(compressionType, batches); testBatchReader(baseOffset, memRecords, batches); } @@ -67,13 +61,8 @@ public void testReadFromMemoryRecords(CompressionType compressionType) { public void testReadFromFileRecords(CompressionType compressionType) throws Exception { long baseOffset = 57; - List> batches = asList( - new BatchReader.Batch<>(baseOffset, 1, asList("a", "b", "c")), - new BatchReader.Batch<>(baseOffset + 3, 2, asList("d", "e")), - new BatchReader.Batch<>(baseOffset + 5, 2, asList("f")) - ); - - MemoryRecords memRecords = buildRecords(compressionType, batches); + List> batches = RecordsIteratorTest.createBatches(baseOffset); + MemoryRecords memRecords = RecordsIteratorTest.buildRecords(compressionType, batches); FileRecords fileRecords = FileRecords.open(tempFile()); fileRecords.append(memRecords); @@ -81,39 +70,10 @@ public void testReadFromFileRecords(CompressionType compressionType) throws Exce testBatchReader(baseOffset, fileRecords, batches); } - private MemoryRecords buildRecords( - CompressionType compressionType, - List> batches - ) { - ByteBuffer buffer = ByteBuffer.allocate(1024); - - for (BatchReader.Batch batch : batches) { - BatchBuilder builder = new BatchBuilder<>( - buffer, - serde, - compressionType, - batch.baseOffset(), - time.milliseconds(), - false, - batch.epoch(), - MAX_BATCH_BYTES - ); - - for (String record : batch.records()) { - builder.appendRecord(record, null); - } - - builder.build(); - } - - buffer.flip(); - return MemoryRecords.readableRecords(buffer); - } - private void testBatchReader( long baseOffset, Records records, - List> expectedBatches + List> expectedBatches ) { BufferSupplier bufferSupplier = Mockito.mock(BufferSupplier.class); Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>()); @@ -134,15 +94,16 @@ private void testBatchReader( @SuppressWarnings("unchecked") CloseListener> closeListener = Mockito.mock(CloseListener.class); - RecordsBatchReader reader = new RecordsBatchReader<>( + RecordsBatchReader reader = RecordsBatchReader.of( baseOffset, records, serde, bufferSupplier, + MAX_BATCH_BYTES, closeListener ); - for (BatchReader.Batch batch : expectedBatches) { + for (Batch batch : expectedBatches) { assertTrue(reader.hasNext()); assertEquals(batch, reader.next()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java new file mode 100644 index 0000000000000..e450b5266440c --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -0,0 +1,188 @@ +/* + * 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.internals; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Random; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import net.jqwik.api.ForAll; +import net.jqwik.api.Property; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.Records; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.Batch; +import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; +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.assertTrue; + +public final class RecordsIteratorTest { + private static final RecordSerde STRING_SERDE = new StringSerde(); + + private static Stream emptyRecords() throws IOException { + return Stream.of( + FileRecords.open(TestUtils.tempFile()), + MemoryRecords.EMPTY + ).map(Arguments::of); + } + + @ParameterizedTest + @MethodSource("emptyRecords") + void testEmptyRecords(Records records) throws IOException { + testIterator(Collections.emptyList(), records); + } + + @Property + public void testMemoryRecords( + @ForAll CompressionType compressionType, + @ForAll long seed + ) { + List> batches = createBatches(seed); + + MemoryRecords memRecords = buildRecords(compressionType, batches); + testIterator(batches, memRecords); + } + + @Property + public void testFileRecords( + @ForAll CompressionType compressionType, + @ForAll long seed + ) throws IOException { + List> batches = createBatches(seed); + + MemoryRecords memRecords = buildRecords(compressionType, batches); + FileRecords fileRecords = FileRecords.open(TestUtils.tempFile()); + fileRecords.append(memRecords); + + testIterator(batches, fileRecords); + } + + private void testIterator( + List> expectedBatches, + Records records + ) { + Set allocatedBuffers = Collections.newSetFromMap(new IdentityHashMap<>()); + + RecordsIterator iterator = createIterator( + records, + mockBufferSupplier(allocatedBuffers) + ); + + for (Batch batch : expectedBatches) { + assertTrue(iterator.hasNext()); + assertEquals(batch, iterator.next()); + } + + assertFalse(iterator.hasNext()); + assertThrows(NoSuchElementException.class, iterator::next); + + iterator.close(); + assertEquals(Collections.emptySet(), allocatedBuffers); + } + + static RecordsIterator createIterator(Records records, BufferSupplier bufferSupplier) { + return new RecordsIterator<>(records, STRING_SERDE, bufferSupplier, Records.HEADER_SIZE_UP_TO_MAGIC); + } + + static BufferSupplier mockBufferSupplier(Set buffers) { + BufferSupplier bufferSupplier = Mockito.mock(BufferSupplier.class); + + Mockito.when(bufferSupplier.get(Mockito.anyInt())).thenAnswer(invocation -> { + int size = invocation.getArgument(0); + ByteBuffer buffer = ByteBuffer.allocate(size); + buffers.add(buffer); + return buffer; + }); + + Mockito.doAnswer(invocation -> { + ByteBuffer released = invocation.getArgument(0); + buffers.remove(released); + return null; + }).when(bufferSupplier).release(Mockito.any(ByteBuffer.class)); + + return bufferSupplier; + } + + public static List> createBatches(long seed) { + Random random = new Random(seed); + long baseOffset = random.nextInt(100); + int epoch = random.nextInt(3) + 1; + + int numberOfBatches = random.nextInt(100) + 1; + List> batches = new ArrayList<>(numberOfBatches); + for (int i = 0; i < numberOfBatches; i++) { + int numberOfRecords = random.nextInt(100) + 1; + List records = random + .ints(numberOfRecords, 0, 10) + .mapToObj(String::valueOf) + .collect(Collectors.toList()); + + batches.add(Batch.of(baseOffset, epoch, records)); + baseOffset += records.size(); + if (i % 5 == 0) { + epoch += random.nextInt(3); + } + } + + return batches; + } + + public static MemoryRecords buildRecords( + CompressionType compressionType, + List> batches + ) { + ByteBuffer buffer = ByteBuffer.allocate(102400); + + for (Batch batch : batches) { + BatchBuilder builder = new BatchBuilder<>( + buffer, + STRING_SERDE, + compressionType, + batch.baseOffset(), + 12345L, + false, + batch.epoch(), + 1024 + ); + + for (String record : batch.records()) { + builder.appendRecord(record, null); + } + + builder.build(); + } + + buffer.flip(); + return MemoryRecords.readableRecords(buffer); + } +} diff --git a/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java b/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java index dc4f6359d6cd1..ef38d465c5440 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/FileRawSnapshotTest.java @@ -62,14 +62,14 @@ public void tearDown() throws IOException { public void testWritingSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; int expectedSize = 0; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { assertEquals(0, snapshot.sizeInBytes()); UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); expectedSize += records.sizeInBytes(); } @@ -88,13 +88,13 @@ public void testWritingSnapshot() throws IOException { public void testWriteReadSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; ByteBuffer expectedBuffer = ByteBuffer.wrap(randomBytes(bufferSize)); try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(expectedBuffer); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } @@ -104,7 +104,10 @@ public void testWriteReadSnapshot() throws IOException { try (FileRawSnapshotReader snapshot = FileRawSnapshotReader.open(tempDir, offsetAndEpoch)) { int countBatches = 0; int countRecords = 0; - for (RecordBatch batch : snapshot) { + + Iterator batches = Utils.covariantCast(snapshot.records().batchIterator()); + while (batches.hasNext()) { + RecordBatch batch = batches.next(); countBatches += 1; Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); @@ -120,8 +123,8 @@ public void testWriteReadSnapshot() throws IOException { } } - assertEquals(batches, countBatches); - assertEquals(batches, countRecords); + assertEquals(numberOfBatches, countBatches); + assertEquals(numberOfBatches, countRecords); } } @@ -151,8 +154,8 @@ public void testPartialWriteReadSnapshot() throws IOException { int totalSize = Math.toIntExact(snapshot.sizeInBytes()); assertEquals(expectedBuffer.remaining(), totalSize); - UnalignedFileRecords record1 = (UnalignedFileRecords) snapshot.read(0, totalSize / 2); - UnalignedFileRecords record2 = (UnalignedFileRecords) snapshot.read(totalSize / 2, totalSize - totalSize / 2); + UnalignedFileRecords record1 = (UnalignedFileRecords) snapshot.slice(0, totalSize / 2); + UnalignedFileRecords record2 = (UnalignedFileRecords) snapshot.slice(totalSize / 2, totalSize - totalSize / 2); assertEquals(buffer1, TestUtils.toBuffer(record1)); assertEquals(buffer2, TestUtils.toBuffer(record2)); @@ -170,10 +173,10 @@ public void testBatchWriteReadSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; int batchSize = 3; - int batches = 10; + int numberOfBatches = 10; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { ByteBuffer[] buffers = IntStream .range(0, batchSize) .mapToObj(ignore -> ByteBuffer.wrap(randomBytes(bufferSize))).toArray(ByteBuffer[]::new); @@ -187,7 +190,10 @@ public void testBatchWriteReadSnapshot() throws IOException { try (FileRawSnapshotReader snapshot = FileRawSnapshotReader.open(tempDir, offsetAndEpoch)) { int countBatches = 0; int countRecords = 0; - for (RecordBatch batch : snapshot) { + + Iterator batches = Utils.covariantCast(snapshot.records().batchIterator()); + while (batches.hasNext()) { + RecordBatch batch = batches.next(); countBatches += 1; Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); @@ -202,8 +208,8 @@ public void testBatchWriteReadSnapshot() throws IOException { } } - assertEquals(batches, countBatches); - assertEquals(batches * batchSize, countRecords); + assertEquals(numberOfBatches, countBatches); + assertEquals(numberOfBatches * batchSize, countRecords); } } @@ -212,11 +218,11 @@ public void testBufferWriteReadSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; int batchSize = 3; - int batches = 10; + int numberOfBatches = 10; int expectedSize = 0; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { ByteBuffer[] buffers = IntStream .range(0, batchSize) .mapToObj(ignore -> ByteBuffer.wrap(randomBytes(bufferSize))).toArray(ByteBuffer[]::new); @@ -239,7 +245,9 @@ public void testBufferWriteReadSnapshot() throws IOException { int countBatches = 0; int countRecords = 0; - for (RecordBatch batch : snapshot) { + Iterator batches = Utils.covariantCast(snapshot.records().batchIterator()); + while (batches.hasNext()) { + RecordBatch batch = batches.next(); countBatches += 1; Iterator records = batch.streamingIterator(new GrowableBufferSupplier()); @@ -254,8 +262,8 @@ public void testBufferWriteReadSnapshot() throws IOException { } } - assertEquals(batches, countBatches); - assertEquals(batches * batchSize, countRecords); + assertEquals(numberOfBatches, countBatches); + assertEquals(numberOfBatches * batchSize, countRecords); } } @@ -263,11 +271,11 @@ public void testBufferWriteReadSnapshot() throws IOException { public void testAbortedSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(20L, 2); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } } @@ -281,11 +289,11 @@ public void testAbortedSnapshot() throws IOException { public void testAppendToFrozenSnapshot() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(10L, 3); int bufferSize = 256; - int batches = 10; + int numberOfBatches = 10; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } @@ -296,18 +304,18 @@ public void testAppendToFrozenSnapshot() throws IOException { // File should exist and the size should be greater than the sum of all the buffers assertTrue(Files.exists(Snapshots.snapshotPath(tempDir, offsetAndEpoch))); - assertTrue(Files.size(Snapshots.snapshotPath(tempDir, offsetAndEpoch)) > bufferSize * batches); + assertTrue(Files.size(Snapshots.snapshotPath(tempDir, offsetAndEpoch)) > bufferSize * numberOfBatches); } @Test public void testCreateSnapshotWithSameId() throws IOException { OffsetAndEpoch offsetAndEpoch = new OffsetAndEpoch(20L, 2); int bufferSize = 256; - int batches = 1; + int numberOfBatches = 1; try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } @@ -317,7 +325,7 @@ public void testCreateSnapshotWithSameId() throws IOException { // Create another snapshot with the same id try (FileRawSnapshotWriter snapshot = createSnapshotWriter(tempDir, offsetAndEpoch)) { UnalignedMemoryRecords records = buildRecords(ByteBuffer.wrap(randomBytes(bufferSize))); - for (int i = 0; i < batches; i++) { + for (int i = 0; i < numberOfBatches; i++) { snapshot.append(records); } diff --git a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java similarity index 77% rename from raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java rename to raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java index 27bdff2fda02d..be862101d73ba 100644 --- a/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterTest.java +++ b/raft/src/test/java/org/apache/kafka/snapshot/SnapshotWriterReaderTest.java @@ -20,19 +20,20 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.Set; -import org.apache.kafka.common.utils.BufferSupplier.GrowableBufferSupplier; -import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.OffsetAndEpoch; import org.apache.kafka.raft.RaftClientTestContext; +import org.apache.kafka.raft.internals.StringSerde; import org.junit.jupiter.api.Test; 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.assertThrows; -final public class SnapshotWriterTest { +final public class SnapshotWriterReaderTest { private final int localId = 0; private final Set voters = Collections.singleton(localId); @@ -49,7 +50,7 @@ public void testWritingSnapshot() throws IOException { snapshot.freeze(); } - try (RawSnapshotReader reader = context.log.readSnapshot(id).get()) { + try (SnapshotReader reader = readSnapshot(context, id, Integer.MAX_VALUE)) { assertSnapshot(expected, reader); } } @@ -66,7 +67,7 @@ public void testAbortedSnapshot() throws IOException { }); } - assertFalse(context.log.readSnapshot(id).isPresent()); + assertEquals(Optional.empty(), context.log.readSnapshot(id)); } @Test @@ -100,16 +101,37 @@ private List> buildRecords(int recordsPerBatch, int batches) { return result; } + private SnapshotReader readSnapshot( + RaftClientTestContext context, + OffsetAndEpoch snapshotId, + int maxBatchSize + ) { + return SnapshotReader.of( + context.log.readSnapshot(snapshotId).get(), + context.serde, + BufferSupplier.create(), + maxBatchSize + ); + } + public static void assertSnapshot(List> batches, RawSnapshotReader reader) { + assertSnapshot( + batches, + SnapshotReader.of(reader, new StringSerde(), BufferSupplier.create(), Integer.MAX_VALUE) + ); + } + + public static void assertSnapshot(List> batches, SnapshotReader reader) { List expected = new ArrayList<>(); batches.forEach(expected::addAll); List actual = new ArrayList<>(expected.size()); - reader.forEach(batch -> { - batch.streamingIterator(new GrowableBufferSupplier()).forEachRemaining(record -> { - actual.add(Utils.utf8(record.value())); - }); - }); + while (reader.hasNext()) { + Batch batch = reader.next(); + for (String value : batch) { + actual.add(value); + } + } assertEquals(expected, actual); } diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index 739e0278d5cda..dde0e40defb2c 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -41,8 +41,10 @@ import org.apache.kafka.metalog.MetaLogListener; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.KafkaEventQueue; +import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.shell.MetadataNode.DirectoryNode; import org.apache.kafka.shell.MetadataNode.FileNode; import org.slf4j.Logger; @@ -83,7 +85,7 @@ public void handleCommit(BatchReader reader) { try { // TODO: handle lastOffset while (reader.hasNext()) { - BatchReader.Batch batch = reader.next(); + Batch batch = reader.next(); for (ApiMessageAndVersion messageAndVersion : batch.records()) { handleMessage(messageAndVersion.message()); } @@ -105,6 +107,20 @@ public void handleCommits(long lastOffset, List messages) { }, null); } + @Override + public void handleSnapshot(SnapshotReader reader) { + try { + while (reader.hasNext()) { + Batch batch = reader.next(); + for (ApiMessageAndVersion messageAndVersion : batch) { + handleMessage(messageAndVersion.message()); + } + } + } finally { + reader.close(); + } + } + @Override public void handleNewLeader(MetaLogLeader leader) { appendEvent("handleNewLeader", () -> { From e73731d8486735c882a721b92df450eb4be59f15 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Sun, 2 May 2021 13:07:08 +0800 Subject: [PATCH 110/155] KAFKA-12661 ConfigEntry#equal does not compare other fields when value is NOT null (#10446) Reviewers: Ismael Juma --- .../kafka/clients/admin/ConfigEntry.java | 18 ++++++++++++------ .../apache/kafka/clients/admin/ConfigTest.java | 15 +++++++++++++++ .../unit/kafka/admin/ConfigCommandTest.scala | 9 ++++++++- 3 files changed, 35 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java index 0fed04f6f5225..e3426662e801d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ConfigEntry.java @@ -150,11 +150,13 @@ public boolean equals(Object o) { ConfigEntry that = (ConfigEntry) o; return this.name.equals(that.name) && - this.value != null ? this.value.equals(that.value) : that.value == null && + Objects.equals(this.value, that.value) && this.isSensitive == that.isSensitive && this.isReadOnly == that.isReadOnly && - this.source == that.source && - Objects.equals(this.synonyms, that.synonyms); + Objects.equals(this.source, that.source) && + Objects.equals(this.synonyms, that.synonyms) && + Objects.equals(this.type, that.type) && + Objects.equals(this.documentation, that.documentation); } @Override @@ -162,11 +164,13 @@ public int hashCode() { final int prime = 31; int result = 1; result = prime * result + name.hashCode(); - result = prime * result + ((value == null) ? 0 : value.hashCode()); + result = prime * result + Objects.hashCode(value); result = prime * result + (isSensitive ? 1 : 0); result = prime * result + (isReadOnly ? 1 : 0); - result = prime * result + source.hashCode(); - result = prime * result + synonyms.hashCode(); + result = prime * result + Objects.hashCode(source); + result = prime * result + Objects.hashCode(synonyms); + result = prime * result + Objects.hashCode(type); + result = prime * result + Objects.hashCode(documentation); return result; } @@ -179,6 +183,8 @@ public String toString() { ", isSensitive=" + isSensitive + ", isReadOnly=" + isReadOnly + ", synonyms=" + synonyms + + ", type=" + type + + ", documentation=" + documentation + ")"; } diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java index 4008a5438261e..59d1150ac3ba8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/ConfigTest.java @@ -82,4 +82,19 @@ public static ConfigEntry newConfigEntry(String name, String value, ConfigEntry. boolean isReadOnly, List synonyms) { return new ConfigEntry(name, value, source, isSensitive, isReadOnly, synonyms, ConfigType.UNKNOWN, null); } + + @Test + public void testHashCodeAndEqualsWithNull() { + ConfigEntry ce0 = new ConfigEntry("abc", null, null, false, false, null, null, null); + ConfigEntry ce1 = new ConfigEntry("abc", null, null, false, false, null, null, null); + assertEquals(ce0, ce1); + assertEquals(ce0.hashCode(), ce1.hashCode()); + } + + @Test + public void testEquals() { + ConfigEntry ce0 = new ConfigEntry("abc", null, ConfigEntry.ConfigSource.DEFAULT_CONFIG, false, false, null, null, null); + ConfigEntry ce1 = new ConfigEntry("abc", null, ConfigEntry.ConfigSource.DYNAMIC_BROKER_CONFIG, false, false, null, null, null); + assertNotEquals(ce0, ce1); + } } diff --git a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala index 8603fbec78942..fde8faabf4b66 100644 --- a/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala @@ -803,7 +803,14 @@ class ConfigCommandTest extends ZooKeeperTestHarness with Logging { new AlterConfigOp(newConfigEntry("min.insync.replicas", "2"), AlterConfigOp.OpType.SET), new AlterConfigOp(newConfigEntry("unclean.leader.election.enable", ""), AlterConfigOp.OpType.DELETE) ) - assertEquals(expectedConfigOps, alterConfigOps.asScala.toSet) + assertEquals(expectedConfigOps.size, alterConfigOps.size) + expectedConfigOps.foreach { expectedOp => + val actual = alterConfigOps.asScala.find(_.configEntry.name == expectedOp.configEntry.name) + assertNotEquals(actual, None) + assertEquals(expectedOp.opType, actual.get.opType) + assertEquals(expectedOp.configEntry.name, actual.get.configEntry.name) + assertEquals(expectedOp.configEntry.value, actual.get.configEntry.value) + } alteredConfigs = true alterResult } From 325cb8853b7682bfe065a96d8b7808f0ef6cb89e Mon Sep 17 00:00:00 2001 From: wenbingshen Date: Mon, 3 May 2021 00:44:32 +0800 Subject: [PATCH 111/155] MINOR: Clean up some redundant code from ReplicaManager (#10623) Reviewers: Chia-Ping Tsai --- core/src/main/scala/kafka/server/KafkaApis.scala | 1 - .../src/main/scala/kafka/server/ReplicaManager.scala | 4 +--- .../test/scala/unit/kafka/server/KafkaApisTest.scala | 2 -- .../scala/unit/kafka/server/ReplicaManagerTest.scala | 12 ++++++------ 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index ef8476381d18e..8abd7dbb92af3 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -284,7 +284,6 @@ class KafkaApis(val requestChannel: RequestChannel, request.context.correlationId, stopReplicaRequest.controllerId, stopReplicaRequest.controllerEpoch, - stopReplicaRequest.brokerEpoch, partitionStates) // Clear the coordinator caches in case we were the leader. In the case of a reassignment, we // cannot rely on the LeaderAndIsr API for this since it is only sent to active replicas. diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 5d05b9bb1bd93..d813241bf3058 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -325,7 +325,7 @@ class ReplicaManager(val config: KafkaConfig, // If inter-broker protocol (IBP) < 1.0, the controller will send LeaderAndIsrRequest V0 which does not include isNew field. // In this case, the broker receiving the request cannot determine whether it is safe to create a partition if a log directory has failed. - // Thus, we choose to halt the broker on any log diretory failure if IBP < 1.0 + // Thus, we choose to halt the broker on any log directory failure if IBP < 1.0 val haltBrokerOnFailure = config.interBrokerProtocolVersion < KAFKA_1_0_IV0 logDirFailureHandler = new LogDirFailureHandler("LogDirFailureHandler", haltBrokerOnFailure) logDirFailureHandler.start() @@ -349,7 +349,6 @@ class ReplicaManager(val config: KafkaConfig, def stopReplicas(correlationId: Int, controllerId: Int, controllerEpoch: Int, - brokerEpoch: Long, partitionStates: Map[TopicPartition, StopReplicaPartitionState] ): (mutable.Map[TopicPartition, Errors], Errors) = { replicaStateChangeLock synchronized { @@ -434,7 +433,6 @@ class ReplicaManager(val config: KafkaConfig, s"controller $controllerId with correlation id $correlationId " + s"epoch $controllerEpoch for partition $topicPartition due to an unexpected " + s"${e.getClass.getName} exception: ${e.getMessage}") - responseMap.put(topicPartition, Errors.forException(e)) } responseMap.put(topicPartition, Errors.forException(e)) } diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 7813c2dca1049..bd0de39286621 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -1674,7 +1674,6 @@ class KafkaApisTest { EasyMock.eq(request.context.correlationId), EasyMock.eq(controllerId), EasyMock.eq(controllerEpoch), - EasyMock.eq(brokerEpoch), EasyMock.eq(stopReplicaRequest.partitionStates().asScala) )).andReturn( (mutable.Map( @@ -2900,7 +2899,6 @@ class KafkaApisTest { EasyMock.eq(request.context.correlationId), EasyMock.eq(controllerId), EasyMock.eq(controllerEpoch), - EasyMock.eq(brokerEpochInRequest), EasyMock.eq(stopReplicaRequest.partitionStates().asScala) )).andStubReturn( (mutable.Map( diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index deb8827bc7dc8..5a6ea2ef8ddd4 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -1358,7 +1358,7 @@ class ReplicaManagerTest { // We have a fetch in purgatory, now receive a stop replica request and // assert that the fetch returns with a NOT_LEADER error - replicaManager.stopReplicas(2, 0, 0, brokerEpoch, + replicaManager.stopReplicas(2, 0, 0, mutable.Map(tp0 -> new StopReplicaPartitionState() .setPartitionIndex(tp0.partition) .setDeletePartition(true) @@ -1398,7 +1398,7 @@ class ReplicaManagerTest { Mockito.when(replicaManager.metadataCache.contains(tp0)).thenReturn(true) - replicaManager.stopReplicas(2, 0, 0, brokerEpoch, + replicaManager.stopReplicas(2, 0, 0, mutable.Map(tp0 -> new StopReplicaPartitionState() .setPartitionIndex(tp0.partition) .setDeletePartition(true) @@ -2022,7 +2022,7 @@ class ReplicaManagerTest { .setDeletePartition(false) ) - val (_, error) = replicaManager.stopReplicas(1, 0, 0, brokerEpoch, partitionStates) + val (_, error) = replicaManager.stopReplicas(1, 0, 0, partitionStates) assertEquals(Errors.STALE_CONTROLLER_EPOCH, error) } @@ -2050,7 +2050,7 @@ class ReplicaManagerTest { .setDeletePartition(false) ) - val (result, error) = replicaManager.stopReplicas(1, 0, 0, brokerEpoch, partitionStates) + val (result, error) = replicaManager.stopReplicas(1, 0, 0, partitionStates) assertEquals(Errors.NONE, error) assertEquals(Map(tp0 -> Errors.KAFKA_STORAGE_ERROR), result) } @@ -2090,7 +2090,7 @@ class ReplicaManagerTest { .setDeletePartition(deletePartitions) ) - val (result, error) = replicaManager.stopReplicas(1, 0, 0, brokerEpoch, partitionStates) + val (result, error) = replicaManager.stopReplicas(1, 0, 0, partitionStates) assertEquals(Errors.NONE, error) if (throwIOException && deletePartitions) { @@ -2217,7 +2217,7 @@ class ReplicaManagerTest { .setDeletePartition(deletePartition) ) - val (result, error) = replicaManager.stopReplicas(1, 0, 0, brokerEpoch, partitionStates) + val (result, error) = replicaManager.stopReplicas(1, 0, 0, partitionStates) assertEquals(Errors.NONE, error) assertEquals(Map(tp0 -> expectedOutput), result) From 711608b3b1ed409afc0a7a3c701861559dd7a416 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 2 May 2021 12:15:54 -0700 Subject: [PATCH 112/155] MINOR: Update test libraries and gradle plugins for better JDK 16/17 support (#10619) Details: * spotbugs gradle plugin from 4.6.0 to 4.7.1: https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.6.1 https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.6.2 https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.7.0 https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.7.1 * spotless gradle plugin from 5.10.2 to 5.12.4: https://github.com/diffplug/spotless/blob/gradle/5.12.4/CHANGES.md * test-retry gradle plugin from 1.2.0 to 1.2.1: https://github.com/gradle/test-retry-gradle-plugin/releases/tag/v1.2.1 * dependency check gradle plugin from 6.1.1 to 6.1.6: https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.2 https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.3 https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.4 https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.5 https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.6 * versions gradle plugin from 0.36.0 to 0.38.0: https://github.com/ben-manes/gradle-versions-plugin/releases/tag/v0.37.0 https://github.com/ben-manes/gradle-versions-plugin/releases/tag/v0.38.0 * easymock from 4.2 to 4.3: https://github.com/easymock/easymock/releases/tag/easymock-4.3 * mockito from 3.6.0 to 3.9.0: https://github.com/mockito/mockito/releases (too many releases to list them all individually) * spotbugs from 4.1.4 to 4.2.2: https://github.com/spotbugs/spotbugs/blob/4.2.2/CHANGELOG.md 4.2.3 has a regression that causes spurious errors related to `Random` usage. Reviewers: Chia-Ping Tsai --- build.gradle | 10 +++++----- gradle/dependencies.gradle | 6 +++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/build.gradle b/build.gradle index 2c21405394895..e7a3a2317d979 100644 --- a/build.gradle +++ b/build.gradle @@ -30,15 +30,15 @@ buildscript { } plugins { - id 'com.diffplug.spotless' version '5.10.2' - id 'com.github.ben-manes.versions' version '0.36.0' + id 'com.diffplug.spotless' version '5.12.4' + id 'com.github.ben-manes.versions' version '0.38.0' id 'idea' id 'java-library' - id 'org.owasp.dependencycheck' version '6.1.1' + id 'org.owasp.dependencycheck' version '6.1.6' id 'org.nosphere.apache.rat' version "0.7.0" - id "com.github.spotbugs" version '4.6.0' apply false - id 'org.gradle.test-retry' version '1.2.0' apply false + id "com.github.spotbugs" version '4.7.1' apply false + id 'org.gradle.test-retry' version '1.2.1' apply false id 'org.scoverage' version '5.0.0' apply false id 'com.github.johnrengelman.shadow' version '6.1.0' apply false } diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 3e54681c269c4..40dc41ad6bcd9 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -64,7 +64,7 @@ versions += [ gradle: "6.8.3", grgit: "4.1.0", httpclient: "4.5.13", - easymock: "4.2", + easymock: "4.3", jackson: "2.10.5", jacksonDatabind: "2.10.5.1", jacoco: "0.8.5", @@ -99,7 +99,7 @@ versions += [ lz4: "1.7.1", mavenArtifact: "3.6.3", metrics: "2.2.0", - mockito: "3.6.0", + mockito: "3.9.0", netty: "4.1.62.Final", powermock: "2.0.9", reflections: "0.9.12", @@ -110,7 +110,7 @@ versions += [ scoverage: "1.4.1", slf4j: "1.7.30", snappy: "1.1.8.1", - spotbugs: "4.1.4", + spotbugs: "4.2.2", zinc: "1.3.5", zookeeper: "3.5.9", zstd: "1.4.9-1" From 3ddc377b056bde60f549ef383f9a8af1152ac3d6 Mon Sep 17 00:00:00 2001 From: dengziming Date: Mon, 3 May 2021 12:41:03 +0800 Subject: [PATCH 113/155] MINOR: Remove duplicate method in test classes (#10535) 1. Remove duplicate serializing auto-generated data in RequestConvertToJsonTest, this is inspired by #9964 2. Remove RequestTestUtils.serializeRequestWithHeader since we added a AbstractRequest.serializeWithHeader in #10142 Reviewers: Chia-Ping Tsai --- .../common/requests/RequestTestUtils.java | 4 ---- .../kafka/server/IntegrationTestUtils.scala | 8 +++---- .../kafka/network/RequestChannelTest.scala | 5 ++-- .../network/RequestConvertToJsonTest.scala | 23 ++++++------------- .../unit/kafka/network/SocketServerTest.scala | 14 +++++------ .../AbstractApiVersionsRequestTest.scala | 10 ++++---- .../server/BaseClientQuotaManagerTest.scala | 15 +++++------- .../unit/kafka/server/BaseRequestTest.scala | 13 +++++------ .../kafka/server/ControllerApisTest.scala | 3 +-- .../kafka/server/ForwardingManagerTest.scala | 2 +- .../unit/kafka/server/KafkaApisTest.scala | 10 ++++---- .../unit/kafka/server/RequestQuotaTest.scala | 4 ++-- 12 files changed, 47 insertions(+), 64 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java index 33348214d9064..4ec1171f1109b 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java @@ -48,10 +48,6 @@ public static ByteBuffer serializeRequestHeader(RequestHeader header) { return buffer; } - public static ByteBuffer serializeRequestWithHeader(RequestHeader header, AbstractRequest request) { - return RequestUtils.serialize(header.data(), header.headerVersion(), request.data(), request.version()); - } - public static ByteBuffer serializeResponseWithHeader(AbstractResponse response, short version, int correlationId) { return response.serializeWithHeader(new ResponseHeader(correlationId, response.apiKey().responseHeaderVersion(version)), version); diff --git a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala index c240cbfe1e0b6..9a21e7f8178b6 100644 --- a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala +++ b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala @@ -21,7 +21,7 @@ import kafka.network.SocketServer import kafka.utils.{NotNothing, TestUtils} import org.apache.kafka.common.network.{ListenerName, Mode} import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, RequestTestUtils, ResponseHeader} +import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.Utils @@ -34,15 +34,15 @@ import scala.reflect.ClassTag object IntegrationTestUtils { - private def sendRequest(socket: Socket, request: Array[Byte]): Unit = { + def sendRequest(socket: Socket, request: Array[Byte]): Unit = { val outgoing = new DataOutputStream(socket.getOutputStream) outgoing.writeInt(request.length) outgoing.write(request) outgoing.flush() } - def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = { - val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request)) + private def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = { + val serializedBytes = Utils.toArray(request.serializeWithHeader(header)) sendRequest(socket, serializedBytes) } diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala index 49e24012d8e3f..78618c2bcea83 100644 --- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala @@ -32,8 +32,8 @@ import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData._ import org.apache.kafka.common.network.{ClientInformation, ListenerName} -import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests.AlterConfigsRequest._ +import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.easymock.EasyMock._ import org.junit.jupiter.api.Assertions._ @@ -190,8 +190,7 @@ class RequestChannelTest { } def request(req: AbstractRequest): RequestChannel.Request = { - val buffer = RequestTestUtils.serializeRequestWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1), - req) + val buffer = req.serializeWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1)) val requestContext = newRequestContext(buffer) new network.RequestChannel.Request(processor = 1, requestContext, diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala index e74e77d67ff43..9b8db57b646a6 100644 --- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala +++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala @@ -26,12 +26,12 @@ import kafka.network.RequestConvertToJson.requestHeaderNode import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.message._ import org.apache.kafka.common.network.{ClientInformation, ListenerName, NetworkSend} -import org.junit.jupiter.api.Test -import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, ObjectSerializationCache} +import org.apache.kafka.common.protocol.{ApiKeys, MessageUtil} import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.easymock.EasyMock.createNiceMock import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test import scala.collection.mutable.ArrayBuffer @@ -42,7 +42,6 @@ class RequestConvertToJsonTest { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach { key => { val version: Short = key.latestVersion() - val cache = new ObjectSerializationCache val message = key match { case ApiKeys.DESCRIBE_ACLS => ApiMessageType.fromApiKey(key.id).newRequest().asInstanceOf[DescribeAclsRequestData] @@ -50,12 +49,9 @@ class RequestConvertToJsonTest { case _ => ApiMessageType.fromApiKey(key.id).newRequest() } - val messageSize = message.size(cache, version) - val bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize)) - message.write(bytes, cache, version) - bytes.flip() - val req = AbstractRequest.parseRequest(key, version, bytes.buffer).request + val bytes = MessageUtil.toByteBuffer(message, version) + val req = AbstractRequest.parseRequest(key, version, bytes).request try { RequestConvertToJson.request(req) } catch { @@ -70,13 +66,9 @@ class RequestConvertToJsonTest { val unhandledKeys = ArrayBuffer[String]() ApiKeys.values().foreach { key => { val version: Short = key.latestVersion() - val cache = new ObjectSerializationCache val message = ApiMessageType.fromApiKey(key.id).newResponse() - val messageSize = message.size(cache, version) - val bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize)) - message.write(bytes, cache, version) - bytes.flip() - val res = AbstractResponse.parseResponse(key, bytes.buffer, version) + val bytes = MessageUtil.toByteBuffer(message, version) + val res = AbstractResponse.parseResponse(key, bytes, version) try { RequestConvertToJson.response(res, version) } catch { @@ -171,8 +163,7 @@ class RequestConvertToJsonTest { } def request(req: AbstractRequest): RequestChannel.Request = { - val buffer = RequestTestUtils.serializeRequestWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1), - req) + val buffer = req.serializeWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1)) val requestContext = newRequestContext(buffer) new network.RequestChannel.Request(processor = 1, requestContext, diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 52aaeef5a7d9e..01e7f202db5dd 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -114,7 +114,7 @@ class SocketServerTest { } def sendApiRequest(socket: Socket, request: AbstractRequest, header: RequestHeader): Unit = { - val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request)) + val serializedBytes = Utils.toArray(request.serializeWithHeader(header)) sendRequest(socket, serializedBytes) } @@ -140,7 +140,7 @@ class SocketServerTest { } def processRequest(channel: RequestChannel, request: RequestChannel.Request): Unit = { - val byteBuffer = RequestTestUtils.serializeRequestWithHeader(request.header, request.body[AbstractRequest]) + val byteBuffer = request.body[AbstractRequest].serializeWithHeader(request.header) val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(byteBuffer)) val headerLog = RequestConvertToJson.requestHeaderNode(request.header) channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(headerLog), None)) @@ -212,13 +212,13 @@ class SocketServerTest { .setTransactionalId(null)) .build() val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId) - Utils.toArray(RequestTestUtils.serializeRequestWithHeader(emptyHeader, emptyRequest)) + Utils.toArray(emptyRequest.serializeWithHeader(emptyHeader)) } private def apiVersionRequestBytes(clientId: String, version: Short): Array[Byte] = { val request = new ApiVersionsRequest.Builder().build(version) val header = new RequestHeader(ApiKeys.API_VERSIONS, request.version(), clientId, -1) - Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request)) + Utils.toArray(request.serializeWithHeader(header)) } @Test @@ -367,7 +367,7 @@ class SocketServerTest { val correlationId = 57 val header = new RequestHeader(ApiKeys.VOTE, 0, "", correlationId) val request = new VoteRequest.Builder(new VoteRequestData()).build() - val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request)) + val serializedBytes = Utils.toArray(request.serializeWithHeader(header)) val socket = connect() @@ -676,7 +676,7 @@ class SocketServerTest { // Mimic a primitive request handler that fetches the request from RequestChannel and place a response with a // throttled channel. val request = receiveRequest(server.dataPlaneRequestChannel) - val byteBuffer = RequestTestUtils.serializeRequestWithHeader(request.header, request.body[AbstractRequest]) + val byteBuffer = request.body[AbstractRequest].serializeWithHeader(request.header) val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(byteBuffer)) val channelThrottlingCallback = new ThrottleCallback { @@ -971,7 +971,7 @@ class SocketServerTest { .setTransactionalId(null)) .build() val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId) - val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(emptyHeader, emptyRequest)) + val serializedBytes = Utils.toArray(emptyRequest.serializeWithHeader(emptyHeader)) sendRequest(sslSocket, serializedBytes) processRequest(overrideServer.dataPlaneRequestChannel) diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 6a2bacab76f93..f0864430a9b49 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -16,16 +16,16 @@ */ package kafka.server -import java.util.Properties - import kafka.test.ClusterInstance import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} +import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, RequestUtils} +import org.apache.kafka.common.utils.Utils import org.junit.jupiter.api.Assertions._ +import java.util.Properties import scala.jdk.CollectionConverters._ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) { @@ -49,7 +49,9 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) { val overrideHeader = IntegrationTestUtils.nextRequestHeader(ApiKeys.API_VERSIONS, Short.MaxValue) val socket = IntegrationTestUtils.connect(cluster.brokerSocketServers().asScala.head, cluster.clientListener()) try { - IntegrationTestUtils.sendWithHeader(request, overrideHeader, socket) + val serializedBytes = Utils.toArray( + RequestUtils.serialize(overrideHeader.data, overrideHeader.headerVersion, request.data, request.version)) + IntegrationTestUtils.sendRequest(socket, serializedBytes) IntegrationTestUtils.receive[ApiVersionsResponse](socket, ApiKeys.API_VERSIONS, 0.toShort) } finally socket.close() } diff --git a/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala index 48379ca08aebf..1fe01bf4ecc5a 100644 --- a/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala @@ -19,18 +19,16 @@ package kafka.server import java.net.InetAddress import java.util import java.util.Collections + import kafka.network.RequestChannel import kafka.network.RequestChannel.Session import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.memory.MemoryPool -import org.apache.kafka.common.metrics.MetricConfig -import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.network.ClientInformation -import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader, RequestTestUtils} +import org.apache.kafka.common.metrics.{MetricConfig, Metrics} +import org.apache.kafka.common.network.{ClientInformation, ListenerName} import org.apache.kafka.common.requests.FetchRequest.PartitionData -import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader} +import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.MockTime import org.easymock.EasyMock import org.junit.jupiter.api.AfterEach @@ -57,8 +55,7 @@ class BaseClientQuotaManagerTest { listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)): (T, RequestChannel.Request) = { val request = builder.build() - val buffer = RequestTestUtils.serializeRequestWithHeader( - new RequestHeader(builder.apiKey, request.version, "", 0), request) + val buffer = request.serializeWithHeader(new RequestHeader(builder.apiKey, request.version, "", 0)) val requestChannelMetrics: RequestChannel.Metrics = EasyMock.createNiceMock(classOf[RequestChannel.Metrics]) // read the header from the buffer first so that the body can be read next from the Request constructor diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index 7b51bfeff2384..3d3d0ca49e53a 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -17,20 +17,19 @@ package kafka.server -import java.io.{DataInputStream, DataOutputStream} -import java.net.Socket -import java.nio.ByteBuffer -import java.util.Properties - import kafka.api.IntegrationTestHarness import kafka.network.SocketServer import kafka.utils.NotNothing import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys -import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, RequestTestUtils, ResponseHeader} +import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.BrokerState +import java.io.{DataInputStream, DataOutputStream} +import java.net.Socket +import java.nio.ByteBuffer +import java.util.Properties import scala.annotation.nowarn import scala.collection.Seq import scala.reflect.ClassTag @@ -136,7 +135,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness { } def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = { - val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request)) + val serializedBytes = Utils.toArray(request.serializeWithHeader(header)) sendRequest(socket, serializedBytes) } diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 44d82e40226f5..3ed4faa2a01a2 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -121,8 +121,7 @@ class ControllerApisTest { request: AbstractRequest, listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) ): RequestChannel.Request = { - val buffer = RequestTestUtils.serializeRequestWithHeader( - new RequestHeader(request.apiKey, request.version, clientID, 0), request) + val buffer = request.serializeWithHeader(new RequestHeader(request.apiKey, request.version, clientID, 0)) // read the header from the buffer first so that the body can be read next from the Request constructor val header = RequestHeader.parse(buffer) diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala index 2fefdac46c05c..d0fc30fbdec21 100644 --- a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala @@ -207,7 +207,7 @@ class ForwardingManagerTest { "clientId", correlationId ) - val buffer = RequestTestUtils.serializeRequestWithHeader(header, body) + val buffer = body.serializeWithHeader(header) // Fast-forward buffer to start of the request as `RequestChannel.Request` expects RequestHeader.parse(buffer) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index bd0de39286621..1f6e0134a062f 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -3109,14 +3109,14 @@ class KafkaApisTest { val listenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT) val requestHeader = new RequestHeader(request.apiKey, request.version, clientId, 0) - val requestBuffer = RequestTestUtils.serializeRequestWithHeader(requestHeader, request) + val requestBuffer = request.serializeWithHeader(requestHeader) val envelopeHeader = new RequestHeader(ApiKeys.ENVELOPE, ApiKeys.ENVELOPE.latestVersion(), clientId, 0) - val envelopeBuffer = RequestTestUtils.serializeRequestWithHeader(envelopeHeader, new EnvelopeRequest.Builder( + val envelopeBuffer = new EnvelopeRequest.Builder( requestBuffer, principalSerde.serialize(KafkaPrincipal.ANONYMOUS), InetAddress.getLocalHost.getAddress - ).build()) + ).build().serializeWithHeader(envelopeHeader) val envelopeContext = new RequestContext(envelopeHeader, "1", InetAddress.getLocalHost, KafkaPrincipal.ANONYMOUS, listenerName, SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY, fromPrivilegedListener, Optional.of(principalSerde)) @@ -3136,8 +3136,8 @@ class KafkaApisTest { listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), fromPrivilegedListener: Boolean = false, requestHeader: Option[RequestHeader] = None): RequestChannel.Request = { - val buffer = RequestTestUtils.serializeRequestWithHeader(requestHeader.getOrElse( - new RequestHeader(request.apiKey, request.version, clientId, 0)), request) + val buffer = request.serializeWithHeader( + requestHeader.getOrElse(new RequestHeader(request.apiKey, request.version, clientId, 0))) // read the header from the buffer first so that the body can be read next from the Request constructor val header = RequestHeader.parse(buffer) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index b992aac7f0631..35f6ff8b8c953 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -612,8 +612,8 @@ class RequestQuotaTest extends BaseRequestTest { "client-id", 0 ) - val embedRequestData = RequestTestUtils.serializeRequestWithHeader(requestHeader, - new AlterClientQuotasRequest.Builder(List.empty.asJava, false).build()) + val embedRequestData = new AlterClientQuotasRequest.Builder(List.empty.asJava, false).build() + .serializeWithHeader(requestHeader) new EnvelopeRequest.Builder(embedRequestData, new Array[Byte](0), InetAddress.getByName("192.168.1.1").getAddress) From 816f5c3b86ce94b1872f0d9d4bac9fdaa37b390f Mon Sep 17 00:00:00 2001 From: Vito Jeng Date: Tue, 4 May 2021 04:53:35 +0800 Subject: [PATCH 114/155] KAFKA-5876: KIP-216 Part 3, Apply StreamsNotStartedException for Interactive Queries (#10597) KIP-216 Part 3: Throw StreamsNotStartedException if KafkaStreams state is CREATED Reviewers: Anna Sophie Blee-Goldman --- checkstyle/suppressions.xml | 2 +- docs/streams/upgrade-guide.html | 9 ++- .../apache/kafka/streams/KafkaStreams.java | 15 ++++- .../kafka/streams/KafkaStreamsTest.java | 66 +++++++++++++++---- 4 files changed, 74 insertions(+), 18 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 1c64a6f7b298c..1093ac895aafd 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -158,7 +158,7 @@ files="StreamThread.java"/> + files="(KafkaStreams|KStreamImpl|KTableImpl).java"/> diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 02f258c1d176e..1990d721c8196 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -96,7 +96,14 @@

      Upgrade Guide and API Changes

      Streams API changes in 3.0.0

      - A new exception may be thrown from KafkaStreams#store(). If the specified store name does not exist in the topology, an UnknownStateStoreException will be thrown instead of the former InvalidStateStoreException. See KIP-216 for more information. + Interactive Queries may throw new exceptions for different errors: +

      +
        +
      • UnknownStateStoreException: If the specified store name does not exist in the topology, an UnknownStateStoreException will be thrown instead of the former InvalidStateStoreException.
      • +
      • StreamsNotStartedException: If Streams state is CREATED, a StreamsNotStartedException will be thrown.
      • +
      +

      + See KIP-216 for more information.

      We deprecated the StreamsConfig processing.guarantee configuration value "exactly_once" (for EOS version 1) in favor of the improved EOS version 2, formerly configured via diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index b7eda15dc8e24..e8a7fa2c1a260 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -42,6 +42,7 @@ import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.errors.StreamsNotStartedException; import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.errors.UnknownStateStoreException; @@ -343,10 +344,16 @@ private boolean isRunningOrRebalancing() { } private void validateIsRunningOrRebalancing() { - if (!isRunningOrRebalancing()) { - throw new IllegalStateException("KafkaStreams is not running. State is " + state + "."); + synchronized (stateLock) { + if (state == State.CREATED) { + throw new StreamsNotStartedException("KafkaStreams has not been started, you can retry after calling start()"); + } + if (!isRunningOrRebalancing()) { + throw new IllegalStateException("KafkaStreams is not running. State is " + state + "."); + } } } + /** * Listen to {@link State} change events. */ @@ -1516,6 +1523,8 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, * * @param storeQueryParameters the parameters used to fetch a queryable store * @return A facade wrapping the local {@link StateStore} instances + * @throws StreamsNotStartedException If Streams state is {@link KafkaStreams.State#CREATED CREATED}. Just + * retry and wait until to {@link KafkaStreams.State#RUNNING RUNNING}. * @throws UnknownStateStoreException If the specified store name does not exist in the topology. * @throws InvalidStateStoreException If the Streams instance isn't in a queryable state. * If the store's type does not match the QueryableStoreType, @@ -1524,6 +1533,7 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, * an InvalidStateStoreException is thrown upon store access. */ public T store(final StoreQueryParameters storeQueryParameters) { + validateIsRunningOrRebalancing(); final String storeName = storeQueryParameters.storeName(); if ((taskTopology == null || !taskTopology.hasStore(storeName)) && (globalTaskTopology == null || !globalTaskTopology.hasStore(storeName))) { @@ -1531,7 +1541,6 @@ public T store(final StoreQueryParameters storeQueryParameters) { "Cannot get state store " + storeName + " because no such store is registered in the topology." ); } - validateIsRunningOrRebalancing(); return queryableStoreProvider.getStore(storeQueryParameters); } diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index 82e8d6b2425c7..63fa5e7e0c378 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.StreamsNotStartedException; import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.errors.UnknownStateStoreException; @@ -94,6 +95,7 @@ import static java.util.Collections.singletonList; import static org.apache.kafka.streams.state.QueryableStoreTypes.keyValueStore; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; import static org.easymock.EasyMock.anyInt; import static org.easymock.EasyMock.anyLong; import static org.easymock.EasyMock.anyObject; @@ -116,6 +118,7 @@ public class KafkaStreamsTest { private static final int NUM_THREADS = 2; private final static String APPLICATION_ID = "appId"; private final static String CLIENT_ID = "test-client"; + private final static Duration DEFAULT_DURATION = Duration.ofSeconds(30); @Rule public TestName testName = new TestName(); @@ -717,21 +720,39 @@ public void shouldThrowOnCleanupWhileRunning() throws InterruptedException { } @Test - public void shouldNotGetAllTasksWhenNotRunning() { - final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); - assertThrows(IllegalStateException.class, streams::allMetadata); + public void shouldNotGetAllTasksWhenNotRunning() throws InterruptedException { + try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { + assertThrows(StreamsNotStartedException.class, streams::allMetadata); + streams.start(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.RUNNING, DEFAULT_DURATION); + streams.close(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.NOT_RUNNING, DEFAULT_DURATION); + assertThrows(IllegalStateException.class, streams::allMetadata); + } } @Test - public void shouldNotGetAllTasksWithStoreWhenNotRunning() { - final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); - assertThrows(IllegalStateException.class, () -> streams.allMetadataForStore("store")); + public void shouldNotGetAllTasksWithStoreWhenNotRunning() throws InterruptedException { + try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { + assertThrows(StreamsNotStartedException.class, () -> streams.allMetadataForStore("store")); + streams.start(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.RUNNING, DEFAULT_DURATION); + streams.close(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.NOT_RUNNING, DEFAULT_DURATION); + assertThrows(IllegalStateException.class, () -> streams.allMetadataForStore("store")); + } } @Test - public void shouldNotGetQueryMetadataWithSerializerWhenNotRunningOrRebalancing() { - final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); - assertThrows(IllegalStateException.class, () -> streams.queryMetadataForKey("store", "key", Serdes.String().serializer())); + public void shouldNotGetQueryMetadataWithSerializerWhenNotRunningOrRebalancing() throws InterruptedException { + try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { + assertThrows(StreamsNotStartedException.class, () -> streams.queryMetadataForKey("store", "key", Serdes.String().serializer())); + streams.start(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.RUNNING, DEFAULT_DURATION); + streams.close(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.NOT_RUNNING, DEFAULT_DURATION); + assertThrows(IllegalStateException.class, () -> streams.queryMetadataForKey("store", "key", Serdes.String().serializer())); + } } @Test @@ -742,19 +763,38 @@ public void shouldGetQueryMetadataWithSerializerWhenRunningOrRebalancing() { } @Test - public void shouldNotGetQueryMetadataWithPartitionerWhenNotRunningOrRebalancing() { - final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); - assertThrows(IllegalStateException.class, () -> streams.queryMetadataForKey("store", "key", (topic, key, value, numPartitions) -> 0)); + public void shouldNotGetQueryMetadataWithPartitionerWhenNotRunningOrRebalancing() throws InterruptedException { + try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { + assertThrows(StreamsNotStartedException.class, () -> streams.queryMetadataForKey("store", "key", (topic, key, value, numPartitions) -> 0)); + streams.start(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.RUNNING, DEFAULT_DURATION); + streams.close(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.NOT_RUNNING, DEFAULT_DURATION); + assertThrows(IllegalStateException.class, () -> streams.queryMetadataForKey("store", "key", (topic, key, value, numPartitions) -> 0)); + } } @Test - public void shouldThrowUnknownStateStoreExceptionWhenStoreNotExist() { + public void shouldThrowUnknownStateStoreExceptionWhenStoreNotExist() throws InterruptedException { try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { streams.start(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.RUNNING, DEFAULT_DURATION); assertThrows(UnknownStateStoreException.class, () -> streams.store(StoreQueryParameters.fromNameAndType("unknown-store", keyValueStore()))); } } + @Test + public void shouldNotGetStoreWhenWhenNotRunningOrRebalancing() throws InterruptedException { + try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { + assertThrows(StreamsNotStartedException.class, () -> streams.store(StoreQueryParameters.fromNameAndType("store", keyValueStore()))); + streams.start(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.RUNNING, DEFAULT_DURATION); + streams.close(); + waitForApplicationState(Collections.singletonList(streams), KafkaStreams.State.NOT_RUNNING, DEFAULT_DURATION); + assertThrows(IllegalStateException.class, () -> streams.store(StoreQueryParameters.fromNameAndType("store", keyValueStore()))); + } + } + @Test public void shouldReturnEmptyLocalStorePartitionLags() { // Mock all calls made to compute the offset lags, From 62221edaff412ca18cf32d81e82fd1f68752eaf6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Sergio=20Pe=C3=B1a?= Date: Mon, 3 May 2021 16:10:05 -0500 Subject: [PATCH 115/155] KAFKA-10847: Add internal flag to disable KAFKA-10847 fix (#10612) Adds an internal flag that can be used to disable the fixes in KAFKA-10847. It defaults to true if the flag is not set or has an invalid boolean value. The flag is named __enable.kstreams.outer.join.spurious.results.fix__. This flag is considered internal only. It is a temporary flag that will be used to help users to disable the join fixes while they do a transition from the previous semantics of left/outer joins. The flag may be removed in future releases. Reviewers: Guozhang Wang --- .../apache/kafka/streams/StreamsConfig.java | 15 +++ .../kstream/internals/KStreamKStreamJoin.java | 11 +- .../internals/graph/StreamStreamJoinNode.java | 8 +- .../internals/InternalTopologyBuilder.java | 17 +++ .../internals/KStreamKStreamLeftJoinTest.java | 124 ++++++++++++++++++ .../KStreamKStreamOuterJoinTest.java | 76 ++++++++++- .../InternalTopologyBuilderTest.java | 12 ++ 7 files changed, 255 insertions(+), 8 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index b619b1f6db340..89154c7b64c61 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -917,6 +917,21 @@ public static class InternalConfig { // This is settable in the main Streams config, but it's a private API for testing public static final String ASSIGNMENT_LISTENER = "__assignment.listener__"; + + // Private API used to disable the fix on left/outer joins (https://issues.apache.org/jira/browse/KAFKA-10847) + public static final String ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX = "__enable.kstreams.outer.join.spurious.results.fix__"; + + public static boolean getBoolean(final Map configs, final String key, final boolean defaultValue) { + final Object value = configs.getOrDefault(key, defaultValue); + if (value instanceof Boolean) { + return (boolean) value; + } else if (value instanceof String) { + return Boolean.parseBoolean((String) value); + } else { + log.warn("Invalid value (" + value + ") on internal configuration '" + key + "'. Please specify a true/false value."); + return defaultValue; + } + } } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index a27586774b138..14d0bdb97d17f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.AbstractProcessor; @@ -35,8 +36,8 @@ import org.slf4j.LoggerFactory; import java.util.Optional; -import java.util.function.Predicate; +import static org.apache.kafka.streams.StreamsConfig.InternalConfig.ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor; class KStreamKStreamJoin implements ProcessorSupplier { @@ -80,9 +81,6 @@ public Processor get() { } private class KStreamKStreamJoinProcessor extends AbstractProcessor { - private final Predicate>> recordWindowHasClosed = - windowedKey -> windowedKey.window().start() + joinAfterMs + joinGraceMs < maxObservedStreamTime.get(); - private WindowStore otherWindowStore; private StreamsMetricsImpl metrics; private Sensor droppedRecordsSensor; @@ -95,7 +93,10 @@ public void init(final ProcessorContext context) { metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensorOrSkippedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); otherWindowStore = context.getStateStore(otherWindowName); - outerJoinWindowStore = outerJoinWindowName.map(name -> context.getStateStore(name)); + + if (StreamsConfig.InternalConfig.getBoolean(context().appConfigs(), ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, true)) { + outerJoinWindowStore = outerJoinWindowName.map(name -> context.getStateStore(name)); + } } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java index eeba0c8211a4c..9de23781e7b6a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals.graph; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.Joined; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; @@ -27,6 +28,8 @@ import java.util.Optional; +import static org.apache.kafka.streams.StreamsConfig.InternalConfig.ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; + /** * Too much information to generalize, so Stream-Stream joins are represented by a specific node. */ @@ -95,7 +98,10 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { topologyBuilder.addStateStore(thisWindowStoreBuilder, thisWindowedStreamProcessorName, otherProcessorName); topologyBuilder.addStateStore(otherWindowStoreBuilder, otherWindowedStreamProcessorName, thisProcessorName); - outerJoinWindowStoreBuilder.ifPresent(builder -> topologyBuilder.addStateStore(builder, thisProcessorName, otherProcessorName)); + final StreamsConfig streamsConfig = topologyBuilder.getStreamsConfig(); + if (streamsConfig == null || StreamsConfig.InternalConfig.getBoolean(streamsConfig.originals(), ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, true)) { + outerJoinWindowStoreBuilder.ifPresent(builder -> topologyBuilder.addStateStore(builder, thisProcessorName, otherProcessorName)); + } } public static StreamStreamJoinNodeBuilder streamStreamJoinNodeBuilder() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 3e2f99dea20cf..6eb451aed80e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -127,6 +127,8 @@ public class InternalTopologyBuilder { private Map> nodeGroups = null; + private StreamsConfig config = null; + public static class StateStoreFactory { private final StoreBuilder builder; private final Set users = new HashSet<>(); @@ -339,6 +341,18 @@ public synchronized final InternalTopologyBuilder setApplicationId(final String return this; } + public synchronized final InternalTopologyBuilder setStreamsConfig(final StreamsConfig config) { + Objects.requireNonNull(config, "config can't be null"); + this.config = config; + + return this; + } + + public synchronized final StreamsConfig getStreamsConfig() { + return config; + } + + public synchronized final InternalTopologyBuilder rewriteTopology(final StreamsConfig config) { Objects.requireNonNull(config, "config can't be null"); @@ -361,6 +375,9 @@ public synchronized final InternalTopologyBuilder rewriteTopology(final StreamsC globalStateStores.put(storeBuilder.name(), storeBuilder.build()); } + // set streams config + setStreamsConfig(config); + return this; } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index b7f7df5117a96..5c4bccfdef747 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.TopologyWrapper; import org.apache.kafka.streams.kstream.Consumed; @@ -28,6 +29,7 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.kstream.StreamJoined; +import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -43,6 +45,9 @@ import java.util.Set; import static java.time.Duration.ofMillis; +import static org.apache.kafka.streams.StreamsConfig.InternalConfig.ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; public class KStreamKStreamLeftJoinTest { @@ -53,6 +58,125 @@ public class KStreamKStreamLeftJoinTest { private final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + @Test + public void testLeftJoinWithInvalidSpuriousResultFixFlag() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + props.put(ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, 5); + + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StreamsConfig.class); + final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + assertThat(appender.getMessages(), hasItem("Invalid value (5) on internal configuration " + + "'" + ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX + "'. Please specify a true/false value.")); + } + } + + @Test + public void testLeftJoinWithSpuriousResultFixDisabled() { + final StreamsBuilder builder = new StreamsBuilder(); + + final int[] expectedKeys = new int[] {0, 1, 2, 3}; + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + props.put(ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, false); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + // push two items to the primary stream; the other window is empty + // w1 {} + // w2 {} + // --> w1 = { 0:A0, 1:A1 } + // --> w2 = {} + for (int i = 0; i < 2; i++) { + inputTopic1.pipeInput(expectedKeys[i], "A" + expectedKeys[i]); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(1, "A1+null", 0)); + + // push two items to the other stream; this should produce two items + // w1 = { 0:A0, 1:A1 } + // w2 {} + // --> w1 = { 0:A0, 1:A1 } + // --> w2 = { 0:a0, 1:a1 } + for (int i = 0; i < 2; i++) { + inputTopic2.pipeInput(expectedKeys[i], "a" + expectedKeys[i]); + } + processor.checkAndClearProcessResult(new KeyValueTimestamp<>(0, "A0+a0", 0), + new KeyValueTimestamp<>(1, "A1+a1", 0)); + } + } + + @Test + public void testLeftJoinDuplicatesWithFixDisabled() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(10)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + props.put(ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, false); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + inputTopic1.pipeInput(0, "A0", 0); + inputTopic1.pipeInput(0, "A0-0", 0); + inputTopic2.pipeInput(0, "a0", 0); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(0, "A0-0+null", 0), + new KeyValueTimestamp<>(0, "A0+a0", 0), + new KeyValueTimestamp<>(0, "A0-0+a0", 0)); + } + } + @Test public void testLeftJoinDuplicates() { final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java index 36cf5877e4c19..c0f88d88944d1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.KeyValueTimestamp; import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.TopologyWrapper; @@ -28,6 +29,7 @@ import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.StreamJoined; +import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -43,16 +45,86 @@ import java.util.Set; import static java.time.Duration.ofMillis; +import static org.apache.kafka.streams.StreamsConfig.InternalConfig.ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; public class KStreamKStreamOuterJoinTest { - private final static KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0]; - private final String topic1 = "topic1"; private final String topic2 = "topic2"; private final Consumed consumed = Consumed.with(Serdes.Integer(), Serdes.String()); private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String()); + @Test + public void testOuterJoinWithInvalidSpuriousResultFixFlag() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + props.put(ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, 5); + + try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(StreamsConfig.class); + final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + assertThat(appender.getMessages(), hasItem("Invalid value (5) on internal configuration " + + "'" + ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX + "'. Please specify a true/false value.")); + } + } + + @Test + public void testOuterJoinDuplicatesWithFixDisabled() { + final StreamsBuilder builder = new StreamsBuilder(); + + final KStream stream1; + final KStream stream2; + final KStream joined; + final MockProcessorSupplier supplier = new MockProcessorSupplier<>(); + stream1 = builder.stream(topic1, consumed); + stream2 = builder.stream(topic2, consumed); + + joined = stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + JoinWindows.of(ofMillis(100)).grace(ofMillis(10)), + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joined.process(supplier); + + props.put(ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, false); + + try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(props), props)) { + final TestInputTopic inputTopic1 = + driver.createInputTopic(topic1, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final TestInputTopic inputTopic2 = + driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); + final MockProcessor processor = supplier.theCapturedProcessor(); + + inputTopic1.pipeInput(0, "A0", 0); + inputTopic1.pipeInput(0, "A0-0", 0); + inputTopic2.pipeInput(0, "a0", 0); + inputTopic2.pipeInput(1, "b1", 0); + + processor.checkAndClearProcessResult( + new KeyValueTimestamp<>(0, "A0+null", 0), + new KeyValueTimestamp<>(0, "A0-0+null", 0), + new KeyValueTimestamp<>(0, "A0+a0", 0), + new KeyValueTimestamp<>(0, "A0-0+a0", 0), + new KeyValueTimestamp<>(1, "null+b1", 0)); + } + } + + @Test public void testOuterJoinDuplicates() { final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 3992f818578d8..2d5cde58983d8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -767,6 +767,11 @@ public void shouldNotSetApplicationIdToNull() { assertThrows(NullPointerException.class, () -> builder.setApplicationId(null)); } + @Test + public void shouldNotSetStreamsConfigToNull() { + assertThrows(NullPointerException.class, () -> builder.setStreamsConfig(null)); + } + @Test public void shouldNotAddNullStateStoreSupplier() { assertThrows(NullPointerException.class, () -> builder.addStateStore(null)); @@ -905,6 +910,13 @@ public void shouldSetCorrectSourceNodesWithRegexUpdatedTopics() { assertTrue(topicGroups.get(2).sourceTopics.contains("topic-3")); } + @Test + public void shouldSetStreamsConfigOnRewriteTopology() { + final StreamsConfig config = new StreamsConfig(StreamsTestUtils.getStreamsConfig()); + final InternalTopologyBuilder topologyBuilder = builder.rewriteTopology(config); + assertThat(topologyBuilder.getStreamsConfig(), equalTo(config)); + } + @Test public void shouldAddTimestampExtractorPerSource() { builder.addSource(null, "source", new MockTimestampExtractor(), null, null, "topic"); From 45f24c41954e5bfeeec5b7050ee8fb873dd8104a Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Tue, 4 May 2021 17:23:30 +0100 Subject: [PATCH 116/155] KAFKA-12450: Remove deprecated methods from ReadOnlyWindowStore (#10294) Implement first part of https://cwiki.apache.org/confluence/display/KAFKA/KIP-667%3A+Remove+deprecated+methods+from+ReadOnlyWindowStore. Reviewers: Guozhang Wang --- .../streams/state/ReadOnlyWindowStore.java | 67 ------------------- .../CompositeReadOnlyWindowStore.java | 54 +++------------ .../internals/ReadOnlyWindowStoreFacade.java | 24 ------- .../kafka/streams/state/NoOpWindowStore.java | 21 ------ .../ReadOnlyWindowStoreFacadeTest.java | 12 ++-- .../internals/ReadOnlyWindowStoreStub.java | 42 ++---------- .../kafka/streams/TopologyTestDriver.java | 37 ++++++++-- 7 files changed, 50 insertions(+), 207 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index c0aae15d9430d..aa84bfcf5f9fe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -46,43 +46,6 @@ public interface ReadOnlyWindowStore { */ V fetch(K key, long time); - /** - * Get all the key-value pairs with the given key and the time range from all the existing windows. - *

      - * This iterator must be closed after use. - *

      - * The time range is inclusive and applies to the starting timestamp of the window. - * For example, if we have the following windows: - *

      -     * +-------------------------------+
      -     * |  key  | start time | end time |
      -     * +-------+------------+----------+
      -     * |   A   |     10     |    20    |
      -     * +-------+------------+----------+
      -     * |   A   |     15     |    25    |
      -     * +-------+------------+----------+
      -     * |   A   |     20     |    30    |
      -     * +-------+------------+----------+
      -     * |   A   |     25     |    35    |
      -     * +--------------------------------
      -     * 
      - * And we call {@code store.fetch("A", 10, 20)} then the results will contain the first - * three windows from the table above, i.e., all those where 10 <= start time <= 20. - *

      - * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest - * available window to the newest/latest window. - * - * @param key the key to fetch - * @param timeFrom time range start (inclusive), where iteration starts. - * @param timeTo time range end (inclusive), where iteration ends. - * @return an iterator over key-value pairs {@code }, from beginning to end of time. - * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for key. - * @deprecated Use {@link #fetch(Object, Instant, Instant)} instead - */ - @Deprecated - WindowStoreIterator fetch(K key, long timeFrom, long timeTo); - /** * Get all the key-value pairs with the given key and the time range from all the existing windows. *

      @@ -159,23 +122,6 @@ default WindowStoreIterator backwardFetch(K key, Instant timeFrom, Instant ti throw new UnsupportedOperationException(); } - /** - * Get all the key-value pairs in the given key range and time range from all the existing windows. - *

      - * This iterator must be closed after use. - * - * @param keyFrom the first key in the range - * @param keyTo the last key in the range - * @param timeFrom time range start (inclusive), where iteration starts. - * @param timeTo time range end (inclusive), where iteration ends. - * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. - * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for any key. - * @deprecated Use {@link #fetch(Object, Object, Instant, Instant)} instead - */ - @Deprecated - KeyValueIterator, V> fetch(K keyFrom, K keyTo, long timeFrom, long timeTo); - /** * Get all the key-value pairs in the given key range and time range from all the existing windows. *

      @@ -233,19 +179,6 @@ default KeyValueIterator, V> backwardAll() { throw new UnsupportedOperationException(); } - /** - * Gets all the key-value pairs that belong to the windows within in the given time range. - * - * @param timeFrom the beginning of the time slot from which to search (inclusive), where iteration starts. - * @param timeTo the end of the time slot from which to search (inclusive), where iteration ends. - * @return an iterator over windowed key-value pairs {@code , value>}, from beginning to end of time. - * @throws InvalidStateStoreException if the store is not initialized - * @throws NullPointerException if {@code null} is used for any key - * @deprecated Use {@link #fetchAll(Instant, Instant)} instead - */ - @Deprecated - KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); - /** * Gets all the key-value pairs that belong to the windows within in the given time range. * diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index f062d758571fc..42278559a9042 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.streams.errors.InvalidStateStoreException; -import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.QueryableStoreType; @@ -28,8 +27,6 @@ import java.util.List; import java.util.Objects; -import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; - /** * Wrapper over the underlying {@link ReadOnlyWindowStore}s found in a {@link * org.apache.kafka.streams.processor.internals.ProcessorTopology} @@ -68,10 +65,9 @@ public V fetch(final K key, final long time) { } @Override - @Deprecated public WindowStoreIterator fetch(final K key, - final long timeFrom, - final long timeTo) { + final Instant timeFrom, + final Instant timeTo) { Objects.requireNonNull(key, "key can't be null"); final List> stores = provider.stores(storeName, windowStoreType); for (final ReadOnlyWindowStore windowStore : stores) { @@ -91,17 +87,6 @@ public WindowStoreIterator fetch(final K key, return KeyValueIterators.emptyWindowStoreIterator(); } - @SuppressWarnings("deprecation") // removing fetch(K from, long from, long to) will fix this - @Override - public WindowStoreIterator fetch(final K key, - final Instant timeFrom, - final Instant timeTo) throws IllegalArgumentException { - return fetch( - key, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); - } - @Override public WindowStoreIterator backwardFetch(final K key, final Instant timeFrom, @@ -125,14 +110,13 @@ public WindowStoreIterator backwardFetch(final K key, return KeyValueIterators.emptyWindowStoreIterator(); } - @SuppressWarnings("deprecation") // removing fetch(K from, K to, long from, long to) will fix this @Override public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, - final long timeFrom, - final long timeTo) { - Objects.requireNonNull(keyFrom, "from can't be null"); - Objects.requireNonNull(keyTo, "to can't be null"); + final Instant timeFrom, + final Instant timeTo) { + Objects.requireNonNull(keyFrom, "keyFrom can't be null"); + Objects.requireNonNull(keyTo, "keyTo can't be null"); final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = store -> store.fetch(keyFrom, keyTo, timeFrom, timeTo); return new DelegatingPeekingKeyValueIterator<>( @@ -142,18 +126,6 @@ public KeyValueIterator, V> fetch(final K keyFrom, nextIteratorFunction)); } - @Override - public KeyValueIterator, V> fetch(final K keyFrom, - final K keyTo, - final Instant timeFrom, - final Instant timeTo) throws IllegalArgumentException { - return fetch( - keyFrom, - keyTo, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "timeTo"))); - } - @Override public KeyValueIterator, V> backwardFetch(final K keyFrom, final K keyTo, @@ -193,9 +165,8 @@ public KeyValueIterator, V> backwardAll() { } @Override - @Deprecated - public KeyValueIterator, V> fetchAll(final long timeFrom, - final long timeTo) { + public KeyValueIterator, V> fetchAll(final Instant timeFrom, + final Instant timeTo) { final NextIteratorFunction, V, ReadOnlyWindowStore> nextIteratorFunction = store -> store.fetchAll(timeFrom, timeTo); return new DelegatingPeekingKeyValueIterator<>( @@ -205,15 +176,6 @@ public KeyValueIterator, V> fetchAll(final long timeFrom, nextIteratorFunction)); } - @SuppressWarnings("deprecation") // removing fetchAll(long from, long to) will fix this - @Override - public KeyValueIterator, V> fetchAll(final Instant timeFrom, - final Instant timeTo) throws IllegalArgumentException { - return fetchAll( - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); - } - @Override public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java index b9fdebeca7c20..281a1c2675767 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacade.java @@ -41,14 +41,6 @@ public V fetch(final K key, return getValueOrNull(inner.fetch(key, time)); } - @Override - @SuppressWarnings("deprecation") - public WindowStoreIterator fetch(final K key, - final long timeFrom, - final long timeTo) { - return new WindowStoreIteratorFacade<>(inner.fetch(key, timeFrom, timeTo)); - } - @Override public WindowStoreIterator fetch(final K key, final Instant timeFrom, @@ -63,15 +55,6 @@ public WindowStoreIterator backwardFetch(final K key, return new WindowStoreIteratorFacade<>(inner.backwardFetch(key, timeFrom, timeTo)); } - @Override - @SuppressWarnings("deprecation") - public KeyValueIterator, V> fetch(final K keyFrom, - final K keyTo, - final long timeFrom, - final long timeTo) { - return new KeyValueIteratorFacade<>(inner.fetch(keyFrom, keyTo, timeFrom, timeTo)); - } - @Override public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, @@ -88,13 +71,6 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, return new KeyValueIteratorFacade<>(inner.backwardFetch(keyFrom, keyTo, timeFrom, timeTo)); } - @Override - @SuppressWarnings("deprecation") - public KeyValueIterator, V> fetchAll(final long timeFrom, - final long timeTo) { - return new KeyValueIteratorFacade<>(inner.fetchAll(timeFrom, timeTo)); - } - @Override public KeyValueIterator, V> fetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java index 0de4890d05309..d64fd0989c662 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/NoOpWindowStore.java @@ -85,12 +85,6 @@ public Object fetch(final Object key, final long time) { return null; } - @Override - @SuppressWarnings("deprecation") - public WindowStoreIterator fetch(final Object key, final long timeFrom, final long timeTo) { - return EMPTY_WINDOW_STORE_ITERATOR; - } - @Override public WindowStoreIterator fetch(final Object key, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; @@ -103,15 +97,6 @@ public WindowStoreIterator backwardFetch(final Object key, return EMPTY_WINDOW_STORE_ITERATOR; } - @Override - @SuppressWarnings("deprecation") - public WindowStoreIterator fetch(final Object keyFrom, - final Object keyTo, - final long timeFrom, - final long timeTo) { - return EMPTY_WINDOW_STORE_ITERATOR; - } - @Override public KeyValueIterator fetch(final Object keyFrom, final Object keyTo, @@ -138,12 +123,6 @@ public WindowStoreIterator backwardAll() { return EMPTY_WINDOW_STORE_ITERATOR; } - @Override - @SuppressWarnings("deprecation") - public WindowStoreIterator fetchAll(final long timeFrom, final long timeTo) { - return EMPTY_WINDOW_STORE_ITERATOR; - } - @Override public KeyValueIterator fetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { return EMPTY_WINDOW_STORE_ITERATOR; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacadeTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacadeTest.java index 47ebeaa132d12..fa8128c6dffa3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacadeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreFacadeTest.java @@ -73,12 +73,12 @@ public void shouldReturnPlainKeyValuePairsOnSingleKeyFetchLongParameters() { expect(mockedWindowTimestampIterator.next()) .andReturn(KeyValue.pair(21L, ValueAndTimestamp.make("value1", 22L))) .andReturn(KeyValue.pair(42L, ValueAndTimestamp.make("value2", 23L))); - expect(mockedWindowTimestampStore.fetch("key1", 21L, 42L)) + expect(mockedWindowTimestampStore.fetch("key1", Instant.ofEpochMilli(21L), Instant.ofEpochMilli(42L))) .andReturn(mockedWindowTimestampIterator); replay(mockedWindowTimestampIterator, mockedWindowTimestampStore); final WindowStoreIterator iterator = - readOnlyWindowStoreFacade.fetch("key1", 21L, 42L); + readOnlyWindowStoreFacade.fetch("key1", Instant.ofEpochMilli(21L), Instant.ofEpochMilli(42L)); assertThat(iterator.next(), is(KeyValue.pair(21L, "value1"))); assertThat(iterator.next(), is(KeyValue.pair(42L, "value2"))); @@ -111,12 +111,12 @@ public void shouldReturnPlainKeyValuePairsOnRangeFetchLongParameters() { .andReturn(KeyValue.pair( new Windowed<>("key2", new TimeWindow(42L, 43L)), ValueAndTimestamp.make("value2", 100L))); - expect(mockedWindowTimestampStore.fetch("key1", "key2", 21L, 42L)) + expect(mockedWindowTimestampStore.fetch("key1", "key2", Instant.ofEpochMilli(21L), Instant.ofEpochMilli(42L))) .andReturn(mockedKeyValueWindowTimestampIterator); replay(mockedKeyValueWindowTimestampIterator, mockedWindowTimestampStore); final KeyValueIterator, String> iterator = - readOnlyWindowStoreFacade.fetch("key1", "key2", 21L, 42L); + readOnlyWindowStoreFacade.fetch("key1", "key2", Instant.ofEpochMilli(21L), Instant.ofEpochMilli(42L)); assertThat(iterator.next(), is(KeyValue.pair(new Windowed<>("key1", new TimeWindow(21L, 22L)), "value1"))); assertThat(iterator.next(), is(KeyValue.pair(new Windowed<>("key2", new TimeWindow(42L, 43L)), "value2"))); @@ -153,12 +153,12 @@ public void shouldReturnPlainKeyValuePairsOnFetchAllLongParameters() { .andReturn(KeyValue.pair( new Windowed<>("key2", new TimeWindow(42L, 43L)), ValueAndTimestamp.make("value2", 100L))); - expect(mockedWindowTimestampStore.fetchAll(21L, 42L)) + expect(mockedWindowTimestampStore.fetchAll(Instant.ofEpochMilli(21L), Instant.ofEpochMilli(42L))) .andReturn(mockedKeyValueWindowTimestampIterator); replay(mockedKeyValueWindowTimestampIterator, mockedWindowTimestampStore); final KeyValueIterator, String> iterator = - readOnlyWindowStoreFacade.fetchAll(21L, 42L); + readOnlyWindowStoreFacade.fetchAll(Instant.ofEpochMilli(21L), Instant.ofEpochMilli(42L)); assertThat(iterator.next(), is(KeyValue.pair(new Windowed<>("key1", new TimeWindow(21L, 22L)), "value1"))); assertThat(iterator.next(), is(KeyValue.pair(new Windowed<>("key2", new TimeWindow(42L, 43L)), "value2"))); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 96c3ae05d9b9d..79d22296fad88 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -61,14 +61,13 @@ public V fetch(final K key, final long time) { } } - @SuppressWarnings("deprecation") @Override - public WindowStoreIterator fetch(final K key, final long timeFrom, final long timeTo) { + public WindowStoreIterator fetch(final K key, final Instant timeFrom, final Instant timeTo) { if (!open) { throw new InvalidStateStoreException("Store is not open"); } final List> results = new ArrayList<>(); - for (long now = timeFrom; now <= timeTo; now++) { + for (long now = timeFrom.toEpochMilli(); now <= timeTo.toEpochMilli(); now++) { final Map kvMap = data.get(now); if (kvMap != null && kvMap.containsKey(key)) { results.add(new KeyValue<>(now, kvMap.get(key))); @@ -77,14 +76,6 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long return new TheWindowStoreIterator<>(results.iterator()); } - @Override - public WindowStoreIterator fetch(final K key, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { - return fetch( - key, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); - } - @Override public WindowStoreIterator backwardFetch(final K key, final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { final long timeFromTs = ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")); @@ -180,15 +171,14 @@ public KeyValue, V> next() { }; } - @SuppressWarnings("deprecation") @Override - public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { + public KeyValueIterator, V> fetchAll(final Instant timeFrom, final Instant timeTo) { if (!open) { throw new InvalidStateStoreException("Store is not open"); } final List, V>> results = new ArrayList<>(); for (final long now : data.keySet()) { - if (!(now >= timeFrom && now <= timeTo)) { + if (!(now >= timeFrom.toEpochMilli() && now <= timeTo.toEpochMilli())) { continue; } final NavigableMap kvMap = data.get(now); @@ -223,13 +213,6 @@ public KeyValue, V> next() { }; } - @Override - public KeyValueIterator, V> fetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { - return fetchAll( - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "from")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "to"))); - } - @Override public KeyValueIterator, V> backwardFetchAll(final Instant timeFrom, final Instant timeTo) throws IllegalArgumentException { final long timeFromTs = ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "timeFrom")); @@ -274,14 +257,13 @@ public KeyValue, V> next() { }; } - @SuppressWarnings("deprecation") @Override - public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, final long timeFrom, final long timeTo) { + public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, final Instant timeFrom, final Instant timeTo) { if (!open) { throw new InvalidStateStoreException("Store is not open"); } final List, V>> results = new ArrayList<>(); - for (long now = timeFrom; now <= timeTo; now++) { + for (long now = timeFrom.toEpochMilli(); now <= timeTo.toEpochMilli(); now++) { final NavigableMap kvMap = data.get(now); if (kvMap != null) { for (final Entry entry : kvMap.subMap(keyFrom, true, keyTo, true).entrySet()) { @@ -314,18 +296,6 @@ public KeyValue, V> next() { }; } - @Override - public KeyValueIterator, V> fetch(final K keyFrom, - final K keyTo, - final Instant timeFrom, - final Instant timeTo) throws IllegalArgumentException { - return fetch( - keyFrom, - keyTo, - ApiUtils.validateMillisecondInstant(timeFrom, prepareMillisCheckFailMsgPrefix(timeFrom, "fromTime")), - ApiUtils.validateMillisecondInstant(timeTo, prepareMillisCheckFailMsgPrefix(timeTo, "toTime"))); - } - @Override public KeyValueIterator, V> backwardFetch(final K from, final K to, diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index ae4ca99651a6a..63e6a278da06d 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -43,17 +43,12 @@ import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.StateStoreContext; -import org.apache.kafka.streams.processor.internals.StreamsProducer; -import org.apache.kafka.streams.state.KeyValueIterator; -import org.apache.kafka.streams.state.WindowStoreIterator; -import org.apache.kafka.streams.state.internals.ReadOnlyKeyValueStoreFacade; -import org.apache.kafka.streams.state.internals.ReadOnlyWindowStoreFacade; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.ChangelogRegister; import org.apache.kafka.streams.processor.internals.ClientUtils; @@ -71,9 +66,11 @@ import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.processor.internals.StreamTask; import org.apache.kafka.streams.processor.internals.StreamThread; +import org.apache.kafka.streams.processor.internals.StreamsProducer; import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; +import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.ReadOnlySessionStore; @@ -83,6 +80,9 @@ import org.apache.kafka.streams.state.TimestampedWindowStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.streams.state.WindowStoreIterator; +import org.apache.kafka.streams.state.internals.ReadOnlyKeyValueStoreFacade; +import org.apache.kafka.streams.state.internals.ReadOnlyWindowStoreFacade; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.streams.test.TestRecord; import org.slf4j.Logger; @@ -1263,6 +1263,13 @@ public void put(final K key, inner.put(key, ValueAndTimestamp.make(value, ConsumerRecord.NO_TIMESTAMP), windowStartTimestamp); } + @Override + public WindowStoreIterator fetch(final K key, + final long timeFrom, + final long timeTo) { + return fetch(key, Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); + } + @Override public WindowStoreIterator backwardFetch(final K key, final long timeFrom, @@ -1270,6 +1277,15 @@ public WindowStoreIterator backwardFetch(final K key, return backwardFetch(key, Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); } + @Override + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo, + final long timeFrom, + final long timeTo) { + return fetch(keyFrom, keyTo, Instant.ofEpochMilli(timeFrom), + Instant.ofEpochMilli(timeTo)); + } + @Override public KeyValueIterator, V> backwardFetch(final K keyFrom, final K keyTo, @@ -1279,7 +1295,14 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, } @Override - public KeyValueIterator, V> backwardFetchAll(final long timeFrom, final long timeTo) { + public KeyValueIterator, V> fetchAll(final long timeFrom, + final long timeTo) { + return fetchAll(Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); + } + + @Override + public KeyValueIterator, V> backwardFetchAll(final long timeFrom, + final long timeTo) { return backwardFetchAll(Instant.ofEpochMilli(timeFrom), Instant.ofEpochMilli(timeTo)); } From 9a71468cb0e5fc9faeba39b56f36d0c93ca73c59 Mon Sep 17 00:00:00 2001 From: vamossagar12 Date: Wed, 5 May 2021 15:56:51 +0530 Subject: [PATCH 117/155] KAFKA-10767: Adding test cases for all, reverseAll and reverseRange for ThreadCache (#9779) The test cases for ThreaCache didn't have the corresponding unit tests for all, reverseAll and reverseRange methods. This PR aims to add the same. Reviewers: Bruno Cadonna --- .../state/internals/ThreadCacheTest.java | 225 +++++++++++++----- 1 file changed, 168 insertions(+), 57 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java index afd5449afd446..c449de9d5231a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import static org.hamcrest.MatcherAssert.assertThat; import org.junit.Test; import java.util.ArrayList; @@ -29,7 +30,9 @@ import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; +import java.util.function.Supplier; +import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -42,19 +45,20 @@ public class ThreadCacheTest { final String namespace1 = "0.1-namespace"; final String namespace2 = "0.2-namespace"; private final LogContext logContext = new LogContext("testCache "); + private final byte[][] bytes = new byte[][]{{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}}; @Test public void basicPutGet() { final List> toInsert = Arrays.asList( - new KeyValue<>("K1", "V1"), - new KeyValue<>("K2", "V2"), - new KeyValue<>("K3", "V3"), - new KeyValue<>("K4", "V4"), - new KeyValue<>("K5", "V5")); + new KeyValue<>("K1", "V1"), + new KeyValue<>("K2", "V2"), + new KeyValue<>("K3", "V3"), + new KeyValue<>("K4", "V4"), + new KeyValue<>("K5", "V5")); final KeyValue kv = toInsert.get(0); final ThreadCache cache = new ThreadCache(logContext, - toInsert.size() * memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""), - new MockStreamsMetrics(new Metrics())); + toInsert.size() * memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""), + new MockStreamsMetrics(new Metrics())); for (final KeyValue kvToInsert : toInsert) { final Bytes key = Bytes.wrap(kvToInsert.key.getBytes()); @@ -132,35 +136,35 @@ public void cacheOverheadsLargeValues() { static int memoryCacheEntrySize(final byte[] key, final byte[] value, final String topic) { return key.length + - value.length + - 1 + // isDirty - 8 + // timestamp - 8 + // offset - 4 + - topic.length() + - // LRU Node entries - key.length + - 8 + // entry - 8 + // previous - 8; // next + value.length + + 1 + // isDirty + 8 + // timestamp + 8 + // offset + 4 + + topic.length() + + // LRU Node entries + key.length + + 8 + // entry + 8 + // previous + 8; // next } @Test public void evict() { final List> received = new ArrayList<>(); final List> expected = Collections.singletonList( - new KeyValue<>("K1", "V1")); + new KeyValue<>("K1", "V1")); final List> toInsert = Arrays.asList( - new KeyValue<>("K1", "V1"), - new KeyValue<>("K2", "V2"), - new KeyValue<>("K3", "V3"), - new KeyValue<>("K4", "V4"), - new KeyValue<>("K5", "V5")); + new KeyValue<>("K1", "V1"), + new KeyValue<>("K2", "V2"), + new KeyValue<>("K3", "V3"), + new KeyValue<>("K4", "V4"), + new KeyValue<>("K5", "V5")); final KeyValue kv = toInsert.get(0); final ThreadCache cache = new ThreadCache(logContext, - memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""), - new MockStreamsMetrics(new Metrics())); + memoryCacheEntrySize(kv.key.getBytes(), kv.value.getBytes(), ""), + new MockStreamsMetrics(new Metrics())); cache.addDirtyEntryFlushListener(namespace, dirty -> { for (final ThreadCache.DirtyEntry dirtyEntry : dirty) { received.add(new KeyValue<>(dirtyEntry.key().toString(), new String(dirtyEntry.newValue()))); @@ -233,46 +237,88 @@ public void shouldNotClashWithOverlappingNames() { assertArrayEquals(name1Byte.get(), cache.get(namespace2, nameByte).value()); } + private ThreadCache setupThreadCache(final int first, final int last, final long entrySize, final boolean reverse) { + final ThreadCache cache = new ThreadCache(logContext, entrySize, new MockStreamsMetrics(new Metrics())); + cache.addDirtyEntryFlushListener(namespace, dirty -> { }); + int index = first; + while ((!reverse && index < last) || (reverse && index >= last)) { + cache.put(namespace, Bytes.wrap(bytes[index]), dirtyEntry(bytes[index])); + if (!reverse) + index++; + else + index--; + } + return cache; + } + @Test public void shouldPeekNextKey() { - final ThreadCache cache = new ThreadCache(logContext, 10000L, new MockStreamsMetrics(new Metrics())); + final ThreadCache cache = setupThreadCache(0, 1, 10000L, false); final Bytes theByte = Bytes.wrap(new byte[]{0}); - cache.put(namespace, theByte, dirtyEntry(theByte.get())); final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, theByte, Bytes.wrap(new byte[]{1})); assertEquals(theByte, iterator.peekNextKey()); assertEquals(theByte, iterator.peekNextKey()); } + @Test + public void shouldPeekNextKeyReverseRange() { + final ThreadCache cache = setupThreadCache(1, 1, 10000L, true); + final Bytes theByte = Bytes.wrap(new byte[]{1}); + final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.reverseRange(namespace, Bytes.wrap(new byte[]{0}), theByte); + assertThat(iterator.peekNextKey(), is(theByte)); + assertThat(iterator.peekNextKey(), is(theByte)); + } + @Test public void shouldGetSameKeyAsPeekNext() { - final ThreadCache cache = new ThreadCache(logContext, 10000L, new MockStreamsMetrics(new Metrics())); + final ThreadCache cache = setupThreadCache(0, 1, 10000L, false); final Bytes theByte = Bytes.wrap(new byte[]{0}); - cache.put(namespace, theByte, dirtyEntry(theByte.get())); final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, theByte, Bytes.wrap(new byte[]{1})); - assertEquals(iterator.peekNextKey(), iterator.next().key); + assertThat(iterator.peekNextKey(), is(iterator.next().key)); } @Test - public void shouldThrowIfNoPeekNextKey() { - final ThreadCache cache = new ThreadCache(logContext, 10000L, new MockStreamsMetrics(new Metrics())); - final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{1})); + public void shouldGetSameKeyAsPeekNextReverseRange() { + final ThreadCache cache = setupThreadCache(1, 1, 10000L, true); + final Bytes theByte = Bytes.wrap(new byte[]{1}); + final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.reverseRange(namespace, Bytes.wrap(new byte[]{0}), theByte); + assertThat(iterator.peekNextKey(), is(iterator.next().key)); + } + + private void shouldThrowIfNoPeekNextKey(final Supplier methodUnderTest) { + final ThreadCache.MemoryLRUCacheBytesIterator iterator = methodUnderTest.get(); assertThrows(NoSuchElementException.class, iterator::peekNextKey); } + @Test + public void shouldThrowIfNoPeekNextKeyRange() { + final ThreadCache cache = setupThreadCache(0, 0, 10000L, false); + shouldThrowIfNoPeekNextKey(() -> cache.range(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{1}))); + } + + @Test + public void shouldThrowIfNoPeekNextKeyReverseRange() { + final ThreadCache cache = setupThreadCache(-1, 0, 10000L, true); + shouldThrowIfNoPeekNextKey(() -> cache.reverseRange(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{1}))); + } + @Test public void shouldReturnFalseIfNoNextKey() { - final ThreadCache cache = new ThreadCache(logContext, 10000L, new MockStreamsMetrics(new Metrics())); + final ThreadCache cache = setupThreadCache(0, 0, 10000L, false); final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{1})); assertFalse(iterator.hasNext()); } + @Test + public void shouldReturnFalseIfNoNextKeyReverseRange() { + final ThreadCache cache = setupThreadCache(-1, 0, 10000L, true); + final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.reverseRange(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{1})); + assertFalse(iterator.hasNext()); + } + @Test public void shouldPeekAndIterateOverRange() { - final ThreadCache cache = new ThreadCache(logContext, 10000L, new MockStreamsMetrics(new Metrics())); - final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}}; - for (final byte[] aByte : bytes) { - cache.put(namespace, Bytes.wrap(aByte), dirtyEntry(aByte)); - } + final ThreadCache cache = setupThreadCache(0, 10, 10000L, false); final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, Bytes.wrap(new byte[]{1}), Bytes.wrap(new byte[]{4})); int bytesIndex = 1; while (iterator.hasNext()) { @@ -286,12 +332,8 @@ public void shouldPeekAndIterateOverRange() { } @Test - public void shouldSkipToEntryWhentoInclusiveIsFalseInRange() { - final ThreadCache cache = new ThreadCache(logContext, 10000L, new MockStreamsMetrics(new Metrics())); - final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}, {10}}; - for (final byte[] aByte : bytes) { - cache.put(namespace, Bytes.wrap(aByte), dirtyEntry(aByte)); - } + public void shouldSkipToEntryWhenToInclusiveIsFalseInRange() { + final ThreadCache cache = setupThreadCache(0, 10, 10000L, false); final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.range(namespace, Bytes.wrap(new byte[]{1}), Bytes.wrap(new byte[]{4}), false); int bytesIndex = 1; while (iterator.hasNext()) { @@ -304,26 +346,95 @@ public void shouldSkipToEntryWhentoInclusiveIsFalseInRange() { assertEquals(4, bytesIndex); } + @Test + public void shouldPeekAndIterateOverReverseRange() { + final ThreadCache cache = setupThreadCache(10, 0, 10000L, true); + final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.reverseRange(namespace, Bytes.wrap(new byte[]{1}), Bytes.wrap(new byte[]{4})); + int bytesIndex = 4; + while (iterator.hasNext()) { + final Bytes peekedKey = iterator.peekNextKey(); + final KeyValue next = iterator.next(); + assertArrayEquals(bytes[bytesIndex], peekedKey.get()); + assertArrayEquals(bytes[bytesIndex], next.key.get()); + bytesIndex--; + } + assertEquals(0, bytesIndex); + } + @Test public void shouldSkipEntriesWhereValueHasBeenEvictedFromCache() { final int entrySize = memoryCacheEntrySize(new byte[1], new byte[1], ""); - final ThreadCache cache = new ThreadCache(logContext, entrySize * 5, new MockStreamsMetrics(new Metrics())); - cache.addDirtyEntryFlushListener(namespace, dirty -> { }); - - final byte[][] bytes = {{0}, {1}, {2}, {3}, {4}, {5}, {6}, {7}, {8}, {9}}; - for (int i = 0; i < 5; i++) { - cache.put(namespace, Bytes.wrap(bytes[i]), dirtyEntry(bytes[i])); - } + final ThreadCache cache = setupThreadCache(0, 5, entrySize * 5, false); assertEquals(5, cache.size()); - // should evict byte[] {0} cache.put(namespace, Bytes.wrap(new byte[]{6}), dirtyEntry(new byte[]{6})); - final ThreadCache.MemoryLRUCacheBytesIterator range = cache.range(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{5})); + assertEquals(Bytes.wrap(new byte[]{1}), range.peekNextKey()); + } + + @Test + public void shouldSkipEntriesWhereValueHasBeenEvictedFromCacheReverseRange() { + final int entrySize = memoryCacheEntrySize(new byte[1], new byte[1], ""); + final ThreadCache cache = setupThreadCache(4, 0, entrySize * 5, true); + assertEquals(5, cache.size()); + // should evict byte[] {4} + cache.put(namespace, Bytes.wrap(new byte[]{6}), dirtyEntry(new byte[]{6})); + final ThreadCache.MemoryLRUCacheBytesIterator range = cache.reverseRange(namespace, Bytes.wrap(new byte[]{0}), Bytes.wrap(new byte[]{5})); + assertEquals(Bytes.wrap(new byte[]{3}), range.peekNextKey()); + } + + @Test + public void shouldFetchAllEntriesInCache() { + final ThreadCache cache = setupThreadCache(0, 11, 10000L, false); + final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.all(namespace); + int bytesIndex = 0; + while (iterator.hasNext()) { + final Bytes peekedKey = iterator.peekNextKey(); + final KeyValue next = iterator.next(); + assertArrayEquals(bytes[bytesIndex], peekedKey.get()); + assertArrayEquals(bytes[bytesIndex], next.key.get()); + bytesIndex++; + } + assertEquals(11, bytesIndex); + } + @Test + public void shouldFetchAllEntriesInCacheInReverseOrder() { + final ThreadCache cache = setupThreadCache(10, 0, 10000L, true); + final ThreadCache.MemoryLRUCacheBytesIterator iterator = cache.reverseAll(namespace); + int bytesIndex = 10; + while (iterator.hasNext()) { + final Bytes peekedKey = iterator.peekNextKey(); + final KeyValue next = iterator.next(); + assertArrayEquals(bytes[bytesIndex], peekedKey.get()); + assertArrayEquals(bytes[bytesIndex], next.key.get()); + bytesIndex--; + } + assertEquals(-1, bytesIndex); + } + + @Test + public void shouldReturnAllUnevictedValuesFromCache() { + final int entrySize = memoryCacheEntrySize(new byte[1], new byte[1], ""); + final ThreadCache cache = setupThreadCache(0, 5, entrySize * 5, false); + assertEquals(5, cache.size()); + // should evict byte[] {0} + cache.put(namespace, Bytes.wrap(new byte[]{6}), dirtyEntry(new byte[]{6})); + final ThreadCache.MemoryLRUCacheBytesIterator range = cache.all(namespace); assertEquals(Bytes.wrap(new byte[]{1}), range.peekNextKey()); } + @Test + public void shouldReturnAllUnevictedValuesFromCacheInReverseOrder() { + final int entrySize = memoryCacheEntrySize(new byte[1], new byte[1], ""); + final ThreadCache cache = setupThreadCache(4, 0, entrySize * 5, true); + assertEquals(5, cache.size()); + // should evict byte[] {4} + cache.put(namespace, Bytes.wrap(new byte[]{6}), dirtyEntry(new byte[]{6})); + final ThreadCache.MemoryLRUCacheBytesIterator range = cache.reverseAll(namespace); + assertEquals(Bytes.wrap(new byte[]{6}), range.peekNextKey()); + } + @Test public void shouldFlushDirtyEntriesForNamespace() { final ThreadCache cache = new ThreadCache(logContext, 100000, new MockStreamsMetrics(new Metrics())); @@ -394,7 +505,7 @@ public void shouldEvictAfterPutAll() { cache.addDirtyEntryFlushListener(namespace, received::addAll); cache.putAll(namespace, Arrays.asList(KeyValue.pair(Bytes.wrap(new byte[]{0}), dirtyEntry(new byte[]{5})), - KeyValue.pair(Bytes.wrap(new byte[]{1}), dirtyEntry(new byte[]{6})))); + KeyValue.pair(Bytes.wrap(new byte[]{1}), dirtyEntry(new byte[]{6})))); assertEquals(cache.evicts(), 2); assertEquals(received.size(), 2); @@ -405,7 +516,7 @@ public void shouldPutAll() { final ThreadCache cache = new ThreadCache(logContext, 100000, new MockStreamsMetrics(new Metrics())); cache.putAll(namespace, Arrays.asList(KeyValue.pair(Bytes.wrap(new byte[]{0}), dirtyEntry(new byte[]{5})), - KeyValue.pair(Bytes.wrap(new byte[]{1}), dirtyEntry(new byte[]{6})))); + KeyValue.pair(Bytes.wrap(new byte[]{1}), dirtyEntry(new byte[]{6})))); assertArrayEquals(new byte[]{5}, cache.get(namespace, Bytes.wrap(new byte[]{0})).value()); assertArrayEquals(new byte[]{6}, cache.get(namespace, Bytes.wrap(new byte[]{1})).value()); From a1367f57f52e7c5bdb2fadf14f3b124dd7148213 Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Wed, 5 May 2021 20:18:52 +0530 Subject: [PATCH 118/155] KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage. (#10271) KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage. This topic will receive events of RemoteLogSegmentMetadata, RemoteLogSegmentUpdate, and RemotePartitionDeleteMetadata. These events are serialized into Kafka protocol message format. Added tests for all the event types for that topic. This is part of the tiered storaqe implementation KIP-405. Reivewers: Kowshik Prakasam , Jun Rao --- build.gradle | 20 ++- checkstyle/import-control.xml | 4 +- checkstyle/suppressions.xml | 2 + .../org/apache/kafka/raft/RecordSerde.java | 2 +- .../metadata/AbstractApiMessageSerde.java | 93 +++++++++++++ .../raft/metadata/MetadataRecordSerde.java | 44 +----- .../log/remote/storage/RemoteLogMetadata.java | 56 ++++++++ .../storage/RemoteLogSegmentMetadata.java | 49 ++----- .../RemoteLogSegmentMetadataUpdate.java | 43 ++---- .../RemotePartitionDeleteMetadata.java | 50 +++---- .../serialization/BytesApiMessageSerde.java | 68 ++++++++++ .../serialization/RemoteLogMetadataSerde.java | 104 +++++++++++++++ .../RemoteLogMetadataTransform.java | 52 ++++++++ .../RemoteLogSegmentMetadataTransform.java | 98 ++++++++++++++ ...moteLogSegmentMetadataUpdateTransform.java | 59 ++++++++ ...emotePartitionDeleteMetadataTransform.java | 54 ++++++++ .../message/RemoteLogSegmentMetadata.json | 126 ++++++++++++++++++ .../RemoteLogSegmentMetadataUpdate.json | 82 ++++++++++++ .../RemotePartitionDeleteMetadata.json | 68 ++++++++++ .../storage/RemoteLogMetadataSerdeTest.java | 111 +++++++++++++++ .../RemoteLogMetadataTransformTest.java | 85 ++++++++++++ 21 files changed, 1122 insertions(+), 148 deletions(-) create mode 100644 raft/src/main/java/org/apache/kafka/raft/metadata/AbstractApiMessageSerde.java create mode 100644 storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/BytesApiMessageSerde.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java create mode 100644 storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java create mode 100644 storage/src/main/resources/message/RemoteLogSegmentMetadata.json create mode 100644 storage/src/main/resources/message/RemoteLogSegmentMetadataUpdate.json create mode 100644 storage/src/main/resources/message/RemotePartitionDeleteMetadata.json create mode 100644 storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java create mode 100644 storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java diff --git a/build.gradle b/build.gradle index e7a3a2317d979..e443d317e870b 100644 --- a/build.gradle +++ b/build.gradle @@ -1411,6 +1411,8 @@ project(':storage') { dependencies { implementation project(':storage:api') implementation project(':clients') + implementation project(':metadata') + implementation project(':raft') implementation libs.slf4jApi implementation libs.jacksonDatabind @@ -1438,19 +1440,33 @@ project(':storage') { } } + task processMessages(type:JavaExec) { + main = "org.apache.kafka.message.MessageGenerator" + classpath = project(':generator').sourceSets.main.runtimeClasspath + args = [ "-p", " org.apache.kafka.server.log.remote.metadata.storage.generated", + "-o", "src/generated/java/org/apache/kafka/server/log/remote/metadata/storage/generated", + "-i", "src/main/resources/message", + "-m", "MessageDataGenerator", "JsonConverterGenerator", + "-t", "MetadataRecordTypeGenerator", "MetadataJsonConvertersGenerator" ] + inputs.dir("src/main/resources/message") + outputs.dir("src/generated/java/org/apache/kafka/server/log/remote/metadata/storage/generated") + } + sourceSets { main { java { - srcDirs = ["src/main/java"] + srcDirs = ["src/generated/java", "src/main/java"] } } test { java { - srcDirs = ["src/test/java"] + srcDirs = ["src/generated/java", "src/test/java"] } } } + compileJava.dependsOn 'processMessages' + jar { dependsOn createVersionFile from("$buildDir") { diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 8506bba5c509f..38db3966fac06 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -267,11 +267,13 @@ + + + - diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 1093ac895aafd..29ec30687c6d2 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -184,6 +184,8 @@ files="streams[\\/]src[\\/](generated|generated-test)[\\/].+.java$"/> + diff --git a/raft/src/main/java/org/apache/kafka/raft/RecordSerde.java b/raft/src/main/java/org/apache/kafka/raft/RecordSerde.java index 9581297de32b1..90a5e1ed286ec 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RecordSerde.java +++ b/raft/src/main/java/org/apache/kafka/raft/RecordSerde.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.protocol.Writable; /** - * Serde interface for records written to the Raft log. This class assumes + * Serde interface for records written to a metadata log. This class assumes * a two-pass serialization, with the first pass used to compute the size of the * serialized record, and the second pass to write the object. */ diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/AbstractApiMessageSerde.java b/raft/src/main/java/org/apache/kafka/raft/metadata/AbstractApiMessageSerde.java new file mode 100644 index 0000000000000..fea70a7ce41cc --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/AbstractApiMessageSerde.java @@ -0,0 +1,93 @@ +/* + * 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.metadata; + +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ObjectSerializationCache; +import org.apache.kafka.common.protocol.Readable; +import org.apache.kafka.common.protocol.Writable; +import org.apache.kafka.common.utils.ByteUtils; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.raft.RecordSerde; + +/** + * This is an implementation of {@code RecordSerde} with {@link ApiMessageAndVersion} but implementors need to implement + * {@link #apiMessageFor(short)} to return a {@code ApiMessage} instance for the given {@code apiKey}. + * + * This can be used as the underlying serialization mechanism for records defined with {@link ApiMessage}s. + *

      + * Serialization format for the given {@code ApiMessageAndVersion} is below: + *

      + *
      + *     [data_frame_version header message]
      + *     header => [api_key version]
      + *
      + *     data_frame_version   : This is the header version, current value is 0. Header includes both api_key and version.
      + *     api_key              : apiKey of {@code ApiMessageAndVersion} object.
      + *     version              : version of {@code ApiMessageAndVersion} object.
      + *     message              : serialized message of {@code ApiMessageAndVersion} object.
      + * 
      + */ +public abstract class AbstractApiMessageSerde implements RecordSerde { + private static final short DEFAULT_FRAME_VERSION = 0; + private static final int DEFAULT_FRAME_VERSION_SIZE = ByteUtils.sizeOfUnsignedVarint(DEFAULT_FRAME_VERSION); + + @Override + public int recordSize(ApiMessageAndVersion data, + ObjectSerializationCache serializationCache) { + int size = DEFAULT_FRAME_VERSION_SIZE; + size += ByteUtils.sizeOfUnsignedVarint(data.message().apiKey()); + size += ByteUtils.sizeOfUnsignedVarint(data.version()); + size += data.message().size(serializationCache, data.version()); + return size; + } + + @Override + public void write(ApiMessageAndVersion data, + ObjectSerializationCache serializationCache, + Writable out) { + out.writeUnsignedVarint(DEFAULT_FRAME_VERSION); + out.writeUnsignedVarint(data.message().apiKey()); + out.writeUnsignedVarint(data.version()); + data.message().write(out, serializationCache, data.version()); + } + + @Override + public ApiMessageAndVersion read(Readable input, + int size) { + short frameVersion = (short) input.readUnsignedVarint(); + if (frameVersion != DEFAULT_FRAME_VERSION) { + throw new SerializationException("Could not deserialize metadata record due to unknown frame version " + + frameVersion + "(only frame version " + DEFAULT_FRAME_VERSION + " is supported)"); + } + + short apiKey = (short) input.readUnsignedVarint(); + short version = (short) input.readUnsignedVarint(); + ApiMessage record = apiMessageFor(apiKey); + record.read(input, version); + return new ApiMessageAndVersion(record, version); + } + + /** + * Return {@code ApiMessage} instance for the given {@code apiKey}. This is used while deserializing the bytes + * payload into the respective {@code ApiMessage} in {@link #read(Readable, int)} method. + * + * @param apiKey apiKey for which a {@code ApiMessage} to be created. + */ + public abstract ApiMessage apiMessageFor(short apiKey); +} diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java index c740497fb317e..5250d3927b2dd 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java @@ -16,51 +16,13 @@ */ package org.apache.kafka.raft.metadata; -import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.metadata.MetadataRecordType; import org.apache.kafka.common.protocol.ApiMessage; -import org.apache.kafka.common.protocol.ObjectSerializationCache; -import org.apache.kafka.common.protocol.Readable; -import org.apache.kafka.common.protocol.Writable; -import org.apache.kafka.common.utils.ByteUtils; -import org.apache.kafka.metadata.ApiMessageAndVersion; -import org.apache.kafka.raft.RecordSerde; -public class MetadataRecordSerde implements RecordSerde { - private static final short DEFAULT_FRAME_VERSION = 0; - private static final int DEFAULT_FRAME_VERSION_SIZE = ByteUtils.sizeOfUnsignedVarint(DEFAULT_FRAME_VERSION); +public class MetadataRecordSerde extends AbstractApiMessageSerde { @Override - public int recordSize(ApiMessageAndVersion data, ObjectSerializationCache serializationCache) { - int size = DEFAULT_FRAME_VERSION_SIZE; - size += ByteUtils.sizeOfUnsignedVarint(data.message().apiKey()); - size += ByteUtils.sizeOfUnsignedVarint(data.version()); - size += data.message().size(serializationCache, data.version()); - return size; + public ApiMessage apiMessageFor(short apiKey) { + return MetadataRecordType.fromId(apiKey).newMetadataRecord(); } - - @Override - public void write(ApiMessageAndVersion data, ObjectSerializationCache serializationCache, Writable out) { - out.writeUnsignedVarint(DEFAULT_FRAME_VERSION); - out.writeUnsignedVarint(data.message().apiKey()); - out.writeUnsignedVarint(data.version()); - data.message().write(out, serializationCache, data.version()); - } - - @Override - public ApiMessageAndVersion read(Readable input, int size) { - short frameVersion = (short) input.readUnsignedVarint(); - if (frameVersion != DEFAULT_FRAME_VERSION) { - throw new SerializationException("Could not deserialize metadata record due to unknown frame version " - + frameVersion + "(only frame version " + DEFAULT_FRAME_VERSION + " is supported)"); - } - - short apiKey = (short) input.readUnsignedVarint(); - short version = (short) input.readUnsignedVarint(); - MetadataRecordType recordType = MetadataRecordType.fromId(apiKey); - ApiMessage record = recordType.newMetadataRecord(); - record.read(input, version); - return new ApiMessageAndVersion(record, version); - } - } diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java new file mode 100644 index 0000000000000..1350288036f03 --- /dev/null +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadata.java @@ -0,0 +1,56 @@ +/* + * 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.server.log.remote.storage; + +import org.apache.kafka.common.annotation.InterfaceStability; + +/** + * Base class for remote log metadata objects like {@link RemoteLogSegmentMetadata}, {@link RemoteLogSegmentMetadataUpdate}, + * and {@link RemotePartitionDeleteMetadata}. + */ +@InterfaceStability.Evolving +public abstract class RemoteLogMetadata { + + /** + * Broker id from which this event is generated. + */ + private final int brokerId; + + /** + * Epoch time in milli seconds at which this event is generated. + */ + private final long eventTimestampMs; + + protected RemoteLogMetadata(int brokerId, long eventTimestampMs) { + this.brokerId = brokerId; + this.eventTimestampMs = eventTimestampMs; + } + + /** + * @return Epoch time in milli seconds at which this event is occurred. + */ + public long eventTimestampMs() { + return eventTimestampMs; + } + + /** + * @return Broker id from which this event is generated. + */ + public int brokerId() { + return brokerId; + } +} diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java index 4a59e3093ca6b..b66309138a945 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadata.java @@ -33,7 +33,7 @@ * {@code RemoteLogSegmentMetadata}. */ @InterfaceStability.Evolving -public class RemoteLogSegmentMetadata { +public class RemoteLogSegmentMetadata extends RemoteLogMetadata { /** * Universally unique remote log segment id. @@ -50,21 +50,11 @@ public class RemoteLogSegmentMetadata { */ private final long endOffset; - /** - * Broker id from which this event is generated. - */ - private final int brokerId; - /** * Maximum timestamp in milli seconds in the segment */ private final long maxTimestampMs; - /** - * Epoch time in milli seconds at which the respective {@link #state} is set. - */ - private final long eventTimestampMs; - /** * LeaderEpoch vs offset for messages within this segment. */ @@ -105,14 +95,13 @@ private RemoteLogSegmentMetadata(RemoteLogSegmentId remoteLogSegmentId, int segmentSizeInBytes, RemoteLogSegmentState state, Map segmentLeaderEpochs) { + super(brokerId, eventTimestampMs); this.remoteLogSegmentId = Objects.requireNonNull(remoteLogSegmentId, "remoteLogSegmentId can not be null"); this.state = Objects.requireNonNull(state, "state can not be null"); this.startOffset = startOffset; this.endOffset = endOffset; - this.brokerId = brokerId; this.maxTimestampMs = maxTimestampMs; - this.eventTimestampMs = eventTimestampMs; this.segmentSizeInBytes = segmentSizeInBytes; if (segmentLeaderEpochs == null || segmentLeaderEpochs.isEmpty()) { @@ -177,13 +166,6 @@ public long endOffset() { return endOffset; } - /** - * @return Epoch time in milli seconds at which this event is occurred. - */ - public long eventTimestampMs() { - return eventTimestampMs; - } - /** * @return Total size of this segment in bytes. */ @@ -205,13 +187,6 @@ public NavigableMap segmentLeaderEpochs() { return segmentLeaderEpochs; } - /** - * @return Broker id from which this event is generated. - */ - public int brokerId() { - return brokerId; - } - /** * Returns the current state of this remote log segment. It can be any of the below *
        @@ -251,17 +226,19 @@ public boolean equals(Object o) { return false; } RemoteLogSegmentMetadata that = (RemoteLogSegmentMetadata) o; - return startOffset == that.startOffset && endOffset == that.endOffset && brokerId == that.brokerId - && maxTimestampMs == that.maxTimestampMs && eventTimestampMs == that.eventTimestampMs - && segmentSizeInBytes == that.segmentSizeInBytes - && Objects.equals(remoteLogSegmentId, that.remoteLogSegmentId) - && Objects.equals(segmentLeaderEpochs, that.segmentLeaderEpochs) && state == that.state; + return startOffset == that.startOffset && endOffset == that.endOffset + && maxTimestampMs == that.maxTimestampMs + && segmentSizeInBytes == that.segmentSizeInBytes + && Objects.equals(remoteLogSegmentId, that.remoteLogSegmentId) + && Objects.equals(segmentLeaderEpochs, that.segmentLeaderEpochs) && state == that.state + && eventTimestampMs() == that.eventTimestampMs() + && brokerId() == that.brokerId(); } @Override public int hashCode() { - return Objects.hash(remoteLogSegmentId, startOffset, endOffset, brokerId, maxTimestampMs, - eventTimestampMs, segmentLeaderEpochs, segmentSizeInBytes, state); + return Objects.hash(remoteLogSegmentId, startOffset, endOffset, brokerId(), maxTimestampMs, + eventTimestampMs(), segmentLeaderEpochs, segmentSizeInBytes, state); } @Override @@ -270,9 +247,9 @@ public String toString() { "remoteLogSegmentId=" + remoteLogSegmentId + ", startOffset=" + startOffset + ", endOffset=" + endOffset + - ", brokerId=" + brokerId + + ", brokerId=" + brokerId() + ", maxTimestampMs=" + maxTimestampMs + - ", eventTimestampMs=" + eventTimestampMs + + ", eventTimestampMs=" + eventTimestampMs() + ", segmentLeaderEpochs=" + segmentLeaderEpochs + ", segmentSizeInBytes=" + segmentSizeInBytes + ", state=" + state + diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java index 56ed8dbff20cc..4487304eb1fdf 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogSegmentMetadataUpdate.java @@ -26,28 +26,18 @@ * This also includes the timestamp of this event. */ @InterfaceStability.Evolving -public class RemoteLogSegmentMetadataUpdate { +public class RemoteLogSegmentMetadataUpdate extends RemoteLogMetadata { /** * Universally unique remote log segment id. */ private final RemoteLogSegmentId remoteLogSegmentId; - /** - * Epoch time in milli seconds at which this event is generated. - */ - private final long eventTimestampMs; - /** * It indicates the state in which the action is executed on this segment. */ private final RemoteLogSegmentState state; - /** - * Broker id from which this event is generated. - */ - private final int brokerId; - /** * @param remoteLogSegmentId Universally unique remote log segment id. * @param eventTimestampMs Epoch time in milli seconds at which the remote log segment is copied to the remote tier storage. @@ -56,10 +46,9 @@ public class RemoteLogSegmentMetadataUpdate { */ public RemoteLogSegmentMetadataUpdate(RemoteLogSegmentId remoteLogSegmentId, long eventTimestampMs, RemoteLogSegmentState state, int brokerId) { + super(brokerId, eventTimestampMs); this.remoteLogSegmentId = Objects.requireNonNull(remoteLogSegmentId, "remoteLogSegmentId can not be null"); this.state = Objects.requireNonNull(state, "state can not be null"); - this.brokerId = brokerId; - this.eventTimestampMs = eventTimestampMs; } /** @@ -69,13 +58,6 @@ public RemoteLogSegmentId remoteLogSegmentId() { return remoteLogSegmentId; } - /** - * @return Epoch time in milli seconds at which this event is generated. - */ - public long eventTimestampMs() { - return eventTimestampMs; - } - /** * It represents the state of the remote log segment. It can be one of the values of {@link RemoteLogSegmentState}. */ @@ -83,13 +65,6 @@ public RemoteLogSegmentState state() { return state; } - /** - * @return Broker id from which this event is generated. - */ - public int brokerId() { - return brokerId; - } - @Override public boolean equals(Object o) { if (this == o) { @@ -99,24 +74,24 @@ public boolean equals(Object o) { return false; } RemoteLogSegmentMetadataUpdate that = (RemoteLogSegmentMetadataUpdate) o; - return eventTimestampMs == that.eventTimestampMs && - Objects.equals(remoteLogSegmentId, that.remoteLogSegmentId) && + return Objects.equals(remoteLogSegmentId, that.remoteLogSegmentId) && state == that.state && - brokerId == that.brokerId; + eventTimestampMs() == that.eventTimestampMs() && + brokerId() == that.brokerId(); } @Override public int hashCode() { - return Objects.hash(remoteLogSegmentId, eventTimestampMs, state, brokerId); + return Objects.hash(remoteLogSegmentId, state, eventTimestampMs(), brokerId()); } @Override public String toString() { return "RemoteLogSegmentMetadataUpdate{" + "remoteLogSegmentId=" + remoteLogSegmentId + - ", eventTimestampMs=" + eventTimestampMs + - ", state=" + state + - ", brokerId=" + brokerId + + ", state=" + state + + ", eventTimestampMs=" + eventTimestampMs() + + ", brokerId=" + brokerId() + '}'; } } diff --git a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java index fdf4e61c1dbec..c84e1d78408fc 100644 --- a/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java +++ b/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemotePartitionDeleteMetadata.java @@ -22,32 +22,30 @@ import java.util.Objects; /** - * This class represents the metadata about the remote partition. It can be updated with {@link RemoteLogMetadataManager#putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata)}. + * This class represents the metadata about the remote partition. It can be created/updated with {@link RemoteLogMetadataManager#putRemotePartitionDeleteMetadata(RemotePartitionDeleteMetadata)}. * Possible state transitions are mentioned at {@link RemotePartitionDeleteState}. */ @InterfaceStability.Evolving -public class RemotePartitionDeleteMetadata { +public class RemotePartitionDeleteMetadata extends RemoteLogMetadata { private final TopicIdPartition topicIdPartition; private final RemotePartitionDeleteState state; - private final long eventTimestamp; - private final int brokerId; /** + * Creates an instance of this class with the given metadata. * - * @param topicIdPartition - * @param state - * @param eventTimestamp - * @param brokerId + * @param topicIdPartition topic partition for which this event is meant for. + * @param state State of the remote topic partition. + * @param eventTimestampMs Epoch time in milli seconds at which this event is occurred. + * @param brokerId Id of the broker in which this event is raised. */ public RemotePartitionDeleteMetadata(TopicIdPartition topicIdPartition, RemotePartitionDeleteState state, - long eventTimestamp, + long eventTimestampMs, int brokerId) { + super(brokerId, eventTimestampMs); this.topicIdPartition = Objects.requireNonNull(topicIdPartition); this.state = Objects.requireNonNull(state); - this.eventTimestamp = eventTimestamp; - this.brokerId = brokerId; } /** @@ -64,27 +62,13 @@ public RemotePartitionDeleteState state() { return state; } - /** - * @return Epoch time at which this event is occurred. - */ - public long eventTimestamp() { - return eventTimestamp; - } - - /** - * @return broker id from which this event is generated. - */ - public int brokerId() { - return brokerId; - } - @Override public String toString() { return "RemotePartitionDeleteMetadata{" + "topicPartition=" + topicIdPartition + ", state=" + state + - ", eventTimestamp=" + eventTimestamp + - ", brokerId=" + brokerId + + ", eventTimestampMs=" + eventTimestampMs() + + ", brokerId=" + brokerId() + '}'; } @@ -97,14 +81,14 @@ public boolean equals(Object o) { return false; } RemotePartitionDeleteMetadata that = (RemotePartitionDeleteMetadata) o; - return eventTimestamp == that.eventTimestamp && - brokerId == that.brokerId && - Objects.equals(topicIdPartition, that.topicIdPartition) && - state == that.state; + return Objects.equals(topicIdPartition, that.topicIdPartition) && + state == that.state && + eventTimestampMs() == that.eventTimestampMs() && + brokerId() == that.brokerId(); } @Override public int hashCode() { - return Objects.hash(topicIdPartition, state, eventTimestamp, brokerId); + return Objects.hash(topicIdPartition, state, eventTimestampMs(), brokerId()); } -} +} \ No newline at end of file diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/BytesApiMessageSerde.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/BytesApiMessageSerde.java new file mode 100644 index 0000000000000..65973b0e920cf --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/BytesApiMessageSerde.java @@ -0,0 +1,68 @@ +/* + * 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.server.log.remote.metadata.storage.serialization; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; +import org.apache.kafka.common.protocol.ObjectSerializationCache; +import org.apache.kafka.common.protocol.Readable; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.raft.metadata.AbstractApiMessageSerde; +import java.nio.ByteBuffer; + +/** + * This class provides conversion of {@code ApiMessageAndVersion} to bytes and vice versa.. This can be used as serialization protocol for any + * metadata records derived of {@code ApiMessage}s. It internally uses {@link AbstractApiMessageSerde} for serialization/deserialization + * mechanism. + *

        + * Implementors need to extend this class and implement {@link #apiMessageFor(short)} method to return a respective + * {@code ApiMessage} for the given {@code apiKey}. This is required to deserialize the bytes to build the respective + * {@code ApiMessage} instance. + */ +public abstract class BytesApiMessageSerde { + + private final AbstractApiMessageSerde metadataRecordSerde = new AbstractApiMessageSerde() { + @Override + public ApiMessage apiMessageFor(short apiKey) { + return BytesApiMessageSerde.this.apiMessageFor(apiKey); + } + }; + + public byte[] serialize(ApiMessageAndVersion messageAndVersion) { + ObjectSerializationCache cache = new ObjectSerializationCache(); + int size = metadataRecordSerde.recordSize(messageAndVersion, cache); + ByteBufferAccessor writable = new ByteBufferAccessor(ByteBuffer.allocate(size)); + metadataRecordSerde.write(messageAndVersion, cache, writable); + + return writable.buffer().array(); + } + + public ApiMessageAndVersion deserialize(byte[] data) { + Readable readable = new ByteBufferAccessor(ByteBuffer.wrap(data)); + + return metadataRecordSerde.read(readable, data.length); + } + + /** + * Return {@code ApiMessage} instance for the given {@code apiKey}. This is used while deserializing the bytes + * payload into the respective {@code ApiMessage} in {@link #deserialize(byte[])} method. + * + * @param apiKey apiKey for which a {@code ApiMessage} to be created. + */ + public abstract ApiMessage apiMessageFor(short apiKey); + +} \ No newline at end of file diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java new file mode 100644 index 0000000000000..4e68cf0b97f3b --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java @@ -0,0 +1,104 @@ +/* + * 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.server.log.remote.metadata.storage.serialization; + +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.generated.MetadataRecordType; +import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord; +import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataUpdateRecord; +import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; + +import java.util.HashMap; +import java.util.Map; + +/** + * This class provides serialization and deserialization for {@link RemoteLogMetadata}. This is the root serde + * for the messages that are stored in internal remote log metadata topic. + */ +public class RemoteLogMetadataSerde { + private static final short REMOTE_LOG_SEGMENT_METADATA_API_KEY = new RemoteLogSegmentMetadataRecord().apiKey(); + private static final short REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY = new RemoteLogSegmentMetadataUpdateRecord().apiKey(); + private static final short REMOTE_PARTITION_DELETE_API_KEY = new RemotePartitionDeleteMetadataRecord().apiKey(); + + private final Map remoteLogStorageClassToApiKey; + private final Map keyToTransform; + private final BytesApiMessageSerde bytesApiMessageSerde; + + public RemoteLogMetadataSerde() { + remoteLogStorageClassToApiKey = createRemoteLogStorageClassToApiKeyMap(); + keyToTransform = createRemoteLogMetadataTransforms(); + bytesApiMessageSerde = new BytesApiMessageSerde() { + @Override + public ApiMessage apiMessageFor(short apiKey) { + return newApiMessage(apiKey); + } + }; + } + + protected ApiMessage newApiMessage(short apiKey) { + return MetadataRecordType.fromId(apiKey).newMetadataRecord(); + } + + protected Map createRemoteLogMetadataTransforms() { + Map map = new HashMap<>(); + map.put(REMOTE_LOG_SEGMENT_METADATA_API_KEY, new RemoteLogSegmentMetadataTransform()); + map.put(REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY, new RemoteLogSegmentMetadataUpdateTransform()); + map.put(REMOTE_PARTITION_DELETE_API_KEY, new RemotePartitionDeleteMetadataTransform()); + return map; + } + + protected Map createRemoteLogStorageClassToApiKeyMap() { + Map map = new HashMap<>(); + map.put(RemoteLogSegmentMetadata.class.getName(), REMOTE_LOG_SEGMENT_METADATA_API_KEY); + map.put(RemoteLogSegmentMetadataUpdate.class.getName(), REMOTE_LOG_SEGMENT_METADATA_UPDATE_API_KEY); + map.put(RemotePartitionDeleteMetadata.class.getName(), REMOTE_PARTITION_DELETE_API_KEY); + return map; + } + + public byte[] serialize(RemoteLogMetadata remoteLogMetadata) { + Short apiKey = remoteLogStorageClassToApiKey.get(remoteLogMetadata.getClass().getName()); + if (apiKey == null) { + throw new IllegalArgumentException("ApiKey for given RemoteStorageMetadata class: " + remoteLogMetadata.getClass() + + " does not exist."); + } + + @SuppressWarnings("unchecked") + ApiMessageAndVersion apiMessageAndVersion = remoteLogMetadataTransform(apiKey).toApiMessageAndVersion(remoteLogMetadata); + + return bytesApiMessageSerde.serialize(apiMessageAndVersion); + } + + public RemoteLogMetadata deserialize(byte[] data) { + ApiMessageAndVersion apiMessageAndVersion = bytesApiMessageSerde.deserialize(data); + + return remoteLogMetadataTransform(apiMessageAndVersion.message().apiKey()).fromApiMessageAndVersion(apiMessageAndVersion); + } + + private RemoteLogMetadataTransform remoteLogMetadataTransform(short apiKey) { + RemoteLogMetadataTransform metadataTransform = keyToTransform.get(apiKey); + if (metadataTransform == null) { + throw new IllegalArgumentException("RemoteLogMetadataTransform for apikey: " + apiKey + " does not exist."); + } + + return metadataTransform; + } +} \ No newline at end of file diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java new file mode 100644 index 0000000000000..8cb4c61af20f5 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java @@ -0,0 +1,52 @@ +/* + * 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.server.log.remote.metadata.storage.serialization; + +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; + +/** + * This interface is about transforming {@link RemoteLogMetadata} objects into the respective {@link ApiMessageAndVersion} or vice versa. + *

        + * Those metadata objects can be {@link org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata}, + * {@link org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate}, or {@link org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata}. + *

        + * @param metadata type. + * + * @see RemoteLogSegmentMetadataTransform + * @see RemoteLogSegmentMetadataUpdateTransform + * @see RemotePartitionDeleteMetadataTransform + */ +public interface RemoteLogMetadataTransform { + + /** + * Transforms the given {@code metadata} object into the respective {@code ApiMessageAndVersion} object. + * + * @param metadata metadata object to be transformed. + * @return transformed {@code ApiMessageAndVersion} object. + */ + ApiMessageAndVersion toApiMessageAndVersion(T metadata); + + /** + * Return the metadata object transformed from the given {@code apiMessageAndVersion}. + * + * @param apiMessageAndVersion ApiMessageAndVersion object to be transformed. + * @return transformed {@code T} metadata object. + */ + T fromApiMessageAndVersion(ApiMessageAndVersion apiMessageAndVersion); + +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java new file mode 100644 index 0000000000000..29e7fc83e1cb7 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java @@ -0,0 +1,98 @@ +/* + * 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.server.log.remote.metadata.storage.serialization; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class RemoteLogSegmentMetadataTransform implements RemoteLogMetadataTransform { + + public ApiMessageAndVersion toApiMessageAndVersion(RemoteLogSegmentMetadata segmentMetadata) { + RemoteLogSegmentMetadataRecord record = new RemoteLogSegmentMetadataRecord() + .setRemoteLogSegmentId(createRemoteLogSegmentIdEntry(segmentMetadata)) + .setStartOffset(segmentMetadata.startOffset()) + .setEndOffset(segmentMetadata.endOffset()) + .setBrokerId(segmentMetadata.brokerId()) + .setEventTimestampMs(segmentMetadata.eventTimestampMs()) + .setMaxTimestampMs(segmentMetadata.maxTimestampMs()) + .setSegmentSizeInBytes(segmentMetadata.segmentSizeInBytes()) + .setSegmentLeaderEpochs(createSegmentLeaderEpochsEntry(segmentMetadata)) + .setRemoteLogSegmentState(segmentMetadata.state().id()); + + return new ApiMessageAndVersion(record, record.highestSupportedVersion()); + } + + private List createSegmentLeaderEpochsEntry(RemoteLogSegmentMetadata data) { + return data.segmentLeaderEpochs().entrySet().stream() + .map(entry -> new RemoteLogSegmentMetadataRecord.SegmentLeaderEpochEntry() + .setLeaderEpoch(entry.getKey()) + .setOffset(entry.getValue())) + .collect(Collectors.toList()); + } + + private RemoteLogSegmentMetadataRecord.RemoteLogSegmentIdEntry createRemoteLogSegmentIdEntry(RemoteLogSegmentMetadata data) { + return new RemoteLogSegmentMetadataRecord.RemoteLogSegmentIdEntry() + .setTopicIdPartition( + new RemoteLogSegmentMetadataRecord.TopicIdPartitionEntry() + .setId(data.remoteLogSegmentId().topicIdPartition().topicId()) + .setName(data.remoteLogSegmentId().topicIdPartition().topicPartition().topic()) + .setPartition(data.remoteLogSegmentId().topicIdPartition().topicPartition().partition())) + .setId(data.remoteLogSegmentId().id()); + } + + @Override + public RemoteLogSegmentMetadata fromApiMessageAndVersion(ApiMessageAndVersion apiMessageAndVersion) { + RemoteLogSegmentMetadataRecord record = (RemoteLogSegmentMetadataRecord) apiMessageAndVersion.message(); + RemoteLogSegmentId remoteLogSegmentId = buildRemoteLogSegmentId(record.remoteLogSegmentId()); + + Map segmentLeaderEpochs = new HashMap<>(); + for (RemoteLogSegmentMetadataRecord.SegmentLeaderEpochEntry segmentLeaderEpoch : record.segmentLeaderEpochs()) { + segmentLeaderEpochs.put(segmentLeaderEpoch.leaderEpoch(), segmentLeaderEpoch.offset()); + } + + RemoteLogSegmentMetadata remoteLogSegmentMetadata = + new RemoteLogSegmentMetadata(remoteLogSegmentId, record.startOffset(), record.endOffset(), + record.maxTimestampMs(), record.brokerId(), + record.eventTimestampMs(), record.segmentSizeInBytes(), + segmentLeaderEpochs); + RemoteLogSegmentMetadataUpdate rlsmUpdate + = new RemoteLogSegmentMetadataUpdate(remoteLogSegmentId, record.eventTimestampMs(), + RemoteLogSegmentState.forId(record.remoteLogSegmentState()), + record.brokerId()); + + return remoteLogSegmentMetadata.createWithUpdates(rlsmUpdate); + } + + private RemoteLogSegmentId buildRemoteLogSegmentId(RemoteLogSegmentMetadataRecord.RemoteLogSegmentIdEntry entry) { + TopicIdPartition topicIdPartition = + new TopicIdPartition(entry.topicIdPartition().id(), + new TopicPartition(entry.topicIdPartition().name(), entry.topicIdPartition().partition())); + + return new RemoteLogSegmentId(topicIdPartition, entry.id()); + } +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java new file mode 100644 index 0000000000000..c0f5b2efa2582 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java @@ -0,0 +1,59 @@ +/* + * 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.server.log.remote.metadata.storage.serialization; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataUpdateRecord; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; + +public class RemoteLogSegmentMetadataUpdateTransform implements RemoteLogMetadataTransform { + + public ApiMessageAndVersion toApiMessageAndVersion(RemoteLogSegmentMetadataUpdate segmentMetadataUpdate) { + RemoteLogSegmentMetadataUpdateRecord record = new RemoteLogSegmentMetadataUpdateRecord() + .setRemoteLogSegmentId(createRemoteLogSegmentIdEntry(segmentMetadataUpdate)) + .setBrokerId(segmentMetadataUpdate.brokerId()) + .setEventTimestampMs(segmentMetadataUpdate.eventTimestampMs()) + .setRemoteLogSegmentState(segmentMetadataUpdate.state().id()); + + return new ApiMessageAndVersion(record, record.highestSupportedVersion()); + } + + public RemoteLogSegmentMetadataUpdate fromApiMessageAndVersion(ApiMessageAndVersion apiMessageAndVersion) { + RemoteLogSegmentMetadataUpdateRecord record = (RemoteLogSegmentMetadataUpdateRecord) apiMessageAndVersion.message(); + RemoteLogSegmentMetadataUpdateRecord.RemoteLogSegmentIdEntry entry = record.remoteLogSegmentId(); + TopicIdPartition topicIdPartition = new TopicIdPartition(entry.topicIdPartition().id(), + new TopicPartition(entry.topicIdPartition().name(), entry.topicIdPartition().partition())); + + return new RemoteLogSegmentMetadataUpdate(new RemoteLogSegmentId(topicIdPartition, entry.id()), + record.eventTimestampMs(), RemoteLogSegmentState.forId(record.remoteLogSegmentState()), record.brokerId()); + } + + private RemoteLogSegmentMetadataUpdateRecord.RemoteLogSegmentIdEntry createRemoteLogSegmentIdEntry(RemoteLogSegmentMetadataUpdate data) { + return new RemoteLogSegmentMetadataUpdateRecord.RemoteLogSegmentIdEntry() + .setId(data.remoteLogSegmentId().id()) + .setTopicIdPartition( + new RemoteLogSegmentMetadataUpdateRecord.TopicIdPartitionEntry() + .setName(data.remoteLogSegmentId().topicIdPartition().topicPartition().topic()) + .setPartition(data.remoteLogSegmentId().topicIdPartition().topicPartition().partition()) + .setId(data.remoteLogSegmentId().topicIdPartition().topicId())); + } + +} diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java new file mode 100644 index 0000000000000..09d142c3d1f7e --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java @@ -0,0 +1,54 @@ +/* + * 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.server.log.remote.metadata.storage.serialization; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteState; + +public final class RemotePartitionDeleteMetadataTransform implements RemoteLogMetadataTransform { + + @Override + public ApiMessageAndVersion toApiMessageAndVersion(RemotePartitionDeleteMetadata partitionDeleteMetadata) { + RemotePartitionDeleteMetadataRecord record = new RemotePartitionDeleteMetadataRecord() + .setTopicIdPartition(createTopicIdPartitionEntry(partitionDeleteMetadata.topicIdPartition())) + .setEventTimestampMs(partitionDeleteMetadata.eventTimestampMs()) + .setBrokerId(partitionDeleteMetadata.brokerId()) + .setRemotePartitionDeleteState(partitionDeleteMetadata.state().id()); + return new ApiMessageAndVersion(record, record.highestSupportedVersion()); + } + + private RemotePartitionDeleteMetadataRecord.TopicIdPartitionEntry createTopicIdPartitionEntry(TopicIdPartition topicIdPartition) { + return new RemotePartitionDeleteMetadataRecord.TopicIdPartitionEntry() + .setName(topicIdPartition.topicPartition().topic()) + .setPartition(topicIdPartition.topicPartition().partition()) + .setId(topicIdPartition.topicId()); + } + + public RemotePartitionDeleteMetadata fromApiMessageAndVersion(ApiMessageAndVersion apiMessageAndVersion) { + RemotePartitionDeleteMetadataRecord record = (RemotePartitionDeleteMetadataRecord) apiMessageAndVersion.message(); + TopicIdPartition topicIdPartition = new TopicIdPartition(record.topicIdPartition().id(), + new TopicPartition(record.topicIdPartition().name(), record.topicIdPartition().partition())); + + return new RemotePartitionDeleteMetadata(topicIdPartition, + RemotePartitionDeleteState.forId(record.remotePartitionDeleteState()), + record.eventTimestampMs(), record.brokerId()); + } +} diff --git a/storage/src/main/resources/message/RemoteLogSegmentMetadata.json b/storage/src/main/resources/message/RemoteLogSegmentMetadata.json new file mode 100644 index 0000000000000..d18144e4dfe12 --- /dev/null +++ b/storage/src/main/resources/message/RemoteLogSegmentMetadata.json @@ -0,0 +1,126 @@ +// 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. + +{ + "apiKey": 0, + "type": "metadata", + "name": "RemoteLogSegmentMetadataRecord", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { + "name": "RemoteLogSegmentId", + "type": "RemoteLogSegmentIdEntry", + "versions": "0+", + "about": "Unique representation of the remote log segment.", + "fields": [ + { + "name": "TopicIdPartition", + "type": "TopicIdPartitionEntry", + "versions": "0+", + "about": "Represents unique topic partition.", + "fields": [ + { + "name": "Name", + "type": "string", + "versions": "0+", + "about": "Topic name." + }, + { + "name": "Id", + "type": "uuid", + "versions": "0+", + "about": "Unique identifier of the topic." + }, + { + "name": "Partition", + "type": "int32", + "versions": "0+", + "about": "Partition number." + } + ] + }, + { + "name": "Id", + "type": "uuid", + "versions": "0+", + "about": "Unique identifier of the remote log segment." + } + ] + }, + { + "name": "StartOffset", + "type": "int64", + "versions": "0+", + "about": "Start offset of the segment." + }, + { + "name": "EndOffset", + "type": "int64", + "versions": "0+", + "about": "End offset of the segment." + }, + { + "name": "BrokerId", + "type": "int32", + "versions": "0+", + "about": "Broker id from which this event is generated." + }, + { + "name": "MaxTimestampMs", + "type": "int64", + "versions": "0+", + "about": "Maximum timestamp in milli seconds with in this segment." + }, + { + "name": "EventTimestampMs", + "type": "int64", + "versions": "0+", + "about": "Epoch time in milli seconds at which this event is generated." + }, + { + "name": "SegmentLeaderEpochs", + "type": "[]SegmentLeaderEpochEntry", + "versions": "0+", + "about": "Leader epoch to start-offset mappings for the records with in this segment.", + "fields": [ + { + "name": "LeaderEpoch", + "type": "int32", + "versions": "0+", + "about": "Leader epoch" + }, + { + "name": "Offset", + "type": "int64", + "versions": "0+", + "about": "Start offset for the leader epoch." + } + ] + }, + { + "name": "SegmentSizeInBytes", + "type": "int32", + "versions": "0+", + "about": "Segment size in bytes." + }, + { + "name": "RemoteLogSegmentState", + "type": "int8", + "versions": "0+", + "about": "State identifier of the remote log segment, which is RemoteLogSegmentState.id()." + } + ] +} \ No newline at end of file diff --git a/storage/src/main/resources/message/RemoteLogSegmentMetadataUpdate.json b/storage/src/main/resources/message/RemoteLogSegmentMetadataUpdate.json new file mode 100644 index 0000000000000..24003dcbce849 --- /dev/null +++ b/storage/src/main/resources/message/RemoteLogSegmentMetadataUpdate.json @@ -0,0 +1,82 @@ +// 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. + +{ + "apiKey": 1, + "type": "metadata", + "name": "RemoteLogSegmentMetadataUpdateRecord", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { + "name": "RemoteLogSegmentId", + "type": "RemoteLogSegmentIdEntry", + "versions": "0+", + "about": "Unique representation of the remote log segment.", + "fields": [ + { + "name": "TopicIdPartition", + "type": "TopicIdPartitionEntry", + "versions": "0+", + "about": "Represents unique topic partition.", + "fields": [ + { + "name": "Name", + "type": "string", + "versions": "0+", + "about": "Topic name." + }, + { + "name": "Id", + "type": "uuid", + "versions": "0+", + "about": "Unique identifier of the topic." + }, + { + "name": "Partition", + "type": "int32", + "versions": "0+", + "about": "Partition number." + } + ] + }, + { + "name": "Id", + "type": "uuid", + "versions": "0+", + "about": "Unique identifier of the remote log segment." + } + ] + }, + { + "name": "BrokerId", + "type": "int32", + "versions": "0+", + "about": "Broker id from which this event is generated." + }, + { + "name": "EventTimestampMs", + "type": "int64", + "versions": "0+", + "about": "Epoch time in milli seconds at which this event is generated." + }, + { + "name": "RemoteLogSegmentState", + "type": "int8", + "versions": "0+", + "about": "State identifier of the remote log segment, which is RemoteLogSegmentState.id()." + } + ] +} \ No newline at end of file diff --git a/storage/src/main/resources/message/RemotePartitionDeleteMetadata.json b/storage/src/main/resources/message/RemotePartitionDeleteMetadata.json new file mode 100644 index 0000000000000..f5e955bceaca8 --- /dev/null +++ b/storage/src/main/resources/message/RemotePartitionDeleteMetadata.json @@ -0,0 +1,68 @@ +// 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. + +{ + "apiKey": 2, + "type": "metadata", + "name": "RemotePartitionDeleteMetadataRecord", + "validVersions": "0", + "flexibleVersions": "0+", + "fields": [ + { + "name": "TopicIdPartition", + "type": "TopicIdPartitionEntry", + "versions": "0+", + "about": "Represents unique topic partition.", + "fields": [ + { + "name": "Name", + "type": "string", + "versions": "0+", + "about": "Topic name." + }, + { + "name": "Id", + "type": "uuid", + "versions": "0+", + "about": "Unique identifier of the topic." + }, + { + "name": "Partition", + "type": "int32", + "versions": "0+", + "about": "Partition number." + } + ] + }, + { + "name": "BrokerId", + "type": "int32", + "versions": "0+", + "about": "Broker (controller or leader) id from which this event is created. DELETE_PARTITION_MARKED is sent by the controller. DELETE_PARTITION_STARTED and DELETE_PARTITION_FINISHED are sent by remote log metadata topic partition leader." + }, + { + "name": "EventTimestampMs", + "type": "int64", + "versions": "0+", + "about": "Epoch time in milli seconds at which this event is generated." + }, + { + "name": "RemotePartitionDeleteState", + "type": "int8", + "versions": "0+", + "about": "Deletion state identifier of the remote partition, which is RemotePartitionDeleteState.id()." + } + ] +} \ No newline at end of file diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java new file mode 100644 index 0000000000000..c4305884fbbfa --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataSerdeTest.java @@ -0,0 +1,111 @@ +/* + * 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.server.log.remote.metadata.storage; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteState; +import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +public class RemoteLogMetadataSerdeTest { + + public static final String TOPIC = "foo"; + private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition(TOPIC, 0)); + private final Time time = new MockTime(1); + + @Test + public void testRemoteLogSegmentMetadataSerde() { + RemoteLogSegmentMetadata remoteLogSegmentMetadata = createRemoteLogSegmentMetadata(); + + doTestRemoteLogMetadataSerde(remoteLogSegmentMetadata); + } + + @Test + public void testRemoteLogSegmentMetadataUpdateSerde() { + RemoteLogSegmentMetadataUpdate remoteLogSegmentMetadataUpdate = createRemoteLogSegmentMetadataUpdate(); + + doTestRemoteLogMetadataSerde(remoteLogSegmentMetadataUpdate); + } + + @Test + public void testRemotePartitionDeleteMetadataSerde() { + RemotePartitionDeleteMetadata remotePartitionDeleteMetadata = createRemotePartitionDeleteMetadata(); + + doTestRemoteLogMetadataSerde(remotePartitionDeleteMetadata); + } + + private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() { + Map segLeaderEpochs = new HashMap<>(); + segLeaderEpochs.put(0, 0L); + segLeaderEpochs.put(1, 20L); + segLeaderEpochs.put(2, 80L); + RemoteLogSegmentId remoteLogSegmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + return new RemoteLogSegmentMetadata(remoteLogSegmentId, 0L, 100L, -1L, 1, + time.milliseconds(), 1024, segLeaderEpochs); + } + + private RemoteLogSegmentMetadataUpdate createRemoteLogSegmentMetadataUpdate() { + RemoteLogSegmentId remoteLogSegmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + return new RemoteLogSegmentMetadataUpdate(remoteLogSegmentId, time.milliseconds(), + RemoteLogSegmentState.COPY_SEGMENT_FINISHED, 2); + } + + private RemotePartitionDeleteMetadata createRemotePartitionDeleteMetadata() { + return new RemotePartitionDeleteMetadata(TP0, RemotePartitionDeleteState.DELETE_PARTITION_MARKED, + time.milliseconds(), 0); + } + + private void doTestRemoteLogMetadataSerde(RemoteLogMetadata remoteLogMetadata) { + // Serialize metadata and get the bytes. + RemoteLogMetadataSerde serializer = new RemoteLogMetadataSerde(); + byte[] metadataBytes = serializer.serialize(remoteLogMetadata); + + // Deserialize the bytes and check the RemoteLogMetadata object is as expected. + // Created another RemoteLogMetadataSerde instance to depict the real usecase of serializer and deserializer having their own instances. + RemoteLogMetadataSerde deserializer = new RemoteLogMetadataSerde(); + RemoteLogMetadata deserializedRemoteLogMetadata = deserializer.deserialize(metadataBytes); + Assertions.assertEquals(remoteLogMetadata, deserializedRemoteLogMetadata); + } + + @Test + public void testInvalidRemoteStorageMetadata() { + // Serializing receives an exception as it does not have the expected RemoteLogMetadata registered in serdes. + Assertions.assertThrows(IllegalArgumentException.class, + () -> new RemoteLogMetadataSerde().serialize(new InvalidRemoteLogMetadata(1, time.milliseconds()))); + } + + private static class InvalidRemoteLogMetadata extends RemoteLogMetadata { + public InvalidRemoteLogMetadata(int brokerId, long eventTimestampMs) { + super(brokerId, eventTimestampMs); + } + } + +} \ No newline at end of file diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java new file mode 100644 index 0000000000000..ea7dae820c168 --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java @@ -0,0 +1,85 @@ +/* + * 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.server.log.remote.metadata.storage; + +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogSegmentMetadataTransform; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogSegmentMetadataUpdateTransform; +import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemotePartitionDeleteMetadataTransform; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; +import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentState; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; +import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteState; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +public class RemoteLogMetadataTransformTest { + private static final TopicIdPartition TP0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + private final Time time = new MockTime(1); + + @Test + public void testRemoteLogSegmentMetadataTransform() { + RemoteLogSegmentMetadataTransform metadataTransform = new RemoteLogSegmentMetadataTransform(); + + RemoteLogSegmentMetadata metadata = createRemoteLogSegmentMetadata(); + ApiMessageAndVersion apiMessageAndVersion = metadataTransform.toApiMessageAndVersion(metadata); + RemoteLogSegmentMetadata remoteLogSegmentMetadataFromRecord = metadataTransform + .fromApiMessageAndVersion(apiMessageAndVersion); + + Assertions.assertEquals(metadata, remoteLogSegmentMetadataFromRecord); + } + + @Test + public void testRemoteLogSegmentMetadataUpdateTransform() { + RemoteLogSegmentMetadataUpdateTransform metadataUpdateTransform = new RemoteLogSegmentMetadataUpdateTransform(); + + RemoteLogSegmentMetadataUpdate metadataUpdate = + new RemoteLogSegmentMetadataUpdate(new RemoteLogSegmentId(TP0, Uuid.randomUuid()), time.milliseconds(), + RemoteLogSegmentState.COPY_SEGMENT_FINISHED, 1); + ApiMessageAndVersion apiMessageAndVersion = metadataUpdateTransform.toApiMessageAndVersion(metadataUpdate); + RemoteLogSegmentMetadataUpdate metadataUpdateFromRecord = metadataUpdateTransform.fromApiMessageAndVersion(apiMessageAndVersion); + + Assertions.assertEquals(metadataUpdate, metadataUpdateFromRecord); + } + + private RemoteLogSegmentMetadata createRemoteLogSegmentMetadata() { + RemoteLogSegmentId remoteLogSegmentId = new RemoteLogSegmentId(TP0, Uuid.randomUuid()); + return new RemoteLogSegmentMetadata(remoteLogSegmentId, 0L, 100L, -1L, 1, + time.milliseconds(), 1024, Collections.singletonMap(0, 0L)); + } + + @Test + public void testRemoteLogPartitionMetadataTransform() { + RemotePartitionDeleteMetadataTransform transform = new RemotePartitionDeleteMetadataTransform(); + + RemotePartitionDeleteMetadata partitionDeleteMetadata + = new RemotePartitionDeleteMetadata(TP0, RemotePartitionDeleteState.DELETE_PARTITION_STARTED, time.milliseconds(), 1); + ApiMessageAndVersion apiMessageAndVersion = transform.toApiMessageAndVersion(partitionDeleteMetadata); + RemotePartitionDeleteMetadata partitionDeleteMetadataFromRecord = transform.fromApiMessageAndVersion(apiMessageAndVersion); + + Assertions.assertEquals(partitionDeleteMetadata, partitionDeleteMetadataFromRecord); + } +} From d915ce58d2adcfd6113f961ecbf337770dbe760b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Sergio=20Pe=C3=B1a?= Date: Wed, 5 May 2021 12:21:43 -0500 Subject: [PATCH 119/155] KAFKA-10847: Set shared outer store to an in-memory store when in-memory stores are supplied (#10613) When users supply in-memory stores for left/outer joins, then the internal shared outer store must be switch to in-memory store too. This will allow users who want to keep all stores in memory to continue doing so. Added unit tests to validate topology and left/outer joins work fine with an in-memory shared store. Reviewers: Guozhang Wang --- .../kstream/internals/KStreamImplJoin.java | 76 +++++--- .../apache/kafka/streams/TopologyTest.java | 165 ++++++++++++++++++ .../internals/KStreamKStreamLeftJoinTest.java | 33 +++- .../KStreamKStreamOuterJoinTest.java | 33 +++- 4 files changed, 276 insertions(+), 31 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java index 0977640fd0668..c5e918f9c92e6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImplJoin.java @@ -144,18 +144,7 @@ public KStream join(final KStream lhs, Optional, LeftOrRightValue>>> outerJoinWindowStore = Optional.empty(); if (leftOuter) { - final String outerJoinSuffix = rightOuter ? "-outer-shared-join" : "-left-shared-join"; - - // Get the suffix index of the joinThisGeneratedName to build the outer join store name. - final String outerJoinStoreGeneratedName = KStreamImpl.OUTERSHARED_NAME - + joinThisGeneratedName.substring( - rightOuter - ? KStreamImpl.OUTERTHIS_NAME.length() - : KStreamImpl.JOINTHIS_NAME.length()); - - final String outerJoinStoreName = userProvidedBaseStoreName == null ? outerJoinStoreGeneratedName : userProvidedBaseStoreName + outerJoinSuffix; - - outerJoinWindowStore = Optional.of(sharedOuterJoinWindowStoreBuilder(outerJoinStoreName, windows, streamJoinedInternal)); + outerJoinWindowStore = Optional.of(sharedOuterJoinWindowStoreBuilder(windows, streamJoinedInternal, joinThisGeneratedName)); } // Time shared between joins to keep track of the maximum stream time @@ -263,20 +252,57 @@ private static StoreBuilder> joinWindowStoreBuilder(fin return builder; } + private String buildOuterJoinWindowStoreName(final StreamJoinedInternal streamJoinedInternal, final String joinThisGeneratedName) { + final String outerJoinSuffix = rightOuter ? "-outer-shared-join" : "-left-shared-join"; + + if (streamJoinedInternal.thisStoreSupplier() != null && !streamJoinedInternal.thisStoreSupplier().name().isEmpty()) { + return streamJoinedInternal.thisStoreSupplier().name() + outerJoinSuffix; + } else if (streamJoinedInternal.storeName() != null) { + return streamJoinedInternal.storeName() + outerJoinSuffix; + } else { + return KStreamImpl.OUTERSHARED_NAME + + joinThisGeneratedName.substring( + rightOuter + ? KStreamImpl.OUTERTHIS_NAME.length() + : KStreamImpl.JOINTHIS_NAME.length()); + } + } + @SuppressWarnings("unchecked") - private static StoreBuilder, LeftOrRightValue>> sharedOuterJoinWindowStoreBuilder(final String storeName, - final JoinWindows windows, - final StreamJoinedInternal streamJoinedInternal) { - final StoreBuilder, LeftOrRightValue>> builder = new TimeOrderedWindowStoreBuilder, LeftOrRightValue>( - persistentTimeOrderedWindowStore( - storeName + "-store", - Duration.ofMillis(windows.size() + windows.gracePeriodMs()), - Duration.ofMillis(windows.size()) - ), - new KeyAndJoinSideSerde<>(streamJoinedInternal.keySerde()), - new LeftOrRightValueSerde(streamJoinedInternal.valueSerde(), streamJoinedInternal.otherValueSerde()), - Time.SYSTEM - ); + private StoreBuilder, LeftOrRightValue>> sharedOuterJoinWindowStoreBuilder(final JoinWindows windows, + final StreamJoinedInternal streamJoinedInternal, + final String joinThisGeneratedName) { + final boolean persistent = streamJoinedInternal.thisStoreSupplier() == null || streamJoinedInternal.thisStoreSupplier().get().persistent(); + final String storeName = buildOuterJoinWindowStoreName(streamJoinedInternal, joinThisGeneratedName); + + final KeyAndJoinSideSerde keyAndJoinSideSerde = new KeyAndJoinSideSerde<>(streamJoinedInternal.keySerde()); + final LeftOrRightValueSerde leftOrRightValueSerde = new LeftOrRightValueSerde(streamJoinedInternal.valueSerde(), streamJoinedInternal.otherValueSerde()); + + final StoreBuilder, LeftOrRightValue>> builder; + if (persistent) { + builder = new TimeOrderedWindowStoreBuilder, LeftOrRightValue>( + persistentTimeOrderedWindowStore( + storeName + "-store", + Duration.ofMillis(windows.size() + windows.gracePeriodMs()), + Duration.ofMillis(windows.size()) + ), + keyAndJoinSideSerde, + leftOrRightValueSerde, + Time.SYSTEM + ); + } else { + builder = Stores.windowStoreBuilder( + Stores.inMemoryWindowStore( + storeName + "-store", + Duration.ofMillis(windows.size() + windows.gracePeriodMs()), + Duration.ofMillis(windows.size()), + false + ), + keyAndJoinSideSerde, + leftOrRightValueSerde + ); + } + if (streamJoinedInternal.loggingEnabled()) { builder.withLoggingEnabled(streamJoinedInternal.logConfig()); } else { diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java index 526284b90f892..66c678d2bd17f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java @@ -36,6 +36,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockKeyValueStore; @@ -45,6 +47,7 @@ import org.junit.Assert; import org.junit.Test; +import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -778,6 +781,60 @@ public void streamStreamJoinTopologyWithCustomStoresNames() { describe.toString()); } + @Test + public void streamStreamJoinTopologyWithCustomStoresSuppliers() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + final JoinWindows joinWindows = JoinWindows.of(ofMillis(100)); + + final WindowBytesStoreSupplier thisStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + final WindowBytesStoreSupplier otherStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store-other", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + stream1.join( + stream2, + MockValueJoiner.TOSTRING_JOINER, + joinWindows, + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + .withThisStoreSupplier(thisStoreSupplier) + .withOtherStoreSupplier(otherStoreSupplier)); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [in-memory-join-store])\n" + + " --> KSTREAM-JOINTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [in-memory-join-store-other])\n" + + " --> KSTREAM-JOINOTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-JOINOTHER-0000000005 (stores: [in-memory-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-JOINTHIS-0000000004 (stores: [in-memory-join-store-other])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-JOINTHIS-0000000004, KSTREAM-JOINOTHER-0000000005\n\n", + describe.toString()); + } + @Test public void streamStreamLeftJoinTopologyWithDefaultStoresNames() { final StreamsBuilder builder = new StreamsBuilder(); @@ -863,6 +920,60 @@ public void streamStreamLeftJoinTopologyWithCustomStoresNames() { describe.toString()); } + @Test + public void streamStreamLeftJoinTopologyWithCustomStoresSuppliers() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + final JoinWindows joinWindows = JoinWindows.of(ofMillis(100)); + + final WindowBytesStoreSupplier thisStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + final WindowBytesStoreSupplier otherStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store-other", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + stream1.leftJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + joinWindows, + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + .withThisStoreSupplier(thisStoreSupplier) + .withOtherStoreSupplier(otherStoreSupplier)); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [in-memory-join-store])\n" + + " --> KSTREAM-JOINTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [in-memory-join-store-other])\n" + + " --> KSTREAM-OUTEROTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-JOINTHIS-0000000004 (stores: [in-memory-join-store-other, in-memory-join-store-left-shared-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-OUTEROTHER-0000000005 (stores: [in-memory-join-store, in-memory-join-store-left-shared-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-JOINTHIS-0000000004, KSTREAM-OUTEROTHER-0000000005\n\n", + describe.toString()); + } + @Test public void streamStreamOuterJoinTopologyWithDefaultStoresNames() { final StreamsBuilder builder = new StreamsBuilder(); @@ -948,6 +1059,60 @@ public void streamStreamOuterJoinTopologyWithCustomStoresNames() { describe.toString()); } + @Test + public void streamStreamOuterJoinTopologyWithCustomStoresSuppliers() { + final StreamsBuilder builder = new StreamsBuilder(); + final KStream stream1; + final KStream stream2; + + stream1 = builder.stream("input-topic1"); + stream2 = builder.stream("input-topic2"); + + final JoinWindows joinWindows = JoinWindows.of(ofMillis(100)); + + final WindowBytesStoreSupplier thisStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + final WindowBytesStoreSupplier otherStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store-other", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + stream1.outerJoin( + stream2, + MockValueJoiner.TOSTRING_JOINER, + joinWindows, + StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()) + .withThisStoreSupplier(thisStoreSupplier) + .withOtherStoreSupplier(otherStoreSupplier)); + + final TopologyDescription describe = builder.build().describe(); + + assertEquals( + "Topologies:\n" + + " Sub-topology: 0\n" + + " Source: KSTREAM-SOURCE-0000000000 (topics: [input-topic1])\n" + + " --> KSTREAM-WINDOWED-0000000002\n" + + " Source: KSTREAM-SOURCE-0000000001 (topics: [input-topic2])\n" + + " --> KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-WINDOWED-0000000002 (stores: [in-memory-join-store])\n" + + " --> KSTREAM-OUTERTHIS-0000000004\n" + + " <-- KSTREAM-SOURCE-0000000000\n" + + " Processor: KSTREAM-WINDOWED-0000000003 (stores: [in-memory-join-store-other])\n" + + " --> KSTREAM-OUTEROTHER-0000000005\n" + + " <-- KSTREAM-SOURCE-0000000001\n" + + " Processor: KSTREAM-OUTEROTHER-0000000005 (stores: [in-memory-join-store-outer-shared-join-store, in-memory-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000003\n" + + " Processor: KSTREAM-OUTERTHIS-0000000004 (stores: [in-memory-join-store-other, in-memory-join-store-outer-shared-join-store])\n" + + " --> KSTREAM-MERGE-0000000006\n" + + " <-- KSTREAM-WINDOWED-0000000002\n" + + " Processor: KSTREAM-MERGE-0000000006 (stores: [])\n" + + " --> none\n" + + " <-- KSTREAM-OUTERTHIS-0000000004, KSTREAM-OUTEROTHER-0000000005\n\n", + describe.toString()); + } + @Test public void topologyWithDynamicRoutingShouldDescribeExtractorClass() { final StreamsBuilder builder = new StreamsBuilder(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 5c4bccfdef747..50c2f86008b3c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -30,6 +30,8 @@ import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.kstream.StreamJoined; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -429,7 +431,32 @@ public void testRightNonJoinedRecordsAreNeverEmittedByTheRightProcessor() { } @Test - public void testLeftJoin() { + public void testLeftJoinWithInMemoryCustomSuppliers() { + final JoinWindows joinWindows = JoinWindows.of(ofMillis(100)); + + final WindowBytesStoreSupplier thisStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + final WindowBytesStoreSupplier otherStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store-other", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + final StreamJoined streamJoined = StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()); + + runLeftJoin(streamJoined.withThisStoreSupplier(thisStoreSupplier).withOtherStoreSupplier(otherStoreSupplier), joinWindows); + } + + @Test + public void testLeftJoinWithDefaultSuppliers() { + final JoinWindows joinWindows = JoinWindows.of(ofMillis(100)); + final StreamJoined streamJoined = StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()); + + runLeftJoin(streamJoined, joinWindows); + } + + public void runLeftJoin(final StreamJoined streamJoined, + final JoinWindows joinWindows) { final StreamsBuilder builder = new StreamsBuilder(); final int[] expectedKeys = new int[] {0, 1, 2, 3}; @@ -444,8 +471,8 @@ public void testLeftJoin() { joined = stream1.leftJoin( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(100)), - StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joinWindows, + streamJoined); joined.process(supplier); final Collection> copartitionGroups = diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java index c0f88d88944d1..2bf7fef9132ff 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -30,6 +30,8 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.StreamJoined; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; +import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.streams.state.WindowBytesStoreSupplier; import org.apache.kafka.test.MockProcessor; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; @@ -498,7 +500,32 @@ public void testGracePeriod() { } @Test - public void testOuterJoin() { + public void testOuterJoinWithInMemoryCustomSuppliers() { + final JoinWindows joinWindows = JoinWindows.of(ofMillis(100)); + + final WindowBytesStoreSupplier thisStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + final WindowBytesStoreSupplier otherStoreSupplier = Stores.inMemoryWindowStore("in-memory-join-store-other", + Duration.ofMillis(joinWindows.size() + joinWindows.gracePeriodMs()), + Duration.ofMillis(joinWindows.size()), true); + + final StreamJoined streamJoined = StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()); + + runOuterJoin(streamJoined.withThisStoreSupplier(thisStoreSupplier).withOtherStoreSupplier(otherStoreSupplier), joinWindows); + } + + @Test + public void testOuterJoinWithDefaultSuppliers() { + final JoinWindows joinWindows = JoinWindows.of(ofMillis(100)); + final StreamJoined streamJoined = StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String()); + + runOuterJoin(streamJoined, joinWindows); + } + + public void runOuterJoin(final StreamJoined streamJoined, + final JoinWindows joinWindows) { final StreamsBuilder builder = new StreamsBuilder(); final int[] expectedKeys = new int[] {0, 1, 2, 3}; @@ -513,8 +540,8 @@ public void testOuterJoin() { joined = stream1.outerJoin( stream2, MockValueJoiner.TOSTRING_JOINER, - JoinWindows.of(ofMillis(100)).grace(ofMillis(0)), - StreamJoined.with(Serdes.Integer(), Serdes.String(), Serdes.String())); + joinWindows, + streamJoined); joined.process(supplier); final Collection> copartitionGroups = From d881d113886b837f382d89a02cf7f3c5715edf6e Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Thu, 6 May 2021 02:12:00 +0800 Subject: [PATCH 120/155] MINOR: fix streams_broker_compatibility_test.py (#10632) The log message was changed and so the system test can't capture expected message. Reviewers: Anna Sophie Blee-Goldman ableegoldman@apache.org> --- .../tests/streams/streams_broker_compatibility_test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py index 3bca7062d8b71..1e7ece7bd1895 100644 --- a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py +++ b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py @@ -168,9 +168,9 @@ def test_fail_fast_on_incompatible_brokers_if_eos_beta_enabled(self, broker_vers with processor.node.account.monitor_log(processor.STDERR_FILE) as monitor: with processor.node.account.monitor_log(processor.LOG_FILE) as log: processor.start() - log.wait_until('Shutting down because the Kafka cluster seems to be on a too old version. Setting processing\.guarantee="exactly_once_beta" requires broker version 2\.5 or higher\.', + log.wait_until('Shutting down because the Kafka cluster seems to be on a too old version. Setting processing\.guarantee="exactly_once_v2"/"exactly_once_beta" requires broker version 2\.5 or higher\.', timeout_sec=60, - err_msg="Never saw 'Shutting down, because the Kafka cluster seems to be on a too old version. Setting `processing.guarantee=\"exaclty_once_beta\"` requires broker version 2.5 or higher.' log message " + str(processor.node.account)) + err_msg="Never saw 'Shutting down because the Kafka cluster seems to be on a too old version. Setting `processing.guarantee=\"exactly_once_v2\"/\"exaclty_once_beta\"` requires broker version 2.5 or higher.' log message " + str(processor.node.account)) monitor.wait_until('FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException', timeout_sec=60, err_msg="Never saw 'FATAL: An unexpected exception org.apache.kafka.common.errors.UnsupportedVersionException' error message " + str(processor.node.account)) From 9ba583f6d6677a48685b37eb4b2bef2a711c94b7 Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Wed, 5 May 2021 17:11:15 -0400 Subject: [PATCH 121/155] KAFKA-12252 and KAFKA-12262: Fix session key rotation when leadership changes (#10014) Author: Chris Egerton Reviewers: Greg Harris , Randall Hauch --- checkstyle/suppressions.xml | 3 + .../distributed/DistributedHerder.java | 9 +- .../distributed/DistributedHerderTest.java | 105 +++++++++++++++++- 3 files changed, 110 insertions(+), 7 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 29ec30687c6d2..03ba807e6d025 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -147,6 +147,9 @@ + + diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index b3b9ba9b94d2e..9c7bf96d942d5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -408,7 +408,7 @@ public void tick() { log.debug("Scheduled rebalance at: {} (now: {} nextRequestTimeoutMs: {}) ", scheduledRebalance, now, nextRequestTimeoutMs); } - if (internalRequestValidationEnabled() && keyExpiration < Long.MAX_VALUE) { + if (isLeader() && internalRequestValidationEnabled() && keyExpiration < Long.MAX_VALUE) { nextRequestTimeoutMs = Math.min(nextRequestTimeoutMs, Math.max(keyExpiration - now, 0)); log.debug("Scheduled next key rotation at: {} (now: {} nextRequestTimeoutMs: {}) ", keyExpiration, now, nextRequestTimeoutMs); @@ -1583,10 +1583,11 @@ public void onSessionKeyUpdate(SessionKey sessionKey) { synchronized (DistributedHerder.this) { DistributedHerder.this.sessionKey = sessionKey.key(); - // Track the expiration of the key if and only if this worker is the leader + // Track the expiration of the key. // Followers will receive rotated keys from the leader and won't be responsible for - // tracking expiration and distributing new keys themselves - if (isLeader() && keyRotationIntervalMs > 0) { + // tracking expiration and distributing new keys themselves, but may become leaders + // later on and will need to know when to update the key. + if (keyRotationIntervalMs > 0) { DistributedHerder.this.keyExpiration = sessionKey.creationTimestamp() + keyRotationIntervalMs; } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 65ec89c8cf99b..1843868c85790 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -2076,6 +2076,84 @@ public void testPutConnectorConfig() throws Exception { PowerMock.verifyAll(); } + @Test + public void testKeyRotationWhenWorkerBecomesLeader() throws Exception { + EasyMock.expect(member.memberId()).andStubReturn("member"); + EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); + + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + // First rebalance: poll indefinitely as no key has been read yet, so expiration doesn't come into play + member.poll(Long.MAX_VALUE); + EasyMock.expectLastCall(); + + expectRebalance(2, Collections.emptyList(), Collections.emptyList()); + SessionKey initialKey = new SessionKey(EasyMock.mock(SecretKey.class), 0); + ClusterConfigState snapshotWithKey = new ClusterConfigState(2, initialKey, Collections.singletonMap(CONN1, 3), + Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, Collections.emptySet()); + expectPostRebalanceCatchup(snapshotWithKey); + // Second rebalance: poll indefinitely as worker is follower, so expiration still doesn't come into play + member.poll(Long.MAX_VALUE); + EasyMock.expectLastCall(); + + expectRebalance(2, Collections.emptyList(), Collections.emptyList(), "member", MEMBER_URL); + Capture updatedKey = EasyMock.newCapture(); + configBackingStore.putSessionKey(EasyMock.capture(updatedKey)); + EasyMock.expectLastCall().andAnswer(() -> { + configUpdateListener.onSessionKeyUpdate(updatedKey.getValue()); + return null; + }); + // Third rebalance: poll for a limited time as worker has become leader and must wake up for key expiration + Capture pollTimeout = EasyMock.newCapture(); + member.poll(EasyMock.captureLong(pollTimeout)); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.tick(); + configUpdateListener.onSessionKeyUpdate(initialKey); + herder.tick(); + herder.tick(); + + assertTrue(pollTimeout.getValue() <= DistributedConfig.INTER_WORKER_KEY_TTL_MS_MS_DEFAULT); + + PowerMock.verifyAll(); + } + + @Test + public void testKeyRotationDisabledWhenWorkerBecomesFollower() throws Exception { + EasyMock.expect(member.memberId()).andStubReturn("member"); + EasyMock.expect(member.currentProtocolVersion()).andStubReturn(CONNECT_PROTOCOL_V2); + + expectRebalance(1, Collections.emptyList(), Collections.emptyList(), "member", MEMBER_URL); + SecretKey initialSecretKey = EasyMock.mock(SecretKey.class); + EasyMock.expect(initialSecretKey.getAlgorithm()).andReturn(DistributedConfig.INTER_WORKER_KEY_GENERATION_ALGORITHM_DEFAULT).anyTimes(); + EasyMock.expect(initialSecretKey.getEncoded()).andReturn(new byte[32]).anyTimes(); + SessionKey initialKey = new SessionKey(initialSecretKey, time.milliseconds()); + ClusterConfigState snapshotWithKey = new ClusterConfigState(1, initialKey, Collections.singletonMap(CONN1, 3), + Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, Collections.emptySet()); + expectPostRebalanceCatchup(snapshotWithKey); + // First rebalance: poll for a limited time as worker is leader and must wake up for key expiration + Capture firstPollTimeout = EasyMock.newCapture(); + member.poll(EasyMock.captureLong(firstPollTimeout)); + EasyMock.expectLastCall(); + + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + // Second rebalance: poll indefinitely as worker is no longer leader, so key expiration doesn't come into play + member.poll(Long.MAX_VALUE); + EasyMock.expectLastCall(); + + PowerMock.replayAll(initialSecretKey); + + configUpdateListener.onSessionKeyUpdate(initialKey); + herder.tick(); + assertTrue(firstPollTimeout.getValue() <= DistributedConfig.INTER_WORKER_KEY_TTL_MS_MS_DEFAULT); + herder.tick(); + + PowerMock.verifyAll(); + } @Test public void testPutTaskConfigsSignatureNotRequiredV0() { @@ -2311,6 +2389,14 @@ private void expectRebalance(final long offset, ConnectProtocol.Assignment.NO_ERROR, offset, assignedConnectors, assignedTasks, 0); } + private void expectRebalance(final long offset, + final List assignedConnectors, + final List assignedTasks, + String leader, String leaderUrl) { + expectRebalance(Collections.emptyList(), Collections.emptyList(), + ConnectProtocol.Assignment.NO_ERROR, offset, leader, leaderUrl, assignedConnectors, assignedTasks, 0); + } + // Handles common initial part of rebalance callback. Does not handle instantiation of connectors and tasks. private void expectRebalance(final Collection revokedConnectors, final List revokedTasks, @@ -2329,21 +2415,34 @@ private void expectRebalance(final Collection revokedConnectors, final List assignedConnectors, final List assignedTasks, int delay) { + expectRebalance(revokedConnectors, revokedTasks, error, offset, "leader", "leaderUrl", assignedConnectors, assignedTasks, delay); + } + + // Handles common initial part of rebalance callback. Does not handle instantiation of connectors and tasks. + private void expectRebalance(final Collection revokedConnectors, + final List revokedTasks, + final short error, + final long offset, + String leader, + String leaderUrl, + final List assignedConnectors, + final List assignedTasks, + int delay) { member.ensureActive(); PowerMock.expectLastCall().andAnswer(() -> { ExtendedAssignment assignment; if (!revokedConnectors.isEmpty() || !revokedTasks.isEmpty()) { - rebalanceListener.onRevoked("leader", revokedConnectors, revokedTasks); + rebalanceListener.onRevoked(leader, revokedConnectors, revokedTasks); } if (connectProtocolVersion == CONNECT_PROTOCOL_V0) { assignment = new ExtendedAssignment( - connectProtocolVersion, error, "leader", "leaderUrl", offset, + connectProtocolVersion, error, leader, leaderUrl, offset, assignedConnectors, assignedTasks, Collections.emptyList(), Collections.emptyList(), 0); } else { assignment = new ExtendedAssignment( - connectProtocolVersion, error, "leader", "leaderUrl", offset, + connectProtocolVersion, error, leader, leaderUrl, offset, assignedConnectors, assignedTasks, new ArrayList<>(revokedConnectors), new ArrayList<>(revokedTasks), delay); } From 6a5992a814364a41ce7ad34aa6b27cc44d0eb78c Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Wed, 5 May 2021 16:16:06 -0700 Subject: [PATCH 122/155] KAFKA-8531: Change default replication factor config (#10532) Implements KIP-733 Reviewers: A. Sophie Blee-Goldman --- .../developer-guide/config-streams.html | 6 +- docs/streams/upgrade-guide.html | 7 ++- .../apache/kafka/streams/StreamsConfig.java | 14 ++--- .../internals/InternalTopicManager.java | 30 ++++++++-- .../internals/InternalTopicManagerTest.java | 57 +++++++++++++++++++ 5 files changed, 99 insertions(+), 15 deletions(-) diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html index 1f5002f08914c..1f8e8f3d7f86b 100644 --- a/docs/streams/developer-guide/config-streams.html +++ b/docs/streams/developer-guide/config-streams.html @@ -304,10 +304,10 @@

        bootstrap.servers

      - + - + The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer. + diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 1990d721c8196..6f13cc5d36c15 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -144,7 +144,12 @@

      Streams API
    • Connect-json: As of Kafka Streams no longer has a compile time dependency on "connect:json" module (KAFKA-5146). Projects that were relying on this transitive dependency will have to explicitly declare it.
    • - +

      + The default value for configuration parameter replication.factor was changed to -1 + (meaning: use broker default replication factor). + The replication.factor value of -1 requires broker version 2.4 or newer. +

      +

      Streams API changes in 2.8.0

      We extended StreamJoined to include the options withLoggingEnabled() and withLoggingDisabled() in diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 89154c7b64c61..0e9e6a149f9b6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -501,8 +501,8 @@ public class StreamsConfig extends AbstractConfig { /** {@code replication.factor} */ @SuppressWarnings("WeakerAccess") public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; - private static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application." + - " If your broker cluster is on version 2.4 or newer, you can set -1 to use the broker default replication factor."; + private static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application." + + " The default of -1 (meaning: use broker default replication factor) requires broker version 2.4 or newer"; /** {@code request.timeout.ms} */ @SuppressWarnings("WeakerAccess") @@ -608,11 +608,6 @@ public class StreamsConfig extends AbstractConfig { Type.LIST, Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) - .define(REPLICATION_FACTOR_CONFIG, - Type.INT, - 1, - Importance.HIGH, - REPLICATION_FACTOR_DOC) .define(STATE_DIR_CONFIG, Type.STRING, System.getProperty("java.io.tmpdir") + File.separator + "kafka-streams", @@ -700,6 +695,11 @@ public class StreamsConfig extends AbstractConfig { in(AT_LEAST_ONCE, EXACTLY_ONCE, EXACTLY_ONCE_BETA, EXACTLY_ONCE_V2), Importance.MEDIUM, PROCESSING_GUARANTEE_DOC) + .define(REPLICATION_FACTOR_CONFIG, + Type.INT, + -1, + Importance.MEDIUM, + REPLICATION_FACTOR_DOC) .define(SECURITY_PROTOCOL_CONFIG, Type.STRING, CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 2ca130a9dd2f2..c612245b0769d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -439,13 +440,34 @@ public Set makeReady(final Map topics) { final Throwable cause = executionException.getCause(); if (cause instanceof TopicExistsException) { // This topic didn't exist earlier or its leader not known before; just retain it for next round of validation. - log.info("Could not create topic {}. Topic is probably marked for deletion (number of partitions is unknown).\n" + - "Will retry to create this topic in {} ms (to let broker finish async delete operation first).\n" + - "Error message was: {}", topicName, retryBackOffMs, cause.toString()); + log.info( + "Could not create topic {}. Topic is probably marked for deletion (number of partitions is unknown).\n" + + + "Will retry to create this topic in {} ms (to let broker finish async delete operation first).\n" + + + "Error message was: {}", topicName, retryBackOffMs, + cause.toString()); } else { log.error("Unexpected error during topic creation for {}.\n" + "Error message was: {}", topicName, cause.toString()); - throw new StreamsException(String.format("Could not create topic %s.", topicName), cause); + + if (cause instanceof UnsupportedVersionException) { + final String errorMessage = cause.getMessage(); + if (errorMessage != null && + errorMessage.startsWith("Creating topics with default partitions/replication factor are only supported in CreateTopicRequest version 4+")) { + + throw new StreamsException(String.format( + "Could not create topic %s, because brokers don't support configuration replication.factor=-1." + + " You can change the replication.factor config or upgrade your brokers to version 2.4 or newer to avoid this error.", + topicName) + ); + } + } else { + throw new StreamsException( + String.format("Could not create topic %s.", topicName), + cause + ); + } } } catch (final TimeoutException retriableException) { log.error("Creating topic {} timed out.\n" + diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java index 653250c9d8a7a..5b2b64e7803a5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.Config; import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.CreateTopicsOptions; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.CreateTopicsResult.TopicMetadataAndConfig; import org.apache.kafka.clients.admin.DeleteTopicsResult; @@ -40,7 +41,13 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicExistsException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.message.CreateTopicsRequestData; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignmentCollection; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection; +import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; @@ -53,6 +60,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -222,6 +230,55 @@ private void shouldRetryCreateTopicWhenRetriableExceptionIsThrown(final Exceptio )); } + @Test + public void shouldThrowInformativeExceptionForOlderBrokers() { + final AdminClient admin = new MockAdminClient() { + @Override + public CreateTopicsResult createTopics(final Collection newTopics, + final CreateTopicsOptions options) { + final CreatableTopic topicToBeCreated = new CreatableTopic(); + topicToBeCreated.setAssignments(new CreatableReplicaAssignmentCollection()); + topicToBeCreated.setNumPartitions((short) 1); + // set unsupported replication factor for older brokers + topicToBeCreated.setReplicationFactor((short) -1); + + final CreatableTopicCollection topicsToBeCreated = new CreatableTopicCollection(); + topicsToBeCreated.add(topicToBeCreated); + + try { + new CreateTopicsRequest.Builder( + new CreateTopicsRequestData() + .setTopics(topicsToBeCreated) + .setTimeoutMs(0) + .setValidateOnly(options.shouldValidateOnly())) + .build((short) 3); // pass in old unsupported request version for old brokers + + throw new IllegalStateException("Building CreateTopicRequest should have thrown."); + } catch (final UnsupportedVersionException expected) { + final KafkaFutureImpl future = new KafkaFutureImpl<>(); + future.completeExceptionally(expected); + + return new CreateTopicsResult(Collections.singletonMap(topic1, future)) { }; + } + } + }; + + final StreamsConfig streamsConfig = new StreamsConfig(config); + final InternalTopicManager topicManager = new InternalTopicManager(Time.SYSTEM, admin, streamsConfig); + + final InternalTopicConfig topicConfig = new RepartitionTopicConfig(topic1, Collections.emptyMap()); + topicConfig.setNumberOfPartitions(1); + + final StreamsException exception = assertThrows( + StreamsException.class, + () -> topicManager.makeReady(Collections.singletonMap(topic1, topicConfig)) + ); + assertThat( + exception.getMessage(), + equalTo("Could not create topic " + topic1 + ", because brokers don't support configuration replication.factor=-1." + + " You can change the replication.factor config or upgrade your brokers to version 2.4 or newer to avoid this error.")); + } + @Test public void shouldThrowTimeoutExceptionIfTopicExistsDuringSetup() { setupTopicInMockAdminClient(topic1, Collections.emptyMap()); From 79225504ed920e63b2a31a968b7d50f88af5ada2 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Thu, 6 May 2021 09:44:59 +0800 Subject: [PATCH 123/155] KAFKA-12464: enhance constrained sticky Assign algorithm (#10509) 1. Make code simpler and cleaner 2. After the PR: the testLargeAssignmentAndGroupWithUniformSubscription (1 million partitions) will run from ~2600 ms down to ~1400 ms, improves 46% of performance, almost 2x faster!! Reviewers: Anna Sophie Blee-Goldman , Guozhang Wang --- .../internals/AbstractStickyAssignor.java | 267 +++++++++++------- .../internals/AbstractStickyAssignorTest.java | 145 ++++++++++ 2 files changed, 313 insertions(+), 99 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java index 7e42e44a7d4c8..24c8107f25e3b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java @@ -28,9 +28,7 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Optional; -import java.util.Queue; import java.util.Set; -import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; import java.util.stream.Collectors; @@ -82,6 +80,8 @@ public Map> assign(Map partitionsP log.debug("Detected that all not consumers were subscribed to same set of topics, falling back to the " + "general case assignment algorithm"); partitionsTransferringOwnership = null; + // we don't need consumerToOwnedPartitions in general assign case + consumerToOwnedPartitions = null; return generalAssign(partitionsPerTopic, subscriptions); } } @@ -149,141 +149,210 @@ private boolean allSubscriptionsEqual(Set allTopics, * This constrainedAssign optimizes the assignment algorithm when all consumers were subscribed to same set of topics. * The method includes the following steps: * - * 1. Reassign as many previously owned partitions as possible, up to the maxQuota - * 2. Fill remaining members up to minQuota - * 3. If we ran out of unassigned partitions before filling all consumers, we need to start stealing partitions - * from the over-full consumers at max capacity - * 4. Otherwise we may have run out of unfilled consumers before assigning all partitions, in which case we - * should just distribute one partition each to all consumers at min capacity + * 1. Reassign previously owned partitions: + * a. if owned less than minQuota partitions, just assign all owned partitions, and put the member into unfilled member list + * b. if owned maxQuota or more, and we're still under the number of expected max capacity members, assign maxQuota partitions + * c. if owned at least "minQuota" of partitions, assign minQuota partitions, and put the member into unfilled member list if + * we're still under the number of expected max capacity members + * 2. Fill remaining members up to the expected numbers of maxQuota partitions, otherwise, to minQuota partitions * * @param partitionsPerTopic The number of partitions for each subscribed topic * @param consumerToOwnedPartitions Each consumer's previously owned and still-subscribed partitions * - * @return Map from each member to the list of partitions assigned to them. + * @return Map from each member to the list of partitions assigned to them. */ private Map> constrainedAssign(Map partitionsPerTopic, Map> consumerToOwnedPartitions) { - SortedSet unassignedPartitions = getTopicPartitions(partitionsPerTopic); + if (log.isDebugEnabled()) { + log.debug("performing constrained assign. partitionsPerTopic: {}, consumerToOwnedPartitions: {}", + partitionsPerTopic, consumerToOwnedPartitions); + } Set allRevokedPartitions = new HashSet<>(); - // Each consumer should end up in exactly one of the below - // the consumers not yet at capacity + // the consumers not yet at expected capacity List unfilledMembers = new LinkedList<>(); - // the members with exactly maxQuota partitions assigned - Queue maxCapacityMembers = new LinkedList<>(); - // the members with exactly minQuota partitions assigned - Queue minCapacityMembers = new LinkedList<>(); int numberOfConsumers = consumerToOwnedPartitions.size(); - int minQuota = (int) Math.floor(((double) unassignedPartitions.size()) / numberOfConsumers); - int maxQuota = (int) Math.ceil(((double) unassignedPartitions.size()) / numberOfConsumers); + int totalPartitionsCount = partitionsPerTopic.values().stream().reduce(0, Integer::sum); + + int minQuota = (int) Math.floor(((double) totalPartitionsCount) / numberOfConsumers); + int maxQuota = (int) Math.ceil(((double) totalPartitionsCount) / numberOfConsumers); + // the expected number of members with maxQuota assignment + int expectedNumMembersHavingMorePartitions = totalPartitionsCount % numberOfConsumers; + // the number of members with exactly maxQuota partitions assigned + int numMembersHavingMorePartitions = 0; - // initialize the assignment map with an empty array of size minQuota for all members + // initialize the assignment map with an empty array of size maxQuota for all members Map> assignment = new HashMap<>( - consumerToOwnedPartitions.keySet().stream().collect(Collectors.toMap(c -> c, c -> new ArrayList<>(minQuota)))); + consumerToOwnedPartitions.keySet().stream().collect(Collectors.toMap(c -> c, c -> new ArrayList<>(maxQuota)))); + List assignedPartitions = new ArrayList<>(); // Reassign as many previously owned partitions as possible for (Map.Entry> consumerEntry : consumerToOwnedPartitions.entrySet()) { String consumer = consumerEntry.getKey(); List ownedPartitions = consumerEntry.getValue(); List consumerAssignment = assignment.get(consumer); - int i = 0; - // assign the first N partitions up to the max quota, and mark the remaining as being revoked - for (TopicPartition tp : ownedPartitions) { - if (i < maxQuota) { - consumerAssignment.add(tp); - unassignedPartitions.remove(tp); - } else { - allRevokedPartitions.add(tp); - } - ++i; - } if (ownedPartitions.size() < minQuota) { + // the expected assignment size is more than consumer have now, so keep all the owned partitions + // and put this member into unfilled member list + if (ownedPartitions.size() > 0) { + consumerAssignment.addAll(ownedPartitions); + assignedPartitions.addAll(ownedPartitions); + } unfilledMembers.add(consumer); + } else if (ownedPartitions.size() >= maxQuota && numMembersHavingMorePartitions < expectedNumMembersHavingMorePartitions) { + // consumer owned the "maxQuota" of partitions or more, and we're still under the number of expected max capacity members + // so keep "maxQuota" of the owned partitions, and revoke the rest of the partitions + numMembersHavingMorePartitions++; + List maxQuotaPartitions = ownedPartitions.subList(0, maxQuota); + consumerAssignment.addAll(maxQuotaPartitions); + assignedPartitions.addAll(maxQuotaPartitions); + allRevokedPartitions.addAll(ownedPartitions.subList(maxQuota, ownedPartitions.size())); } else { - // It's possible for a consumer to be at both min and max capacity if minQuota == maxQuota - if (consumerAssignment.size() == minQuota) - minCapacityMembers.add(consumer); - if (consumerAssignment.size() == maxQuota) - maxCapacityMembers.add(consumer); + // consumer owned at least "minQuota" of partitions + // so keep "minQuota" of the owned partitions, and revoke the rest of the partitions + List minQuotaPartitions = ownedPartitions.subList(0, minQuota); + consumerAssignment.addAll(minQuotaPartitions); + assignedPartitions.addAll(minQuotaPartitions); + allRevokedPartitions.addAll(ownedPartitions.subList(minQuota, ownedPartitions.size())); + // this consumer is potential maxQuota candidate since we're still under the number of expected max capacity members + if (numMembersHavingMorePartitions < expectedNumMembersHavingMorePartitions) { + unfilledMembers.add(consumer); + } } } + List unassignedPartitions = getUnassignedPartitions(totalPartitionsCount, partitionsPerTopic, assignedPartitions); + assignedPartitions = null; + + if (log.isDebugEnabled()) { + log.debug("After reassigning previously owned partitions, unfilled members: {}, unassigned partitions: {}, " + + "current assignment: {}", unfilledMembers, unassignedPartitions, assignment); + } + Collections.sort(unfilledMembers); - Iterator unassignedPartitionsIter = unassignedPartitions.iterator(); - - // Fill remaining members up to minQuota - while (!unfilledMembers.isEmpty() && !unassignedPartitions.isEmpty()) { - Iterator unfilledConsumerIter = unfilledMembers.iterator(); - - while (unfilledConsumerIter.hasNext()) { - String consumer = unfilledConsumerIter.next(); - List consumerAssignment = assignment.get(consumer); - - if (unassignedPartitionsIter.hasNext()) { - TopicPartition tp = unassignedPartitionsIter.next(); - consumerAssignment.add(tp); - unassignedPartitionsIter.remove(); - // We already assigned all possible ownedPartitions, so we know this must be newly to this consumer - if (allRevokedPartitions.contains(tp)) - partitionsTransferringOwnership.put(tp, consumer); - } else { - break; - } - if (consumerAssignment.size() == minQuota) { - minCapacityMembers.add(consumer); - unfilledConsumerIter.remove(); + Iterator unfilledConsumerIter = unfilledMembers.iterator(); + // Round-Robin filling remaining members up to the expected numbers of maxQuota, otherwise, to minQuota + for (TopicPartition unassignedPartition : unassignedPartitions) { + if (!unfilledConsumerIter.hasNext()) { + if (unfilledMembers.isEmpty()) { + // Should not enter here since we have calculated the exact number to assign to each consumer + // There might be issues in the assigning algorithm, or maybe assigning the same partition to two owners. + throw new IllegalStateException("No more unfilled consumers to be assigned."); } + unfilledConsumerIter = unfilledMembers.iterator(); } - } - - // If we ran out of unassigned partitions before filling all consumers, we need to start stealing partitions - // from the over-full consumers at max capacity - for (String consumer : unfilledMembers) { + String consumer = unfilledConsumerIter.next(); List consumerAssignment = assignment.get(consumer); - int remainingCapacity = minQuota - consumerAssignment.size(); - while (remainingCapacity > 0) { - String overloadedConsumer = maxCapacityMembers.poll(); - if (overloadedConsumer == null) { - throw new IllegalStateException("Some consumers are under capacity but all partitions have been assigned"); + consumerAssignment.add(unassignedPartition); + + // We already assigned all possible ownedPartitions, so we know this must be newly assigned to this consumer + if (allRevokedPartitions.contains(unassignedPartition)) + partitionsTransferringOwnership.put(unassignedPartition, consumer); + + int currentAssignedCount = consumerAssignment.size(); + int expectedAssignedCount = numMembersHavingMorePartitions < expectedNumMembersHavingMorePartitions ? maxQuota : minQuota; + if (currentAssignedCount == expectedAssignedCount) { + if (currentAssignedCount == maxQuota) { + numMembersHavingMorePartitions++; } - TopicPartition swappedPartition = assignment.get(overloadedConsumer).remove(0); - consumerAssignment.add(swappedPartition); - --remainingCapacity; - // This partition is by definition transferring ownership, the swapped partition must have come from - // the max capacity member's owned partitions since it can only reach max capacity with owned partitions - partitionsTransferringOwnership.put(swappedPartition, consumer); + unfilledConsumerIter.remove(); } - minCapacityMembers.add(consumer); } - // Otherwise we may have run out of unfilled consumers before assigning all partitions, in which case we - // should just distribute one partition each to all consumers at min capacity - for (TopicPartition unassignedPartition : unassignedPartitions) { - String underCapacityConsumer = minCapacityMembers.poll(); - if (underCapacityConsumer == null) { - throw new IllegalStateException("Some partitions are unassigned but all consumers are at maximum capacity"); + if (!unfilledMembers.isEmpty()) { + // we expected all the remaining unfilled members have minQuota partitions and we're already at the allowed number + // of max capacity members. Otherwise, there must be error here. + if (numMembersHavingMorePartitions != expectedNumMembersHavingMorePartitions) { + throw new IllegalStateException(String.format("We haven't reached the allowed number of max capacity members, " + + "but no more partitions to be assigned to unfilled consumers: %s", unfilledMembers)); + } else { + for (String unfilledMember : unfilledMembers) { + int assignedPartitionsCount = assignment.get(unfilledMember).size(); + if (assignedPartitionsCount != minQuota) { + throw new IllegalStateException(String.format("Consumer: [%s] should have %d partitions, but got %d partitions, " + + "and no more partitions to be assigned", unfilledMember, minQuota, assignedPartitionsCount)); + } + } } - // We can skip the bookkeeping of unassignedPartitions and maxCapacityMembers here since we are at the end - assignment.get(underCapacityConsumer).add(unassignedPartition); + } - if (allRevokedPartitions.contains(unassignedPartition)) - partitionsTransferringOwnership.put(unassignedPartition, underCapacityConsumer); + if (log.isDebugEnabled()) { + log.debug("Final assignment of partitions to consumers: \n{}", assignment); } return assignment; } - private SortedSet getTopicPartitions(Map partitionsPerTopic) { - SortedSet allPartitions = - new TreeSet<>(Comparator.comparing(TopicPartition::topic).thenComparing(TopicPartition::partition)); - for (Entry entry: partitionsPerTopic.entrySet()) { - String topic = entry.getKey(); - for (int i = 0; i < entry.getValue(); ++i) { + /** + * get the unassigned partition list by computing the difference set of all sorted partitions + * and sortedAssignedPartitions. If no assigned partitions, we'll just return all topic partitions. + * + * To compute the difference set, we use two pointers technique here: + * + * We loop through the all sorted topics, and then iterate all partitions the topic has, + * compared with the ith element in sortedAssignedPartitions(i starts from 0): + * - if not equal to the ith element, add to unassignedPartitions + * - if equal to the the ith element, get next element from sortedAssignedPartitions + * + * @param totalPartitionsCount all partitions counts in this assignment + * @param partitionsPerTopic the number of partitions for each subscribed topic. + * @param sortedAssignedPartitions sorted partitions, all are included in the sortedPartitions + * @return the partitions not yet assigned to any consumers + */ + private List getUnassignedPartitions(int totalPartitionsCount, + Map partitionsPerTopic, + List sortedAssignedPartitions) { + List sortedAllTopics = new ArrayList<>(partitionsPerTopic.keySet()); + // sort all topics first, then we can have sorted all topic partitions by adding partitions starting from 0 + Collections.sort(sortedAllTopics); + + if (sortedAssignedPartitions.isEmpty()) { + // no assigned partitions means all partitions are unassigned partitions + return getAllTopicPartitions(partitionsPerTopic, sortedAllTopics, totalPartitionsCount); + } + + List unassignedPartitions = new ArrayList<>(totalPartitionsCount - sortedAssignedPartitions.size()); + + Collections.sort(sortedAssignedPartitions, Comparator.comparing(TopicPartition::topic).thenComparing(TopicPartition::partition)); + + boolean shouldAddDirectly = false; + Iterator sortedAssignedPartitionsIter = sortedAssignedPartitions.iterator(); + TopicPartition nextAssignedPartition = sortedAssignedPartitionsIter.next(); + + for (String topic : sortedAllTopics) { + int partitionCount = partitionsPerTopic.get(topic); + for (int i = 0; i < partitionCount; i++) { + if (shouldAddDirectly || !(nextAssignedPartition.topic().equals(topic) && nextAssignedPartition.partition() == i)) { + unassignedPartitions.add(new TopicPartition(topic, i)); + } else { + // this partition is in assignedPartitions, don't add to unassignedPartitions, just get next assigned partition + if (sortedAssignedPartitionsIter.hasNext()) { + nextAssignedPartition = sortedAssignedPartitionsIter.next(); + } else { + // add the remaining directly since there is no more sortedAssignedPartitions + shouldAddDirectly = true; + } + } + } + } + + return unassignedPartitions; + } + + + private List getAllTopicPartitions(Map partitionsPerTopic, + List sortedAllTopics, + int totalPartitionsCount) { + List allPartitions = new ArrayList<>(totalPartitionsCount); + + for (String topic : sortedAllTopics) { + int partitionCount = partitionsPerTopic.get(topic); + for (int i = 0; i < partitionCount; ++i) { allPartitions.add(new TopicPartition(topic, i)); } } @@ -303,7 +372,7 @@ private SortedSet getTopicPartitions(Map partit * @param partitionsPerTopic The number of partitions for each subscribed topic. * @param subscriptions Map from the member id to their respective topic subscription * - * @return Map from each member to the list of partitions assigned to them. + * @return Map from each member to the list of partitions assigned to them. */ private Map> generalAssign(Map partitionsPerTopic, Map subscriptions) { @@ -446,10 +515,10 @@ private void prepopulateCurrentAssignments(Map subscriptio /** * determine if the current assignment is a balanced one * - * @param currentAssignment: the assignment whose balance needs to be checked - * @param sortedCurrentSubscriptions: an ascending sorted set of consumers based on how many topic partitions are already assigned to them - * @param allSubscriptions: a mapping of all consumers to all potential topic partitions that can be assigned to them - * @return true if the given assignment is balanced; false otherwise + * @param currentAssignment the assignment whose balance needs to be checked + * @param sortedCurrentSubscriptions an ascending sorted set of consumers based on how many topic partitions are already assigned to them + * @param allSubscriptions a mapping of all consumers to all potential topic partitions that can be assigned to them + * @return true if the given assignment is balanced; false otherwise */ private boolean isBalanced(Map> currentAssignment, TreeSet sortedCurrentSubscriptions, @@ -527,8 +596,8 @@ private int getBalanceScore(Map> assignment) { * Sort valid partitions so they are processed in the potential reassignment phase in the proper order * that causes minimal partition movement among consumers (hence honoring maximal stickiness) * - * @param partition2AllPotentialConsumers a mapping of partitions to their potential consumers - * @return an ascending sorted list of topic partitions based on how many consumers can potentially use them + * @param partition2AllPotentialConsumers a mapping of partitions to their potential consumers + * @return an ascending sorted list of topic partitions based on how many consumers can potentially use them */ private List sortPartitions(Map> partition2AllPotentialConsumers) { List sortedPartitions = new ArrayList<>(partition2AllPotentialConsumers.keySet()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java index 35785401770a4..c8f6c14c5e82a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignorTest.java @@ -222,6 +222,101 @@ public void testTwoConsumersTwoTopicsSixPartitions() { assertTrue(isFullyBalanced(assignment)); } + /** + * This unit test is testing consumer owned minQuota partitions, and expected to have maxQuota partitions situation + */ + @Test + public void testConsumerOwningMinQuotaExpectedMaxQuota() { + String topic1 = "topic1"; + String topic2 = "topic2"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, 2); + partitionsPerTopic.put(topic2, 3); + + List subscribedTopics = topics(topic1, topic2); + + subscriptions.put(consumer1, + buildSubscription(subscribedTopics, partitions(tp(topic1, 0), tp(topic2, 1)))); + subscriptions.put(consumer2, + buildSubscription(subscribedTopics, partitions(tp(topic1, 1), tp(topic2, 2)))); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(partitions(tp(topic1, 0), tp(topic2, 1), tp(topic2, 0)), assignment.get(consumer1)); + assertEquals(partitions(tp(topic1, 1), tp(topic2, 2)), assignment.get(consumer2)); + + verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic); + assertTrue(isFullyBalanced(assignment)); + } + + /** + * This unit test is testing consumers owned maxQuota partitions are more than numExpectedMaxCapacityMembers situation + */ + @Test + public void testMaxQuotaConsumerMoreThanNumExpectedMaxCapacityMembers() { + String topic1 = "topic1"; + String topic2 = "topic2"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + String consumer3 = "consumer3"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, 2); + partitionsPerTopic.put(topic2, 2); + + List subscribedTopics = topics(topic1, topic2); + + subscriptions.put(consumer1, + buildSubscription(subscribedTopics, partitions(tp(topic1, 0), tp(topic2, 0)))); + subscriptions.put(consumer2, + buildSubscription(subscribedTopics, partitions(tp(topic1, 1), tp(topic2, 1)))); + subscriptions.put(consumer3, buildSubscription(subscribedTopics, Collections.emptyList())); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + + verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic); + assertEquals(partitions(tp(topic1, 0)), assignment.get(consumer1)); + assertEquals(partitions(tp(topic1, 1), tp(topic2, 1)), assignment.get(consumer2)); + assertEquals(partitions(tp(topic2, 0)), assignment.get(consumer3)); + + assertTrue(isFullyBalanced(assignment)); + } + + /** + * This unit test is testing all consumers owned less than minQuota partitions situation + */ + @Test + public void testAllConsumerAreUnderMinQuota() { + String topic1 = "topic1"; + String topic2 = "topic2"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + String consumer3 = "consumer3"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, 2); + partitionsPerTopic.put(topic2, 3); + + List subscribedTopics = topics(topic1, topic2); + + subscriptions.put(consumer1, + buildSubscription(subscribedTopics, partitions(tp(topic1, 0)))); + subscriptions.put(consumer2, + buildSubscription(subscribedTopics, partitions(tp(topic1, 1)))); + subscriptions.put(consumer3, buildSubscription(subscribedTopics, Collections.emptyList())); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + + verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic); + assertEquals(partitions(tp(topic1, 0), tp(topic2, 0)), assignment.get(consumer1)); + assertEquals(partitions(tp(topic1, 1), tp(topic2, 1)), assignment.get(consumer2)); + assertEquals(partitions(tp(topic2, 2)), assignment.get(consumer3)); + + assertTrue(isFullyBalanced(assignment)); + } + @Test public void testAddRemoveConsumerOneTopic() { String consumer1 = "consumer1"; @@ -256,6 +351,56 @@ public void testAddRemoveConsumerOneTopic() { assertTrue(isFullyBalanced(assignment)); } + @Test + public void testAddRemoveTwoConsumersTwoTopics() { + String topic1 = "topic1"; + String topic2 = "topic2"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + String consumer3 = "consumer3"; + String consumer4 = "consumer4"; + List allTopics = topics(topic1, topic2); + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, 3); + partitionsPerTopic.put(topic2, 4); + subscriptions.put(consumer1, new Subscription(allTopics)); + subscriptions.put(consumer2, new Subscription(allTopics)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(partitions(tp(topic1, 0), tp(topic1, 2), tp(topic2, 1), tp(topic2, 3)), assignment.get(consumer1)); + assertEquals(partitions(tp(topic1, 1), tp(topic2, 0), tp(topic2, 2)), assignment.get(consumer2)); + + verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic); + assertTrue(isFullyBalanced(assignment)); + + // add 2 consumers + subscriptions.put(consumer1, buildSubscription(allTopics, assignment.get(consumer1))); + subscriptions.put(consumer2, buildSubscription(allTopics, assignment.get(consumer2))); + subscriptions.put(consumer3, buildSubscription(allTopics, Collections.emptyList())); + subscriptions.put(consumer4, buildSubscription(allTopics, Collections.emptyList())); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + + verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic); + assertEquals(partitions(tp(topic1, 0), tp(topic1, 2)), assignment.get(consumer1)); + assertEquals(partitions(tp(topic1, 1), tp(topic2, 0)), assignment.get(consumer2)); + assertEquals(partitions(tp(topic2, 1), tp(topic2, 3)), assignment.get(consumer3)); + assertEquals(partitions(tp(topic2, 2)), assignment.get(consumer4)); + assertTrue(isFullyBalanced(assignment)); + + // remove 2 consumers + subscriptions.remove(consumer1); + subscriptions.remove(consumer2); + subscriptions.put(consumer3, buildSubscription(allTopics, assignment.get(consumer3))); + subscriptions.put(consumer4, buildSubscription(allTopics, assignment.get(consumer4))); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(partitions(tp(topic2, 1), tp(topic2, 3), tp(topic1, 0), tp(topic1, 2)), assignment.get(consumer3)); + assertEquals(partitions(tp(topic2, 2), tp(topic1, 1), tp(topic2, 0)), assignment.get(consumer4)); + + verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic); + assertTrue(isFullyBalanced(assignment)); + } + /** * This unit test performs sticky assignment for a scenario that round robin assignor handles poorly. * Topics (partitions per topic): topic1 (2), topic2 (1), topic3 (2), topic4 (1), topic5 (2) From 80aea23beb2654b2a53e0248387f2b4dbc0f0dc1 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Thu, 6 May 2021 12:58:27 +0800 Subject: [PATCH 124/155] KAFKA-9295: increase startup timeout for flaky test in KTableKTableForeignKeyInnerJoinMultiIntegrationTest (#10635) Try to address the extreme flakiness of shouldInnerJoinMultiPartitionQueryable since the recent test cleanup. Since we need to wait for 3 streams reach RUNNING state, it makes sense to increase the waiting time to make the test more reliable. Reviewers: Anna Sophie Blee-Goldman --- .../KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java index 6156c484b290a..92520fb0186a7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableForeignKeyInnerJoinMultiIntegrationTest.java @@ -194,7 +194,7 @@ private void verifyKTableKTableJoin(final Set> expecte streamsThree = prepareTopology(queryableName, queryableNameTwo, streamsConfigThree); final List kafkaStreamsList = asList(streams, streamsTwo, streamsThree); - startApplicationAndWaitUntilRunning(kafkaStreamsList, ofSeconds(60)); + startApplicationAndWaitUntilRunning(kafkaStreamsList, ofSeconds(120)); final Set> result = new HashSet<>(IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( CONSUMER_CONFIG, From ebd4748eba80c9fc9264f434bbb135336e38133e Mon Sep 17 00:00:00 2001 From: Chris Egerton Date: Thu, 6 May 2021 04:57:45 -0400 Subject: [PATCH 125/155] MINOR: Stop logging raw record contents above TRACE level in WorkerSourceTask (#10630) Reviewers: Tom Bentley --- .../org/apache/kafka/connect/runtime/WorkerSourceTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 48660f3b33233..5a877bac3d59d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -370,7 +370,7 @@ private boolean sendRecords() { (recordMetadata, e) -> { if (e != null) { log.error("{} failed to send record to {}: ", WorkerSourceTask.this, topic, e); - log.debug("{} Failed record: {}", WorkerSourceTask.this, preTransformRecord); + log.trace("{} Failed record: {}", WorkerSourceTask.this, preTransformRecord); producerSendException.compareAndSet(null, e); } else { recordSent(producerRecord); @@ -396,7 +396,7 @@ private boolean sendRecords() { } catch (ConnectException e) { log.warn("{} Failed to send record to topic '{}' and partition '{}' due to an unrecoverable exception: ", this, producerRecord.topic(), producerRecord.partition(), e); - log.warn("{} Failed to send {} with unrecoverable exception: ", this, producerRecord, e); + log.trace("{} Failed to send {} with unrecoverable exception: ", this, producerRecord, e); throw e; } catch (KafkaException e) { throw new ConnectException("Unrecoverable exception trying to send", e); @@ -475,7 +475,7 @@ private synchronized void recordSent(final ProducerRecord record removed = outstandingMessagesBacklog.remove(record); // But if neither one had it, something is very wrong if (removed == null) { - log.error("{} CRITICAL Saw callback for record that was not present in the outstanding message set: {}", this, record); + log.error("{} CRITICAL Saw callback for record from topic {} partition {} that was not present in the outstanding message set: {}", this, record.topic(), record.partition()); } else if (flushing && outstandingMessages.isEmpty()) { // flush thread may be waiting on the outstanding messages to clear this.notifyAll(); From 03690d7a1f5e90fb85690352d7f2b7651e554dc3 Mon Sep 17 00:00:00 2001 From: leah Date: Thu, 6 May 2021 05:05:36 -0500 Subject: [PATCH 126/155] MINOR: Stop using hamcrest in system tests (#10631) We currently use hamcrest imports to check the outputs of the RelationalSmokeTest, but with the new gradle updates, the proper hamcrest imports are no longer included in the test jar. This is a bit of a workaround to remove the hamcrest usage so we can get system tests up and running again. Potential follow-up could be to update the way we create the test-jar to pull in the proper dependencies. Reviewers: Bruno Cadonna --- .../streams/tests/RelationalSmokeTest.java | 94 +++++++++---------- 1 file changed, 45 insertions(+), 49 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java index bd2ebc16e7381..883c5cc06b6d8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java @@ -43,9 +43,6 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.StringDescription; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,10 +65,6 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkProperties; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.startsWith; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.hasKey; /** * This test builds on a basic relational data caricature: @@ -806,20 +799,13 @@ public static boolean verifySync(final String broker, final Instant deadline) th } } - public static void assertThat(final AtomicBoolean pass, - final StringBuilder failures, - final String message, - final T actual, - final Matcher matcher) { - if (!matcher.matches(actual)) { + public static void assertThat(final AtomicBoolean pass, + final StringBuilder failures, + final String message, + final boolean passed) { + if (!passed) { if (failures != null) { - final Description description = new StringDescription(failures); - description.appendText("\n" + message) - .appendText("\nExpected: ") - .appendDescriptionOf(matcher) - .appendText("\n but: "); - matcher.describeMismatch(actual, description); - description.appendText("\n"); + failures.append("\n").append(message); } pass.set(false); } @@ -833,22 +819,36 @@ static boolean verifySync(final boolean logResults, final AtomicBoolean pass = new AtomicBoolean(true); final StringBuilder report = logResults ? new StringBuilder() : null; - assertThat(pass, report, "one article", consumedArticles.size(), greaterThan(0)); - assertThat(pass, report, "one comment", consumedComments.size(), greaterThan(0)); + assertThat( + pass, + report, + "Expected 1 article, got " + consumedArticles.size(), + consumedArticles.size() > 0 + ); + assertThat( + pass, + report, + "Expected 1 comment, got " + consumedComments.size(), + consumedComments.size() > 0 + ); assertThat( pass, report, - "article size", - consumedAugmentedArticles.size(), - is(consumedArticles.size()) + "Mismatched article size between augmented articles (size " + + consumedAugmentedArticles.size() + + ") and consumed articles (size " + + consumedArticles.size() + ")", + consumedAugmentedArticles.size() == consumedArticles.size() ); assertThat( pass, report, - "comment size", - consumedAugmentedComments.size(), - is(consumedComments.size()) + "Mismatched comments size between augmented comments (size " + + consumedAugmentedComments.size() + + ") and consumed comments (size " + + consumedComments.size() + ")", + consumedAugmentedComments.size() == consumedComments.size() ); final Map commentCounts = new TreeMap<>(); @@ -856,41 +856,40 @@ static boolean verifySync(final boolean logResults, for (final RelationalSmokeTest.AugmentedComment augmentedComment : consumedAugmentedComments.values()) { final int key = augmentedComment.getKey(); assertThat(pass, - report, - "comment missing, but found in augmentedComment: " + key, - consumedComments, - hasKey(key)); + report, + "comment missing, but found in augmentedComment: " + key, + consumedComments.containsKey(key) + ); final Comment comment = consumedComments.get(key); if (comment != null) { assertThat( pass, report, - "comment articleId [" + comment.getArticleId() + "] didn't match " + - "augmentedComment articleId [" + augmentedComment.getArticleId() + "]", - comment.getArticleId(), - is(augmentedComment.getArticleId()) + "comment missing, but found in augmentedComment: " + key, + consumedComments.containsKey(key) ); } - commentCounts.put(augmentedComment.getArticleId(), - commentCounts.getOrDefault(augmentedComment.getArticleId(), 0L) + 1); + commentCounts.put( + augmentedComment.getArticleId(), + commentCounts.getOrDefault(augmentedComment.getArticleId(), 0L) + 1 + ); assertThat( pass, report, "augmentedArticle [" + augmentedComment.getArticleId() + "] " + "missing for augmentedComment [" + augmentedComment.getKey() + "]", - consumedAugmentedArticles, - hasKey(augmentedComment.getArticleId()) + consumedAugmentedArticles.containsKey(augmentedComment.getArticleId()) ); - final AugmentedArticle augmentedArticle = consumedAugmentedArticles.get(augmentedComment.getArticleId()); + final AugmentedArticle augmentedArticle = + consumedAugmentedArticles.get(augmentedComment.getArticleId()); if (augmentedArticle != null) { assertThat( pass, report, "articlePrefix didn't match augmentedArticle: " + augmentedArticle.getText(), - augmentedArticle.getText(), - startsWith(augmentedComment.getArticlePrefix()) + augmentedArticle.getText().startsWith(augmentedComment.getArticlePrefix()) ); } @@ -898,8 +897,7 @@ static boolean verifySync(final boolean logResults, pass, report, "article " + augmentedComment.getArticleId() + " missing from consumedArticles", - consumedArticles, - hasKey(augmentedComment.getArticleId()) + consumedArticles.containsKey(augmentedComment.getArticleId()) ); final Article article = consumedArticles.get(augmentedComment.getArticleId()); if (article != null) { @@ -907,8 +905,7 @@ static boolean verifySync(final boolean logResults, pass, report, "articlePrefix didn't match article: " + article.getText(), - article.getText(), - startsWith(augmentedComment.getArticlePrefix()) + article.getText().startsWith(augmentedComment.getArticlePrefix()) ); } } @@ -919,8 +916,7 @@ static boolean verifySync(final boolean logResults, pass, report, "article " + augmentedArticle.getKey() + " comment count mismatch", - augmentedArticle.getCommentCount(), - is(commentCounts.getOrDefault(augmentedArticle.getKey(), 0L)) + augmentedArticle.getCommentCount() == commentCounts.getOrDefault(augmentedArticle.getKey(), 0L) ); } From b73d639adc9e4b5f1940d364f5ece481604b802a Mon Sep 17 00:00:00 2001 From: Shay Elkin <2046772+shayelkin@users.noreply.github.com> Date: Thu, 6 May 2021 07:44:03 -0700 Subject: [PATCH 127/155] KAFKA-12752: Bump Jersey deps to 2.34 due to CVE-2021-28168 (#10636) The version of the Eclipse Jersey library brought as dependences, 2.31, has a known vulnerability, CVE-2021-28168 (https://github.com/advisories/GHSA-c43q-5hpj-4crv). This replaces it with 2.34, which is fully compatible with 2.31, except for bugs and vulnerabilities. Reviewers: Manikumar Reddy --- gradle/dependencies.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 40dc41ad6bcd9..f8bfb2c39f45d 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -70,7 +70,7 @@ versions += [ jacoco: "0.8.5", javassist: "3.27.0-GA", jetty: "9.4.39.v20210325", - jersey: "2.31", + jersey: "2.34", jline: "3.12.1", jmh: "1.27", hamcrest: "2.2", From 94be57d6106fa6d0dea16f9574b7e8fcbffcf88b Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Thu, 6 May 2021 17:25:02 +0200 Subject: [PATCH 128/155] MINOR: Fix formatting in RelationalSmokeTest (#10639) Fixes formatting in RelationalSmokeTest. Reviewers: Leah Thomas --- .../streams/tests/RelationalSmokeTest.java | 67 ++++++++++--------- 1 file changed, 36 insertions(+), 31 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java index 883c5cc06b6d8..8688cf2d2b372 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/RelationalSmokeTest.java @@ -138,10 +138,10 @@ public byte[] serialize(final String topic, final Article data) { final ByteBuffer buffer = ByteBuffer.allocate(length) - .putInt(data.getKey()) - .putLong(data.getTimestamp()) - .putInt(serialText.length) - .put(serialText); + .putInt(data.getKey()) + .putLong(data.getTimestamp()) + .putInt(serialText.length) + .put(serialText); return Serdes.ByteBuffer().serializer().serialize(topic, buffer); } @@ -228,11 +228,11 @@ public byte[] serialize(final String topic, final Comment data) { final ByteBuffer buffer = ByteBuffer.allocate(length) - .putInt(data.key) - .putLong(data.timestamp) - .putInt(serialText.length) - .put(serialText) - .putInt(data.articleId); + .putInt(data.key) + .putLong(data.timestamp) + .putInt(serialText.length) + .put(serialText) + .putInt(data.articleId); return Serdes.ByteBuffer().serializer().serialize(topic, buffer); } @@ -278,7 +278,6 @@ public static final class DataSet { private final Comment[] comments; private DataSet(final Article[] articles, final Comment[] comments) { - this.articles = articles; this.comments = comments; } @@ -464,8 +463,8 @@ public byte[] serialize(final String topic, final AugmentedArticle data) { final int length = serializedArticle.length + Long.BYTES; final ByteBuffer buffer = ByteBuffer.allocate(length) - .put(serializedArticle) - .putLong(data.getCommentCount()); + .put(serializedArticle) + .putLong(data.getCommentCount()); return Serdes.ByteBuffer().serializer().serialize(topic, buffer); } } @@ -540,9 +539,9 @@ public byte[] serialize(final String topic, final AugmentedComment data) { final int length = serializedComment.length + Integer.BYTES + serializedPrefix.length; final ByteBuffer buffer = ByteBuffer.allocate(length) - .put(serializedComment) - .putInt(serializedPrefix.length) - .put(serializedPrefix); + .put(serializedComment) + .putInt(serializedPrefix.length) + .put(serializedPrefix); return Serdes.ByteBuffer().serializer().serialize(topic, buffer); } } @@ -606,14 +605,18 @@ public static Topology getTopology() { final KTable commentCounts = - comments.groupBy((key, value) -> new KeyValue<>(value.getArticleId(), (short) 1), - Grouped.with(Serdes.Integer(), Serdes.Short())) - .count(); + comments.groupBy( + (key, value) -> new KeyValue<>(value.getArticleId(), (short) 1), + Grouped.with(Serdes.Integer(), Serdes.Short()) + ) + .count(); articles - .leftJoin(commentCounts, - AugmentedArticle.joiner(), - Materialized.with(null, new AugmentedArticle.AugmentedArticleSerde())) + .leftJoin( + commentCounts, + AugmentedArticle.joiner(), + Materialized.with(null, new AugmentedArticle.AugmentedArticleSerde()) + ) .toStream() .to(ARTICLE_RESULT_SINK); @@ -621,8 +624,8 @@ public static Topology getTopology() { Comment::getArticleId, AugmentedComment.joiner(), Materialized.with(null, new AugmentedComment.AugmentedCommentSerde())) - .toStream() - .to(COMMENT_RESULT_SINK); + .toStream() + .to(COMMENT_RESULT_SINK); return streamsBuilder.build(); } @@ -698,7 +701,8 @@ public static boolean verifySync(final String broker, final Instant deadline) th Stream.concat( articlePartitions.stream().map(p -> new TopicPartition(p.topic(), p.partition())), augmentedArticlePartitions.stream().map(p -> new TopicPartition(p.topic(), p.partition())) - ), Stream.concat( + ), + Stream.concat( commentPartitions.stream().map(p -> new TopicPartition(p.topic(), p.partition())), augmentedCommentPartitions.stream().map(p -> new TopicPartition(p.topic(), p.partition())) ) @@ -845,9 +849,9 @@ static boolean verifySync(final boolean logResults, pass, report, "Mismatched comments size between augmented comments (size " - + consumedAugmentedComments.size() + - ") and consumed comments (size " + - consumedComments.size() + ")", + + consumedAugmentedComments.size() + + ") and consumed comments (size " + + consumedComments.size() + ")", consumedAugmentedComments.size() == consumedComments.size() ); @@ -855,10 +859,11 @@ static boolean verifySync(final boolean logResults, for (final RelationalSmokeTest.AugmentedComment augmentedComment : consumedAugmentedComments.values()) { final int key = augmentedComment.getKey(); - assertThat(pass, - report, - "comment missing, but found in augmentedComment: " + key, - consumedComments.containsKey(key) + assertThat( + pass, + report, + "comment missing, but found in augmentedComment: " + key, + consumedComments.containsKey(key) ); final Comment comment = consumedComments.get(key); From 12a1e68aeb2d1e38c06d92c957dabc239e2252d5 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Thu, 6 May 2021 19:25:23 +0100 Subject: [PATCH 129/155] KAFKA-12451: Remove deprecation annotation on long-based read operations in WindowStore (#10296) Complete https://cwiki.apache.org/confluence/display/KAFKA/KIP-667%3A+Remove+deprecated+methods+from+ReadOnlyWindowStore by removing deprecation annotation on long-based read operations in WindowStore. Reviewers: Guozhang Wang --- .../internals/AbstractReadWriteDecorator.java | 3 --- .../org/apache/kafka/streams/state/WindowStore.java | 12 ++++++------ .../streams/state/internals/CachingWindowStore.java | 3 --- .../internals/ChangeLoggingWindowBytesStore.java | 3 --- .../streams/state/internals/MeteredWindowStore.java | 3 --- .../streams/state/internals/RocksDBWindowStore.java | 3 --- 6 files changed, 6 insertions(+), 21 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java index 86dd33fb8566c..eac606cdbe302 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java @@ -167,7 +167,6 @@ public V fetch(final K key, return wrapped().fetch(key, time); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public WindowStoreIterator fetch(final K key, final long timeFrom, @@ -182,7 +181,6 @@ public WindowStoreIterator backwardFetch(final K key, return wrapped().backwardFetch(key, timeFrom, timeTo); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, @@ -199,7 +197,6 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, return wrapped().backwardFetch(keyFrom, keyTo, timeFrom, timeTo); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index 3ac11b8c94651..31c6eb107cd68 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -81,8 +81,8 @@ public interface WindowStore extends StateStore, ReadOnlyWindowStore * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if the given key is {@code null} */ - // note, this method must be kept if super#fetch(...) is removed - @SuppressWarnings("deprecation") + // WindowStore keeps a long-based implementation of ReadOnlyWindowStore#fetch Instant-based + // if super#fetch is removed, keep this implementation as it serves PAPI Stores. WindowStoreIterator fetch(K key, long timeFrom, long timeTo); @Override @@ -124,8 +124,8 @@ default WindowStoreIterator backwardFetch(final K key, * @throws InvalidStateStoreException if the store is not initialized * @throws NullPointerException if one of the given keys is {@code null} */ - // note, this method must be kept if super#fetch(...) is removed - @SuppressWarnings("deprecation") + // WindowStore keeps a long-based implementation of ReadOnlyWindowStore#fetch Instant-based + // if super#fetch is removed, keep this implementation as it serves PAPI Stores. KeyValueIterator, V> fetch(K keyFrom, K keyTo, long timeFrom, long timeTo); @Override @@ -167,8 +167,8 @@ default KeyValueIterator, V> backwardFetch(final K keyFrom, * @return an iterator over windowed key-value pairs {@code , value>} * @throws InvalidStateStoreException if the store is not initialized */ - // note, this method must be kept if super#fetchAll(...) is removed - @SuppressWarnings("deprecation") + // WindowStore keeps a long-based implementation of ReadOnlyWindowStore#fetch Instant-based + // if super#fetch is removed, keep this implementation as it serves PAPI Stores. KeyValueIterator, V> fetchAll(long timeFrom, long timeTo); @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index cc7e5c6828386..ad5e92cec7112 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -183,7 +183,6 @@ public byte[] fetch(final Bytes key, } } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public synchronized WindowStoreIterator fetch(final Bytes key, final long timeFrom, @@ -240,7 +239,6 @@ public synchronized WindowStoreIterator backwardFetch(final Bytes key, return new MergedSortedCacheWindowStoreIterator(filteredCacheIterator, underlyingIterator, false); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, @@ -330,7 +328,6 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFr ); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java index 70c89495af0fc..5ce965482d2d7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingWindowBytesStore.java @@ -75,7 +75,6 @@ public byte[] fetch(final Bytes key, return wrapped().fetch(key, timestamp); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public WindowStoreIterator fetch(final Bytes key, final long from, @@ -90,7 +89,6 @@ public WindowStoreIterator backwardFetch(final Bytes key, return wrapped().backwardFetch(key, timeFrom, timeTo); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, @@ -118,7 +116,6 @@ public KeyValueIterator, byte[]> backwardAll() { return wrapped().backwardAll(); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java index 041026b1a8b2b..91b4387fbbdc5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java @@ -194,7 +194,6 @@ public V fetch(final K key, ); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public WindowStoreIterator fetch(final K key, final long timeFrom, @@ -223,7 +222,6 @@ public WindowStoreIterator backwardFetch(final K key, ); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo, @@ -254,7 +252,6 @@ public KeyValueIterator, V> backwardFetch(final K keyFrom, time); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, V> fetchAll(final long timeFrom, final long timeTo) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 394950525a322..8f48dcaaeda3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -53,7 +53,6 @@ public byte[] fetch(final Bytes key, final long timestamp) { return wrapped().get(WindowKeySchema.toStoreKeyBinary(key, timestamp, seqnum)); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public WindowStoreIterator fetch(final Bytes key, final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = wrapped().fetch(key, timeFrom, timeTo); @@ -66,7 +65,6 @@ public WindowStoreIterator backwardFetch(final Bytes key, final long tim return new WindowStoreIteratorWrapper(bytesIterator, windowSize).valuesIterator(); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetch(...) is removed @Override public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo, @@ -97,7 +95,6 @@ public KeyValueIterator, byte[]> backwardAll() { return new WindowStoreIteratorWrapper(bytesIterator, windowSize).keyValueIterator(); } - @SuppressWarnings("deprecation") // note, this method must be kept if super#fetchAll(...) is removed @Override public KeyValueIterator, byte[]> fetchAll(final long timeFrom, final long timeTo) { final KeyValueIterator bytesIterator = wrapped().fetchAll(timeFrom, timeTo); From 90fc875e24121fd0a39143983ce9eb8a93849d7e Mon Sep 17 00:00:00 2001 From: Bruno Cadonna Date: Fri, 7 May 2021 00:29:26 +0200 Subject: [PATCH 130/155] KAFKA-8897: Upgrade RocksDB to 6.19.3 (#10568) This PR upgrades RocksDB to 6.19.3. After the upgrade the Gradle build exited with code 134 due to SIGABRT signals ("Pure virtual function called!") coming from the C++ part of RocksDB. This error was caused by RocksDB state stores not properly closed in Streams' code. This PR adds the missing closings and updates the RocksDB option adapter. Reviewers: Anna Sophie Blee-Goldman , Guozhang Wang --- gradle/dependencies.gradle | 2 +- .../internals/GlobalStateManagerImpl.java | 17 +- .../internals/GlobalStreamThread.java | 21 +- .../ChangeLoggingKeyValueBytesStore.java | 4 +- .../ChangeLoggingSessionBytesStore.java | 4 +- ...ToDbOptionsColumnFamilyOptionsAdapter.java | 352 ++++++++++++++++-- .../streams/state/internals/RocksDBStore.java | 2 +- .../internals/RocksDBTimestampedStore.java | 2 +- .../internals/KStreamKTableJoinTest.java | 3 +- ...amSessionWindowAggregateProcessorTest.java | 65 +--- .../internals/GlobalStreamThreadTest.java | 32 +- .../processor/internals/StreamThreadTest.java | 5 + ...bstractRocksDBSegmentedBytesStoreTest.java | 12 + .../AbstractSessionBytesStoreTest.java | 8 + .../AbstractWindowBytesStoreTest.java | 7 + .../CachingPersistentSessionStoreTest.java | 1 + .../CachingPersistentWindowStoreTest.java | 3 +- .../state/internals/KeyValueSegmentTest.java | 10 + ...OptionsColumnFamilyOptionsAdapterTest.java | 24 ++ .../state/internals/RocksDBStoreTest.java | 2 +- .../internals/RocksDBWindowStoreTest.java | 2 +- .../internals/TimestampedSegmentTest.java | 14 + .../WindowedWordCountProcessorTest.java | 4 + 23 files changed, 486 insertions(+), 110 deletions(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index f8bfb2c39f45d..f4ca693c3362d 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -103,7 +103,7 @@ versions += [ netty: "4.1.62.Final", powermock: "2.0.9", reflections: "0.9.12", - rocksDB: "5.18.4", + rocksDB: "6.19.3", scalaCollectionCompat: "2.3.0", scalafmt: "1.5.1", scalaJava8Compat : "0.9.1", diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 54ead671e49ba..ae13753715ccc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -374,7 +374,7 @@ public void flush() { @Override public void close() { - if (globalStores.isEmpty()) { + if (globalStateStores.isEmpty() && globalStores.isEmpty()) { return; } final StringBuilder closeFailed = new StringBuilder(); @@ -396,6 +396,21 @@ public void close() { log.info("Skipping to close non-initialized store {}", entry.getKey()); } } + for (final StateStore store : globalStateStores) { + if (store.isOpen()) { + try { + store.close(); + } catch (final RuntimeException e) { + log.error("Failed to close global state store {}", store.name(), e); + closeFailed.append("Failed to close global state store:") + .append(store.name()) + .append(". Reason: ") + .append(e) + .append("\n"); + } + } + } + if (closeFailed.length() > 0) { throw new ProcessorStateException("Exceptions caught during close of 1 or more global state globalStores\n" + closeFailed); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java index 2f8c0ed51b0b5..ddad121ef5ecd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java @@ -354,6 +354,7 @@ public void resize(final long cacheSize) { } private StateConsumer initialize() { + StateConsumer stateConsumer = null; try { final GlobalStateManager stateMgr = new GlobalStateManagerImpl( logContext, @@ -374,7 +375,7 @@ private StateConsumer initialize() { ); stateMgr.setGlobalProcessorContext(globalProcessorContext); - final StateConsumer stateConsumer = new StateConsumer( + stateConsumer = new StateConsumer( logContext, globalConsumer, new GlobalStateUpdateTask( @@ -397,11 +398,7 @@ private StateConsumer initialize() { recoverableException ); - try { - stateConsumer.close(true); - } catch (final IOException e) { - log.error("Failed to close state consumer due to the following error:", e); - } + closeStateConsumer(stateConsumer, true); throw new StreamsException( "Bootstrapping global state failed. You can restart KafkaStreams to recover from this error.", @@ -411,13 +408,25 @@ private StateConsumer initialize() { return stateConsumer; } catch (final StreamsException fatalException) { + closeStateConsumer(stateConsumer, false); startupException = fatalException; } catch (final Exception fatalException) { + closeStateConsumer(stateConsumer, false); startupException = new StreamsException("Exception caught during initialization of GlobalStreamThread", fatalException); } return null; } + private void closeStateConsumer(final StateConsumer stateConsumer, final boolean wipeStateStore) { + if (stateConsumer != null) { + try { + stateConsumer.close(wipeStateStore); + } catch (final IOException e) { + log.error("Failed to close state consumer due to the following error:", e); + } + } + } + @Override public synchronized void start() { super.start(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java index 3770696808ded..88c9292f07f1c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java @@ -44,16 +44,16 @@ public class ChangeLoggingKeyValueBytesStore @Override public void init(final ProcessorContext context, final StateStore root) { - super.init(context, root); this.context = asInternalProcessorContext(context); + super.init(context, root); maybeSetEvictionListener(); } @Override public void init(final StateStoreContext context, final StateStore root) { - super.init(context, root); this.context = asInternalProcessorContext(context); + super.init(context, root); maybeSetEvictionListener(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java index 556a67e60ee90..f70eabd19f2f5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java @@ -44,14 +44,14 @@ class ChangeLoggingSessionBytesStore @Deprecated @Override public void init(final ProcessorContext context, final StateStore root) { - super.init(context, root); this.context = asInternalProcessorContext(context); + super.init(context, root); } @Override public void init(final StateStoreContext context, final StateStore root) { - super.init(context, root); this.context = asInternalProcessorContext(context); + super.init(context, root); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java index 2e2cb41100005..96a482bb6e6ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java @@ -19,7 +19,9 @@ import org.rocksdb.AbstractCompactionFilter; import org.rocksdb.AbstractCompactionFilterFactory; import org.rocksdb.AbstractComparator; +import org.rocksdb.AbstractEventListener; import org.rocksdb.AbstractSlice; +import org.rocksdb.AbstractWalFilter; import org.rocksdb.AccessHint; import org.rocksdb.BuiltinComparator; import org.rocksdb.Cache; @@ -30,6 +32,7 @@ import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionOptions; import org.rocksdb.CompressionType; +import org.rocksdb.ConcurrentTaskLimiter; import org.rocksdb.DBOptions; import org.rocksdb.DbPath; import org.rocksdb.Env; @@ -40,9 +43,11 @@ import org.rocksdb.Options; import org.rocksdb.RateLimiter; import org.rocksdb.SstFileManager; +import org.rocksdb.SstPartitionerFactory; import org.rocksdb.Statistics; import org.rocksdb.TableFormatConfig; import org.rocksdb.WALRecoveryMode; +import org.rocksdb.WalFilter; import org.rocksdb.WriteBufferManager; import org.slf4j.LoggerFactory; @@ -99,35 +104,7 @@ public Env getEnv() { @Override public Options prepareForBulkLoad() { - /* From https://github.com/facebook/rocksdb/wiki/RocksDB-FAQ - * - * Q: What's the fastest way to load data into RocksDB? - * - * A: A fast way to direct insert data to the DB: - * - * 1. using single writer thread and insert in sorted order - * 2. batch hundreds of keys into one write batch - * 3. use vector memtable - * 4. make sure options.max_background_flushes is at least 4 - * 5. before inserting the data, - * disable automatic compaction, - * set options.level0_file_num_compaction_trigger, - * options.level0_slowdown_writes_trigger - * and options.level0_stop_writes_trigger to very large. - * After inserting all the data, issue a manual compaction. - * - * 3-5 will be automatically done if you call Options::PrepareForBulkLoad() to your option - */ - // (1) not in our control - // (2) is done via bulk-loading API - // (3) skipping because, not done in actual PrepareForBulkLoad() code in https://github.com/facebook/rocksdb/blob/master/options/options.cc - //columnFamilyOptions.setMemTableConfig(new VectorMemTableConfig()); - // (4-5) below: - dbOptions.setMaxBackgroundFlushes(4); - columnFamilyOptions.setDisableAutoCompactions(true); - columnFamilyOptions.setLevel0FileNumCompactionTrigger(1 << 30); - columnFamilyOptions.setLevel0SlowdownWritesTrigger(1 << 30); - columnFamilyOptions.setLevel0StopWritesTrigger(1 << 30); + super.prepareForBulkLoad(); return this; } @@ -185,7 +162,7 @@ public Options setComparator(final BuiltinComparator builtinComparator) { } @Override - public Options setComparator(final AbstractComparator> comparator) { + public Options setComparator(final AbstractComparator comparator) { columnFamilyOptions.setComparator(comparator); return this; } @@ -342,6 +319,7 @@ public Options setDeleteObsoleteFilesPeriodMicros(final long micros) { return this; } + @Deprecated @Override public int maxBackgroundCompactions() { return dbOptions.maxBackgroundCompactions(); @@ -358,6 +336,7 @@ public Statistics statistics() { return dbOptions.statistics(); } + @Deprecated @Override public void setBaseBackgroundCompactions(final int baseBackgroundCompactions) { dbOptions.setBaseBackgroundCompactions(baseBackgroundCompactions); @@ -368,6 +347,7 @@ public int baseBackgroundCompactions() { return dbOptions.baseBackgroundCompactions(); } + @Deprecated @Override public Options setMaxBackgroundCompactions(final int maxBackgroundCompactions) { dbOptions.setMaxBackgroundCompactions(maxBackgroundCompactions); @@ -375,8 +355,9 @@ public Options setMaxBackgroundCompactions(final int maxBackgroundCompactions) { } @Override - public void setMaxSubcompactions(final int maxSubcompactions) { + public Options setMaxSubcompactions(final int maxSubcompactions) { dbOptions.setMaxSubcompactions(maxSubcompactions); + return this; } @Override @@ -384,11 +365,13 @@ public int maxSubcompactions() { return dbOptions.maxSubcompactions(); } + @Deprecated @Override public int maxBackgroundFlushes() { return dbOptions.maxBackgroundFlushes(); } + @Deprecated @Override public Options setMaxBackgroundFlushes(final int maxBackgroundFlushes) { dbOptions.setMaxBackgroundFlushes(maxBackgroundFlushes); @@ -1381,6 +1364,313 @@ public WriteBufferManager writeBufferManager() { return dbOptions.writeBufferManager(); } + @Override + public Options setMaxWriteBatchGroupSizeBytes(final long maxWriteBatchGroupSizeBytes) { + dbOptions.setMaxWriteBatchGroupSizeBytes(maxWriteBatchGroupSizeBytes); + return this; + } + + @Override + public long maxWriteBatchGroupSizeBytes() { + return dbOptions.maxWriteBatchGroupSizeBytes(); + } + + @Override + public Options oldDefaults(final int majorVersion, final int minorVersion) { + columnFamilyOptions.oldDefaults(majorVersion, minorVersion); + return this; + } + + @Override + public Options optimizeForSmallDb(final Cache cache) { + return super.optimizeForSmallDb(cache); + } + + @Override + public AbstractCompactionFilter> compactionFilter() { + return columnFamilyOptions.compactionFilter(); + } + + @Override + public AbstractCompactionFilterFactory> compactionFilterFactory() { + return columnFamilyOptions.compactionFilterFactory(); + } + + @Override + public Options setStatsPersistPeriodSec(final int statsPersistPeriodSec) { + dbOptions.setStatsPersistPeriodSec(statsPersistPeriodSec); + return this; + } + + @Override + public int statsPersistPeriodSec() { + return dbOptions.statsPersistPeriodSec(); + } + + @Override + public Options setStatsHistoryBufferSize(final long statsHistoryBufferSize) { + dbOptions.setStatsHistoryBufferSize(statsHistoryBufferSize); + return this; + } + + @Override + public long statsHistoryBufferSize() { + return dbOptions.statsHistoryBufferSize(); + } + + @Override + public Options setStrictBytesPerSync(final boolean strictBytesPerSync) { + dbOptions.setStrictBytesPerSync(strictBytesPerSync); + return this; + } + + @Override + public boolean strictBytesPerSync() { + return dbOptions.strictBytesPerSync(); + } + + @Override + public Options setListeners(final List listeners) { + dbOptions.setListeners(listeners); + return this; + } + + @Override + public List listeners() { + return dbOptions.listeners(); + } + + @Override + public Options setEnablePipelinedWrite(final boolean enablePipelinedWrite) { + dbOptions.setEnablePipelinedWrite(enablePipelinedWrite); + return this; + } + + @Override + public boolean enablePipelinedWrite() { + return dbOptions.enablePipelinedWrite(); + } + + @Override + public Options setUnorderedWrite(final boolean unorderedWrite) { + dbOptions.setUnorderedWrite(unorderedWrite); + return this; + } + + @Override + public boolean unorderedWrite() { + return dbOptions.unorderedWrite(); + } + + @Override + public Options setSkipCheckingSstFileSizesOnDbOpen(final boolean skipCheckingSstFileSizesOnDbOpen) { + dbOptions.setSkipCheckingSstFileSizesOnDbOpen(skipCheckingSstFileSizesOnDbOpen); + return this; + } + + @Override + public boolean skipCheckingSstFileSizesOnDbOpen() { + return dbOptions.skipCheckingSstFileSizesOnDbOpen(); + } + + @Override + public Options setWalFilter(final AbstractWalFilter walFilter) { + dbOptions.setWalFilter(walFilter); + return this; + } + + @Override + public WalFilter walFilter() { + return dbOptions.walFilter(); + } + + @Override + public Options setAllowIngestBehind(final boolean allowIngestBehind) { + dbOptions.setAllowIngestBehind(allowIngestBehind); + return this; + } + + @Override + public boolean allowIngestBehind() { + return dbOptions.allowIngestBehind(); + } + + @Override + public Options setPreserveDeletes(final boolean preserveDeletes) { + dbOptions.setPreserveDeletes(preserveDeletes); + return this; + } + + @Override + public boolean preserveDeletes() { + return dbOptions.preserveDeletes(); + } + + @Override + public Options setTwoWriteQueues(final boolean twoWriteQueues) { + dbOptions.setTwoWriteQueues(twoWriteQueues); + return this; + } + + @Override + public boolean twoWriteQueues() { + return dbOptions.twoWriteQueues(); + } + + @Override + public Options setManualWalFlush(final boolean manualWalFlush) { + dbOptions.setManualWalFlush(manualWalFlush); + return this; + } + + @Override + public boolean manualWalFlush() { + return dbOptions.manualWalFlush(); + } + + @Override + public Options setCfPaths(final Collection cfPaths) { + columnFamilyOptions.setCfPaths(cfPaths); + return this; + } + + @Override + public List cfPaths() { + return columnFamilyOptions.cfPaths(); + } + + @Override + public Options setBottommostCompressionOptions(final CompressionOptions bottommostCompressionOptions) { + columnFamilyOptions.setBottommostCompressionOptions(bottommostCompressionOptions); + return this; + } + + @Override + public CompressionOptions bottommostCompressionOptions() { + return columnFamilyOptions.bottommostCompressionOptions(); + } + + @Override + public Options setTtl(final long ttl) { + columnFamilyOptions.setTtl(ttl); + return this; + } + + @Override + public long ttl() { + return columnFamilyOptions.ttl(); + } + + @Override + public Options setAtomicFlush(final boolean atomicFlush) { + dbOptions.setAtomicFlush(atomicFlush); + return this; + } + + @Override + public boolean atomicFlush() { + return dbOptions.atomicFlush(); + } + + @Override + public Options setAvoidUnnecessaryBlockingIO(final boolean avoidUnnecessaryBlockingIO) { + dbOptions.setAvoidUnnecessaryBlockingIO(avoidUnnecessaryBlockingIO); + return this; + } + + @Override + public boolean avoidUnnecessaryBlockingIO() { + return dbOptions.avoidUnnecessaryBlockingIO(); + } + + @Override + public Options setPersistStatsToDisk(final boolean persistStatsToDisk) { + dbOptions.setPersistStatsToDisk(persistStatsToDisk); + return this; + } + + @Override + public boolean persistStatsToDisk() { + return dbOptions.persistStatsToDisk(); + } + + @Override + public Options setWriteDbidToManifest(final boolean writeDbidToManifest) { + dbOptions.setWriteDbidToManifest(writeDbidToManifest); + return this; + } + + @Override + public boolean writeDbidToManifest() { + return dbOptions.writeDbidToManifest(); + } + + @Override + public Options setLogReadaheadSize(final long logReadaheadSize) { + dbOptions.setLogReadaheadSize(logReadaheadSize); + return this; + } + + @Override + public long logReadaheadSize() { + return dbOptions.logReadaheadSize(); + } + + @Override + public Options setBestEffortsRecovery(final boolean bestEffortsRecovery) { + dbOptions.setBestEffortsRecovery(bestEffortsRecovery); + return this; + } + + @Override + public boolean bestEffortsRecovery() { + return dbOptions.bestEffortsRecovery(); + } + + @Override + public Options setMaxBgErrorResumeCount(final int maxBgerrorResumeCount) { + dbOptions.setMaxBgErrorResumeCount(maxBgerrorResumeCount); + return this; + } + + @Override + public int maxBgerrorResumeCount() { + return dbOptions.maxBgerrorResumeCount(); + } + + @Override + public Options setBgerrorResumeRetryInterval(final long bgerrorResumeRetryInterval) { + dbOptions.setBgerrorResumeRetryInterval(bgerrorResumeRetryInterval); + return this; + } + + @Override + public long bgerrorResumeRetryInterval() { + return dbOptions.bgerrorResumeRetryInterval(); + } + + @Override + public Options setSstPartitionerFactory(final SstPartitionerFactory sstPartitionerFactory) { + columnFamilyOptions.setSstPartitionerFactory(sstPartitionerFactory); + return this; + } + + @Override + public SstPartitionerFactory sstPartitionerFactory() { + return columnFamilyOptions.sstPartitionerFactory(); + } + + @Override + public Options setCompactionThreadLimiter(final ConcurrentTaskLimiter compactionThreadLimiter) { + columnFamilyOptions.setCompactionThreadLimiter(compactionThreadLimiter); + return this; + } + + @Override + public ConcurrentTaskLimiter compactionThreadLimiter() { + return columnFamilyOptions.compactionThreadLimiter(); + } + public Options setCompactionFilter(final AbstractCompactionFilter> compactionFilter) { columnFamilyOptions.setCompactionFilter(compactionFilter); return this; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 5596bbf76ae73..f501d139378c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -133,7 +133,7 @@ void openDB(final Map configs, final File stateDir) { tableConfig.setBlockSize(BLOCK_SIZE); filter = new BloomFilter(); - tableConfig.setFilter(filter); + tableConfig.setFilterPolicy(filter); userSpecifiedOptions.optimizeFiltersForHits(); userSpecifiedOptions.setTableFormatConfig(tableConfig); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java index dafbc8f059f58..cd5a00ffcb3bc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBTimestampedStore.java @@ -75,7 +75,7 @@ void openRocksDB(final DBOptions dbOptions, db = RocksDB.open(dbOptions, dbDir.getAbsolutePath(), columnFamilyDescriptors, columnFamilies); setDbAccessor(columnFamilies.get(0), columnFamilies.get(1)); } catch (final RocksDBException e) { - if ("Column family not found: : keyValueWithTimestamp".equals(e.getMessage())) { + if ("Column family not found: keyValueWithTimestamp".equals(e.getMessage())) { try { db = RocksDB.open(dbOptions, dbDir.getAbsolutePath(), columnFamilyDescriptors.subList(0, 1), columnFamilies); columnFamilies.add(db.createColumnFamily(columnFamilyDescriptors.get(1))); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java index 80b776eb10811..f6a191a3ad301 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java @@ -242,6 +242,7 @@ private void shouldLogAndMeterWhenSkippingNullLeftKey(final String builtInMetric props.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, builtInMetricsVersion); try (final LogCaptureAppender appender = LogCaptureAppender.createAndRegister(KStreamKTableJoin.class)) { + driver.close(); driver = new TopologyTestDriver(builder.build(), props); final TestInputTopic inputTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer()); @@ -272,7 +273,7 @@ public void shouldLogAndMeterWhenSkippingNullLeftValueWithBuiltInMetricsVersion0 private void shouldLogAndMeterWhenSkippingNullLeftValue(final String builtInMetricsVersion) { props.setProperty(StreamsConfig.BUILT_IN_METRICS_VERSION_CONFIG, StreamsConfig.METRICS_0100_TO_24); - + driver.close(); driver = new TopologyTestDriver(builder.build(), props); final TestInputTopic inputTopic = driver.createInputTopic(streamTopic, new IntegerSerializer(), new StringSerializer()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index a836c565d54b9..09823376cf052 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -34,7 +34,6 @@ import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.To; -import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.ToInternal; @@ -52,7 +51,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.File; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -91,22 +89,23 @@ public class KStreamSessionWindowAggregateProcessorTest { private final Processor processor = sessionAggregator.get(); private SessionStore sessionStore; private InternalMockProcessorContext context; - private Metrics metrics; + private final Metrics metrics = new Metrics(); @Before - public void initializeStore() { - final File stateDir = TestUtils.tempDirectory(); - metrics = new Metrics(); - final MockStreamsMetrics metrics = new MockStreamsMetrics(KStreamSessionWindowAggregateProcessorTest.this.metrics); + public void setup() { + setup(StreamsConfig.METRICS_LATEST, true); + } + private void setup(final String builtInMetricsVersion, final boolean enableCache) { + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", builtInMetricsVersion, new MockTime()); context = new InternalMockProcessorContext( - stateDir, + TestUtils.tempDirectory(), Serdes.String(), Serdes.String(), - metrics, + streamsMetrics, new StreamsConfig(StreamsTestUtils.getStreamsConfig()), MockRecordCollector::new, - new ThreadCache(new LogContext("testCache "), 100000, metrics), + new ThreadCache(new LogContext("testCache "), 100000, streamsMetrics), Time.SYSTEM ) { @SuppressWarnings("unchecked") @@ -116,8 +115,9 @@ public void forward(final Object key, final Object value, final To to) { results.add(new KeyValueTimestamp<>((Windowed) key, (Change) value, toInternal.timestamp())); } }; + TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), streamsMetrics); - initStore(true); + initStore(enableCache); processor.init(context); } @@ -133,6 +133,9 @@ private void initStore(final boolean enableCaching) { storeBuilder.withCachingEnabled(); } + if (sessionStore != null) { + sessionStore.close(); + } sessionStore = storeBuilder.build(); sessionStore.init((StateStoreContext) context, sessionStore); } @@ -383,8 +386,7 @@ public void shouldLogAndMeterWhenSkippingNullKeyWithBuiltInMetricsVersionLatest( } private void shouldLogAndMeterWhenSkippingNullKeyWithBuiltInMetrics(final String builtInMetricsVersion) { - final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); - processor.init(context); + setup(builtInMetricsVersion, false); context.setRecordContext( new ProcessorRecordContext(-1, -2, -3, "topic", null) ); @@ -424,7 +426,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithZeroGraceWithBuiltInMetri } private void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace(final String builtInMetricsVersion) { - final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); + setup(builtInMetricsVersion, false); final Processor processor = new KStreamSessionWindowAggregate<>( SessionWindows.with(ofMillis(10L)).grace(ofMillis(0L)), STORE_NAME, @@ -432,7 +434,6 @@ private void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace(final String b aggregator, sessionMerger ).get(); - initStore(false); processor.init(context); // dummy record to establish stream time = 0 @@ -522,7 +523,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGraceWithBuiltInMe } private void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace(final String builtInMetricsVersion) { - final InternalMockProcessorContext context = createInternalMockProcessorContext(builtInMetricsVersion); + setup(builtInMetricsVersion, false); final Processor processor = new KStreamSessionWindowAggregate<>( SessionWindows.with(ofMillis(10L)).grace(ofMillis(1L)), STORE_NAME, @@ -530,7 +531,6 @@ private void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace(final Strin aggregator, sessionMerger ).get(); - initStore(false); processor.init(context); try (final LogCaptureAppender appender = @@ -616,35 +616,4 @@ private void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace(final Strin (Double) metrics.metrics().get(dropRate).metricValue(), greaterThan(0.0)); } - - private InternalMockProcessorContext createInternalMockProcessorContext(final String builtInMetricsVersion) { - final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(metrics, "test", builtInMetricsVersion, new MockTime()); - final InternalMockProcessorContext context = new InternalMockProcessorContext( - TestUtils.tempDirectory(), - Serdes.String(), - Serdes.String(), - streamsMetrics, - new StreamsConfig(StreamsTestUtils.getStreamsConfig()), - MockRecordCollector::new, - new ThreadCache(new LogContext("testCache "), 100000, streamsMetrics), - Time.SYSTEM - ) { - @SuppressWarnings("unchecked") - @Override - public void forward(final Object key, final Object value, final To to) { - toInternal.update(to); - results.add(new KeyValueTimestamp<>((Windowed) key, (Change) value, toInternal.timestamp())); - } - }; - TaskMetrics.droppedRecordsSensorOrSkippedRecordsSensor(threadId, context.taskId().toString(), streamsMetrics); - final StoreBuilder> storeBuilder = - Stores.sessionStoreBuilder( - Stores.persistentSessionStore(STORE_NAME, ofMillis(GAP_MS * 3)), - Serdes.String(), - Serdes.Long()) - .withLoggingDisabled(); - final SessionStore sessionStore = storeBuilder.build(); - sessionStore.init((StateStoreContext) context, sessionStore); - return context; - } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java index 0476e707db979..a2c980ecfe287 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java @@ -50,6 +50,7 @@ import static org.apache.kafka.streams.processor.internals.GlobalStreamThread.State.RUNNING; import static org.apache.kafka.streams.processor.internals.testutil.ConsumerRecordUtil.record; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.Assert.assertEquals; @@ -119,26 +120,30 @@ public String newStoreName(final String prefix) { } @Test - public void shouldThrowStreamsExceptionOnStartupIfThereIsAStreamsException() { + public void shouldThrowStreamsExceptionOnStartupIfThereIsAStreamsException() throws Exception { // should throw as the MockConsumer hasn't been configured and there are no // partitions available + final StateStore globalStore = builder.globalStateStores().get(GLOBAL_STORE_NAME); try { globalStreamThread.start(); fail("Should have thrown StreamsException if start up failed"); } catch (final StreamsException e) { // ok } + globalStreamThread.join(); + assertThat(globalStore.isOpen(), is(false)); assertFalse(globalStreamThread.stillRunning()); } @Test - public void shouldThrowStreamsExceptionOnStartupIfExceptionOccurred() { + public void shouldThrowStreamsExceptionOnStartupIfExceptionOccurred() throws Exception { final MockConsumer mockConsumer = new MockConsumer(OffsetResetStrategy.EARLIEST) { @Override public List partitionsFor(final String topic) { throw new RuntimeException("KABOOM!"); } }; + final StateStore globalStore = builder.globalStateStores().get(GLOBAL_STORE_NAME); globalStreamThread = new GlobalStreamThread( builder.buildGlobalStateTopology(), config, @@ -159,14 +164,19 @@ public List partitionsFor(final String topic) { assertThat(e.getCause(), instanceOf(RuntimeException.class)); assertThat(e.getCause().getMessage(), equalTo("KABOOM!")); } + globalStreamThread.join(); + assertThat(globalStore.isOpen(), is(false)); assertFalse(globalStreamThread.stillRunning()); } @Test - public void shouldBeRunningAfterSuccessfulStart() { + public void shouldBeRunningAfterSuccessfulStart() throws Exception { initializeConsumer(); startAndSwallowError(); assertTrue(globalStreamThread.stillRunning()); + + globalStreamThread.shutdown(); + globalStreamThread.join(); } @Test(timeout = 30000) @@ -189,16 +199,6 @@ public void shouldCloseStateStoresOnClose() throws Exception { assertFalse(globalStore.isOpen()); } - @Test - public void shouldTransitionToDeadOnClose() throws Exception { - initializeConsumer(); - startAndSwallowError(); - globalStreamThread.shutdown(); - globalStreamThread.join(); - - assertEquals(GlobalStreamThread.State.DEAD, globalStreamThread.state()); - } - @Test public void shouldStayDeadAfterTwoCloses() throws Exception { initializeConsumer(); @@ -225,6 +225,7 @@ public void shouldTransitionToRunningOnStart() throws Exception { @Test public void shouldDieOnInvalidOffsetExceptionDuringStartup() throws Exception { + final StateStore globalStore = builder.globalStateStores().get(GLOBAL_STORE_NAME); initializeConsumer(); mockConsumer.setPollException(new InvalidOffsetException("Try Again!") { @Override @@ -240,12 +241,15 @@ public Set partitions() { 10 * 1000, "GlobalStreamThread should have died." ); + globalStreamThread.join(); + assertThat(globalStore.isOpen(), is(false)); assertFalse(new File(baseDirectoryName + File.separator + "testAppId" + File.separator + "global").exists()); } @Test public void shouldDieOnInvalidOffsetExceptionWhileRunning() throws Exception { + final StateStore globalStore = builder.globalStateStores().get(GLOBAL_STORE_NAME); initializeConsumer(); startAndSwallowError(); @@ -274,7 +278,9 @@ public Set partitions() { 10 * 1000, "GlobalStreamThread should have died." ); + globalStreamThread.join(); + assertThat(globalStore.isOpen(), is(false)); assertFalse(new File(baseDirectoryName + File.separator + "testAppId" + File.separator + "global").exists()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index bbb1c60639767..71e6af16bf088 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -1535,6 +1535,7 @@ public void shouldReinitializeRevivedTasksInAnyState() { assertThat(processed.get(), is(false)); thread.runOnce(); assertThat(processed.get(), is(true)); + thread.taskManager().shutdown(true); } @Test @@ -1736,6 +1737,8 @@ public void shouldReturnStandbyTaskMetadataWhileRunningState() { assertEquals(StreamThread.State.RUNNING.name(), threadMetadata.threadState()); assertTrue(threadMetadata.standbyTasks().contains(new TaskMetadata(task1.toString(), Utils.mkSet(t1p1), new HashMap<>(), new HashMap<>(), Optional.empty()))); assertTrue(threadMetadata.activeTasks().isEmpty()); + + thread.taskManager().shutdown(true); } @SuppressWarnings("unchecked") @@ -1816,6 +1819,8 @@ public void shouldUpdateStandbyTask() throws Exception { assertEquals(10L, store1.approximateNumEntries()); assertEquals(4L, store2.approximateNumEntries()); + + thread.taskManager().shutdown(true); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java index 53d184652d0b9..b103e989d8df0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractRocksDBSegmentedBytesStoreTest.java @@ -209,6 +209,8 @@ public void shouldRollSegments() { ), results ); + + segments.close(); } @Test @@ -237,6 +239,8 @@ public void shouldGetAllSegments() { ), results ); + + segments.close(); } @Test @@ -265,6 +269,8 @@ public void shouldFetchAllSegments() { ), results ); + + segments.close(); } @Test @@ -299,6 +305,8 @@ public void shouldLoadSegmentsWithOldStyleDateFormattedName() { ) ) ); + + segments.close(); } @Test @@ -329,6 +337,8 @@ public void shouldLoadSegmentsWithOldStyleColonFormattedName() { ) ) ); + + segments.close(); } @Test @@ -470,6 +480,8 @@ private void shouldLogAndMeasureExpiredRecords(final String builtInMetricsVersio } assertEquals(1.0, dropTotal.metricValue()); assertNotEquals(0.0, dropRate.metricValue()); + + bytesStore.close(); } private Set segmentDirs() { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java index 7b63182251ff2..ed60837400d2f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractSessionBytesStoreTest.java @@ -370,6 +370,7 @@ public void shouldBackwardFindSessionsToMerge() { @Test public void shouldFetchExactKeys() { + sessionStore.close(); sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long()); sessionStore.init((StateStoreContext) context, sessionStore); @@ -407,6 +408,7 @@ public void shouldFetchExactKeys() { @Test public void shouldBackwardFetchExactKeys() { + sessionStore.close(); sessionStore = buildSessionStore(0x7a00000000000000L, Serdes.String(), Serdes.Long()); sessionStore.init((StateStoreContext) context, sessionStore); @@ -469,6 +471,8 @@ public void shouldFetchAndIterateOverExactBinaryKeys() { assertThat(valuesToSet(sessionStore.findSessions(key2, 0L, Long.MAX_VALUE)), equalTo(expectedKey2)); final Set expectedKey3 = new HashSet<>(asList("3", "6", "9")); assertThat(valuesToSet(sessionStore.findSessions(key3, 0L, Long.MAX_VALUE)), equalTo(expectedKey3)); + + sessionStore.close(); } @Test @@ -498,6 +502,8 @@ public void shouldBackwardFetchAndIterateOverExactBinaryKeys() { assertThat(valuesToSet(sessionStore.backwardFindSessions(key2, 0L, Long.MAX_VALUE)), equalTo(expectedKey2)); final Set expectedKey3 = new HashSet<>(asList("3", "6", "9")); assertThat(valuesToSet(sessionStore.backwardFindSessions(key3, 0L, Long.MAX_VALUE)), equalTo(expectedKey3)); + + sessionStore.close(); } @Test @@ -681,6 +687,8 @@ private void shouldLogAndMeasureExpiredRecords(final String builtInMetricsVersio } assertEquals(1.0, dropTotal.metricValue()); assertNotEquals(0.0, dropRate.metricValue()); + + sessionStore.close(); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java index 8d73a2d24e36d..08fcb6dc114d9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractWindowBytesStoreTest.java @@ -783,6 +783,7 @@ public void testPutAndFetchAfter() { @Test public void testPutSameKeyTimestamp() { + windowStore.close(); windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); windowStore.init((StateStoreContext) context, windowStore); @@ -899,6 +900,7 @@ public void shouldFetchAndIterateOverExactKeys() { windowedPair("aa", "0004", 1, windowSize) ))) ); + windowStore.close(); } @Test @@ -978,6 +980,8 @@ public void shouldFetchAndIterateOverExactBinaryKeys() { valuesToSet(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))), equalTo(expectedKey3) ); + + windowStore.close(); } @Test @@ -1101,6 +1105,8 @@ private void shouldLogAndMeasureExpiredRecords(final String builtInMetricsVersio } assertEquals(1.0, dropTotal.metricValue()); assertNotEquals(0.0, dropRate.metricValue()); + + windowStore.close(); } @Test @@ -1168,6 +1174,7 @@ public void shouldNotThrowConcurrentModificationException() { @Test public void testFetchDuplicates() { + windowStore.close(); windowStore = buildWindowStore(RETENTION_PERIOD, WINDOW_SIZE, true, Serdes.Integer(), Serdes.String()); windowStore.init((StateStoreContext) context, windowStore); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java index d472c7f5637db..7f8a394c278c0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentSessionStoreTest.java @@ -201,6 +201,7 @@ public void shouldCloseCacheAfterErrorDuringWrappedStoreClose() { } private void setUpCloseTests() { + underlyingStore.close(); underlyingStore = EasyMock.createNiceMock(SessionStore.class); EasyMock.expect(underlyingStore.name()).andStubReturn("store-name"); EasyMock.expect(underlyingStore.isOpen()).andStubReturn(true); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java index 735101f8158d1..13f1f2b395b8d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingPersistentWindowStoreTest.java @@ -196,7 +196,6 @@ public void close() { } }, "store-name"); - final String bootstrapServers = "localhost:9092"; final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); @@ -231,6 +230,8 @@ public void close() { for (int i = 0; i < 5; i++) { inputTopic.pipeInput(UUID.randomUUID().toString(), UUID.randomUUID().toString()); } + + driver.close(); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java index 71841bdebd2b6..859aea1168a96 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/KeyValueSegmentTest.java @@ -75,6 +75,8 @@ public void shouldDeleteStateDirectoryOnDestroy() throws Exception { segment.destroy(); assertFalse(new File(directoryPath + File.separator + "window", "segment").exists()); assertTrue(new File(directoryPath, "window").exists()); + + segment.close(); } @Test @@ -89,6 +91,8 @@ public void shouldBeEqualIfIdIsEqual() { assertThat(segment, not(equalTo(segmentDifferentId))); assertThat(segment, not(equalTo(null))); assertThat(segment, not(equalTo("anyName"))); + + segment.close(); } @Test @@ -102,6 +106,8 @@ public void shouldHashOnSegmentIdOnly() { assertTrue(set.add(segment)); assertFalse(set.add(segmentSameId)); assertTrue(set.add(segmentDifferentId)); + + segment.close(); } @Test @@ -117,5 +123,9 @@ public void shouldCompareSegmentIdOnly() { assertThat(segment3.compareTo(segment1), equalTo(-1)); assertThat(segment2.compareTo(segment3), equalTo(1)); assertThat(segment3.compareTo(segment2), equalTo(-1)); + + segment1.close(); + segment2.close(); + segment3.close(); } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java index 10afe8a6ec24e..2ba0f46c29657 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java @@ -23,6 +23,7 @@ import org.rocksdb.AbstractCompactionFilter; import org.rocksdb.AbstractCompactionFilter.Context; import org.rocksdb.AbstractCompactionFilterFactory; +import org.rocksdb.AbstractWalFilter; import org.rocksdb.AccessHint; import org.rocksdb.BuiltinComparator; import org.rocksdb.ColumnFamilyOptions; @@ -45,6 +46,8 @@ import org.rocksdb.StringAppendOperator; import org.rocksdb.VectorMemTableConfig; import org.rocksdb.WALRecoveryMode; +import org.rocksdb.WalProcessingOption; +import org.rocksdb.WriteBatch; import org.rocksdb.WriteBufferManager; import org.rocksdb.util.BytewiseComparator; @@ -53,6 +56,7 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; +import java.util.Map; import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.replay; @@ -74,6 +78,7 @@ public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest { private final List ignoreMethods = new LinkedList() { { add("isOwningHandle"); + add("getNativeHandle"); add("dispose"); add("wait"); add("equals"); @@ -82,6 +87,7 @@ public class RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest { add("notify"); add("notifyAll"); add("toString"); + add("getOptionStringFromProps"); } }; @@ -174,6 +180,24 @@ protected void log(final InfoLogLevel infoLogLevel, final String logMsg) {} case "org.rocksdb.WriteBufferManager": parameters[i] = new WriteBufferManager(1L, new LRUCache(1L)); break; + case "org.rocksdb.AbstractWalFilter": + class TestWalFilter extends AbstractWalFilter { + @Override + public void columnFamilyLogNumberMap(final Map cfLognumber, final Map cfNameId) { + } + + @Override + public LogRecordFoundResult logRecordFound(final long logNumber, final String logFileName, final WriteBatch batch, final WriteBatch newBatch) { + return new LogRecordFoundResult(WalProcessingOption.CONTINUE_PROCESSING, false); + } + + @Override + public String name() { + return "TestWalFilter"; + } + } + parameters[i] = new TestWalFilter(); + break; default: parameters[i] = parameterTypes[i].getConstructor().newInstance(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index 0568df2d5689c..93e65f02ebddb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -855,7 +855,7 @@ public void setConfig(final String storeName, final Options options, final Map Date: Thu, 6 May 2021 19:27:23 -0500 Subject: [PATCH 131/155] KAFKA-10847: Set StreamsConfig on InternalTopologyDriver before writing topology (#10640) Reviewers: Guozhang Wang , Matthias J. Sax --- .../kstream/internals/InternalStreamsBuilder.java | 4 ++-- .../streams/kstream/internals/graph/GlobalStoreNode.java | 4 +++- .../kafka/streams/kstream/internals/graph/GraphNode.java | 3 ++- .../graph/GroupedTableOperationRepartitionNode.java | 4 +++- .../kstream/internals/graph/KTableKTableJoinNode.java | 3 ++- .../internals/graph/OptimizableRepartitionNode.java | 4 +++- .../kstream/internals/graph/ProcessorGraphNode.java | 4 +++- .../streams/kstream/internals/graph/StateStoreNode.java | 4 +++- .../kstream/internals/graph/StatefulProcessorNode.java | 3 ++- .../streams/kstream/internals/graph/StreamSinkNode.java | 4 +++- .../kstream/internals/graph/StreamSourceNode.java | 3 ++- .../kstream/internals/graph/StreamStreamJoinNode.java | 9 ++++++--- .../kstream/internals/graph/StreamTableJoinNode.java | 3 ++- .../kstream/internals/graph/StreamToTableNode.java | 4 +++- .../kstream/internals/graph/TableProcessorNode.java | 3 ++- .../streams/kstream/internals/graph/TableSourceNode.java | 3 ++- .../internals/graph/UnoptimizableRepartitionNode.java | 4 +++- .../kstream/internals/KStreamKStreamLeftJoinTest.java | 9 +++++++++ .../kstream/internals/KStreamKStreamOuterJoinTest.java | 6 ++++++ .../kstream/internals/graph/TableSourceNodeTest.java | 4 +++- 20 files changed, 64 insertions(+), 21 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java index 6fa65c13894d4..1527263af5cef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/InternalStreamsBuilder.java @@ -74,7 +74,7 @@ public class InternalStreamsBuilder implements InternalNameProvider { protected final GraphNode root = new GraphNode(TOPOLOGY_ROOT) { @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { // no-op for root node } }; @@ -290,7 +290,7 @@ public void buildAndOptimizeTopology(final Properties props) { } if (streamGraphNode.allParentsWrittenToTopology() && !streamGraphNode.hasWrittenToTopology()) { - streamGraphNode.writeToTopology(internalTopologyBuilder); + streamGraphNode.writeToTopology(internalTopologyBuilder, props); streamGraphNode.setHasWrittenToTopology(true); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java index 753e0768cd576..90d9efbab2cf7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GlobalStoreNode.java @@ -22,6 +22,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.StoreBuilder; +import java.util.Properties; + public class GlobalStoreNode extends StateStoreNode { private final String sourceName; @@ -47,7 +49,7 @@ public GlobalStoreNode(final StoreBuilder storeBuilder, } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { storeBuilder.withLoggingDisabled(); topologyBuilder.addGlobalStore(storeBuilder, sourceName, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java index 76c2b5c3d604e..c55395a640765 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphNode.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.LinkedHashSet; +import java.util.Properties; public abstract class GraphNode { @@ -117,7 +118,7 @@ public Integer buildPriority() { return this.buildPriority; } - public abstract void writeToTopology(final InternalTopologyBuilder topologyBuilder); + public abstract void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props); public boolean hasWrittenToTopology() { return hasWrittenToTopology; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java index a7ba30d1ffd0c..1117e5ec7fe7c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GroupedTableOperationRepartitionNode.java @@ -26,6 +26,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopicProperties; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import java.util.Properties; + public class GroupedTableOperationRepartitionNode extends BaseRepartitionNode { @@ -78,7 +80,7 @@ public String toString() { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties); topologyBuilder.addSink( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java index e2abfb5d38a4a..0ca1e35f3b9f9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/KTableKTableJoinNode.java @@ -25,6 +25,7 @@ import org.apache.kafka.streams.state.TimestampedKeyValueStore; import java.util.Arrays; +import java.util.Properties; /** * Too much specific information to generalize so the KTable-KTable join requires a specific node. @@ -96,7 +97,7 @@ public KTableKTableJoinMerger joinMerger() { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final String thisProcessorName = thisProcessorParameters().processorName(); final String otherProcessorName = otherProcessorParameters().processorName(); final String mergeProcessorName = mergeProcessorParameters().processorName(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java index a9693ec814cb8..c7ee03d79910f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/OptimizableRepartitionNode.java @@ -23,6 +23,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopicProperties; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import java.util.Properties; + public class OptimizableRepartitionNode extends BaseRepartitionNode { private OptimizableRepartitionNode(final String nodeName, @@ -64,7 +66,7 @@ public String toString() { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties); topologyBuilder.addProcessor( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java index a38f516bc1c57..c3dd4f62419b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorGraphNode.java @@ -19,6 +19,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import java.util.Properties; + /** * Used to represent any type of stateless operation: * @@ -55,7 +57,7 @@ public String toString() { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { topologyBuilder.addProcessor(processorParameters.processorName(), processorParameters.processorSupplier(), parentNodeNames()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StateStoreNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StateStoreNode.java index 32dc93dda31bd..ae01580f0cc3f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StateStoreNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StateStoreNode.java @@ -20,6 +20,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.state.StoreBuilder; +import java.util.Properties; + public class StateStoreNode extends GraphNode { protected final StoreBuilder storeBuilder; @@ -31,7 +33,7 @@ public StateStoreNode(final StoreBuilder storeBuilder) { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { topologyBuilder.addStateStore(storeBuilder); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java index 28af95b6dd68c..ad6de37f45cbe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java @@ -22,6 +22,7 @@ import org.apache.kafka.streams.state.StoreBuilder; import java.util.Arrays; +import java.util.Properties; import java.util.Set; import java.util.stream.Stream; @@ -78,7 +79,7 @@ public String toString() { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final String processorName = processorParameters().processorName(); final ProcessorSupplier processorSupplier = processorParameters().processorSupplier(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java index f12a9e5b9bf2c..8d67ac1f21726 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSinkNode.java @@ -26,6 +26,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor; +import java.util.Properties; + public class StreamSinkNode extends GraphNode { private final TopicNameExtractor topicNameExtractor; @@ -52,7 +54,7 @@ public String toString() { @Override @SuppressWarnings("unchecked") - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final Serializer keySerializer = producedInternal.keySerde() == null ? null : producedInternal.keySerde().serializer(); final Serializer valSerializer = producedInternal.valueSerde() == null ? null : producedInternal.valueSerde().serializer(); final String[] parentNames = parentNodeNames(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java index f4f9842995e90..d4adc894de5e0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import java.util.Collection; +import java.util.Properties; import java.util.regex.Pattern; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,7 +70,7 @@ public String toString() { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { if (topicPattern() != null) { topologyBuilder.addSource(consumedInternal().offsetResetPolicy(), diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java index 9de23781e7b6a..e940c718f9d1f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamStreamJoinNode.java @@ -26,7 +26,9 @@ import org.apache.kafka.streams.state.internals.KeyAndJoinSide; import org.apache.kafka.streams.state.internals.LeftOrRightValue; +import java.util.HashMap; import java.util.Optional; +import java.util.Properties; import static org.apache.kafka.streams.StreamsConfig.InternalConfig.ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; @@ -34,6 +36,7 @@ * Too much information to generalize, so Stream-Stream joins are represented by a specific node. */ public class StreamStreamJoinNode extends BaseJoinProcessorNode { + private static final Properties EMPTY_PROPERTIES = new Properties(); private final ProcessorParameters thisWindowedStreamProcessorParameters; private final ProcessorParameters otherWindowedStreamProcessorParameters; @@ -84,8 +87,9 @@ public String toString() { "} " + super.toString(); } + @SuppressWarnings("unchecked") @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final String thisProcessorName = thisProcessorParameters().processorName(); final String otherProcessorName = otherProcessorParameters().processorName(); @@ -98,8 +102,7 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { topologyBuilder.addStateStore(thisWindowStoreBuilder, thisWindowedStreamProcessorName, otherProcessorName); topologyBuilder.addStateStore(otherWindowStoreBuilder, otherWindowedStreamProcessorName, thisProcessorName); - final StreamsConfig streamsConfig = topologyBuilder.getStreamsConfig(); - if (streamsConfig == null || StreamsConfig.InternalConfig.getBoolean(streamsConfig.originals(), ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, true)) { + if (props == null || StreamsConfig.InternalConfig.getBoolean(new HashMap(props), ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX, true)) { outerJoinWindowStoreBuilder.ifPresent(builder -> topologyBuilder.addStateStore(builder, thisProcessorName, otherProcessorName)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java index a4db1ba24cd01..d5bd2b8d3e6e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamTableJoinNode.java @@ -21,6 +21,7 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import java.util.Arrays; +import java.util.Properties; /** * Represents a join between a KStream and a KTable or GlobalKTable @@ -54,7 +55,7 @@ public String toString() { } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final String processorName = processorParameters.processorName(); final ProcessorSupplier processorSupplier = processorParameters.processorSupplier(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java index 3b0a572c1141b..54231d3bebd08 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamToTableNode.java @@ -26,6 +26,8 @@ import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.TimestampedKeyValueStore; +import java.util.Properties; + /** * Represents a KTable convert From KStream */ @@ -52,7 +54,7 @@ public String toString() { @SuppressWarnings("unchecked") @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final StoreBuilder> storeBuilder = new TimestampedKeyValueStoreMaterializer<>((MaterializedInternal>) materializedInternal).materialize(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java index b9a25680c1c8d..5254c5757f1b9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNode.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Objects; +import java.util.Properties; public class TableProcessorNode extends GraphNode { @@ -58,7 +59,7 @@ public String toString() { @SuppressWarnings("unchecked") @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final String processorName = processorParameters.processorName(); topologyBuilder.addProcessor(processorName, processorParameters.processorSupplier(), parentNodeNames()); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java index 203f3afbe6641..b708961073401 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNode.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.state.TimestampedKeyValueStore; import java.util.Collections; +import java.util.Properties; /** * Used to represent either a KTable source or a GlobalKTable source. A boolean flag is used to indicate if this represents a GlobalKTable a {@link @@ -81,7 +82,7 @@ public static TableSourceNodeBuilder tableSourceNodeBuilder() { @Override @SuppressWarnings("unchecked") - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { final String topicName = topicNames().iterator().next(); // TODO: we assume source KTables can only be timestamped-key-value stores for now. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java index daac9bdd532e9..a231d23837a95 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/UnoptimizableRepartitionNode.java @@ -22,6 +22,8 @@ import org.apache.kafka.streams.processor.internals.InternalTopicProperties; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import java.util.Properties; + /** * Repartition node that is not subject of optimization algorithm */ @@ -50,7 +52,7 @@ private UnoptimizableRepartitionNode(final String nodeName, } @Override - public void writeToTopology(final InternalTopologyBuilder topologyBuilder) { + public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final Properties props) { topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties); topologyBuilder.addProcessor( diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 50c2f86008b3c..d0fb2bed71159 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -116,6 +116,9 @@ public void testLeftJoinWithSpuriousResultFixDisabled() { driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final MockProcessor processor = supplier.theCapturedProcessor(); + // Only 2 window stores should be available + assertEquals(2, driver.getAllStateStores().size()); + // push two items to the primary stream; the other window is empty // w1 {} // w2 {} @@ -167,6 +170,9 @@ public void testLeftJoinDuplicatesWithFixDisabled() { driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final MockProcessor processor = supplier.theCapturedProcessor(); + // Only 2 window stores should be available + assertEquals(2, driver.getAllStateStores().size()); + inputTopic1.pipeInput(0, "A0", 0); inputTopic1.pipeInput(0, "A0-0", 0); inputTopic2.pipeInput(0, "a0", 0); @@ -488,6 +494,9 @@ public void runLeftJoin(final StreamJoined streamJoined driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final MockProcessor processor = supplier.theCapturedProcessor(); + // 2 window stores + 1 shared window store should be available + assertEquals(3, driver.getAllStateStores().size()); + // push two items to the primary stream; the other window is empty // w1 {} // w2 {} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java index 2bf7fef9132ff..9c16c59c16c1d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java @@ -112,6 +112,9 @@ public void testOuterJoinDuplicatesWithFixDisabled() { driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final MockProcessor processor = supplier.theCapturedProcessor(); + // Only 2 window stores should be available + assertEquals(2, driver.getAllStateStores().size()); + inputTopic1.pipeInput(0, "A0", 0); inputTopic1.pipeInput(0, "A0-0", 0); inputTopic2.pipeInput(0, "a0", 0); @@ -557,6 +560,9 @@ public void runOuterJoin(final StreamJoined streamJoine driver.createInputTopic(topic2, new IntegerSerializer(), new StringSerializer(), Instant.ofEpochMilli(0L), Duration.ZERO); final MockProcessor processor = supplier.theCapturedProcessor(); + // 2 window stores + 1 shared window store should be available + assertEquals(3, driver.getAllStateStores().size()); + // push two items to the primary stream; the other window is empty; this should not // produce any items because window has not expired // w1 {} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java index 66c55c0f3414e..1d74e8926e220 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableSourceNodeTest.java @@ -30,6 +30,8 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import java.util.Properties; + @RunWith(PowerMockRunner.class) @PrepareForTest({InternalTopologyBuilder.class}) public class TableSourceNodeTest { @@ -71,6 +73,6 @@ private void buildTableSourceNode(final boolean shouldReuseSourceTopicForChangel .build(); tableSourceNode.reuseSourceTopicForChangeLog(shouldReuseSourceTopicForChangelog); - tableSourceNode.writeToTopology(topologyBuilder); + tableSourceNode.writeToTopology(topologyBuilder, new Properties()); } } From 15d1cc8b5435eda7c36e42ae0898b0671b70a96f Mon Sep 17 00:00:00 2001 From: Scott Hendricks <49452084+scott-hendricks@users.noreply.github.com> Date: Thu, 6 May 2021 20:46:01 -0400 Subject: [PATCH 132/155] MINOR: Improvements and fixes for Trogdor payload generators. (#10621) * Changes the new Throughput Generators to track messages per window instead of making per-second calculations which can have rounding errors. Also, one of these had a calculation error which prompted this change in the first place. * Fixes a couple typos. * Fixes an error where certain JSON fields were not exposed, causing the workloads to not behave as intended. * Fixes a bug where we use wait not in a loop, which exits too quickly. * Adds additional constant payload generators. * Fixes problems with an example spec. * Fixes several off-by-one comparisons. Reviewers: Colin P. McCabe --- .../workload/ConstantThroughputGenerator.java | 39 +++--- .../workload/GaussianFlushGenerator.java | 6 +- .../workload/GaussianThroughputGenerator.java | 49 ++++--- ...sianTimestampConstantPayloadGenerator.java | 125 ++++++++++++++++++ ...ussianTimestampRandomPayloadGenerator.java | 11 +- .../trogdor/workload/PayloadGenerator.java | 4 +- .../TimestampConstantPayloadGenerator.java | 78 +++++++++++ .../workload/TimestampRecordProcessor.java | 2 +- 8 files changed, 260 insertions(+), 54 deletions(-) create mode 100644 trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java create mode 100644 trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java index 19e5af0559bf6..9e5eeb967e96e 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConstantThroughputGenerator.java @@ -26,26 +26,21 @@ * The lower the window size, the smoother the traffic will be. Using a 100ms window offers no noticeable spikes in * traffic while still being long enough to avoid too much overhead. * - * WARNING: Due to binary nature of throughput in terms of messages sent in a window, each window will send at least 1 - * message, and each window sends the same number of messages, rounded down. For example, 99 messages per second with a - * 100ms window will only send 90 messages per second, or 9 messages per window. Another example, in order to send only - * 5 messages per second, a window size of 200ms is required. In cases like these, both the `messagesPerSecond` and - * `windowSizeMs` parameters should be adjusted together to achieve more accurate throughput. - * * Here is an example spec: * * { * "type": "constant", - * "messagesPerSecond": 500, + * "messagesPerWindow": 50, * "windowSizeMs": 100 * } * * This will produce a workload that runs 500 messages per second, with a maximum resolution of 50 messages per 100 * millisecond. + * + * If `messagesPerWindow` is less than or equal to 0, `throttle` will not throttle at all and will return immediately. */ public class ConstantThroughputGenerator implements ThroughputGenerator { - private final int messagesPerSecond; private final int messagesPerWindow; private final long windowSizeMs; @@ -53,23 +48,25 @@ public class ConstantThroughputGenerator implements ThroughputGenerator { private int messageTracker = 0; @JsonCreator - public ConstantThroughputGenerator(@JsonProperty("messagesPerSecond") int messagesPerSecond, + public ConstantThroughputGenerator(@JsonProperty("messagesPerWindow") int messagesPerWindow, @JsonProperty("windowSizeMs") long windowSizeMs) { - // Calcualte the default values. + // Calculate the default values. if (windowSizeMs <= 0) { windowSizeMs = 100; } this.windowSizeMs = windowSizeMs; - this.messagesPerSecond = messagesPerSecond; - - // Use the rest of the parameters to calculate window properties. - this.messagesPerWindow = (int) ((long) messagesPerSecond / windowSizeMs); + this.messagesPerWindow = messagesPerWindow; calculateNextWindow(); } @JsonProperty - public int messagesPerSecond() { - return messagesPerSecond; + public long windowSizeMs() { + return windowSizeMs; + } + + @JsonProperty + public int messagesPerWindow() { + return messagesPerWindow; } private void calculateNextWindow() { @@ -79,7 +76,7 @@ private void calculateNextWindow() { // Calculate the next window start time. long now = Time.SYSTEM.milliseconds(); if (nextWindowStarts > 0) { - while (nextWindowStarts < now) { + while (nextWindowStarts <= now) { nextWindowStarts += windowSizeMs; } } else { @@ -89,8 +86,8 @@ private void calculateNextWindow() { @Override public synchronized void throttle() throws InterruptedException { - // Run unthrottled if messagesPerSecond is negative. - if (messagesPerSecond < 0) { + // Run unthrottled if messagesPerWindow is not positive. + if (messagesPerWindow <= 0) { return; } @@ -106,7 +103,9 @@ public synchronized void throttle() throws InterruptedException { if (messageTracker >= messagesPerWindow) { // Wait the difference in time between now and when the next window starts. - wait(nextWindowStarts - Time.SYSTEM.milliseconds()); + while (nextWindowStarts > Time.SYSTEM.milliseconds()) { + wait(nextWindowStarts - Time.SYSTEM.milliseconds()); + } } } } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java index a3157dbad4f01..eb6845e38046e 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianFlushGenerator.java @@ -52,7 +52,7 @@ public class GaussianFlushGenerator implements FlushGenerator { private final int messagesPerFlushAverage; - private final int messagesPerFlushDeviation; + private final double messagesPerFlushDeviation; private final Random random = new Random(); @@ -61,7 +61,7 @@ public class GaussianFlushGenerator implements FlushGenerator { @JsonCreator public GaussianFlushGenerator(@JsonProperty("messagesPerFlushAverage") int messagesPerFlushAverage, - @JsonProperty("messagesPerFlushDeviation") int messagesPerFlushDeviation) { + @JsonProperty("messagesPerFlushDeviation") double messagesPerFlushDeviation) { this.messagesPerFlushAverage = messagesPerFlushAverage; this.messagesPerFlushDeviation = messagesPerFlushDeviation; calculateFlushSize(); @@ -73,7 +73,7 @@ public int messagesPerFlushAverage() { } @JsonProperty - public long messagesPerFlushDeviation() { + public double messagesPerFlushDeviation() { return messagesPerFlushDeviation; } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java index 6d71ff5f121ca..a77298f0be8b0 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianThroughputGenerator.java @@ -25,21 +25,17 @@ /* * This throughput generator configures throughput with a gaussian normal distribution on a per-window basis. You can * specify how many windows to keep the throughput at the rate before changing. All traffic will follow a gaussian - * distribution centered around `messagesPerSecondAverage` with a deviation of `messagesPerSecondDeviation`. + * distribution centered around `messagesPerWindowAverage` with a deviation of `messagesPerWindowDeviation`. * * The lower the window size, the smoother the traffic will be. Using a 100ms window offers no noticeable spikes in * traffic while still being long enough to avoid too much overhead. * - * WARNING: Due to binary nature of throughput in terms of messages sent in a window, this does not work well for an - * average throughput of less than 5 messages per window. In cases where you want lower throughput, please adjust the - * `windowSizeMs` accordingly. - * * Here is an example spec: * * { * "type": "gaussian", - * "messagesPerSecondAverage": 500, - * "messagesPerSecondDeviation": 50, + * "messagesPerWindowAverage": 50, + * "messagesPerWindowDeviation": 5, * "windowsUntilRateChange": 100, * "windowSizeMs": 100 * } @@ -56,10 +52,8 @@ */ public class GaussianThroughputGenerator implements ThroughputGenerator { - private final int messagesPerSecondAverage; - private final int messagesPerSecondDeviation; private final int messagesPerWindowAverage; - private final int messagesPerWindowDeviation; + private final double messagesPerWindowDeviation; private final int windowsUntilRateChange; private final long windowSizeMs; @@ -71,35 +65,31 @@ public class GaussianThroughputGenerator implements ThroughputGenerator { private int throttleMessages = 0; @JsonCreator - public GaussianThroughputGenerator(@JsonProperty("messagesPerSecondAverage") int messagesPerSecondAverage, - @JsonProperty("messagesPerSecondDeviation") int messagesPerSecondDeviation, + public GaussianThroughputGenerator(@JsonProperty("messagesPerWindowAverage") int messagesPerWindowAverage, + @JsonProperty("messagesPerWindowDeviation") double messagesPerWindowDeviation, @JsonProperty("windowsUntilRateChange") int windowsUntilRateChange, @JsonProperty("windowSizeMs") long windowSizeMs) { - // Calcualte the default values. + // Calculate the default values. if (windowSizeMs <= 0) { windowSizeMs = 100; } this.windowSizeMs = windowSizeMs; - this.messagesPerSecondAverage = messagesPerSecondAverage; - this.messagesPerSecondDeviation = messagesPerSecondDeviation; + this.messagesPerWindowAverage = messagesPerWindowAverage; + this.messagesPerWindowDeviation = messagesPerWindowDeviation; this.windowsUntilRateChange = windowsUntilRateChange; - // Take per-second calculations and convert them to per-window calculations. - messagesPerWindowAverage = (int) (messagesPerSecondAverage * windowSizeMs / 1000); - messagesPerWindowDeviation = (int) (messagesPerSecondDeviation * windowSizeMs / 1000); - - // Calcualte the first window. + // Calculate the first window. calculateNextWindow(true); } @JsonProperty - public int messagesPerSecondAverage() { - return messagesPerSecondAverage; + public int messagesPerWindowAverage() { + return messagesPerWindowAverage; } @JsonProperty - public long messagesPerSecondDeviation() { - return messagesPerSecondDeviation; + public double messagesPerWindowDeviation() { + return messagesPerWindowDeviation; } @JsonProperty @@ -107,6 +97,11 @@ public long windowsUntilRateChange() { return windowsUntilRateChange; } + @JsonProperty + public long windowSizeMs() { + return windowSizeMs; + } + private synchronized void calculateNextWindow(boolean force) { // Reset the message count. messageTracker = 0; @@ -127,7 +122,7 @@ private synchronized void calculateNextWindow(boolean force) { // Calculate the number of messages allowed in this window using a normal distribution. // The formula is: Messages = Gaussian * Deviation + Average - throttleMessages = Math.max((int) (random.nextGaussian() * (double) messagesPerWindowDeviation) + messagesPerWindowAverage, 1); + throttleMessages = Math.max((int) (random.nextGaussian() * messagesPerWindowDeviation) + messagesPerWindowAverage, 1); } windowTracker += 1; } @@ -146,7 +141,9 @@ public synchronized void throttle() throws InterruptedException { if (messageTracker >= throttleMessages) { // Wait the difference in time between now and when the next window starts. - wait(nextWindowStarts - Time.SYSTEM.milliseconds()); + while (nextWindowStarts > Time.SYSTEM.milliseconds()) { + wait(nextWindowStarts - Time.SYSTEM.milliseconds()); + } } } } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java new file mode 100644 index 0000000000000..8660ed329a293 --- /dev/null +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampConstantPayloadGenerator.java @@ -0,0 +1,125 @@ +/* + * 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.trogdor.workload; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.kafka.common.utils.Time; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Random; + +/** + * This class behaves identically to TimestampConstantPayloadGenerator, except the message size follows a gaussian + * distribution. + * + * This should be used in conjunction with TimestampRecordProcessor in the Consumer to measure true end-to-end latency + * of a system. + * + * `messageSizeAverage` - The average size in bytes of each message. + * `messageSizeDeviation` - The standard deviation to use when calculating message size. + * `messagesUntilSizeChange` - The number of messages to keep at the same size. + * + * Here is an example spec: + * + * { + * "type": "gaussianTimestampConstant", + * "messageSizeAverage": 512, + * "messageSizeDeviation": 100, + * "messagesUntilSizeChange": 100 + * } + * + * This will generate messages on a gaussian distribution with an average size each 512-bytes. The message sizes will + * have a standard deviation of 100 bytes, and the size will only change every 100 messages. The distribution of + * messages will be as follows: + * + * The average size of the messages are 512 bytes. + * ~68% of the messages are between 412 and 612 bytes + * ~95% of the messages are between 312 and 712 bytes + * ~99% of the messages are between 212 and 812 bytes + */ + +public class GaussianTimestampConstantPayloadGenerator implements PayloadGenerator { + private final int messageSizeAverage; + private final double messageSizeDeviation; + private final int messagesUntilSizeChange; + private final long seed; + + private final Random random = new Random(); + private final ByteBuffer buffer; + + private int messageTracker = 0; + private int messageSize = 0; + + @JsonCreator + public GaussianTimestampConstantPayloadGenerator(@JsonProperty("messageSizeAverage") int messageSizeAverage, + @JsonProperty("messageSizeDeviation") double messageSizeDeviation, + @JsonProperty("messagesUntilSizeChange") int messagesUntilSizeChange, + @JsonProperty("seed") long seed) { + this.messageSizeAverage = messageSizeAverage; + this.messageSizeDeviation = messageSizeDeviation; + this.seed = seed; + this.messagesUntilSizeChange = messagesUntilSizeChange; + buffer = ByteBuffer.allocate(Long.BYTES); + buffer.order(ByteOrder.LITTLE_ENDIAN); + } + + @JsonProperty + public int messageSizeAverage() { + return messageSizeAverage; + } + + @JsonProperty + public double messageSizeDeviation() { + return messageSizeDeviation; + } + + @JsonProperty + public int messagesUntilSizeChange() { + return messagesUntilSizeChange; + } + + @JsonProperty + public long seed() { + return seed; + } + + @Override + public synchronized byte[] generate(long position) { + // Make the random number generator deterministic for unit tests. + random.setSeed(seed + position); + + // Calculate the next message size based on a gaussian distribution. + if ((messageSize == 0) || (messageTracker >= messagesUntilSizeChange)) { + messageTracker = 0; + messageSize = Math.max((int) (random.nextGaussian() * messageSizeDeviation) + messageSizeAverage, Long.BYTES); + } + messageTracker += 1; + + // Generate the byte array before the timestamp generation. + byte[] result = new byte[messageSize]; + + // Do the timestamp generation as the very last task. + buffer.clear(); + buffer.putLong(Time.SYSTEM.milliseconds()); + buffer.rewind(); + System.arraycopy(buffer.array(), 0, result, 0, Long.BYTES); + return result; + } +} diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java index 90fe279d4389c..48261a454f32a 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/GaussianTimestampRandomPayloadGenerator.java @@ -58,7 +58,7 @@ public class GaussianTimestampRandomPayloadGenerator implements PayloadGenerator { private final int messageSizeAverage; - private final int messageSizeDeviation; + private final double messageSizeDeviation; private final int messagesUntilSizeChange; private final long seed; @@ -70,7 +70,7 @@ public class GaussianTimestampRandomPayloadGenerator implements PayloadGenerator @JsonCreator public GaussianTimestampRandomPayloadGenerator(@JsonProperty("messageSizeAverage") int messageSizeAverage, - @JsonProperty("messageSizeDeviation") int messageSizeDeviation, + @JsonProperty("messageSizeDeviation") double messageSizeDeviation, @JsonProperty("messagesUntilSizeChange") int messagesUntilSizeChange, @JsonProperty("seed") long seed) { this.messageSizeAverage = messageSizeAverage; @@ -87,10 +87,15 @@ public int messageSizeAverage() { } @JsonProperty - public long messageSizeDeviation() { + public double messageSizeDeviation() { return messageSizeDeviation; } + @JsonProperty + public int messagesUntilSizeChange() { + return messagesUntilSizeChange; + } + @JsonProperty public long seed() { return seed; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java index 225a66379955c..6d7393d0812b2 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PayloadGenerator.java @@ -37,7 +37,9 @@ @JsonSubTypes.Type(value = NullPayloadGenerator.class, name = "null"), @JsonSubTypes.Type(value = RandomComponentPayloadGenerator.class, name = "randomComponent"), @JsonSubTypes.Type(value = TimestampRandomPayloadGenerator.class, name = "timestampRandom"), - @JsonSubTypes.Type(value = GaussianTimestampRandomPayloadGenerator.class, name = "gaussianTimestampRandom") + @JsonSubTypes.Type(value = TimestampConstantPayloadGenerator.class, name = "timestampConstant"), + @JsonSubTypes.Type(value = GaussianTimestampRandomPayloadGenerator.class, name = "gaussianTimestampRandom"), + @JsonSubTypes.Type(value = GaussianTimestampConstantPayloadGenerator.class, name = "gaussianTimestampConstant") }) public interface PayloadGenerator { /** diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java new file mode 100644 index 0000000000000..e9c4bc8a54068 --- /dev/null +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampConstantPayloadGenerator.java @@ -0,0 +1,78 @@ +/* + * 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.trogdor.workload; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.kafka.common.utils.Time; + +import java.nio.ByteOrder; +import java.nio.ByteBuffer; + +/** + * A PayloadGenerator which generates a timestamped constant payload. + * + * The timestamp used for this class is in milliseconds since epoch, encoded directly to the first several bytes of the + * payload. + * + * This should be used in conjunction with TimestampRecordProcessor in the Consumer to measure true end-to-end latency + * of a system. + * + * `size` - The size in bytes of each message. + * + * Here is an example spec: + * + * { + * "type": "timestampConstant", + * "size": 512 + * } + * + * This will generate a 512-byte message with the first several bytes encoded with the timestamp. + */ +public class TimestampConstantPayloadGenerator implements PayloadGenerator { + private final int size; + private final ByteBuffer buffer; + + @JsonCreator + public TimestampConstantPayloadGenerator(@JsonProperty("size") int size) { + this.size = size; + if (size < Long.BYTES) { + throw new RuntimeException("The size of the payload must be greater than or equal to " + Long.BYTES + "."); + } + buffer = ByteBuffer.allocate(Long.BYTES); + buffer.order(ByteOrder.LITTLE_ENDIAN); + } + + @JsonProperty + public int size() { + return size; + } + + @Override + public synchronized byte[] generate(long position) { + // Generate the byte array before the timestamp generation. + byte[] result = new byte[size]; + + // Do the timestamp generation as the very last task. + buffer.clear(); + buffer.putLong(Time.SYSTEM.milliseconds()); + buffer.rewind(); + System.arraycopy(buffer.array(), 0, result, 0, Long.BYTES); + return result; + } +} diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java index 658016a454da2..035d45928398a 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java @@ -38,7 +38,7 @@ * * Example spec: * { - * "type": "timestampRandom", + * "type": "timestamp", * "histogramMaxMs": 10000, * "histogramMinMs": 0, * "histogramStepMs": 1 From f109240236a991f2051cea1e319f97e1fdcc3206 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 7 May 2021 18:24:33 +0800 Subject: [PATCH 133/155] MINOR: remove storage/src/generated from tracked files (#10637) Reviewers: Kowshik Prakasam , Jun Rao , Satish Duggana --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 514b580a57cc1..613e10aba6242 100644 --- a/.gitignore +++ b/.gitignore @@ -60,3 +60,4 @@ raft/src/generated raft/.jqwik-database core/src/generated metadata/src/generated +storage/src/generated From 91b3be44a246a49084bce0c957127aa759c616cc Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Fri, 7 May 2021 21:16:58 +0800 Subject: [PATCH 134/155] MINOR: replace deprecated Class.newInstance() to new one (#10610) * replace deprecated Class.newInstance() to class.getDeclaredConstructor().newInstance() * throw ReflectiveOperationException to cover all other exceptions Reviewers: Tom Bentley --- .../runtime/isolation/DelegatingClassLoader.java | 15 ++++++++------- .../main/scala/kafka/tools/ConsoleConsumer.scala | 2 +- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java index c894dd6232fbe..43ceba34203f8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java @@ -236,13 +236,13 @@ private void initPluginLoader(String path) { log.error("Invalid path in plugin path: {}. Ignoring.", path, e); } catch (IOException e) { log.error("Could not get listing for plugin path: {}. Ignoring.", path, e); - } catch (InstantiationException | IllegalAccessException e) { + } catch (ReflectiveOperationException e) { log.error("Could not instantiate plugins in: {}. Ignoring: {}", path, e); } } private void registerPlugin(Path pluginLocation) - throws InstantiationException, IllegalAccessException, IOException { + throws IOException, ReflectiveOperationException { log.info("Loading plugin from: {}", pluginLocation); List pluginUrls = new ArrayList<>(); for (Path path : PluginUtils.pluginUrls(pluginLocation)) { @@ -264,7 +264,7 @@ private void scanUrlsAndAddPlugins( ClassLoader loader, URL[] urls, Path pluginLocation - ) throws InstantiationException, IllegalAccessException { + ) throws ReflectiveOperationException { PluginScanResult plugins = scanPluginPath(loader, urls); log.info("Registered loader: {}", loader); if (!plugins.isEmpty()) { @@ -322,7 +322,7 @@ private void loadJdbcDrivers(final ClassLoader loader) { private PluginScanResult scanPluginPath( ClassLoader loader, URL[] urls - ) throws InstantiationException, IllegalAccessException { + ) throws ReflectiveOperationException { ConfigurationBuilder builder = new ConfigurationBuilder(); builder.setClassLoaders(new ClassLoader[]{loader}); builder.addUrls(urls); @@ -346,7 +346,7 @@ private Collection> getPluginDesc( Reflections reflections, Class klass, ClassLoader loader - ) throws InstantiationException, IllegalAccessException { + ) throws ReflectiveOperationException { Set> plugins; try { plugins = reflections.getSubTypesOf(klass); @@ -387,9 +387,10 @@ private static String versionFor(T pluginImpl) { return pluginImpl instanceof Versioned ? ((Versioned) pluginImpl).version() : UNDEFINED_VERSION; } - private static String versionFor(Class pluginKlass) throws IllegalAccessException, InstantiationException { + private static String versionFor(Class pluginKlass) throws ReflectiveOperationException { // Temporary workaround until all the plugins are versioned. - return Connector.class.isAssignableFrom(pluginKlass) ? versionFor(pluginKlass.newInstance()) : UNDEFINED_VERSION; + return Connector.class.isAssignableFrom(pluginKlass) ? + versionFor(pluginKlass.getDeclaredConstructor().newInstance()) : UNDEFINED_VERSION; } @Override diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index a40c0416cabf8..5e450c2c41183 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -578,7 +578,7 @@ class DefaultMessageFormatter extends MessageFormatter { } private def getDeserializerProperty(isKey: Boolean)(configs: Map[String, _], propertyName: String): Deserializer[_] = { - val deserializer = Class.forName(configs.get(propertyName).asInstanceOf[String]).newInstance().asInstanceOf[Deserializer[_]] + val deserializer = Class.forName(configs.get(propertyName).asInstanceOf[String]).getDeclaredConstructor().newInstance().asInstanceOf[Deserializer[_]] val deserializerConfig = propertiesWithKeyPrefixStripped(propertyName + ".", configs) .asScala .asJava From 8f8f914efc2dfb2a6638553038dcb17393d7de96 Mon Sep 17 00:00:00 2001 From: Jorge Esteban Quilcate Otoya Date: Fri, 7 May 2021 19:24:41 +0100 Subject: [PATCH 135/155] KAFKA-12536: Add Instant-based methods to ReadOnlySessionStore (#10390) Implements: KIP-666 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-666%3A+Add+Instant-based+methods+to+ReadOnlySessionStore) Reviewers: John Roesler --- .../internals/AbstractReadOnlyDecorator.java | 10 +- .../internals/AbstractReadWriteDecorator.java | 12 +- .../streams/state/ReadOnlySessionStore.java | 250 ++++++++++++++---- .../kafka/streams/state/SessionStore.java | 77 +++++- .../state/internals/CachingSessionStore.java | 29 +- .../ChangeLoggingSessionBytesStore.java | 12 +- .../CompositeReadOnlySessionStore.java | 22 +- .../state/internals/InMemorySessionStore.java | 27 +- .../state/internals/MeteredSessionStore.java | 28 +- .../state/internals/RocksDBSessionStore.java | 18 +- .../kafka/test/ReadOnlySessionStoreStub.java | 15 +- 11 files changed, 359 insertions(+), 141 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java index 6c0c97599df65..be47e89c765c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadOnlyDecorator.java @@ -273,8 +273,8 @@ public void put(final Windowed sessionKey, } @Override - public AGG fetchSession(final K key, final long startTime, final long endTime) { - return wrapped().fetchSession(key, startTime, endTime); + public AGG fetchSession(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { + return wrapped().fetchSession(key, earliestSessionEndTime, latestSessionStartTime); } @Override @@ -283,9 +283,9 @@ public KeyValueIterator, AGG> fetch(final K key) { } @Override - public KeyValueIterator, AGG> fetch(final K from, - final K to) { - return wrapped().fetch(from, to); + public KeyValueIterator, AGG> fetch(final K keyFrom, + final K keyTo) { + return wrapped().fetch(keyFrom, keyTo); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java index eac606cdbe302..60fa8f436fbb6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractReadWriteDecorator.java @@ -265,9 +265,9 @@ public void put(final Windowed sessionKey, @Override public AGG fetchSession(final K key, - final long startTime, - final long endTime) { - return wrapped().fetchSession(key, startTime, endTime); + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return wrapped().fetchSession(key, earliestSessionEndTime, latestSessionStartTime); } @Override @@ -276,9 +276,9 @@ public KeyValueIterator, AGG> fetch(final K key) { } @Override - public KeyValueIterator, AGG> fetch(final K from, - final K to) { - return wrapped().fetch(from, to); + public KeyValueIterator, AGG> fetch(final K keyFrom, + final K keyTo) { + return wrapped().fetch(keyFrom, keyTo); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index 8874908d18072..0ade24286fdd6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -19,162 +19,298 @@ import org.apache.kafka.streams.kstream.Windowed; +import java.time.Instant; + /** - * A session store that only supports read operations. - * Implementations should be thread-safe as concurrent reads and writes - * are expected. + * A session store that only supports read operations. Implementations should be thread-safe as + * concurrent reads and writes are expected. * - * @param the key type + * @param the key type * @param the aggregated value type */ public interface ReadOnlySessionStore { /** - * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime iterating from earliest to latest. + * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime + * and the sessions start is ≤ latestSessionStartTime iterating from earliest to latest. *

      * This iterator must be closed after use. * * @param key the key to return sessions for - * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where iteration starts. - * @param latestSessionStartTime the end timestamp of the latest session to search for, where iteration ends. - * @return iterator of sessions with the matching key and aggregated values, from earliest to latest session time. + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration starts. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration ends. + * @return iterator of sessions with the matching key and aggregated values, from earliest to + * latest session time. * @throws NullPointerException If null is used for key. */ default KeyValueIterator, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException("This API is not supported by this implementation of ReadOnlySessionStore."); + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); } /** - * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime iterating from latest to earliest. + * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime + * and the sessions start is ≤ latestSessionStartTime iterating from earliest to latest. *

      * This iterator must be closed after use. * * @param key the key to return sessions for - * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where iteration ends. - * @param latestSessionStartTime the end timestamp of the latest session to search for, where iteration starts. - * @return backward iterator of sessions with the matching key and aggregated values, from latest to earliest session time. + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration starts. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration ends. + * @return iterator of sessions with the matching key and aggregated values, from earliest to + * latest session time. + * @throws NullPointerException If null is used for key. + */ + default KeyValueIterator, AGG> findSessions(final K key, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); + } + + /** + * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime + * and the sessions start is ≤ latestSessionStartTime iterating from latest to earliest. + *

      + * This iterator must be closed after use. + * + * @param key the key to return sessions for + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration ends. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration starts. + * @return backward iterator of sessions with the matching key and aggregated values, from + * latest to earliest session time. * @throws NullPointerException If null is used for key. */ default KeyValueIterator, AGG> backwardFindSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException("This API is not supported by this implementation of ReadOnlySessionStore."); + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); + } + + /** + * Fetch any sessions with the matching key and the sessions end is ≥ earliestSessionEndTime + * and the sessions start is ≤ latestSessionStartTime iterating from latest to earliest. + *

      + * This iterator must be closed after use. + * + * @param key the key to return sessions for + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration ends. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration starts. + * @return backward iterator of sessions with the matching key and aggregated values, from + * latest to earliest session time. + * @throws NullPointerException If null is used for key. + */ + default KeyValueIterator, AGG> backwardFindSessions(final K key, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); } /** - * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime iterating from earliest to latest. + * Fetch any sessions in the given range of keys and the sessions end is ≥ + * earliestSessionEndTime and the sessions start is ≤ latestSessionStartTime iterating from + * earliest to latest. *

      * This iterator must be closed after use. * * @param keyFrom The first key that could be in the range * @param keyTo The last key that could be in the range - * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where iteration starts. - * @param latestSessionStartTime the end timestamp of the latest session to search for, where iteration ends. - * @return iterator of sessions with the matching keys and aggregated values, from earliest to latest session time. + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration starts. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration ends. + * @return iterator of sessions with the matching keys and aggregated values, from earliest to + * latest session time. * @throws NullPointerException If null is used for any key. */ default KeyValueIterator, AGG> findSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException("This API is not supported by this implementation of ReadOnlySessionStore."); + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); } + /** + * Fetch any sessions in the given range of keys and the sessions end is ≥ + * earliestSessionEndTime and the sessions start is ≤ latestSessionStartTime iterating from + * earliest to latest. + *

      + * This iterator must be closed after use. + * + * @param keyFrom The first key that could be in the range + * @param keyTo The last key that could be in the range + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration starts. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration ends. + * @return iterator of sessions with the matching keys and aggregated values, from earliest to + * latest session time. + * @throws NullPointerException If null is used for any key. + */ + default KeyValueIterator, AGG> findSessions(final K keyFrom, + final K keyTo, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); + } /** - * Fetch any sessions in the given range of keys and the sessions end is ≥ earliestSessionEndTime and the sessions - * start is ≤ latestSessionStartTime iterating from latest to earliest. + * Fetch any sessions in the given range of keys and the sessions end is ≥ + * earliestSessionEndTime and the sessions start is ≤ latestSessionStartTime iterating from + * latest to earliest. *

      * This iterator must be closed after use. * * @param keyFrom The first key that could be in the range * @param keyTo The last key that could be in the range - * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where iteration ends. - * @param latestSessionStartTime the end timestamp of the latest session to search for, where iteration starts. - * @return backward iterator of sessions with the matching keys and aggregated values, from latest to earliest session time. + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration ends. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration starts. + * @return backward iterator of sessions with the matching keys and aggregated values, from + * latest to earliest session time. * @throws NullPointerException If null is used for any key. */ default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, final K keyTo, final long earliestSessionEndTime, final long latestSessionStartTime) { - throw new UnsupportedOperationException("This API is not supported by this implementation of ReadOnlySessionStore."); + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); + } + + /** + * Fetch any sessions in the given range of keys and the sessions end is ≥ + * earliestSessionEndTime and the sessions start is ≤ latestSessionStartTime iterating from + * latest to earliest. + *

      + * This iterator must be closed after use. + * + * @param keyFrom The first key that could be in the range + * @param keyTo The last key that could be in the range + * @param earliestSessionEndTime the end timestamp of the earliest session to search for, where + * iteration ends. + * @param latestSessionStartTime the end timestamp of the latest session to search for, where + * iteration starts. + * @return backward iterator of sessions with the matching keys and aggregated values, from + * latest to earliest session time. + * @throws NullPointerException If null is used for any key. + */ + default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, + final K keyTo, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); } /** * Get the value of key from a single session. * - * @param key the key to fetch - * @param startTime start timestamp of the session - * @param endTime end timestamp of the session + * @param key the key to fetch + * @param earliestSessionEndTime start timestamp of the session + * @param latestSessionStartTime end timestamp of the session * @return The value or {@code null} if no session associated with the key can be found * @throws NullPointerException If {@code null} is used for any key. */ - default AGG fetchSession(final K key, final long startTime, final long endTime) { - throw new UnsupportedOperationException("This API is not supported by this implementation of ReadOnlySessionStore."); + default AGG fetchSession(final K key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); } /** - * Retrieve all aggregated sessions for the provided key. - * This iterator must be closed after use. + * Get the value of key from a single session. + * + * @param key the key to fetch + * @param earliestSessionEndTime start timestamp of the session + * @param latestSessionStartTime end timestamp of the session + * @return The value or {@code null} if no session associated with the key can be found + * @throws NullPointerException If {@code null} is used for any key. + */ + default AGG fetchSession(final K key, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); + } + + /** + * Retrieve all aggregated sessions for the provided key. This iterator must be closed after + * use. *

      * For each key, the iterator guarantees ordering of sessions, starting from the oldest/earliest * available session to the newest/latest session. * - * @param key record key to find aggregated session values for - * @return KeyValueIterator containing all sessions for the provided key, from oldest to newest session. - * @throws NullPointerException If null is used for key. - * + * @param key record key to find aggregated session values for + * @return KeyValueIterator containing all sessions for the provided key, from oldest to newest + * session. + * @throws NullPointerException If null is used for key. */ KeyValueIterator, AGG> fetch(final K key); /** - * Retrieve all aggregated sessions for the provided key. - * This iterator must be closed after use. + * Retrieve all aggregated sessions for the provided key. This iterator must be closed after + * use. *

      * For each key, the iterator guarantees ordering of sessions, starting from the newest/latest * available session to the oldest/earliest session. * * @param key record key to find aggregated session values for - * @return backward KeyValueIterator containing all sessions for the provided key, from newest to oldest session. + * @return backward KeyValueIterator containing all sessions for the provided key, from newest + * to oldest session. * @throws NullPointerException If null is used for key. */ default KeyValueIterator, AGG> backwardFetch(final K key) { - throw new UnsupportedOperationException("This API is not supported by this implementation of ReadOnlySessionStore."); + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); } /** - * Retrieve all aggregated sessions for the given range of keys. - * This iterator must be closed after use. + * Retrieve all aggregated sessions for the given range of keys. This iterator must be closed + * after use. *

      * For each key, the iterator guarantees ordering of sessions, starting from the oldest/earliest * available session to the newest/latest session. * - * @param from first key in the range to find aggregated session values for - * @param to last key in the range to find aggregated session values for - * @return KeyValueIterator containing all sessions for the provided key, from oldest to newest session. - * @throws NullPointerException If null is used for any of the keys. + * @param keyFrom first key in the range to find aggregated session values for + * @param keyTo last key in the range to find aggregated session values for + * @return KeyValueIterator containing all sessions for the provided key, from oldest to newest + * session. + * @throws NullPointerException If null is used for any of the keys. */ - KeyValueIterator, AGG> fetch(final K from, final K to); + KeyValueIterator, AGG> fetch(final K keyFrom, final K keyTo); /** - * Retrieve all aggregated sessions for the given range of keys. - * This iterator must be closed after use. + * Retrieve all aggregated sessions for the given range of keys. This iterator must be closed + * after use. *

      * For each key, the iterator guarantees ordering of sessions, starting from the newest/latest * available session to the oldest/earliest session. * - * @param from first key in the range to find aggregated session values for - * @param to last key in the range to find aggregated session values for - * @return backward KeyValueIterator containing all sessions for the provided key, from newest to oldest session. + * @param keyFrom first key in the range to find aggregated session values for + * @param keyTo last key in the range to find aggregated session values for + * @return backward KeyValueIterator containing all sessions for the provided key, from newest + * to oldest session. * @throws NullPointerException If null is used for any of the keys. */ - default KeyValueIterator, AGG> backwardFetch(final K from, final K to) { - throw new UnsupportedOperationException("This API is not supported by this implementation of ReadOnlySessionStore."); + default KeyValueIterator, AGG> backwardFetch(final K keyFrom, final K keyTo) { + throw new UnsupportedOperationException( + "This API is not supported by this implementation of ReadOnlySessionStore."); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java index 47f48d5e8143c..926cddc4d2a43 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/SessionStore.java @@ -16,26 +16,90 @@ */ package org.apache.kafka.streams.state; +import org.apache.kafka.streams.internals.ApiUtils; import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.StateStore; +import java.time.Instant; + +import static org.apache.kafka.streams.internals.ApiUtils.prepareMillisCheckFailMsgPrefix; + /** * Interface for storing the aggregated values of sessions. *

      - * The key is internally represented as {@link Windowed Windowed<K>} that comprises the plain key - * and the {@link Window} that represents window start- and end-timestamp. + * The key is internally represented as {@link Windowed Windowed<K>} that comprises the plain + * key and the {@link Window} that represents window start- and end-timestamp. *

      - * If two sessions are merged, a new session with new start- and end-timestamp must be inserted into the store - * while the two old sessions must be deleted. + * If two sessions are merged, a new session with new start- and end-timestamp must be inserted into + * the store while the two old sessions must be deleted. * * @param type of the record keys * @param type of the aggregated values */ public interface SessionStore extends StateStore, ReadOnlySessionStore { + @Override + default KeyValueIterator, AGG> findSessions(final K key, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + return findSessions( + key, + ApiUtils.validateMillisecondInstant(earliestSessionEndTime, + prepareMillisCheckFailMsgPrefix(earliestSessionEndTime, "earliestSessionEndTime")), + ApiUtils.validateMillisecondInstant(latestSessionStartTime, + prepareMillisCheckFailMsgPrefix(latestSessionStartTime, "latestSessionStartTime"))); + } + + @Override + default KeyValueIterator, AGG> backwardFindSessions(final K key, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + return backwardFindSessions( + key, + ApiUtils.validateMillisecondInstant(earliestSessionEndTime, + prepareMillisCheckFailMsgPrefix(earliestSessionEndTime, "earliestSessionEndTime")), + ApiUtils.validateMillisecondInstant(latestSessionStartTime, + prepareMillisCheckFailMsgPrefix(latestSessionStartTime, "latestSessionStartTime"))); + } + + default KeyValueIterator, AGG> findSessions(final K keyFrom, + final K keyTo, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + return findSessions( + keyFrom, + keyTo, + ApiUtils.validateMillisecondInstant(earliestSessionEndTime, + prepareMillisCheckFailMsgPrefix(earliestSessionEndTime, "earliestSessionEndTime")), + ApiUtils.validateMillisecondInstant(latestSessionStartTime, + prepareMillisCheckFailMsgPrefix(latestSessionStartTime, "latestSessionStartTime"))); + } + + default KeyValueIterator, AGG> backwardFindSessions(final K keyFrom, + final K keyTo, + final Instant earliestSessionEndTime, + final Instant latestSessionStartTime) { + return backwardFindSessions( + keyFrom, + keyTo, + ApiUtils.validateMillisecondInstant(earliestSessionEndTime, + prepareMillisCheckFailMsgPrefix(earliestSessionEndTime, "earliestSessionEndTime")), + ApiUtils.validateMillisecondInstant(latestSessionStartTime, + prepareMillisCheckFailMsgPrefix(latestSessionStartTime, "latestSessionStartTime"))); + } + + default AGG fetchSession(final K key, final Instant earliestSessionEndTime, final Instant latestSessionStartTime) { + return fetchSession(key, + ApiUtils.validateMillisecondInstant(earliestSessionEndTime, + prepareMillisCheckFailMsgPrefix(earliestSessionEndTime, "startTime")), + ApiUtils.validateMillisecondInstant(latestSessionStartTime, + prepareMillisCheckFailMsgPrefix(latestSessionStartTime, "endTime"))); + } + /** * Remove the session aggregated with provided {@link Windowed} key from the store + * * @param sessionKey key of the session to remove * @throws NullPointerException If null is used for sessionKey. */ @@ -43,9 +107,10 @@ public interface SessionStore extends StateStore, ReadOnlySessionStore sessionKey, final AGG aggregate); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index d0fe25a6050ca..1cfb8ce498a64 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -271,17 +271,18 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte } @Override - public byte[] fetchSession(final Bytes key, final long startTime, final long endTime) { + public byte[] fetchSession(final Bytes key, final long earliestSessionEndTime, final long latestSessionStartTime) { Objects.requireNonNull(key, "key cannot be null"); validateStoreOpen(); if (context.cache() == null) { - return wrapped().fetchSession(key, startTime, endTime); + return wrapped().fetchSession(key, earliestSessionEndTime, latestSessionStartTime); } else { - final Bytes bytesKey = SessionKeySchema.toBinary(key, startTime, endTime); + final Bytes bytesKey = SessionKeySchema.toBinary(key, earliestSessionEndTime, + latestSessionStartTime); final Bytes cacheKey = cacheFunction.cacheKey(bytesKey); final LRUCacheEntry entry = context.cache().get(cacheName, cacheKey); if (entry == null) { - return wrapped().fetchSession(key, startTime, endTime); + return wrapped().fetchSession(key, earliestSessionEndTime, latestSessionStartTime); } else { return entry.value(); } @@ -301,19 +302,19 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes key) } @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, - final Bytes to) { - Objects.requireNonNull(from, "from cannot be null"); - Objects.requireNonNull(to, "to cannot be null"); - return findSessions(from, to, 0, Long.MAX_VALUE); + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, + final Bytes keyTo) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); + return findSessions(keyFrom, keyTo, 0, Long.MAX_VALUE); } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, - final Bytes to) { - Objects.requireNonNull(from, "from cannot be null"); - Objects.requireNonNull(to, "to cannot be null"); - return backwardFindSessions(from, to, 0, Long.MAX_VALUE); + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, + final Bytes keyTo) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); + return backwardFindSessions(keyFrom, keyTo, 0, Long.MAX_VALUE); } public void flush() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java index f70eabd19f2f5..baa9846023a39 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSessionBytesStore.java @@ -91,8 +91,8 @@ public void put(final Windowed sessionKey, final byte[] aggregate) { } @Override - public byte[] fetchSession(final Bytes key, final long startTime, final long endTime) { - return wrapped().fetchSession(key, startTime, endTime); + public byte[] fetchSession(final Bytes key, final long earliestSessionEndTime, final long latestSessionStartTime) { + return wrapped().fetchSession(key, earliestSessionEndTime, latestSessionStartTime); } @Override @@ -106,12 +106,12 @@ public KeyValueIterator, byte[]> fetch(final Bytes key) { } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to) { - return wrapped().backwardFetch(from, to); + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, final Bytes keyTo) { + return wrapped().backwardFetch(keyFrom, keyTo); } @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { - return wrapped().fetch(from, to); + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo) { + return wrapped().fetch(keyFrom, keyTo); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java index 72233122a6286..fb5fb61ae6108 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStore.java @@ -155,12 +155,12 @@ public KeyValueIterator, V> backwardFindSessions(final K keyFrom, } @Override - public V fetchSession(final K key, final long startTime, final long endTime) { + public V fetchSession(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { Objects.requireNonNull(key, "key can't be null"); final List> stores = storeProvider.stores(storeName, queryableStoreType); for (final ReadOnlySessionStore store : stores) { try { - return store.fetchSession(key, startTime, endTime); + return store.fetchSession(key, earliestSessionEndTime, latestSessionStartTime); } catch (final InvalidStateStoreException ise) { throw new InvalidStateStoreException( "State store [" + storeName + "] is not available anymore" + @@ -220,10 +220,11 @@ public KeyValueIterator, V> backwardFetch(final K key) { } @Override - public KeyValueIterator, V> fetch(final K from, final K to) { - Objects.requireNonNull(from, "from can't be null"); - Objects.requireNonNull(to, "to can't be null"); - final NextIteratorFunction, V, ReadOnlySessionStore> nextIteratorFunction = store -> store.fetch(from, to); + public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo) { + Objects.requireNonNull(keyFrom, "keyFrom can't be null"); + Objects.requireNonNull(keyTo, "keyTo can't be null"); + final NextIteratorFunction, V, ReadOnlySessionStore> nextIteratorFunction = + store -> store.fetch(keyFrom, keyTo); return new DelegatingPeekingKeyValueIterator<>(storeName, new CompositeKeyValueIterator<>( storeProvider.stores(storeName, queryableStoreType).iterator(), @@ -231,10 +232,11 @@ public KeyValueIterator, V> fetch(final K from, final K to) { } @Override - public KeyValueIterator, V> backwardFetch(final K from, final K to) { - Objects.requireNonNull(from, "from can't be null"); - Objects.requireNonNull(to, "to can't be null"); - final NextIteratorFunction, V, ReadOnlySessionStore> nextIteratorFunction = store -> store.backwardFetch(from, to); + public KeyValueIterator, V> backwardFetch(final K keyFrom, final K keyTo) { + Objects.requireNonNull(keyFrom, "keyFrom can't be null"); + Objects.requireNonNull(keyTo, "keyTo can't be null"); + final NextIteratorFunction, V, ReadOnlySessionStore> nextIteratorFunction = + store -> store.backwardFetch(keyFrom, keyTo); return new DelegatingPeekingKeyValueIterator<>( storeName, new CompositeKeyValueIterator<>( diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java index b35eaa2204286..affd47ac14292 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemorySessionStore.java @@ -150,18 +150,20 @@ public void remove(final Windowed sessionKey) { } @Override - public byte[] fetchSession(final Bytes key, final long startTime, final long endTime) { + public byte[] fetchSession(final Bytes key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { removeExpiredSegments(); Objects.requireNonNull(key, "key cannot be null"); // Only need to search if the record hasn't expired yet - if (endTime > observedStreamTime - retentionPeriod) { - final ConcurrentNavigableMap> keyMap = endTimeMap.get(endTime); + if (latestSessionStartTime > observedStreamTime - retentionPeriod) { + final ConcurrentNavigableMap> keyMap = endTimeMap.get(latestSessionStartTime); if (keyMap != null) { final ConcurrentNavigableMap startTimeMap = keyMap.get(key); if (startTimeMap != null) { - return startTimeMap.get(startTime); + return startTimeMap.get(earliestSessionEndTime); } } } @@ -273,25 +275,26 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes key) } @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo) { - Objects.requireNonNull(from, "from key cannot be null"); - Objects.requireNonNull(to, "to key cannot be null"); + Objects.requireNonNull(keyFrom, "from key cannot be null"); + Objects.requireNonNull(keyTo, "to key cannot be null"); removeExpiredSegments(); - return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.entrySet().iterator(), false); + return registerNewIterator(keyFrom, keyTo, Long.MAX_VALUE, endTimeMap.entrySet().iterator(), false); } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to) { - Objects.requireNonNull(from, "from key cannot be null"); - Objects.requireNonNull(to, "to key cannot be null"); + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, final Bytes keyTo) { + Objects.requireNonNull(keyFrom, "from key cannot be null"); + Objects.requireNonNull(keyTo, "to key cannot be null"); removeExpiredSegments(); - return registerNewIterator(from, to, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), true); + return registerNewIterator( + keyFrom, keyTo, Long.MAX_VALUE, endTimeMap.descendingMap().entrySet().iterator(), true); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java index 7a31b170ea4e6..1fbc8db095799 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSessionStore.java @@ -200,12 +200,16 @@ public void remove(final Windowed sessionKey) { } @Override - public V fetchSession(final K key, final long startTime, final long endTime) { + public V fetchSession(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { Objects.requireNonNull(key, "key cannot be null"); return maybeMeasureLatency( () -> { final Bytes bytesKey = keyBytes(key); - final byte[] result = wrapped().fetchSession(bytesKey, startTime, endTime); + final byte[] result = wrapped().fetchSession( + bytesKey, + earliestSessionEndTime, + latestSessionStartTime + ); if (result == null) { return null; } @@ -240,12 +244,12 @@ public KeyValueIterator, V> backwardFetch(final K key) { } @Override - public KeyValueIterator, V> fetch(final K from, - final K to) { - Objects.requireNonNull(from, "from cannot be null"); - Objects.requireNonNull(to, "to cannot be null"); + public KeyValueIterator, V> fetch(final K keyFrom, + final K keyTo) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); return new MeteredWindowedKeyValueIterator<>( - wrapped().fetch(keyBytes(from), keyBytes(to)), + wrapped().fetch(keyBytes(keyFrom), keyBytes(keyTo)), fetchSensor, streamsMetrics, serdes, @@ -253,12 +257,12 @@ public KeyValueIterator, V> fetch(final K from, } @Override - public KeyValueIterator, V> backwardFetch(final K from, - final K to) { - Objects.requireNonNull(from, "from cannot be null"); - Objects.requireNonNull(to, "to cannot be null"); + public KeyValueIterator, V> backwardFetch(final K keyFrom, + final K keyTo) { + Objects.requireNonNull(keyFrom, "keyFrom cannot be null"); + Objects.requireNonNull(keyTo, "keyTo cannot be null"); return new MeteredWindowedKeyValueIterator<>( - wrapped().backwardFetch(keyBytes(from), keyBytes(to)), + wrapped().backwardFetch(keyBytes(keyFrom), keyBytes(keyTo)), fetchSensor, streamsMetrics, serdes, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java index 338769abea4a5..f5d710894f767 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java @@ -83,8 +83,14 @@ public KeyValueIterator, byte[]> backwardFindSessions(final Byte } @Override - public byte[] fetchSession(final Bytes key, final long startTime, final long endTime) { - return wrapped().get(SessionKeySchema.toBinary(key, startTime, endTime)); + public byte[] fetchSession(final Bytes key, + final long earliestSessionEndTime, + final long latestSessionStartTime) { + return wrapped().get(SessionKeySchema.toBinary( + key, + earliestSessionEndTime, + latestSessionStartTime + )); } @Override @@ -98,13 +104,13 @@ public KeyValueIterator, byte[]> backwardFetch(final Bytes key) } @Override - public KeyValueIterator, byte[]> fetch(final Bytes from, final Bytes to) { - return findSessions(from, to, 0, Long.MAX_VALUE); + public KeyValueIterator, byte[]> fetch(final Bytes keyFrom, final Bytes keyTo) { + return findSessions(keyFrom, keyTo, 0, Long.MAX_VALUE); } @Override - public KeyValueIterator, byte[]> backwardFetch(final Bytes from, final Bytes to) { - return backwardFindSessions(from, to, 0, Long.MAX_VALUE); + public KeyValueIterator, byte[]> backwardFetch(final Bytes keyFrom, final Bytes keyTo) { + return backwardFindSessions(keyFrom, keyTo, 0, Long.MAX_VALUE); } @Override diff --git a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java index ff37e2586cd94..06640559cb4bb 100644 --- a/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/test/ReadOnlySessionStoreStub.java @@ -63,7 +63,7 @@ public KeyValueIterator, V> backwardFindSessions(K keyFrom, K keyTo, } @Override - public V fetchSession(K key, long startTime, long endTime) { + public V fetchSession(K key, long earliestSessionEndTime, long latestSessionStartTime) { throw new UnsupportedOperationException("Moved from Session Store. Implement if needed"); } @@ -90,14 +90,15 @@ public KeyValueIterator, V> backwardFetch(K key) { } @Override - public KeyValueIterator, V> fetch(final K from, final K to) { + public KeyValueIterator, V> fetch(final K keyFrom, final K keyTo) { if (!open) { throw new InvalidStateStoreException("not open"); } - if (sessions.subMap(from, true, to, true).isEmpty()) { + if (sessions.subMap(keyFrom, true, keyTo, true).isEmpty()) { return new KeyValueIteratorStub<>(Collections., V>>emptyIterator()); } - final Iterator, V>>> keysIterator = sessions.subMap(from, true, to, true).values().iterator(); + final Iterator, V>>> keysIterator = sessions.subMap(keyFrom, true, + keyTo, true).values().iterator(); return new KeyValueIteratorStub<>( new Iterator, V>>() { @@ -124,15 +125,15 @@ public KeyValue, V> next() { } @Override - public KeyValueIterator, V> backwardFetch(K from, K to) { + public KeyValueIterator, V> backwardFetch(K keyFrom, K keyTo) { if (!open) { throw new InvalidStateStoreException("not open"); } - if (sessions.subMap(from, true, to, true).isEmpty()) { + if (sessions.subMap(keyFrom, true, keyTo, true).isEmpty()) { return new KeyValueIteratorStub<>(Collections.emptyIterator()); } final Iterator, V>>> keysIterator = - sessions.subMap(from, true, to, true).descendingMap().values().iterator(); + sessions.subMap(keyFrom, true, keyTo, true).descendingMap().values().iterator(); return new KeyValueIteratorStub<>( new Iterator, V>>() { From f1ef21f70afafa2e9778b355eeffdec804f5c7d9 Mon Sep 17 00:00:00 2001 From: Luke Chen Date: Sun, 9 May 2021 09:11:40 +0800 Subject: [PATCH 136/155] KAFKA-12464: minor code cleanup and additional logging in constrained sticky assignment (#10645) This is the follow up PR to address the remaining comments in #10509. Reviewers: Anna Sophie Blee-Goldman --- .../internals/AbstractStickyAssignor.java | 53 ++++++++++++------- 1 file changed, 33 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java index 24c8107f25e3b..5798909927461 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractStickyAssignor.java @@ -178,17 +178,17 @@ private Map> constrainedAssign(Map int minQuota = (int) Math.floor(((double) totalPartitionsCount) / numberOfConsumers); int maxQuota = (int) Math.ceil(((double) totalPartitionsCount) / numberOfConsumers); - // the expected number of members with maxQuota assignment - int expectedNumMembersHavingMorePartitions = totalPartitionsCount % numberOfConsumers; - // the number of members with exactly maxQuota partitions assigned - int numMembersHavingMorePartitions = 0; + // the expected number of members with over minQuota assignment + int expectedNumMembersAssignedOverMinQuota = totalPartitionsCount % numberOfConsumers; + // the number of members with over minQuota partitions assigned + int numMembersAssignedOverMinQuota = 0; // initialize the assignment map with an empty array of size maxQuota for all members Map> assignment = new HashMap<>( consumerToOwnedPartitions.keySet().stream().collect(Collectors.toMap(c -> c, c -> new ArrayList<>(maxQuota)))); List assignedPartitions = new ArrayList<>(); - // Reassign as many previously owned partitions as possible + // Reassign previously owned partitions to the expected number for (Map.Entry> consumerEntry : consumerToOwnedPartitions.entrySet()) { String consumer = consumerEntry.getKey(); List ownedPartitions = consumerEntry.getValue(); @@ -203,10 +203,10 @@ private Map> constrainedAssign(Map assignedPartitions.addAll(ownedPartitions); } unfilledMembers.add(consumer); - } else if (ownedPartitions.size() >= maxQuota && numMembersHavingMorePartitions < expectedNumMembersHavingMorePartitions) { - // consumer owned the "maxQuota" of partitions or more, and we're still under the number of expected max capacity members - // so keep "maxQuota" of the owned partitions, and revoke the rest of the partitions - numMembersHavingMorePartitions++; + } else if (ownedPartitions.size() >= maxQuota && numMembersAssignedOverMinQuota < expectedNumMembersAssignedOverMinQuota) { + // consumer owned the "maxQuota" of partitions or more, and we're still under the number of expected members + // with more than the minQuota partitions, so keep "maxQuota" of the owned partitions, and revoke the rest of the partitions + numMembersAssignedOverMinQuota++; List maxQuotaPartitions = ownedPartitions.subList(0, maxQuota); consumerAssignment.addAll(maxQuotaPartitions); assignedPartitions.addAll(maxQuotaPartitions); @@ -218,8 +218,10 @@ private Map> constrainedAssign(Map consumerAssignment.addAll(minQuotaPartitions); assignedPartitions.addAll(minQuotaPartitions); allRevokedPartitions.addAll(ownedPartitions.subList(minQuota, ownedPartitions.size())); - // this consumer is potential maxQuota candidate since we're still under the number of expected max capacity members - if (numMembersHavingMorePartitions < expectedNumMembersHavingMorePartitions) { + // this consumer is potential maxQuota candidate since we're still under the number of expected members + // with more than the minQuota partitions. Note, if the number of expected members with more than + // the minQuota partitions is 0, it means minQuota == maxQuota, so they won't be put into unfilledMembers + if (numMembersAssignedOverMinQuota < expectedNumMembersAssignedOverMinQuota) { unfilledMembers.add(consumer); } } @@ -242,6 +244,9 @@ private Map> constrainedAssign(Map if (unfilledMembers.isEmpty()) { // Should not enter here since we have calculated the exact number to assign to each consumer // There might be issues in the assigning algorithm, or maybe assigning the same partition to two owners. + int currentPartitionIndex = unassignedPartitions.indexOf(unassignedPartition); + log.error("No more unfilled consumers to be assigned. The remaining unassigned partitions are: {}", + unassignedPartitions.subList(currentPartitionIndex, unassignedPartitions.size())); throw new IllegalStateException("No more unfilled consumers to be assigned."); } unfilledConsumerIter = unfilledMembers.iterator(); @@ -255,27 +260,35 @@ private Map> constrainedAssign(Map partitionsTransferringOwnership.put(unassignedPartition, consumer); int currentAssignedCount = consumerAssignment.size(); - int expectedAssignedCount = numMembersHavingMorePartitions < expectedNumMembersHavingMorePartitions ? maxQuota : minQuota; + int expectedAssignedCount = numMembersAssignedOverMinQuota < expectedNumMembersAssignedOverMinQuota ? maxQuota : minQuota; if (currentAssignedCount == expectedAssignedCount) { if (currentAssignedCount == maxQuota) { - numMembersHavingMorePartitions++; + numMembersAssignedOverMinQuota++; } unfilledConsumerIter.remove(); } } if (!unfilledMembers.isEmpty()) { - // we expected all the remaining unfilled members have minQuota partitions and we're already at the allowed number - // of max capacity members. Otherwise, there must be error here. - if (numMembersHavingMorePartitions != expectedNumMembersHavingMorePartitions) { - throw new IllegalStateException(String.format("We haven't reached the allowed number of max capacity members, " + - "but no more partitions to be assigned to unfilled consumers: %s", unfilledMembers)); + // we expected all the remaining unfilled members have minQuota partitions and we're already at the expected number + // of members with more than the minQuota partitions. Otherwise, there must be error here. + if (numMembersAssignedOverMinQuota != expectedNumMembersAssignedOverMinQuota) { + log.error("Current number of members with more than the minQuota partitions: {}, is less than the expected number " + + "of members with more than the minQuota partitions: {}, and no more partitions to be assigned to the remaining unfilled consumers: {}", + numMembersAssignedOverMinQuota, expectedNumMembersAssignedOverMinQuota, unfilledMembers); + throw new IllegalStateException("We haven't reached the expected number of members with " + + "more than the minQuota partitions, but no more partitions to be assigned"); } else { for (String unfilledMember : unfilledMembers) { int assignedPartitionsCount = assignment.get(unfilledMember).size(); if (assignedPartitionsCount != minQuota) { - throw new IllegalStateException(String.format("Consumer: [%s] should have %d partitions, but got %d partitions, " + - "and no more partitions to be assigned", unfilledMember, minQuota, assignedPartitionsCount)); + log.error("Consumer: [{}] should have {} partitions, but got {} partitions, and no more partitions " + + "to be assigned. The remaining unfilled consumers are: {}", unfilledMember, minQuota, assignedPartitionsCount, unfilledMembers); + throw new IllegalStateException(String.format("Consumer: [%s] doesn't reach minQuota partitions, " + + "and no more partitions to be assigned", unfilledMember)); + } else { + log.trace("skip over this unfilled member: [{}] because we've reached the expected number of " + + "members with more than the minQuota partitions, and this member already have minQuota partitions", unfilledMember); } } } From d934647484ad61f0ca82f303738b28d9b7c54328 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dejan=20Stojadinovi=C4=87?= Date: Mon, 10 May 2021 04:03:14 +0200 Subject: [PATCH 137/155] MINOR: checkstyle version upgrade: 8.20 -> 8.36.2 (#10656) Details: * Release notes: https://checkstyle.org/releasenotes.html#Release_8.36.2 * Checkstyle version 8.42 should be skipped (lots of false positives, see here: https://github.com/checkstyle/checkstyle/issues/9957) * More recent Checkstyle versions (i.e. 8.37 and above) are imposing more strict indentation rules. --- gradle/dependencies.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index f4ca693c3362d..347a1b3334884 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -59,7 +59,7 @@ versions += [ apacheds: "2.0.0-M24", argparse4j: "0.7.0", bcpkix: "1.66", - checkstyle: "8.20", + checkstyle: "8.36.2", commonsCli: "1.4", gradle: "6.8.3", grgit: "4.1.0", From 9bec36256ef0fa151da97d0d970d381d4d66d6e1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dejan=20Stojadinovi=C4=87?= Date: Mon, 10 May 2021 04:22:29 +0200 Subject: [PATCH 138/155] MINOR: Remove unused `scalatest` definition from `dependencies.gradle` (#10655) Related PR where the `scalatest` usage was removed: #9858 Reviewers: Ismael Juma --- gradle/dependencies.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 347a1b3334884..788aad0f347aa 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -185,7 +185,6 @@ libs += [ scalaLibrary: "org.scala-lang:scala-library:$versions.scala", scalaLogging: "com.typesafe.scala-logging:scala-logging_$versions.baseScala:$versions.scalaLogging", scalaReflect: "org.scala-lang:scala-reflect:$versions.scala", - scalatest: "org.scalatest:scalatest_$versions.baseScala:$versions.scalatest", slf4jApi: "org.slf4j:slf4j-api:$versions.slf4j", slf4jlog4j: "org.slf4j:slf4j-log4j12:$versions.slf4j", snappy: "org.xerial.snappy:snappy-java:$versions.snappy", From 8a574adcae5111f1795f52d638d43fac382fd547 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Mon, 10 May 2021 17:22:26 +0800 Subject: [PATCH 139/155] MINOR: remove unnecessary placeholder from WorkerSourceTask#recordSent (#10659) Reviewers: Tom Bentley --- .../java/org/apache/kafka/connect/runtime/WorkerSourceTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 5a877bac3d59d..c66e0fc99ce01 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -475,7 +475,7 @@ private synchronized void recordSent(final ProducerRecord record removed = outstandingMessagesBacklog.remove(record); // But if neither one had it, something is very wrong if (removed == null) { - log.error("{} CRITICAL Saw callback for record from topic {} partition {} that was not present in the outstanding message set: {}", this, record.topic(), record.partition()); + log.error("{} CRITICAL Saw callback for record from topic {} partition {} that was not present in the outstanding message set", this, record.topic(), record.partition()); } else if (flushing && outstandingMessages.isEmpty()) { // flush thread may be waiting on the outstanding messages to clear this.notifyAll(); From 25f4ee879cf9e13f586d4d45f9451cf0090e084d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 10 May 2021 12:32:51 -0700 Subject: [PATCH 140/155] KAFKA-12747: Fix flakiness in shouldReturnUUIDsWithStringPrefix (#10643) Consecutive UUID generation could result in same prefix. Reviewers: Josep Prat , Anna Sophie Blee-Goldman --- .../streams/state/internals/RocksDBStoreTest.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java index 93e65f02ebddb..c9e9a8d94692c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBStoreTest.java @@ -74,6 +74,7 @@ import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.notNull; import static org.easymock.EasyMock.reset; +import static org.hamcrest.CoreMatchers.either; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.notNullValue; @@ -439,6 +440,8 @@ public void shouldReturnUUIDsWithStringPrefix() { final UUID uuid1 = UUID.randomUUID(); final UUID uuid2 = UUID.randomUUID(); final String prefix = uuid1.toString().substring(0, 4); + final int numMatches = uuid2.toString().substring(0, 4).equals(prefix) ? 2 : 1; + entries.add(new KeyValue<>( new Bytes(uuidSerializer.serialize(null, uuid1)), stringSerializer.serialize(null, "a"))); @@ -460,8 +463,12 @@ public void shouldReturnUUIDsWithStringPrefix() { numberOfKeysReturned++; } - assertThat(numberOfKeysReturned, is(1)); - assertThat(valuesWithPrefix.get(0), is("a")); + assertThat(numberOfKeysReturned, is(numMatches)); + if (numMatches == 2) { + assertThat(valuesWithPrefix.get(0), either(is("a")).or(is("b"))); + } else { + assertThat(valuesWithPrefix.get(0), is("a")); + } } @Test From fae0784ce32a448a0d8c69e0b55f957290df1b5a Mon Sep 17 00:00:00 2001 From: Vito Jeng Date: Tue, 11 May 2021 08:29:58 +0800 Subject: [PATCH 141/155] KAFKA-5876: KIP-216 Part 4, Apply InvalidStateStorePartitionException for Interactive Queries (#10657) KIP-216, part 4 - apply InvalidStateStorePartitionException Reviewers: Anna Sophie Blee-Goldman --- docs/streams/upgrade-guide.html | 1 + .../main/java/org/apache/kafka/streams/KafkaStreams.java | 2 ++ .../streams/state/internals/WrappingStoreProvider.java | 3 ++- .../state/internals/WrappingStoreProviderTest.java | 8 ++++++++ 4 files changed, 13 insertions(+), 1 deletion(-) diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index 6f13cc5d36c15..de747bb862f33 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -101,6 +101,7 @@

      Streams API
      • UnknownStateStoreException: If the specified store name does not exist in the topology, an UnknownStateStoreException will be thrown instead of the former InvalidStateStoreException.
      • StreamsNotStartedException: If Streams state is CREATED, a StreamsNotStartedException will be thrown.
      • +
      • InvalidStateStorePartitionException: If the specified partition does not exist, a InvalidStateStorePartitionException will be thrown.

      See KIP-216 for more information. diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index e8a7fa2c1a260..f3f8e8ea64536 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -46,6 +46,7 @@ import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.errors.UnknownStateStoreException; +import org.apache.kafka.streams.errors.InvalidStateStorePartitionException; import org.apache.kafka.streams.internals.metrics.ClientMetrics; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.StateRestoreListener; @@ -1526,6 +1527,7 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, * @throws StreamsNotStartedException If Streams state is {@link KafkaStreams.State#CREATED CREATED}. Just * retry and wait until to {@link KafkaStreams.State#RUNNING RUNNING}. * @throws UnknownStateStoreException If the specified store name does not exist in the topology. + * @throws InvalidStateStorePartitionException If the specified partition does not exist. * @throws InvalidStateStoreException If the Streams instance isn't in a queryable state. * If the store's type does not match the QueryableStoreType, * the Streams instance is not in a queryable state with respect diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java index 26c5db0e192fd..03ac0ae2c9134 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappingStoreProvider.java @@ -18,6 +18,7 @@ import org.apache.kafka.streams.StoreQueryParameters; import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.errors.InvalidStateStorePartitionException; import org.apache.kafka.streams.state.QueryableStoreType; import java.util.ArrayList; @@ -57,7 +58,7 @@ public List stores(final String storeName, } if (allStores.isEmpty()) { if (storeQueryParameters.partition() != null) { - throw new InvalidStateStoreException( + throw new InvalidStateStorePartitionException( String.format("The specified partition %d for store %s does not exist.", storeQueryParameters.partition(), storeName)); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java index 0d364730dc372..2a5551a7c75dd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.StoreQueryParameters; import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.errors.InvalidStateStorePartitionException; import org.apache.kafka.streams.state.NoOpWindowStore; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; @@ -82,6 +83,13 @@ public void shouldThrowInvalidStoreExceptionIfNoStoreOfTypeFound() { assertThrows(InvalidStateStoreException.class, () -> wrappingStoreProvider.stores("doesn't exist", QueryableStoreTypes.keyValueStore())); } + @Test + public void shouldThrowInvalidStoreExceptionIfNoPartitionFound() { + final int invalidPartition = numStateStorePartitions + 1; + wrappingStoreProvider.setStoreQueryParameters(StoreQueryParameters.fromNameAndType("kv", QueryableStoreTypes.keyValueStore()).withPartition(invalidPartition)); + assertThrows(InvalidStateStorePartitionException.class, () -> wrappingStoreProvider.stores("kv", QueryableStoreTypes.keyValueStore())); + } + @Test public void shouldReturnAllStoreWhenQueryWithoutPartition() { wrappingStoreProvider.setStoreQueryParameters(StoreQueryParameters.fromNameAndType("kv", QueryableStoreTypes.keyValueStore())); From 80a468e2b3eea28173f03651b33c35f8b1547597 Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Tue, 11 May 2021 21:34:40 +0530 Subject: [PATCH 142/155] MINOR Removed copying storage libraries specifically as they are already copied. (#10647) Reviewers: Ismael Juma , Jun Rao --- build.gradle | 4 ---- 1 file changed, 4 deletions(-) diff --git a/build.gradle b/build.gradle index e443d317e870b..c01d994903362 100644 --- a/build.gradle +++ b/build.gradle @@ -1003,10 +1003,6 @@ project(':core') { from(project(':connect:mirror').configurations.runtimeClasspath) { into("libs/") } from(project(':connect:mirror-client').jar) { into("libs/") } from(project(':connect:mirror-client').configurations.runtimeClasspath) { into("libs/") } - from(project(':storage').jar) { into("libs/") } - from(project(':storage').configurations.runtimeClasspath) { into("libs/") } - from(project(':storage:api').jar) { into("libs/") } - from(project(':storage:api').configurations.runtimeClasspath) { into("libs/") } from(project(':streams').jar) { into("libs/") } from(project(':streams').configurations.runtimeClasspath) { into("libs/") } from(project(':streams:streams-scala').jar) { into("libs/") } From 7ef38794291fc5d2317a3c3e33b463e2525b12a9 Mon Sep 17 00:00:00 2001 From: Satish Duggana Date: Tue, 11 May 2021 22:28:28 +0530 Subject: [PATCH 143/155] KAFKA-12758 Added `server-common` module to have server side common classes. (#10638) Added server-common module to have server side common classes. Moved ApiMessageAndVersion, RecordSerde, AbstractApiMessageSerde, and BytesApiMessageSerde to server-common module. Reivewers: Kowshik Prakasam , Jun Rao --- build.gradle | 64 ++++++++++++++++++- checkstyle/import-control-core.xml | 1 + checkstyle/import-control.xml | 18 ++++-- .../main/scala/kafka/raft/RaftManager.scala | 5 +- .../scala/kafka/server/ControllerApis.scala | 4 +- .../scala/kafka/server/ControllerServer.scala | 4 +- .../scala/kafka/server/KafkaRaftServer.scala | 3 +- .../scala/kafka/tools/TestRaftServer.scala | 3 +- .../kafka/testkit/KafkaClusterTestKit.java | 2 +- .../kafka/raft/KafkaMetadataLogTest.scala | 4 +- .../kafka/server/ControllerApisTest.scala | 5 +- .../kafka/tools/DumpLogSegmentsTest.scala | 3 +- .../controller/ClientQuotaControlManager.java | 2 +- .../controller/ClusterControlManager.java | 2 +- .../ConfigurationControlManager.java | 2 +- .../kafka/controller/ControllerResult.java | 2 +- .../controller/ControllerResultAndOffset.java | 2 +- .../controller/FeatureControlManager.java | 2 +- .../kafka/controller/NoOpSnapshotWriter.java | 2 +- .../kafka/controller/QuorumController.java | 2 +- .../controller/ReplicationControlManager.java | 2 +- .../kafka/controller/SnapshotGenerator.java | 2 +- .../kafka/controller/SnapshotWriter.java | 2 +- .../apache/kafka/metalog/MetaLogManager.java | 2 +- .../ClientQuotaControlManagerTest.java | 2 +- .../controller/ClusterControlManagerTest.java | 2 +- .../ConfigurationControlManagerTest.java | 2 +- .../kafka/controller/ControllerTestUtils.java | 2 +- .../controller/FeatureControlManagerTest.java | 2 +- .../kafka/controller/MockSnapshotWriter.java | 2 +- .../controller/QuorumControllerTest.java | 2 +- .../ReplicationControlManagerTest.java | 2 +- .../controller/SnapshotGeneratorTest.java | 2 +- .../apache/kafka/metalog/LocalLogManager.java | 2 +- .../kafka/metalog/LocalLogManagerTest.java | 2 +- .../org/apache/kafka/raft/BatchReader.java | 2 +- .../apache/kafka/raft/KafkaRaftClient.java | 1 + .../raft/internals/BatchAccumulator.java | 2 +- .../kafka/raft/internals/BatchBuilder.java | 2 +- .../raft/internals/RecordsBatchReader.java | 2 +- .../kafka/raft/internals/RecordsIterator.java | 2 +- .../kafka/raft/internals/StringSerde.java | 2 +- .../kafka/raft/metadata/MetaLogRaftShim.java | 2 +- .../raft/metadata/MetadataRecordSerde.java | 1 + .../apache/kafka/snapshot/SnapshotReader.java | 2 +- .../apache/kafka/snapshot/SnapshotWriter.java | 2 +- .../kafka/raft/RaftClientTestContext.java | 1 + .../kafka/raft/RaftEventSimulationTest.java | 1 + .../raft/internals/RecordsIteratorTest.java | 2 +- .../metadata/MetadataRecordSerdeTest.java | 2 +- .../server/common}/ApiMessageAndVersion.java | 2 +- .../AbstractApiMessageSerde.java | 5 +- .../serialization/BytesApiMessageSerde.java | 14 ++-- .../common/serialization}/RecordSerde.java | 2 +- settings.gradle | 1 + .../kafka/shell/MetadataNodeManager.java | 2 +- .../org/apache/kafka/shell/MetadataShell.java | 2 +- .../kafka/shell/SnapshotFileReader.java | 2 +- .../serialization/RemoteLogMetadataSerde.java | 3 +- .../RemoteLogMetadataTransform.java | 2 +- .../RemoteLogSegmentMetadataTransform.java | 2 +- ...moteLogSegmentMetadataUpdateTransform.java | 2 +- ...emotePartitionDeleteMetadataTransform.java | 2 +- .../RemoteLogMetadataTransformTest.java | 2 +- 64 files changed, 152 insertions(+), 79 deletions(-) rename {metadata/src/main/java/org/apache/kafka/metadata => server-common/src/main/java/org/apache/kafka/server/common}/ApiMessageAndVersion.java (97%) rename {raft/src/main/java/org/apache/kafka/raft/metadata => server-common/src/main/java/org/apache/kafka/server/common/serialization}/AbstractApiMessageSerde.java (96%) rename {storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage => server-common/src/main/java/org/apache/kafka/server/common}/serialization/BytesApiMessageSerde.java (84%) rename {raft/src/main/java/org/apache/kafka/raft => server-common/src/main/java/org/apache/kafka/server/common/serialization}/RecordSerde.java (97%) diff --git a/build.gradle b/build.gradle index c01d994903362..b10fc53eab3c6 100644 --- a/build.gradle +++ b/build.gradle @@ -779,6 +779,7 @@ project(':core') { api project(':clients') api libs.scalaLibrary + implementation project(':server-common') implementation project(':metadata') implementation project(':raft') implementation project(':storage') @@ -1066,6 +1067,7 @@ project(':metadata') { archivesBaseName = "kafka-metadata" dependencies { + implementation project(':server-common') implementation project(':clients') implementation libs.jacksonDatabind implementation libs.jacksonJDK8Datatypes @@ -1263,11 +1265,13 @@ project(':raft') { archivesBaseName = "kafka-raft" dependencies { + implementation project(':server-common') implementation project(':clients') implementation project(':metadata') implementation libs.slf4jApi implementation libs.jacksonDatabind + testImplementation project(':server-common') testImplementation project(':clients') testImplementation project(':clients').sourceSets.test.output testImplementation libs.junitJupiter @@ -1341,6 +1345,62 @@ project(':raft') { } } +project(':server-common') { + archivesBaseName = "kafka-server-common" + + dependencies { + api project(':clients') + implementation libs.slf4jApi + + testImplementation project(':clients') + testImplementation project(':clients').sourceSets.test.output + testImplementation libs.junitJupiter + testImplementation libs.mockitoCore + + testRuntimeOnly libs.slf4jlog4j + } + + task createVersionFile(dependsOn: determineCommitId) { + ext.receiptFile = file("$buildDir/kafka/$buildVersionFileName") + outputs.file receiptFile + outputs.upToDateWhen { false } + doLast { + def data = [ + commitId: commitId, + version: version, + ] + + receiptFile.parentFile.mkdirs() + def content = data.entrySet().collect { "$it.key=$it.value" }.sort().join("\n") + receiptFile.setText(content, "ISO-8859-1") + } + } + + sourceSets { + main { + java { + srcDirs = ["src/main/java"] + } + } + test { + java { + srcDirs = ["src/test/java"] + } + } + } + + jar { + dependsOn createVersionFile + from("$buildDir") { + include "kafka/$buildVersionFileName" + } + } + + clean.doFirst { + delete "$buildDir/kafka/" + } +} + project(':storage:api') { archivesBaseName = "kafka-storage-api" @@ -1406,9 +1466,8 @@ project(':storage') { dependencies { implementation project(':storage:api') + implementation project(':server-common') implementation project(':clients') - implementation project(':metadata') - implementation project(':raft') implementation libs.slf4jApi implementation libs.jacksonDatabind @@ -1581,6 +1640,7 @@ project(':shell') { implementation libs.jacksonJDK8Datatypes implementation libs.jline implementation libs.slf4jApi + implementation project(':server-common') implementation project(':clients') implementation project(':core') implementation project(':log4j-appender') diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 7d6374c5b0adc..b2037b784404f 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -54,6 +54,7 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 38db3966fac06..318384fca9c0b 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -221,6 +221,7 @@ + @@ -231,6 +232,7 @@ + @@ -240,6 +242,7 @@ + @@ -266,13 +269,16 @@ - - + + + + + + - - + @@ -287,6 +293,7 @@ + @@ -410,6 +417,8 @@ + + @@ -418,6 +427,7 @@ + diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index ee06a95a78ab4..f0d37be8a6837 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -21,7 +21,6 @@ import java.nio.file.Files import java.util import java.util.OptionalInt import java.util.concurrent.CompletableFuture - import kafka.log.Log import kafka.raft.KafkaRaftManager.RaftIoThread import kafka.server.{KafkaConfig, MetaProperties} @@ -37,8 +36,8 @@ import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.raft.RaftConfig.{AddressSpec, InetAddressSpec, NON_ROUTABLE_ADDRESS, UnknownAddressSpec} -import org.apache.kafka.raft.{FileBasedStateStore, KafkaRaftClient, RaftClient, RaftConfig, RaftRequest, RecordSerde} - +import org.apache.kafka.raft.{FileBasedStateStore, KafkaRaftClient, RaftClient, RaftConfig, RaftRequest} +import org.apache.kafka.server.common.serialization.RecordSerde import scala.jdk.CollectionConverters._ object KafkaRaftManager { diff --git a/core/src/main/scala/kafka/server/ControllerApis.scala b/core/src/main/scala/kafka/server/ControllerApis.scala index cf5be56a9540b..b68da34b0084d 100644 --- a/core/src/main/scala/kafka/server/ControllerApis.scala +++ b/core/src/main/scala/kafka/server/ControllerApis.scala @@ -21,7 +21,6 @@ import java.util import java.util.Collections import java.util.Map.Entry import java.util.concurrent.{CompletableFuture, ExecutionException} - import kafka.network.RequestChannel import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers @@ -50,8 +49,9 @@ import org.apache.kafka.common.resource.ResourceType.{CLUSTER, TOPIC} import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.controller.Controller -import org.apache.kafka.metadata.{ApiMessageAndVersion, BrokerHeartbeatReply, BrokerRegistrationReply, VersionRange} +import org.apache.kafka.metadata.{BrokerHeartbeatReply, BrokerRegistrationReply, VersionRange} import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.common.ApiMessageAndVersion import scala.jdk.CollectionConverters._ diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala index 6a2844af8a25d..638e23ac84c4c 100644 --- a/core/src/main/scala/kafka/server/ControllerServer.scala +++ b/core/src/main/scala/kafka/server/ControllerServer.scala @@ -20,7 +20,6 @@ package kafka.server import java.util.concurrent.{CompletableFuture, TimeUnit} import java.util import java.util.concurrent.locks.ReentrantLock - import kafka.cluster.Broker.ServerInfo import kafka.log.LogConfig import kafka.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector} @@ -37,11 +36,12 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.{ClusterResource, Endpoint} import org.apache.kafka.controller.{Controller, QuorumController, QuorumControllerMetrics} -import org.apache.kafka.metadata.{ApiMessageAndVersion, VersionRange} +import org.apache.kafka.metadata.VersionRange import org.apache.kafka.metalog.MetaLogManager import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig.AddressSpec import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.common.ApiMessageAndVersion import scala.jdk.CollectionConverters._ diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala index e1fc81f09ab8e..1f96eaa7bacf6 100644 --- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala +++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala @@ -18,7 +18,6 @@ package kafka.server import java.io.File import java.util.concurrent.CompletableFuture - import kafka.common.{InconsistentNodeIdException, KafkaException} import kafka.log.Log import kafka.metrics.{KafkaMetricsReporter, KafkaYammerMetrics} @@ -27,9 +26,9 @@ import kafka.server.KafkaRaftServer.{BrokerRole, ControllerRole} import kafka.utils.{CoreUtils, Logging, Mx4jLoader, VerifiableProperties} import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.utils.{AppInfoParser, Time} -import org.apache.kafka.metadata.ApiMessageAndVersion import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.metadata.{MetaLogRaftShim, MetadataRecordSerde} +import org.apache.kafka.server.common.ApiMessageAndVersion import scala.collection.Seq diff --git a/core/src/main/scala/kafka/tools/TestRaftServer.scala b/core/src/main/scala/kafka/tools/TestRaftServer.scala index 3d432159ce4c9..c21f4b7e86c63 100644 --- a/core/src/main/scala/kafka/tools/TestRaftServer.scala +++ b/core/src/main/scala/kafka/tools/TestRaftServer.scala @@ -35,7 +35,8 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{TopicPartition, Uuid, protocol} -import org.apache.kafka.raft.{Batch, BatchReader, RaftClient, RaftConfig, RecordSerde} +import org.apache.kafka.raft.{Batch, BatchReader, RaftClient, RaftConfig} +import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.snapshot.SnapshotReader import scala.jdk.CollectionConverters._ diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 77e6a54fff8b9..87083a240c2d0 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -36,7 +36,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.controller.Controller; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metalog.MetaLogManager; import org.apache.kafka.raft.RaftConfig; import org.apache.kafka.raft.metadata.MetaLogRaftShim; diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala index c54fd4780b626..f9ec73e453f78 100644 --- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala +++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala @@ -20,7 +20,6 @@ import java.io.File import java.nio.ByteBuffer import java.nio.file.{Files, Path} import java.util.{Collections, Optional} - import kafka.log.Log import kafka.server.KafkaRaftServer import kafka.utils.{MockTime, TestUtils} @@ -30,7 +29,8 @@ import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.utils.Utils import org.apache.kafka.raft.internals.BatchBuilder -import org.apache.kafka.raft.{KafkaRaftClient, LogAppendInfo, LogOffsetMetadata, OffsetAndEpoch, RecordSerde, ReplicatedLog, ValidOffsetAndEpoch} +import org.apache.kafka.raft.{KafkaRaftClient, LogAppendInfo, LogOffsetMetadata, OffsetAndEpoch, ReplicatedLog, ValidOffsetAndEpoch} +import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.snapshot.{SnapshotPath, Snapshots} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertThrows, assertTrue} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 3ed4faa2a01a2..b0fa2b36b940f 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -21,7 +21,6 @@ import java.net.InetAddress import java.util import java.util.Properties import java.util.concurrent.ExecutionException - import kafka.network.RequestChannel import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers @@ -43,7 +42,7 @@ import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicRe import org.apache.kafka.common.message.DeleteTopicsRequestData.DeleteTopicState import org.apache.kafka.common.message.DeleteTopicsResponseData.DeletableTopicResult import org.apache.kafka.common.message._ -import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResourceCollection, AlterableConfig, AlterableConfigCollection, AlterConfigsResource} +import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData.{AlterConfigsResource, AlterConfigsResourceCollection, AlterableConfig, AlterableConfigCollection} import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterConfigsResourceCollection => OldAlterConfigsResourceCollection} import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterConfigsResource => OldAlterConfigsResource} import org.apache.kafka.common.message.AlterConfigsRequestData.{AlterableConfigCollection => OldAlterableConfigCollection} @@ -56,8 +55,8 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.controller.Controller -import org.apache.kafka.metadata.ApiMessageAndVersion import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} +import org.apache.kafka.server.common.ApiMessageAndVersion import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.mockito.ArgumentMatchers._ diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 256262a99b4e0..965dd2d64fb7b 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -21,7 +21,6 @@ import java.io.{ByteArrayOutputStream, File, PrintWriter} import java.nio.ByteBuffer import java.util import java.util.Properties - import kafka.log.{Log, LogConfig, LogManager, LogTest} import kafka.server.{BrokerTopicStats, LogDirFailureChannel} import kafka.tools.DumpLogSegments.TimeIndexDumpErrors @@ -31,8 +30,8 @@ import org.apache.kafka.common.metadata.{PartitionChangeRecord, RegisterBrokerRe import org.apache.kafka.common.protocol.{ByteBufferAccessor, ObjectSerializationCache} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.utils.Utils -import org.apache.kafka.metadata.ApiMessageAndVersion import org.apache.kafka.raft.metadata.MetadataRecordSerde +import org.apache.kafka.server.common.ApiMessageAndVersion import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java index c1a98016f4e46..317d58e097db3 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java @@ -26,7 +26,7 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration; import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.requests.ApiError; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java index 312d788ef930c..6d34024ced922 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClusterControlManager.java @@ -33,7 +33,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metadata.FeatureMapAndEpoch; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index b53926e21fb62..6820d6b00a45c 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -27,7 +27,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; import org.slf4j.Logger; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java index e6ae031b9b3b8..d130de59282d4 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java @@ -17,7 +17,7 @@ package org.apache.kafka.controller; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import java.util.Collections; import java.util.List; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerResultAndOffset.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerResultAndOffset.java index 8b8ca8dea80da..1b725653d3aff 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ControllerResultAndOffset.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerResultAndOffset.java @@ -17,7 +17,7 @@ package org.apache.kafka.controller; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import java.util.Objects; import java.util.stream.Collectors; diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index fb6d7dbcce740..42a82502a22fb 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -31,7 +31,7 @@ import org.apache.kafka.common.metadata.FeatureLevelRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.FeatureMap; import org.apache.kafka.metadata.FeatureMapAndEpoch; import org.apache.kafka.metadata.VersionRange; diff --git a/metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriter.java b/metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriter.java index 0263dd556a58e..1a4d1825e0c2f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriter.java +++ b/metadata/src/main/java/org/apache/kafka/controller/NoOpSnapshotWriter.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.util.List; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; /** diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 86faa5ede8e6c..2645b8615ce60 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -74,7 +74,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.controller.SnapshotGenerator.Section; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metadata.FeatureMapAndEpoch; diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index ea94a0052d34b..b270e35205adb 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -60,7 +60,7 @@ import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.controller.BrokersToIsrs.TopicIdPartition; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.timeline.SnapshotRegistry; diff --git a/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java b/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java index 91295c2a2f769..f182d28f158fd 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java +++ b/metadata/src/main/java/org/apache/kafka/controller/SnapshotGenerator.java @@ -24,7 +24,7 @@ import org.apache.kafka.common.utils.ExponentialBackoff; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.slf4j.Logger; diff --git a/metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java b/metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java index 595f4c181b2fa..93a429d97642a 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java +++ b/metadata/src/main/java/org/apache/kafka/controller/SnapshotWriter.java @@ -19,7 +19,7 @@ import java.io.IOException; import java.util.List; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; interface SnapshotWriter extends AutoCloseable { diff --git a/metadata/src/main/java/org/apache/kafka/metalog/MetaLogManager.java b/metadata/src/main/java/org/apache/kafka/metalog/MetaLogManager.java index 9126245ef3855..1ba358c4f06c2 100644 --- a/metadata/src/main/java/org/apache/kafka/metalog/MetaLogManager.java +++ b/metadata/src/main/java/org/apache/kafka/metalog/MetaLogManager.java @@ -17,7 +17,7 @@ package org.apache.kafka.metalog; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import java.util.List; diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java index 47726964c2e1a..ffe891d09768d 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java @@ -25,7 +25,7 @@ import org.apache.kafka.common.quota.ClientQuotaEntity; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 9b9ddb0cc1a78..04fc8168a952e 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -34,7 +34,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 608b428f68a85..4afbf7d2c8804 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -23,7 +23,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; import java.util.AbstractMap.SimpleImmutableEntry; diff --git a/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java b/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java index 10def7ecb352a..5ad037a647a26 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ControllerTestUtils.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Message; import org.apache.kafka.common.utils.ImplicitLinkedHashCollection; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import java.lang.reflect.Field; import java.lang.reflect.Method; diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index 0ee5ee1eb3f26..f417be6aed9fa 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -27,7 +27,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.FeatureMap; import org.apache.kafka.metadata.FeatureMapAndEpoch; import org.apache.kafka.metadata.VersionRange; diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java b/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java index 76b4934482390..de481a0f7f935 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java +++ b/metadata/src/test/java/org/apache/kafka/controller/MockSnapshotWriter.java @@ -18,7 +18,7 @@ package org.apache.kafka.controller; import org.apache.kafka.common.protocol.ApiMessage; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import java.io.IOException; import java.util.ArrayList; diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index 8c64cec8bd712..b6eea362c9024 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -48,7 +48,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.controller.BrokersToIsrs.TopicIdPartition; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistrationReply; import org.apache.kafka.metalog.LocalLogManagerTestEnv; diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 2a456be830bb9..f7e0277eb87bf 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -43,7 +43,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metadata.BrokerHeartbeatReply; import org.apache.kafka.metadata.BrokerRegistration; import org.apache.kafka.timeline.SnapshotRegistry; diff --git a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java index e8346211f1614..a2e703153b55a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/SnapshotGeneratorTest.java @@ -24,7 +24,7 @@ import org.apache.kafka.common.utils.ExponentialBackoff; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.controller.SnapshotGenerator.Section; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index e58848eda4ed2..6442bb9dbaad7 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.KafkaEventQueue; import org.slf4j.Logger; diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java index ac578fb635807..e13ebfe953526 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java @@ -18,7 +18,7 @@ package org.apache.kafka.metalog; import org.apache.kafka.common.metadata.RegisterBrokerRecord; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; diff --git a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java index 6469af2095f57..79e6614e2e7c8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/BatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/BatchReader.java @@ -28,7 +28,7 @@ * of the Raft IO thread. This helps to ensure that a slow state machine will not * affect replication. * - * @param record type (see {@link org.apache.kafka.raft.RecordSerde}) + * @param record type (see {@link org.apache.kafka.server.common.serialization.RecordSerde}) */ public interface BatchReader extends Iterator>, AutoCloseable { diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 1c759514cd6c9..190f39bb7207a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -68,6 +68,7 @@ import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.raft.internals.RecordsBatchReader; import org.apache.kafka.raft.internals.ThresholdPurgatory; +import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RawSnapshotReader; import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.SnapshotReader; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java index 84f744fc7826a..939cfc118f25d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java @@ -22,7 +22,7 @@ import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.common.message.LeaderChangeMessage; import java.io.Closeable; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java index 93dc6546eb73a..4a7a24398db1a 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java @@ -28,7 +28,7 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.ByteUtils; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; import java.io.DataOutputStream; import java.nio.ByteBuffer; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java index a55815d0b5e5e..e95206100a305 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsBatchReader.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; import java.util.NoSuchElementException; import java.util.Optional; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java index 46155b57ee4f6..c12a0849915b4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RecordsIterator.java @@ -33,7 +33,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; public final class RecordsIterator implements Iterator>, AutoCloseable { private final Records records; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/StringSerde.java b/raft/src/main/java/org/apache/kafka/raft/internals/StringSerde.java index cf096dfe69a9e..c2a011a687dbc 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/StringSerde.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/StringSerde.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Writable; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; public class StringSerde implements RecordSerde { diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java index 0bcf2c67055b1..dba84123466a8 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetaLogRaftShim.java @@ -17,7 +17,7 @@ package org.apache.kafka.raft.metadata; import org.apache.kafka.common.protocol.ApiMessage; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.metalog.MetaLogLeader; import org.apache.kafka.metalog.MetaLogListener; import org.apache.kafka.metalog.MetaLogManager; diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java b/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java index 5250d3927b2dd..123de4c4853d6 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java +++ b/raft/src/main/java/org/apache/kafka/raft/metadata/MetadataRecordSerde.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.metadata.MetadataRecordType; import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.server.common.serialization.AbstractApiMessageSerde; public class MetadataRecordSerde extends AbstractApiMessageSerde { diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java index af00cdb2286bf..0ddecd704e6ac 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotReader.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.OffsetAndEpoch; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.raft.internals.RecordsIterator; /** diff --git a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java index e9b3c64e0362f..a61b333c3bb8b 100644 --- a/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java +++ b/raft/src/main/java/org/apache/kafka/snapshot/SnapshotWriter.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.utils.Time; import org.apache.kafka.raft.OffsetAndEpoch; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.raft.internals.BatchAccumulator; import org.apache.kafka.raft.internals.BatchAccumulator.CompletedBatch; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index a70e5b638225b..c4bf9199e5b16 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -55,6 +55,7 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.raft.internals.BatchBuilder; import org.apache.kafka.raft.internals.StringSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.RawSnapshotWriter; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.test.TestCondition; diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java index 32e701a3854e6..d0016614b3892 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftEventSimulationTest.java @@ -35,6 +35,7 @@ import org.apache.kafka.raft.MockLog.LogBatch; import org.apache.kafka.raft.MockLog.LogEntry; import org.apache.kafka.raft.internals.BatchMemoryPool; +import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.snapshot.SnapshotReader; import org.junit.jupiter.api.Tag; diff --git a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java index e450b5266440c..4852c1f210351 100644 --- a/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/internals/RecordsIteratorTest.java @@ -35,7 +35,7 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.raft.Batch; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.serialization.RecordSerde; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; diff --git a/raft/src/test/java/org/apache/kafka/raft/metadata/MetadataRecordSerdeTest.java b/raft/src/test/java/org/apache/kafka/raft/metadata/MetadataRecordSerdeTest.java index 2071814ed9532..de844e6e75ffc 100644 --- a/raft/src/test/java/org/apache/kafka/raft/metadata/MetadataRecordSerdeTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/metadata/MetadataRecordSerdeTest.java @@ -22,7 +22,7 @@ import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.utils.ByteUtils; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; diff --git a/metadata/src/main/java/org/apache/kafka/metadata/ApiMessageAndVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/ApiMessageAndVersion.java similarity index 97% rename from metadata/src/main/java/org/apache/kafka/metadata/ApiMessageAndVersion.java rename to server-common/src/main/java/org/apache/kafka/server/common/ApiMessageAndVersion.java index 75ccb4807b1d0..66a625be5ef3e 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/ApiMessageAndVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/ApiMessageAndVersion.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.metadata; +package org.apache.kafka.server.common; import org.apache.kafka.common.protocol.ApiMessage; diff --git a/raft/src/main/java/org/apache/kafka/raft/metadata/AbstractApiMessageSerde.java b/server-common/src/main/java/org/apache/kafka/server/common/serialization/AbstractApiMessageSerde.java similarity index 96% rename from raft/src/main/java/org/apache/kafka/raft/metadata/AbstractApiMessageSerde.java rename to server-common/src/main/java/org/apache/kafka/server/common/serialization/AbstractApiMessageSerde.java index fea70a7ce41cc..67c067d1a2b07 100644 --- a/raft/src/main/java/org/apache/kafka/raft/metadata/AbstractApiMessageSerde.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/serialization/AbstractApiMessageSerde.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft.metadata; +package org.apache.kafka.server.common.serialization; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.protocol.ApiMessage; @@ -22,8 +22,7 @@ import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Writable; import org.apache.kafka.common.utils.ByteUtils; -import org.apache.kafka.metadata.ApiMessageAndVersion; -import org.apache.kafka.raft.RecordSerde; +import org.apache.kafka.server.common.ApiMessageAndVersion; /** * This is an implementation of {@code RecordSerde} with {@link ApiMessageAndVersion} but implementors need to implement diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/BytesApiMessageSerde.java b/server-common/src/main/java/org/apache/kafka/server/common/serialization/BytesApiMessageSerde.java similarity index 84% rename from storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/BytesApiMessageSerde.java rename to server-common/src/main/java/org/apache/kafka/server/common/serialization/BytesApiMessageSerde.java index 65973b0e920cf..668bbfb248860 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/BytesApiMessageSerde.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/serialization/BytesApiMessageSerde.java @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.server.log.remote.metadata.storage.serialization; +package org.apache.kafka.server.common.serialization; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.Readable; -import org.apache.kafka.metadata.ApiMessageAndVersion; -import org.apache.kafka.raft.metadata.AbstractApiMessageSerde; +import org.apache.kafka.server.common.ApiMessageAndVersion; + import java.nio.ByteBuffer; /** @@ -35,7 +35,7 @@ */ public abstract class BytesApiMessageSerde { - private final AbstractApiMessageSerde metadataRecordSerde = new AbstractApiMessageSerde() { + private final AbstractApiMessageSerde apiMessageSerde = new AbstractApiMessageSerde() { @Override public ApiMessage apiMessageFor(short apiKey) { return BytesApiMessageSerde.this.apiMessageFor(apiKey); @@ -44,9 +44,9 @@ public ApiMessage apiMessageFor(short apiKey) { public byte[] serialize(ApiMessageAndVersion messageAndVersion) { ObjectSerializationCache cache = new ObjectSerializationCache(); - int size = metadataRecordSerde.recordSize(messageAndVersion, cache); + int size = apiMessageSerde.recordSize(messageAndVersion, cache); ByteBufferAccessor writable = new ByteBufferAccessor(ByteBuffer.allocate(size)); - metadataRecordSerde.write(messageAndVersion, cache, writable); + apiMessageSerde.write(messageAndVersion, cache, writable); return writable.buffer().array(); } @@ -54,7 +54,7 @@ public byte[] serialize(ApiMessageAndVersion messageAndVersion) { public ApiMessageAndVersion deserialize(byte[] data) { Readable readable = new ByteBufferAccessor(ByteBuffer.wrap(data)); - return metadataRecordSerde.read(readable, data.length); + return apiMessageSerde.read(readable, data.length); } /** diff --git a/raft/src/main/java/org/apache/kafka/raft/RecordSerde.java b/server-common/src/main/java/org/apache/kafka/server/common/serialization/RecordSerde.java similarity index 97% rename from raft/src/main/java/org/apache/kafka/raft/RecordSerde.java rename to server-common/src/main/java/org/apache/kafka/server/common/serialization/RecordSerde.java index 90a5e1ed286ec..70642c652e977 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RecordSerde.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/serialization/RecordSerde.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.raft; +package org.apache.kafka.server.common.serialization; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.Readable; diff --git a/settings.gradle b/settings.gradle index 8ed1d0eb19d02..f29866136b731 100644 --- a/settings.gradle +++ b/settings.gradle @@ -29,6 +29,7 @@ include 'clients', 'log4j-appender', 'metadata', 'raft', + 'server-common', 'shell', 'storage', 'storage:api', diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java index dde0e40defb2c..55986f3c28d88 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataNodeManager.java @@ -36,7 +36,6 @@ import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.metadata.ApiMessageAndVersion; import org.apache.kafka.metalog.MetaLogLeader; import org.apache.kafka.metalog.MetaLogListener; import org.apache.kafka.queue.EventQueue; @@ -44,6 +43,7 @@ import org.apache.kafka.raft.Batch; import org.apache.kafka.raft.BatchReader; import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.snapshot.SnapshotReader; import org.apache.kafka.shell.MetadataNode.DirectoryNode; import org.apache.kafka.shell.MetadataNode.FileNode; diff --git a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java index 9ba70d83064e4..1d99623044e03 100644 --- a/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java +++ b/shell/src/main/java/org/apache/kafka/shell/MetadataShell.java @@ -24,7 +24,7 @@ import net.sourceforge.argparse4j.inf.Namespace; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/shell/src/main/java/org/apache/kafka/shell/SnapshotFileReader.java b/shell/src/main/java/org/apache/kafka/shell/SnapshotFileReader.java index 907b4db467ae1..3078505980cc4 100644 --- a/shell/src/main/java/org/apache/kafka/shell/SnapshotFileReader.java +++ b/shell/src/main/java/org/apache/kafka/shell/SnapshotFileReader.java @@ -26,12 +26,12 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.metadata.ApiMessageAndVersion; import org.apache.kafka.metalog.MetaLogLeader; import org.apache.kafka.metalog.MetaLogListener; import org.apache.kafka.raft.metadata.MetadataRecordSerde; import org.apache.kafka.queue.EventQueue; import org.apache.kafka.queue.KafkaEventQueue; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java index 4e68cf0b97f3b..94975b91ea866 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataSerde.java @@ -17,7 +17,8 @@ package org.apache.kafka.server.log.remote.metadata.storage.serialization; import org.apache.kafka.common.protocol.ApiMessage; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.server.common.serialization.BytesApiMessageSerde; import org.apache.kafka.server.log.remote.metadata.storage.generated.MetadataRecordType; import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord; import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataUpdateRecord; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java index 8cb4c61af20f5..b6e35820ae810 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogMetadataTransform.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.server.log.remote.metadata.storage.serialization; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.log.remote.storage.RemoteLogMetadata; /** diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java index 29e7fc83e1cb7..375c533d176e4 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataTransform.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataRecord; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java index c0f5b2efa2582..4ad8277257843 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemoteLogSegmentMetadataUpdateTransform.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.log.remote.metadata.storage.generated.RemoteLogSegmentMetadataUpdateRecord; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadataUpdate; diff --git a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java index 09d142c3d1f7e..27f4f1f4c8b28 100644 --- a/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java +++ b/storage/src/main/java/org/apache/kafka/server/log/remote/metadata/storage/serialization/RemotePartitionDeleteMetadataTransform.java @@ -18,7 +18,7 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.log.remote.metadata.storage.generated.RemotePartitionDeleteMetadataRecord; import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteMetadata; import org.apache.kafka.server.log.remote.storage.RemotePartitionDeleteState; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java index ea7dae820c168..87e7683329332 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataTransformTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.metadata.ApiMessageAndVersion; +import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogSegmentMetadataTransform; import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogSegmentMetadataUpdateTransform; import org.apache.kafka.server.log.remote.metadata.storage.serialization.RemotePartitionDeleteMetadataTransform; From 2debbb98c8dae8fabb2d3d88a1a8d8e68cf14caa Mon Sep 17 00:00:00 2001 From: dengziming Date: Wed, 12 May 2021 09:14:38 +0800 Subject: [PATCH 144/155] KAFKA-12772: Move all transaction state transition rules into their states (#10667) Co-authored-by: dengziming --- .../transaction/TransactionMetadata.scala | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala index b30094384d906..1027468553828 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionMetadata.scala @@ -64,6 +64,8 @@ private[transaction] sealed trait TransactionState { * Get the name of this state. This is exposed through the `DescribeTransactions` API. */ def name: String + + def validPreviousStates: Set[TransactionState] } /** @@ -75,6 +77,7 @@ private[transaction] sealed trait TransactionState { private[transaction] case object Empty extends TransactionState { val id: Byte = 0 val name: String = "Empty" + val validPreviousStates: Set[TransactionState] = Set(Empty, CompleteCommit, CompleteAbort) } /** @@ -88,6 +91,7 @@ private[transaction] case object Empty extends TransactionState { private[transaction] case object Ongoing extends TransactionState { val id: Byte = 1 val name: String = "Ongoing" + val validPreviousStates: Set[TransactionState] = Set(Ongoing, Empty, CompleteCommit, CompleteAbort) } /** @@ -98,6 +102,7 @@ private[transaction] case object Ongoing extends TransactionState { private[transaction] case object PrepareCommit extends TransactionState { val id: Byte = 2 val name: String = "PrepareCommit" + val validPreviousStates: Set[TransactionState] = Set(Ongoing) } /** @@ -108,6 +113,7 @@ private[transaction] case object PrepareCommit extends TransactionState { private[transaction] case object PrepareAbort extends TransactionState { val id: Byte = 3 val name: String = "PrepareAbort" + val validPreviousStates: Set[TransactionState] = Set(Ongoing, PrepareEpochFence) } /** @@ -118,6 +124,7 @@ private[transaction] case object PrepareAbort extends TransactionState { private[transaction] case object CompleteCommit extends TransactionState { val id: Byte = 4 val name: String = "CompleteCommit" + val validPreviousStates: Set[TransactionState] = Set(PrepareCommit) } /** @@ -128,6 +135,7 @@ private[transaction] case object CompleteCommit extends TransactionState { private[transaction] case object CompleteAbort extends TransactionState { val id: Byte = 5 val name: String = "CompleteAbort" + val validPreviousStates: Set[TransactionState] = Set(PrepareAbort) } /** @@ -136,6 +144,7 @@ private[transaction] case object CompleteAbort extends TransactionState { private[transaction] case object Dead extends TransactionState { val id: Byte = 6 val name: String = "Dead" + val validPreviousStates: Set[TransactionState] = Set(Empty, CompleteAbort, CompleteCommit) } /** @@ -145,6 +154,7 @@ private[transaction] case object Dead extends TransactionState { private[transaction] case object PrepareEpochFence extends TransactionState { val id: Byte = 7 val name: String = "PrepareEpochFence" + val validPreviousStates: Set[TransactionState] = Set(Ongoing) } private[transaction] object TransactionMetadata { @@ -162,20 +172,6 @@ private[transaction] object TransactionMetadata { new TransactionMetadata(transactionalId, producerId, lastProducerId, producerEpoch, lastProducerEpoch, txnTimeoutMs, state, collection.mutable.Set.empty[TopicPartition], timestamp, timestamp) - def isValidTransition(oldState: TransactionState, newState: TransactionState): Boolean = - TransactionMetadata.validPreviousStates(newState).contains(oldState) - - private val validPreviousStates: Map[TransactionState, Set[TransactionState]] = - Map(Empty -> Set(Empty, CompleteCommit, CompleteAbort), - Ongoing -> Set(Ongoing, Empty, CompleteCommit, CompleteAbort), - PrepareCommit -> Set(Ongoing), - PrepareAbort -> Set(Ongoing, PrepareEpochFence), - CompleteCommit -> Set(PrepareCommit), - CompleteAbort -> Set(PrepareAbort), - Dead -> Set(Empty, CompleteAbort, CompleteCommit), - PrepareEpochFence -> Set(Ongoing) - ) - def isEpochExhausted(producerEpoch: Short): Boolean = producerEpoch >= Short.MaxValue - 1 } @@ -385,7 +381,7 @@ private[transaction] class TransactionMetadata(val transactionalId: String, throw new IllegalArgumentException(s"Illegal new producer epoch $newEpoch") // check that the new state transition is valid and update the pending state if necessary - if (TransactionMetadata.validPreviousStates(newState).contains(state)) { + if (newState.validPreviousStates.contains(state)) { val transitMetadata = TxnTransitMetadata(newProducerId, producerId, newEpoch, newLastEpoch, newTxnTimeoutMs, newState, newTopicPartitions, newTxnStartTimestamp, updateTimestamp) debug(s"TransactionalId $transactionalId prepare transition from $state to $transitMetadata") From 9c9ea3044e7306249441c6ee6c4540afd8ff0bc0 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Wed, 12 May 2021 15:27:49 +0800 Subject: [PATCH 145/155] MINOR: exclude all `src/generated` and `src/generated-test` (#10671) Reviewers: Anna Sophie Blee-Goldman --- .gitignore | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/.gitignore b/.gitignore index 613e10aba6242..3080a0d60b1d8 100644 --- a/.gitignore +++ b/.gitignore @@ -51,13 +51,8 @@ docs/generated/ kafkatest.egg-info/ systest/ *.swp -clients/src/generated -clients/src/generated-test jmh-benchmarks/generated jmh-benchmarks/src/main/generated -streams/src/generated -raft/src/generated raft/.jqwik-database -core/src/generated -metadata/src/generated -storage/src/generated +**/src/generated +**/src/generated-test From 13ffebe2f1012da473189c4fe86fd14b83809962 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 12 May 2021 08:13:03 -0700 Subject: [PATCH 146/155] MINOR: Update jacoco to 0.8.7 for JDK 16 support (#10654) Details: * https://github.com/jacoco/jacoco/releases/tag/v0.8.6 * https://github.com/jacoco/jacoco/releases/tag/v0.8.7 Ran `./gradlew clients:reportCoverage -PenableTestCoverage=true -Dorg.gradle.parallel=false` successfully with Java 15 (see https://github.com/gradle/gradle/issues/15730 and https://github.com/scoverage/gradle-scoverage/issues/150 for the reason why `-Dorg.gradle.parallel=false` is required). Also updated `README.md` to include `-Dorg.gradle.parallel=false` alongside `reportCoverage`. Reviewers: Chia-Ping Tsai --- README.md | 4 ++-- gradle/dependencies.gradle | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 94e957cd384f0..2ee837174b09c 100644 --- a/README.md +++ b/README.md @@ -60,11 +60,11 @@ See [Test Retry Gradle Plugin](https://github.com/gradle/test-retry-gradle-plugi ### Generating test coverage reports ### Generate coverage reports for the whole project: - ./gradlew reportCoverage -PenableTestCoverage=true + ./gradlew reportCoverage -PenableTestCoverage=true -Dorg.gradle.parallel=false Generate coverage for a single module, i.e.: - ./gradlew clients:reportCoverage -PenableTestCoverage=true + ./gradlew clients:reportCoverage -PenableTestCoverage=true -Dorg.gradle.parallel=false ### Building a binary release gzipped tar ball ### ./gradlew clean releaseTarGz diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 788aad0f347aa..2c67d137865f7 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -67,7 +67,7 @@ versions += [ easymock: "4.3", jackson: "2.10.5", jacksonDatabind: "2.10.5.1", - jacoco: "0.8.5", + jacoco: "0.8.7", javassist: "3.27.0-GA", jetty: "9.4.39.v20210325", jersey: "2.34", From 4b2736570c9d5f4b6e501ab8cd5efe696a5142b7 Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 13 May 2021 11:25:18 -0700 Subject: [PATCH 147/155] KAFKA-12648: MINOR - Add TopologyMetadata.Subtopology class for subtopology metadata (#10676) Introduce a Subtopology class to wrap the topicGroupId and namedTopology metadata. Reviewers: Walker Carlson --- .../kafka/streams/TopologyDescription.java | 10 ++-- .../kafka/streams/processor/TaskId.java | 7 +-- .../processor/internals/ChangelogTopics.java | 18 ++++--- .../internals/InternalTopologyBuilder.java | 48 ++++++++++------- .../processor/internals/PartitionGrouper.java | 10 ++-- .../internals/RepartitionTopics.java | 9 ++-- .../internals/StreamsPartitionAssignor.java | 17 +++--- .../processor/internals/TopologyMetadata.java | 51 ++++++++++++++++++ .../internals/namedtopology/NamedTaskId.java | 40 ++++++++++++++ .../kafka/streams/StreamsBuilderTest.java | 9 ++-- .../apache/kafka/streams/TopologyTest.java | 51 +++++++++--------- .../internals/KStreamKStreamJoinTest.java | 6 ++- .../kstream/internals/KTableImplTest.java | 3 +- .../internals/ChangelogTopicsTest.java | 24 +++++---- .../InternalTopologyBuilderTest.java | 54 ++++++++++--------- .../internals/PartitionGrouperTest.java | 41 +++++++------- .../internals/RepartitionTopicsTest.java | 25 +++++---- .../StreamsPartitionAssignorTest.java | 15 +++--- .../assignment/AssignmentTestUtils.java | 6 +++ 19 files changed, 286 insertions(+), 158 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTaskId.java diff --git a/streams/src/main/java/org/apache/kafka/streams/TopologyDescription.java b/streams/src/main/java/org/apache/kafka/streams/TopologyDescription.java index 5d9e1cb61d07e..abeb385573370 100644 --- a/streams/src/main/java/org/apache/kafka/streams/TopologyDescription.java +++ b/streams/src/main/java/org/apache/kafka/streams/TopologyDescription.java @@ -26,7 +26,7 @@ /** * A meta representation of a {@link Topology topology}. *

      - * The nodes of a topology are grouped into {@link Subtopology sub-topologies} if they are connected. + * The nodes of a topology are grouped into {@link SubtopologyDescription sub-topologies} if they are connected. * In contrast, two sub-topologies are not connected but can be linked to each other via topics, i.e., if one * sub-topology {@link Topology#addSink(String, String, String...) writes} into a topic and another sub-topology * {@link Topology#addSource(String, String...) reads} from the same topic. @@ -43,7 +43,7 @@ public interface TopologyDescription { * {@link Topology#connectProcessorAndStateStores(String, String...) state stores} * (i.e., if multiple processors share the same state). */ - interface Subtopology { + interface SubtopologyDescription { /** * Internally assigned unique ID. * @return the ID of the sub-topology @@ -62,10 +62,10 @@ interface Subtopology { * org.apache.kafka.common.serialization.Deserializer, org.apache.kafka.common.serialization.Deserializer, String, * String, org.apache.kafka.streams.processor.api.ProcessorSupplier) global store}. * Adding a global store results in adding a source node and one stateful processor node. - * Note, that all added global stores form a single unit (similar to a {@link Subtopology}) even if different + * Note, that all added global stores form a single unit (similar to a {@link SubtopologyDescription}) even if different * global stores are not connected to each other. * Furthermore, global stores are available to all processors without connecting them explicitly, and thus global - * stores will never be part of any {@link Subtopology}. + * stores will never be part of any {@link SubtopologyDescription}. */ interface GlobalStore { /** @@ -168,7 +168,7 @@ interface Sink extends Node { * All sub-topologies of the represented topology. * @return set of all sub-topologies */ - Set subtopologies(); + Set subtopologies(); /** * All global stores of the represented topology. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java index ff6a0c7e260e2..59e7ef4bbc841 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Objects; -import java.util.Optional; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,7 +40,7 @@ public class TaskId implements Comparable { /** The ID of the partition. */ public final int partition; /** The namedTopology that this task belongs to, or null if it does not belong to one */ - private final String namedTopology; + protected final String namedTopology; public TaskId(final int topicGroupId, final int partition) { this(topicGroupId, partition, null); @@ -59,10 +58,6 @@ public TaskId(final int topicGroupId, final int partition, final String namedTop } } - public Optional namedTopology() { - return namedTopology == null ? Optional.empty() : Optional.of(namedTopology); - } - @Override public String toString() { return namedTopology != null ? namedTopology + "_" + topicGroupId + "_" + partition : topicGroupId + "_" + partition; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java index f47db270f611c..c498bd030f9c8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ChangelogTopics.java @@ -20,6 +20,8 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; + import org.slf4j.Logger; import java.util.Collections; @@ -34,8 +36,8 @@ public class ChangelogTopics { private final InternalTopicManager internalTopicManager; - private final Map topicGroups; - private final Map> tasksForTopicGroup; + private final Map topicGroups; + private final Map> tasksForTopicGroup; private final Map> changelogPartitionsForStatefulTask = new HashMap<>(); private final Map> preExistingChangelogPartitionsForTask = new HashMap<>(); private final Set preExistingNonSourceTopicBasedChangelogPartitions = new HashSet<>(); @@ -44,8 +46,8 @@ public class ChangelogTopics { private final Logger log; public ChangelogTopics(final InternalTopicManager internalTopicManager, - final Map topicGroups, - final Map> tasksForTopicGroup, + final Map topicGroups, + final Map> tasksForTopicGroup, final String logPrefix) { this.internalTopicManager = internalTopicManager; this.topicGroups = topicGroups; @@ -57,13 +59,13 @@ public ChangelogTopics(final InternalTopicManager internalTopicManager, public void setup() { // add tasks to state change log topic subscribers final Map changelogTopicMetadata = new HashMap<>(); - for (final Map.Entry entry : topicGroups.entrySet()) { - final int topicGroupId = entry.getKey(); + for (final Map.Entry entry : topicGroups.entrySet()) { + final Subtopology subtopology = entry.getKey(); final TopicsInfo topicsInfo = entry.getValue(); - final Set topicGroupTasks = tasksForTopicGroup.get(topicGroupId); + final Set topicGroupTasks = tasksForTopicGroup.get(subtopology); if (topicGroupTasks == null) { - log.debug("No tasks found for topic group {}", topicGroupId); + log.debug("No tasks found for subtopology {}", subtopology); continue; } else if (topicsInfo.stateChangelogTopics.isEmpty()) { continue; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 6eb451aed80e8..60ca59dd5134e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.TimestampExtractor; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.internals.SessionStoreBuilder; import org.apache.kafka.streams.state.internals.TimestampedWindowStoreBuilder; @@ -129,6 +130,17 @@ public class InternalTopologyBuilder { private StreamsConfig config = null; + // The name of the topology this builder belongs to, or null if none + private final String namedTopology; + + public InternalTopologyBuilder() { + this.namedTopology = null; + } + + public InternalTopologyBuilder(final String namedTopology) { + this.namedTopology = namedTopology; + } + public static class StateStoreFactory { private final StoreBuilder builder; private final Set users = new HashSet<>(); @@ -352,7 +364,6 @@ public synchronized final StreamsConfig getStreamsConfig() { return config; } - public synchronized final InternalTopologyBuilder rewriteTopology(final StreamsConfig config) { Objects.requireNonNull(config, "config can't be null"); @@ -1065,8 +1076,8 @@ public Set allStateStoreName() { * * @return groups of topic names */ - public synchronized Map topicGroups() { - final Map topicGroups = new LinkedHashMap<>(); + public synchronized Map topicGroups() { + final Map topicGroups = new LinkedHashMap<>(); if (nodeGroups == null) { nodeGroups = makeNodeGroups(); @@ -1129,7 +1140,7 @@ public synchronized Map topicGroups() { } } if (!sourceTopics.isEmpty()) { - topicGroups.put(entry.getKey(), new TopicsInfo( + topicGroups.put(new Subtopology(entry.getKey(), namedTopology), new TopicsInfo( Collections.unmodifiableSet(sinkTopics), Collections.unmodifiableSet(sourceTopics), Collections.unmodifiableMap(repartitionTopics), @@ -1457,7 +1468,7 @@ private void describeSubtopology(final TopologyDescription description, } } - description.addSubtopology(new Subtopology( + description.addSubtopology(new SubtopologyDescription( subtopologyId, new HashSet<>(nodesByName.values()))); } @@ -1743,11 +1754,11 @@ public int hashCode() { } } - public final static class Subtopology implements org.apache.kafka.streams.TopologyDescription.Subtopology { + public final static class SubtopologyDescription implements org.apache.kafka.streams.TopologyDescription.SubtopologyDescription { private final int id; private final Set nodes; - public Subtopology(final int id, final Set nodes) { + public SubtopologyDescription(final int id, final Set nodes) { this.id = id; this.nodes = new TreeSet<>(NODE_COMPARATOR); this.nodes.addAll(nodes); @@ -1792,7 +1803,7 @@ public boolean equals(final Object o) { return false; } - final Subtopology that = (Subtopology) o; + final SubtopologyDescription that = (SubtopologyDescription) o; return id == that.id && nodes.equals(that.nodes); } @@ -1880,10 +1891,10 @@ public int compare(final TopologyDescription.GlobalStore globalStore1, private final static GlobalStoreComparator GLOBALSTORE_COMPARATOR = new GlobalStoreComparator(); - private static class SubtopologyComparator implements Comparator, Serializable { + private static class SubtopologyComparator implements Comparator, Serializable { @Override - public int compare(final TopologyDescription.Subtopology subtopology1, - final TopologyDescription.Subtopology subtopology2) { + public int compare(final org.apache.kafka.streams.TopologyDescription.SubtopologyDescription subtopology1, + final org.apache.kafka.streams.TopologyDescription.SubtopologyDescription subtopology2) { if (subtopology1.equals(subtopology2)) { return 0; } @@ -1894,10 +1905,10 @@ public int compare(final TopologyDescription.Subtopology subtopology1, private final static SubtopologyComparator SUBTOPOLOGY_COMPARATOR = new SubtopologyComparator(); public final static class TopologyDescription implements org.apache.kafka.streams.TopologyDescription { - private final TreeSet subtopologies = new TreeSet<>(SUBTOPOLOGY_COMPARATOR); + private final TreeSet subtopologies = new TreeSet<>(SUBTOPOLOGY_COMPARATOR); private final TreeSet globalStores = new TreeSet<>(GLOBALSTORE_COMPARATOR); - public void addSubtopology(final TopologyDescription.Subtopology subtopology) { + public void addSubtopology(final SubtopologyDescription subtopology) { subtopologies.add(subtopology); } @@ -1906,7 +1917,7 @@ public void addGlobalStore(final TopologyDescription.GlobalStore globalStore) { } @Override - public Set subtopologies() { + public Set subtopologies() { return Collections.unmodifiableSet(subtopologies); } @@ -1919,8 +1930,8 @@ public Set globalStores() { public String toString() { final StringBuilder sb = new StringBuilder(); sb.append("Topologies:\n "); - final TopologyDescription.Subtopology[] sortedSubtopologies = - subtopologies.descendingSet().toArray(new Subtopology[0]); + final SubtopologyDescription[] sortedSubtopologies = + subtopologies.descendingSet().toArray(new SubtopologyDescription[0]); final TopologyDescription.GlobalStore[] sortedGlobalStores = globalStores.descendingSet().toArray(new GlobalStore[0]); int expectedId = 0; @@ -1928,7 +1939,7 @@ public String toString() { int globalStoresIndex = sortedGlobalStores.length - 1; while (subtopologiesIndex != -1 && globalStoresIndex != -1) { sb.append(" "); - final TopologyDescription.Subtopology subtopology = sortedSubtopologies[subtopologiesIndex]; + final SubtopologyDescription subtopology = sortedSubtopologies[subtopologiesIndex]; final TopologyDescription.GlobalStore globalStore = sortedGlobalStores[globalStoresIndex]; if (subtopology.id() == expectedId) { sb.append(subtopology); @@ -1940,7 +1951,7 @@ public String toString() { expectedId++; } while (subtopologiesIndex != -1) { - final TopologyDescription.Subtopology subtopology = sortedSubtopologies[subtopologiesIndex]; + final SubtopologyDescription subtopology = sortedSubtopologies[subtopologiesIndex]; sb.append(" "); sb.append(subtopology); subtopologiesIndex--; @@ -2030,7 +2041,6 @@ private void updateSubscribedTopics(final Set topics, final String logPr setRegexMatchedTopicToStateStore(); } - public synchronized List fullSourceTopicNames() { return maybeDecorateInternalSourceTopics(sourceTopicNames); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java index 8dcb8233913a6..69ee09e6113d8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGrouper.java @@ -21,6 +21,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,11 +50,11 @@ public class PartitionGrouper { * @param metadata metadata of the consuming cluster * @return The map from generated task ids to the assigned partitions */ - public Map> partitionGroups(final Map> topicGroups, final Cluster metadata) { + public Map> partitionGroups(final Map> topicGroups, final Cluster metadata) { final Map> groups = new HashMap<>(); - for (final Map.Entry> entry : topicGroups.entrySet()) { - final Integer topicGroupId = entry.getKey(); + for (final Map.Entry> entry : topicGroups.entrySet()) { + final Subtopology subtopology = entry.getKey(); final Set topicGroup = entry.getValue(); final int maxNumPartitions = maxNumPartitions(metadata, topicGroup); @@ -66,7 +68,7 @@ public Map> partitionGroups(final Map topicGroups = internalTopologyBuilder.topicGroups(); + final Map topicGroups = internalTopologyBuilder.topicGroups(); final Map repartitionTopicMetadata = computeRepartitionTopicConfig(topicGroups, clusterMetadata); // ensure the co-partitioning topics within the group have the same number of partitions, @@ -90,7 +91,7 @@ public Map topicPartitionsInfo() { return Collections.unmodifiableMap(topicPartitionInfos); } - private Map computeRepartitionTopicConfig(final Map topicGroups, + private Map computeRepartitionTopicConfig(final Map topicGroups, final Cluster clusterMetadata) { final Map repartitionTopicConfigs = new HashMap<>(); @@ -129,7 +130,7 @@ private void checkIfExternalSourceTopicsExist(final TopicsInfo topicsInfo, * Computes the number of partitions and sets it for each repartition topic in repartitionTopicMetadata */ private void setRepartitionSourceTopicPartitionCount(final Map repartitionTopicMetadata, - final Map topicGroups, + final Map topicGroups, final Cluster clusterMetadata) { boolean partitionCountNeeded; do { @@ -167,7 +168,7 @@ private void setRepartitionSourceTopicPartitionCount(final Map repartitionTopicMetadata, - final Map topicGroups, + final Map topicGroups, final Cluster clusterMetadata, final String repartitionSourceTopic) { Integer partitionCount = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 2e517f1bcbb69..6c3d6571def24 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -37,6 +37,7 @@ import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration.AssignmentConfigs; @@ -49,6 +50,7 @@ import org.apache.kafka.streams.processor.internals.assignment.StickyTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo; import org.apache.kafka.streams.processor.internals.assignment.TaskAssignor; +import org.apache.kafka.streams.processor.internals.namedtopology.NamedTaskId; import org.apache.kafka.streams.state.HostInfo; import org.slf4j.Logger; @@ -369,10 +371,10 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr // construct the assignment of tasks to clients - final Map topicGroups = taskManager.builder().topicGroups(); + final Map topicGroups = taskManager.builder().topicGroups(); final Set allSourceTopics = new HashSet<>(); - final Map> sourceTopicsByGroup = new HashMap<>(); - for (final Map.Entry entry : topicGroups.entrySet()) { + final Map> sourceTopicsByGroup = new HashMap<>(); + for (final Map.Entry entry : topicGroups.entrySet()) { allSourceTopics.addAll(entry.getValue().sourceTopics); sourceTopicsByGroup.put(entry.getKey(), entry.getValue().sourceTopics); } @@ -385,7 +387,6 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr final boolean probingRebalanceNeeded = assignTasksToClients(fullMetadata, allSourceTopics, topicGroups, clientMetadataMap, partitionsForTask, statefulTasks); - // ---------------- Step Three ---------------- // // construct the global partition assignment per host map @@ -497,7 +498,7 @@ private Map prepareRepartitionTopics(final Cluste * @param fullMetadata the cluster metadata */ private void populateTasksForMaps(final Map taskForPartition, - final Map> tasksForTopicGroup, + final Map> tasksForTopicGroup, final Set allSourceTopics, final Map> partitionsForTask, final Cluster fullMetadata) { @@ -515,7 +516,7 @@ private void populateTasksForMaps(final Map taskForParti } allAssignedPartitions.addAll(partitions); - tasksForTopicGroup.computeIfAbsent(id.topicGroupId, k -> new HashSet<>()).add(id); + tasksForTopicGroup.computeIfAbsent(new Subtopology(id.topicGroupId, NamedTaskId.namedTopology(id)), k -> new HashSet<>()).add(id); } checkAllPartitions(allSourceTopics, partitionsForTask, allAssignedPartitions, fullMetadata); @@ -554,7 +555,7 @@ private void checkAllPartitions(final Set allSourceTopics, */ private boolean assignTasksToClients(final Cluster fullMetadata, final Set allSourceTopics, - final Map topicGroups, + final Map topicGroups, final Map clientMetadataMap, final Map> partitionsForTask, final Set statefulTasks) { @@ -563,7 +564,7 @@ private boolean assignTasksToClients(final Cluster fullMetadata, } final Map taskForPartition = new HashMap<>(); - final Map> tasksForTopicGroup = new HashMap<>(); + final Map> tasksForTopicGroup = new HashMap<>(); populateTasksForMaps(taskForPartition, tasksForTopicGroup, allSourceTopics, partitionsForTask, fullMetadata); final ChangelogTopics changelogTopics = new ChangelogTopics( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java new file mode 100644 index 0000000000000..2b7392ab5bbb7 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TopologyMetadata.java @@ -0,0 +1,51 @@ +/* + * 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.streams.processor.internals; + +import java.util.Objects; + +public class TopologyMetadata { + //TODO KAFKA-12648: the TopologyMetadata class is filled in by Pt. 2 (PR #10683) + + public static class Subtopology { + final int nodeGroupId; + final String namedTopology; + + public Subtopology(final int nodeGroupId, final String namedTopology) { + this.nodeGroupId = nodeGroupId; + this.namedTopology = namedTopology; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Subtopology that = (Subtopology) o; + return nodeGroupId == that.nodeGroupId && + Objects.equals(namedTopology, that.namedTopology); + } + + @Override + public int hashCode() { + return Objects.hash(nodeGroupId, namedTopology); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTaskId.java new file mode 100644 index 0000000000000..82be29cdb9e6f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/namedtopology/NamedTaskId.java @@ -0,0 +1,40 @@ +/* + * 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.streams.processor.internals.namedtopology; + +import org.apache.kafka.streams.processor.TaskId; + +public class NamedTaskId extends TaskId { + public NamedTaskId(final int topicGroupId, final int partition, final String namedTopology) { + super(topicGroupId, partition, namedTopology); + if (namedTopology == null) { + throw new IllegalStateException("NamedTopology is required for a NamedTaskId"); + } + } + + public String namedTopology() { + return namedTopology; + } + + public static String namedTopology(final TaskId taskId) { + if (taskId instanceof NamedTaskId) { + return ((NamedTaskId) taskId).namedTopology(); + } else { + return null; + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java index 54ec9e84b7cc9..129d29aecfa5c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java @@ -64,6 +64,9 @@ import java.util.Properties; import java.util.regex.Pattern; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_1; + import static java.util.Arrays.asList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -458,7 +461,7 @@ public void shouldReuseSourceTopicAsChangelogsWithOptimization20() { internalTopologyBuilder.stateStores().get("store").loggingEnabled(), equalTo(false)); assertThat( - internalTopologyBuilder.topicGroups().get(0).nonSourceChangelogTopics().isEmpty(), + internalTopologyBuilder.topicGroups().get(SUBTOPOLOGY_0).nonSourceChangelogTopics().isEmpty(), equalTo(true)); } @@ -486,7 +489,7 @@ public void shouldNotReuseRepartitionTopicAsChangelogs() { equalTo(true) ); assertThat( - internalTopologyBuilder.topicGroups().get(1).stateChangelogTopics.keySet(), + internalTopologyBuilder.topicGroups().get(SUBTOPOLOGY_1).stateChangelogTopics.keySet(), equalTo(Collections.singleton("appId-store-changelog")) ); } @@ -509,7 +512,7 @@ public void shouldNotReuseSourceTopicAsChangelogsByDefault() { internalTopologyBuilder.stateStores().get("store").loggingEnabled(), equalTo(true)); assertThat( - internalTopologyBuilder.topicGroups().get(0).stateChangelogTopics.keySet(), + internalTopologyBuilder.topicGroups().get(SUBTOPOLOGY_0).stateChangelogTopics.keySet(), equalTo(Collections.singleton("appId-store-changelog"))); } diff --git a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java index 66c678d2bd17f..8068c4839eec4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/TopologyTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -427,8 +428,8 @@ public void singleSourceShouldHaveSingleSubtopology() { final TopologyDescription.Source expectedSourceNode = addSource("source", "topic"); expectedDescription.addSubtopology( - new InternalTopologyBuilder.Subtopology(0, - Collections.singleton(expectedSourceNode))); + new SubtopologyDescription(0, + Collections.singleton(expectedSourceNode))); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -439,8 +440,8 @@ public void singleSourceWithListOfTopicsShouldHaveSingleSubtopology() { final TopologyDescription.Source expectedSourceNode = addSource("source", "topic1", "topic2", "topic3"); expectedDescription.addSubtopology( - new InternalTopologyBuilder.Subtopology(0, - Collections.singleton(expectedSourceNode))); + new SubtopologyDescription(0, + Collections.singleton(expectedSourceNode))); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -451,8 +452,8 @@ public void singleSourcePatternShouldHaveSingleSubtopology() { final TopologyDescription.Source expectedSourceNode = addSource("source", Pattern.compile("topic[0-9]")); expectedDescription.addSubtopology( - new InternalTopologyBuilder.Subtopology(0, - Collections.singleton(expectedSourceNode))); + new SubtopologyDescription(0, + Collections.singleton(expectedSourceNode))); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -462,18 +463,18 @@ public void singleSourcePatternShouldHaveSingleSubtopology() { public void multipleSourcesShouldHaveDistinctSubtopologies() { final TopologyDescription.Source expectedSourceNode1 = addSource("source1", "topic1"); expectedDescription.addSubtopology( - new InternalTopologyBuilder.Subtopology(0, - Collections.singleton(expectedSourceNode1))); + new SubtopologyDescription(0, + Collections.singleton(expectedSourceNode1))); final TopologyDescription.Source expectedSourceNode2 = addSource("source2", "topic2"); expectedDescription.addSubtopology( - new InternalTopologyBuilder.Subtopology(1, - Collections.singleton(expectedSourceNode2))); + new SubtopologyDescription(1, + Collections.singleton(expectedSourceNode2))); final TopologyDescription.Source expectedSourceNode3 = addSource("source3", "topic3"); expectedDescription.addSubtopology( - new InternalTopologyBuilder.Subtopology(2, - Collections.singleton(expectedSourceNode3))); + new SubtopologyDescription(2, + Collections.singleton(expectedSourceNode3))); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -487,7 +488,7 @@ public void sourceAndProcessorShouldHaveSingleSubtopology() { final Set allNodes = new HashSet<>(); allNodes.add(expectedSourceNode); allNodes.add(expectedProcessorNode); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -503,7 +504,7 @@ public void sourceAndProcessorWithStateShouldHaveSingleSubtopology() { final Set allNodes = new HashSet<>(); allNodes.add(expectedSourceNode); allNodes.add(expectedProcessorNode); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -520,7 +521,7 @@ public void sourceAndProcessorWithMultipleStatesShouldHaveSingleSubtopology() { final Set allNodes = new HashSet<>(); allNodes.add(expectedSourceNode); allNodes.add(expectedProcessorNode); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -536,7 +537,7 @@ public void sourceWithMultipleProcessorsShouldHaveSingleSubtopology() { allNodes.add(expectedSourceNode); allNodes.add(expectedProcessorNode1); allNodes.add(expectedProcessorNode2); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -552,7 +553,7 @@ public void processorWithMultipleSourcesShouldHaveSingleSubtopology() { allNodes.add(expectedSourceNode1); allNodes.add(expectedSourceNode2); allNodes.add(expectedProcessorNode); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -572,17 +573,17 @@ public void multipleSourcesWithProcessorsShouldHaveDistinctSubtopologies() { final Set allNodes1 = new HashSet<>(); allNodes1.add(expectedSourceNode1); allNodes1.add(expectedProcessorNode1); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes1)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes1)); final Set allNodes2 = new HashSet<>(); allNodes2.add(expectedSourceNode2); allNodes2.add(expectedProcessorNode2); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(1, allNodes2)); + expectedDescription.addSubtopology(new SubtopologyDescription(1, allNodes2)); final Set allNodes3 = new HashSet<>(); allNodes3.add(expectedSourceNode3); allNodes3.add(expectedProcessorNode3); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(2, allNodes3)); + expectedDescription.addSubtopology(new SubtopologyDescription(2, allNodes3)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -602,17 +603,17 @@ public void multipleSourcesWithSinksShouldHaveDistinctSubtopologies() { final Set allNodes1 = new HashSet<>(); allNodes1.add(expectedSourceNode1); allNodes1.add(expectedSinkNode1); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes1)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes1)); final Set allNodes2 = new HashSet<>(); allNodes2.add(expectedSourceNode2); allNodes2.add(expectedSinkNode2); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(1, allNodes2)); + expectedDescription.addSubtopology(new SubtopologyDescription(1, allNodes2)); final Set allNodes3 = new HashSet<>(); allNodes3.add(expectedSourceNode3); allNodes3.add(expectedSinkNode3); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(2, allNodes3)); + expectedDescription.addSubtopology(new SubtopologyDescription(2, allNodes3)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -644,7 +645,7 @@ public void processorsWithSameSinkShouldHaveSameSubtopology() { allNodes.add(expectedSourceNode3); allNodes.add(expectedProcessorNode3); allNodes.add(expectedSinkNode); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); @@ -675,7 +676,7 @@ public void processorsWithSharedStateShouldHaveSameSubtopology() { allNodes.add(expectedProcessorNode2); allNodes.add(expectedSourceNode3); allNodes.add(expectedProcessorNode3); - expectedDescription.addSubtopology(new InternalTopologyBuilder.Subtopology(0, allNodes)); + expectedDescription.addSubtopology(new SubtopologyDescription(0, allNodes)); assertThat(topology.describe(), equalTo(expectedDescription)); assertThat(topology.describe().hashCode(), equalTo(expectedDescription.hashCode())); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index ca26393bb33a0..c3e32447b7f61 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -53,6 +53,8 @@ import java.util.Set; import static java.time.Duration.ofMillis; + +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0; import static org.apache.kafka.test.StreamsTestUtils.getMetricByName; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; @@ -210,8 +212,8 @@ public void shouldEnableLoggingWithCustomConfigOnStreamJoined() { assertThat(internalTopologyBuilder.stateStores().get("store-this-join-store").loggingEnabled(), equalTo(true)); assertThat(internalTopologyBuilder.stateStores().get("store-other-join-store").loggingEnabled(), equalTo(true)); - assertThat(internalTopologyBuilder.topicGroups().get(0).stateChangelogTopics.size(), equalTo(2)); - for (final InternalTopicConfig config : internalTopologyBuilder.topicGroups().get(0).stateChangelogTopics.values()) { + assertThat(internalTopologyBuilder.topicGroups().get(SUBTOPOLOGY_0).stateChangelogTopics.size(), equalTo(2)); + for (final InternalTopicConfig config : internalTopologyBuilder.topicGroups().get(SUBTOPOLOGY_0).stateChangelogTopics.values()) { assertThat( config.getProperties(Collections.emptyMap(), 0).get("test"), equalTo("property") diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 348750a725975..c1814241db6c2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.TestInputTopic; import org.apache.kafka.streams.Topology; import org.apache.kafka.streams.TopologyDescription; +import org.apache.kafka.streams.TopologyDescription.SubtopologyDescription; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.TopologyTestDriverWrapper; import org.apache.kafka.streams.kstream.Consumed; @@ -405,7 +406,7 @@ public void shouldEnableSendingOldValuesIfNotMaterializedAlreadyButForcedToMater } private void assertTopologyContainsProcessor(final Topology topology, final String processorName) { - for (final TopologyDescription.Subtopology subtopology: topology.describe().subtopologies()) { + for (final SubtopologyDescription subtopology: topology.describe().subtopologies()) { for (final TopologyDescription.Node node: subtopology.nodes()) { if (node.name().equals(processorName)) { return; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java index 06480b3e1cf25..17db61d7c5b13 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ChangelogTopicsTest.java @@ -19,6 +19,8 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.TopicsInfo; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; + import org.junit.Test; import java.util.Collections; @@ -28,6 +30,8 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0; + import static org.easymock.EasyMock.expect; import static org.easymock.EasyMock.mock; import static org.easymock.EasyMock.replay; @@ -80,8 +84,8 @@ public class ChangelogTopicsTest { @Test public void shouldNotContainChangelogsForStatelessTasks() { expect(internalTopicManager.makeReady(Collections.emptyMap())).andStubReturn(Collections.emptySet()); - final Map topicGroups = mkMap(mkEntry(0, TOPICS_INFO2)); - final Map> tasksForTopicGroup = mkMap(mkEntry(0, mkSet(TASK_0_0, TASK_0_1, TASK_0_2))); + final Map topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO2)); + final Map> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, mkSet(TASK_0_0, TASK_0_1, TASK_0_2))); replay(internalTopicManager); final ChangelogTopics changelogTopics = @@ -100,9 +104,9 @@ public void shouldNotContainChangelogsForStatelessTasks() { public void shouldNotContainAnyPreExistingChangelogsIfChangelogIsNewlyCreated() { expect(internalTopicManager.makeReady(mkMap(mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG)))) .andStubReturn(mkSet(CHANGELOG_TOPIC_NAME1)); - final Map topicGroups = mkMap(mkEntry(0, TOPICS_INFO1)); + final Map topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1)); final Set tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); - final Map> tasksForTopicGroup = mkMap(mkEntry(0, tasks)); + final Map> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks)); replay(internalTopicManager); final ChangelogTopics changelogTopics = @@ -122,9 +126,9 @@ public void shouldNotContainAnyPreExistingChangelogsIfChangelogIsNewlyCreated() public void shouldOnlyContainPreExistingNonSourceBasedChangelogs() { expect(internalTopicManager.makeReady(mkMap(mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG)))) .andStubReturn(Collections.emptySet()); - final Map topicGroups = mkMap(mkEntry(0, TOPICS_INFO1)); + final Map topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1)); final Set tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); - final Map> tasksForTopicGroup = mkMap(mkEntry(0, tasks)); + final Map> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks)); replay(internalTopicManager); final ChangelogTopics changelogTopics = @@ -149,9 +153,9 @@ public void shouldOnlyContainPreExistingNonSourceBasedChangelogs() { @Test public void shouldOnlyContainPreExistingSourceBasedChangelogs() { expect(internalTopicManager.makeReady(Collections.emptyMap())).andStubReturn(Collections.emptySet()); - final Map topicGroups = mkMap(mkEntry(0, TOPICS_INFO3)); + final Map topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO3)); final Set tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); - final Map> tasksForTopicGroup = mkMap(mkEntry(0, tasks)); + final Map> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks)); replay(internalTopicManager); final ChangelogTopics changelogTopics = @@ -176,9 +180,9 @@ public void shouldOnlyContainPreExistingSourceBasedChangelogs() { public void shouldContainBothTypesOfPreExistingChangelogs() { expect(internalTopicManager.makeReady(mkMap(mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG)))) .andStubReturn(Collections.emptySet()); - final Map topicGroups = mkMap(mkEntry(0, TOPICS_INFO4)); + final Map topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO4)); final Set tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2); - final Map> tasksForTopicGroup = mkMap(mkEntry(0, tasks)); + final Map> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks)); replay(internalTopicManager); final ChangelogTopics changelogTopics = diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java index 2d5cde58983d8..ef5bebb5d8201 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilderTest.java @@ -26,6 +26,8 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -53,6 +55,10 @@ import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkProperties; import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_1; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_2; + import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; @@ -570,12 +576,12 @@ public void testTopicGroups() { builder.addProcessor("processor-3", new MockApiProcessorSupplier<>(), "source-3", "source-4"); - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); - final Map expectedTopicGroups = new HashMap<>(); - expectedTopicGroups.put(0, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-1", "X-topic-1x", "topic-2"), Collections.emptyMap(), Collections.emptyMap())); - expectedTopicGroups.put(1, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-3", "topic-4"), Collections.emptyMap(), Collections.emptyMap())); - expectedTopicGroups.put(2, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-5"), Collections.emptyMap(), Collections.emptyMap())); + final Map expectedTopicGroups = new HashMap<>(); + expectedTopicGroups.put(SUBTOPOLOGY_0, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-1", "X-topic-1x", "topic-2"), Collections.emptyMap(), Collections.emptyMap())); + expectedTopicGroups.put(SUBTOPOLOGY_1, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-3", "topic-4"), Collections.emptyMap(), Collections.emptyMap())); + expectedTopicGroups.put(SUBTOPOLOGY_2, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-5"), Collections.emptyMap(), Collections.emptyMap())); assertEquals(3, topicGroups.size()); assertEquals(expectedTopicGroups, topicGroups); @@ -607,21 +613,21 @@ public void testTopicGroupsByStateStore() { builder.connectProcessorAndStateStores("processor-5", "store-3"); builder.buildTopology(); - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); - final Map expectedTopicGroups = new HashMap<>(); + final Map expectedTopicGroups = new HashMap<>(); final String store1 = ProcessorStateManager.storeChangelogTopic("X", "store-1"); final String store2 = ProcessorStateManager.storeChangelogTopic("X", "store-2"); final String store3 = ProcessorStateManager.storeChangelogTopic("X", "store-3"); - expectedTopicGroups.put(0, new InternalTopologyBuilder.TopicsInfo( + expectedTopicGroups.put(SUBTOPOLOGY_0, new InternalTopologyBuilder.TopicsInfo( Collections.emptySet(), mkSet("topic-1", "topic-1x", "topic-2"), Collections.emptyMap(), Collections.singletonMap(store1, new UnwindowedChangelogTopicConfig(store1, Collections.emptyMap())))); - expectedTopicGroups.put(1, new InternalTopologyBuilder.TopicsInfo( + expectedTopicGroups.put(SUBTOPOLOGY_1, new InternalTopologyBuilder.TopicsInfo( Collections.emptySet(), mkSet("topic-3", "topic-4"), Collections.emptyMap(), Collections.singletonMap(store2, new UnwindowedChangelogTopicConfig(store2, Collections.emptyMap())))); - expectedTopicGroups.put(2, new InternalTopologyBuilder.TopicsInfo( + expectedTopicGroups.put(SUBTOPOLOGY_2, new InternalTopologyBuilder.TopicsInfo( Collections.emptySet(), mkSet("topic-5"), Collections.emptyMap(), Collections.singletonMap(store3, new UnwindowedChangelogTopicConfig(store3, Collections.emptyMap())))); @@ -837,7 +843,7 @@ public void shouldAddInternalTopicConfigForWindowStores() { "processor" ); builder.buildTopology(); - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); final InternalTopologyBuilder.TopicsInfo topicsInfo = topicGroups.values().iterator().next(); final InternalTopicConfig topicConfig1 = topicsInfo.stateChangelogTopics.get("appId-store1-changelog"); final Map properties1 = topicConfig1.getProperties(Collections.emptyMap(), 10000); @@ -862,7 +868,7 @@ public void shouldAddInternalTopicConfigForNonWindowStores() { builder.addProcessor("processor", new MockApiProcessorSupplier<>(), "source"); builder.addStateStore(storeBuilder, "processor"); builder.buildTopology(); - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); final InternalTopologyBuilder.TopicsInfo topicsInfo = topicGroups.values().iterator().next(); final InternalTopicConfig topicConfig = topicsInfo.stateChangelogTopics.get("appId-testStore-changelog"); final Map properties = topicConfig.getProperties(Collections.emptyMap(), 10000); @@ -903,11 +909,11 @@ public void shouldSetCorrectSourceNodesWithRegexUpdatedTopics() { builder.addSubscribedTopicsFromMetadata(updatedTopics, null); builder.setApplicationId("test-id"); - final Map topicGroups = builder.topicGroups(); - assertTrue(topicGroups.get(0).sourceTopics.contains("topic-foo")); - assertTrue(topicGroups.get(1).sourceTopics.contains("topic-A")); - assertTrue(topicGroups.get(1).sourceTopics.contains("topic-B")); - assertTrue(topicGroups.get(2).sourceTopics.contains("topic-3")); + final Map topicGroups = builder.topicGroups(); + assertTrue(topicGroups.get(SUBTOPOLOGY_0).sourceTopics.contains("topic-foo")); + assertTrue(topicGroups.get(SUBTOPOLOGY_1).sourceTopics.contains("topic-A")); + assertTrue(topicGroups.get(SUBTOPOLOGY_1).sourceTopics.contains("topic-B")); + assertTrue(topicGroups.get(SUBTOPOLOGY_2).sourceTopics.contains("topic-3")); } @Test @@ -943,7 +949,7 @@ public void shouldSortProcessorNodesCorrectly() { assertEquals(1, builder.describe().subtopologies().size()); - final Iterator iterator = ((InternalTopologyBuilder.Subtopology) builder.describe().subtopologies().iterator().next()).nodesInOrder(); + final Iterator iterator = ((SubtopologyDescription) builder.describe().subtopologies().iterator().next()).nodesInOrder(); assertTrue(iterator.hasNext()); InternalTopologyBuilder.AbstractNode node = (InternalTopologyBuilder.AbstractNode) iterator.next(); @@ -1093,9 +1099,9 @@ public void shouldHaveCorrectInternalTopicConfigWhenInternalTopicPropertiesArePr builder.addInternalTopic("topic-1z", new InternalTopicProperties(numberOfPartitions)); builder.addSource(null, "source-1", null, null, null, "topic-1z"); - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); - final Map repartitionSourceTopics = topicGroups.get(0).repartitionSourceTopics; + final Map repartitionSourceTopics = topicGroups.get(SUBTOPOLOGY_0).repartitionSourceTopics; assertEquals( repartitionSourceTopics.get("Z-topic-1z"), @@ -1114,9 +1120,9 @@ public void shouldHandleWhenTopicPropertiesNumberOfPartitionsIsNull() { builder.addInternalTopic("topic-1t", InternalTopicProperties.empty()); builder.addSource(null, "source-1", null, null, null, "topic-1t"); - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); - final Map repartitionSourceTopics = topicGroups.get(0).repartitionSourceTopics; + final Map repartitionSourceTopics = topicGroups.get(SUBTOPOLOGY_0).repartitionSourceTopics; assertEquals( repartitionSourceTopics.get("T-topic-1t"), @@ -1133,9 +1139,9 @@ public void shouldHaveCorrectInternalTopicConfigWhenInternalTopicPropertiesAreNo builder.addInternalTopic("topic-1y", InternalTopicProperties.empty()); builder.addSource(null, "source-1", null, null, null, "topic-1y"); - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); - final Map repartitionSourceTopics = topicGroups.get(0).repartitionSourceTopics; + final Map repartitionSourceTopics = topicGroups.get(SUBTOPOLOGY_0).repartitionSourceTopics; assertEquals( repartitionSourceTopics.get("Y-topic-1y"), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java index 669f710ab97c4..93d2dced7b25f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGrouperTest.java @@ -21,6 +21,8 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; + import org.junit.Test; import java.util.Arrays; @@ -31,6 +33,9 @@ import java.util.Set; import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_1; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; @@ -55,18 +60,16 @@ public class PartitionGrouperTest { public void shouldComputeGroupingForTwoGroups() { final PartitionGrouper grouper = new PartitionGrouper(); final Map> expectedPartitionsForTask = new HashMap<>(); - final Map> topicGroups = new HashMap<>(); - - int topicGroupId = 0; + final Map> topicGroups = new HashMap<>(); - topicGroups.put(topicGroupId, mkSet("topic1")); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic1", 0))); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic1", 1))); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 2), mkSet(new TopicPartition("topic1", 2))); + topicGroups.put(SUBTOPOLOGY_0, mkSet("topic1")); + expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 0, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 0))); + expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 1, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 1))); + expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 2, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 2))); - topicGroups.put(++topicGroupId, mkSet("topic2")); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic2", 0))); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic2", 1))); + topicGroups.put(SUBTOPOLOGY_1, mkSet("topic2")); + expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_1.nodeGroupId, 0, SUBTOPOLOGY_1.namedTopology), mkSet(new TopicPartition("topic2", 0))); + expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_1.nodeGroupId, 1, SUBTOPOLOGY_1.namedTopology), mkSet(new TopicPartition("topic2", 1))); assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); } @@ -75,19 +78,17 @@ public void shouldComputeGroupingForTwoGroups() { public void shouldComputeGroupingForSingleGroupWithMultipleTopics() { final PartitionGrouper grouper = new PartitionGrouper(); final Map> expectedPartitionsForTask = new HashMap<>(); - final Map> topicGroups = new HashMap<>(); + final Map> topicGroups = new HashMap<>(); - final int topicGroupId = 0; - - topicGroups.put(topicGroupId, mkSet("topic1", "topic2")); + topicGroups.put(SUBTOPOLOGY_0, mkSet("topic1", "topic2")); expectedPartitionsForTask.put( - new TaskId(topicGroupId, 0), + new TaskId(SUBTOPOLOGY_0.nodeGroupId, 0, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))); expectedPartitionsForTask.put( - new TaskId(topicGroupId, 1), + new TaskId(SUBTOPOLOGY_0.nodeGroupId, 1, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1))); expectedPartitionsForTask.put( - new TaskId(topicGroupId, 2), + new TaskId(SUBTOPOLOGY_0.nodeGroupId, 2, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 2))); assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); @@ -96,11 +97,9 @@ public void shouldComputeGroupingForSingleGroupWithMultipleTopics() { @Test public void shouldNotCreateAnyTasksBecauseOneTopicHasUnknownPartitions() { final PartitionGrouper grouper = new PartitionGrouper(); - final Map> topicGroups = new HashMap<>(); - - final int topicGroupId = 0; + final Map> topicGroups = new HashMap<>(); - topicGroups.put(topicGroupId, mkSet("topic1", "unknownTopic", "topic2")); + topicGroups.put(SUBTOPOLOGY_0, mkSet("topic1", "unknownTopic", "topic2")); assertThrows(RuntimeException.class, () -> grouper.partitionGroups(topicGroups, metadata)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionTopicsTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionTopicsTest.java index 61a3716fef43a..30a4641e3d2a9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionTopicsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionTopicsTest.java @@ -38,6 +38,9 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0; +import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_1; + import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.eq; import static org.easymock.EasyMock.expect; @@ -96,7 +99,7 @@ public class RepartitionTopicsTest { @Test public void shouldSetupRepartitionTopics() { expect(internalTopologyBuilder.topicGroups()) - .andReturn(mkMap(mkEntry(0, TOPICS_INFO1), mkEntry(1, TOPICS_INFO2))); + .andReturn(mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1), mkEntry(SUBTOPOLOGY_1, TOPICS_INFO2))); final Set coPartitionGroup1 = mkSet(SOURCE_TOPIC_NAME1, SOURCE_TOPIC_NAME2); final Set coPartitionGroup2 = mkSet(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_NAME2); final List> coPartitionGroups = Arrays.asList(coPartitionGroup1, coPartitionGroup2); @@ -135,7 +138,7 @@ public void shouldSetupRepartitionTopics() { @Test public void shouldThrowMissingSourceTopicException() { expect(internalTopologyBuilder.topicGroups()) - .andReturn(mkMap(mkEntry(0, TOPICS_INFO1), mkEntry(1, TOPICS_INFO2))); + .andReturn(mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1), mkEntry(SUBTOPOLOGY_1, TOPICS_INFO2))); expect(internalTopologyBuilder.copartitionGroups()).andReturn(Collections.emptyList()); copartitionedTopicsEnforcer.enforce(eq(Collections.emptySet()), anyObject(), eq(clusterMetadata)); expect(internalTopicManager.makeReady( @@ -162,8 +165,8 @@ public void shouldThrowTaskAssignmentExceptionIfPartitionCountCannotBeComputedFo new RepartitionTopicConfig(REPARTITION_WITHOUT_PARTITION_COUNT, TOPIC_CONFIG5); expect(internalTopologyBuilder.topicGroups()) .andReturn(mkMap( - mkEntry(0, TOPICS_INFO1), - mkEntry(1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) + mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1), + mkEntry(SUBTOPOLOGY_1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) )); expect(internalTopologyBuilder.copartitionGroups()).andReturn(Collections.emptyList()); copartitionedTopicsEnforcer.enforce(eq(Collections.emptySet()), anyObject(), eq(clusterMetadata)); @@ -200,8 +203,8 @@ public void shouldThrowTaskAssignmentExceptionIfSourceTopicHasNoPartitionCount() ); expect(internalTopologyBuilder.topicGroups()) .andReturn(mkMap( - mkEntry(0, topicsInfo), - mkEntry(1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) + mkEntry(SUBTOPOLOGY_0, topicsInfo), + mkEntry(SUBTOPOLOGY_1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) )); expect(internalTopologyBuilder.copartitionGroups()).andReturn(Collections.emptyList()); copartitionedTopicsEnforcer.enforce(eq(Collections.emptySet()), anyObject(), eq(clusterMetadata)); @@ -243,8 +246,8 @@ public void shouldSetRepartitionTopicPartitionCountFromUpstreamExternalSourceTop ); expect(internalTopologyBuilder.topicGroups()) .andReturn(mkMap( - mkEntry(0, topicsInfo), - mkEntry(1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) + mkEntry(SUBTOPOLOGY_0, topicsInfo), + mkEntry(SUBTOPOLOGY_1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) )); expect(internalTopologyBuilder.copartitionGroups()).andReturn(Collections.emptyList()); copartitionedTopicsEnforcer.enforce(eq(Collections.emptySet()), anyObject(), eq(clusterMetadata)); @@ -297,8 +300,8 @@ public void shouldSetRepartitionTopicPartitionCountFromUpstreamInternalRepartiti ); expect(internalTopologyBuilder.topicGroups()) .andReturn(mkMap( - mkEntry(0, topicsInfo), - mkEntry(1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) + mkEntry(SUBTOPOLOGY_0, topicsInfo), + mkEntry(SUBTOPOLOGY_1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount)) )); expect(internalTopologyBuilder.copartitionGroups()).andReturn(Collections.emptyList()); copartitionedTopicsEnforcer.enforce(eq(Collections.emptySet()), anyObject(), eq(clusterMetadata)); @@ -345,7 +348,7 @@ public void shouldNotSetupRepartitionTopicsWhenTopologyDoesNotContainAnyRepartit Collections.emptyMap() ); expect(internalTopologyBuilder.topicGroups()) - .andReturn(mkMap(mkEntry(0, topicsInfo))); + .andReturn(mkMap(mkEntry(SUBTOPOLOGY_0, topicsInfo))); expect(internalTopologyBuilder.copartitionGroups()).andReturn(Collections.emptySet()); expect(internalTopicManager.makeReady(Collections.emptyMap())).andReturn(Collections.emptySet()); setupCluster(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 7e58a8ae61d0a..3ef892a7c803d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -56,6 +56,7 @@ import org.apache.kafka.streams.kstream.internals.InternalStreamsBuilder; import org.apache.kafka.streams.kstream.internals.MaterializedInternal; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; import org.apache.kafka.streams.processor.internals.assignment.AssignmentInfo; import org.apache.kafka.streams.processor.internals.assignment.AssignorConfiguration; import org.apache.kafka.streams.processor.internals.assignment.AssignorError; @@ -811,7 +812,7 @@ public void testAssignWithStates() { assertEquals(new HashSet<>(tasks), allTasks); // check tasks for state topics - final Map topicGroups = builder.topicGroups(); + final Map topicGroups = builder.topicGroups(); assertEquals(mkSet(TASK_0_0, TASK_0_1, TASK_0_2), tasksForState("store1", tasks, topicGroups)); assertEquals(mkSet(TASK_1_0, TASK_1_1, TASK_1_2), tasksForState("store2", tasks, topicGroups)); @@ -820,16 +821,16 @@ public void testAssignWithStates() { private static Set tasksForState(final String storeName, final List tasks, - final Map topicGroups) { + final Map topicGroups) { final String changelogTopic = ProcessorStateManager.storeChangelogTopic(APPLICATION_ID, storeName); final Set ids = new HashSet<>(); - for (final Map.Entry entry : topicGroups.entrySet()) { + for (final Map.Entry entry : topicGroups.entrySet()) { final Set stateChangelogTopics = entry.getValue().stateChangelogTopics.keySet(); if (stateChangelogTopics.contains(changelogTopic)) { for (final TaskId id : tasks) { - if (id.topicGroupId == entry.getKey()) { + if (id.topicGroupId == entry.getKey().nodeGroupId) { ids.add(id); } } @@ -2024,13 +2025,13 @@ public void shouldThrowTaskAssignmentExceptionWhenUnableToResolvePartitionCount( } private static class CorruptedInternalTopologyBuilder extends InternalTopologyBuilder { - private Map corruptedTopicGroups; + private Map corruptedTopicGroups; @Override - public synchronized Map topicGroups() { + public synchronized Map topicGroups() { if (corruptedTopicGroups == null) { corruptedTopicGroups = new HashMap<>(); - for (final Map.Entry topicGroupEntry : super.topicGroups().entrySet()) { + for (final Map.Entry topicGroupEntry : super.topicGroups().entrySet()) { final TopicsInfo originalInfo = topicGroupEntry.getValue(); corruptedTopicGroups.put( topicGroupEntry.getKey(), diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java index 58567529346b7..1afa76f22a6bf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentTestUtils.java @@ -25,6 +25,8 @@ import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task; +import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology; + import org.easymock.EasyMock; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; @@ -90,6 +92,10 @@ public final class AssignmentTestUtils { public static final TaskId NAMED_TASK_1_0 = new TaskId(1, 0, "topology1"); public static final TaskId NAMED_TASK_1_1 = new TaskId(1, 1, "topology1"); + public static final Subtopology SUBTOPOLOGY_0 = new Subtopology(0, null); + public static final Subtopology SUBTOPOLOGY_1 = new Subtopology(1, null); + public static final Subtopology SUBTOPOLOGY_2 = new Subtopology(2, null); + public static final Set EMPTY_TASKS = emptySet(); public static final Map EMPTY_CHANGELOG_END_OFFSETS = new HashMap<>(); From e69571aecc75d4b3d37c8b43d39cae83e250bedb Mon Sep 17 00:00:00 2001 From: Ryan Dielhenn <35785891+dielhennr@users.noreply.github.com> Date: Thu, 13 May 2021 12:08:18 -0700 Subject: [PATCH 148/155] KAFKA-12697: Add Global Topic and Partition count metrics to the Quorum Controller (#10679) Reviewers: Colin P. McCabe --- .../kafka/controller/ControllerMetrics.java | 8 +++ .../kafka/controller/QuorumController.java | 2 +- .../controller/QuorumControllerMetrics.java | 42 +++++++++++++++ .../controller/ReplicationControlManager.java | 22 +++++++- .../controller/MockControllerMetrics.java | 25 ++++++++- .../ReplicationControlManagerTest.java | 51 ++++++++++++++++++- 6 files changed, 145 insertions(+), 5 deletions(-) diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java index fd4f3befb805f..406a533e7d7e4 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerMetrics.java @@ -26,4 +26,12 @@ public interface ControllerMetrics { void updateEventQueueTime(long durationMs); void updateEventQueueProcessingTime(long durationMs); + + void setGlobalTopicsCount(int topicCount); + + int globalTopicsCount(); + + void setGlobalPartitionCount(int partitionCount); + + int globalPartitionCount(); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 2645b8615ce60..7995b4b5134ea 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -928,7 +928,7 @@ private QuorumController(LogContext logContext, this.snapshotGeneratorManager = new SnapshotGeneratorManager(snapshotWriterBuilder); this.replicationControl = new ReplicationControlManager(snapshotRegistry, logContext, defaultReplicationFactor, defaultNumPartitions, - configurationControl, clusterControl); + configurationControl, clusterControl, controllerMetrics); this.logManager = logManager; this.metaLogListener = new QuorumMetaLogListener(); this.curClaimEpoch = -1L; diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java index ad56faf3da99c..a9de1ff0a6836 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumControllerMetrics.java @@ -30,14 +30,24 @@ public final class QuorumControllerMetrics implements ControllerMetrics { "kafka.controller", "ControllerEventManager", "EventQueueTimeMs", null); private final static MetricName EVENT_QUEUE_PROCESSING_TIME_MS = new MetricName( "kafka.controller", "ControllerEventManager", "EventQueueProcessingTimeMs", null); + private final static MetricName GLOBAL_TOPIC_COUNT = new MetricName( + "kafka.controller", "KafkaController", "GlobalTopicCount", null); + private final static MetricName GLOBAL_PARTITION_COUNT = new MetricName( + "kafka.controller", "KafkaController", "GlobalPartitionCount", null); private volatile boolean active; + private volatile int globalTopicCount; + private volatile int globalPartitionCount; private final Gauge activeControllerCount; + private final Gauge globalPartitionCountGauge; + private final Gauge globalTopicCountGauge; private final Histogram eventQueueTime; private final Histogram eventQueueProcessingTime; public QuorumControllerMetrics(MetricsRegistry registry) { this.active = false; + this.globalTopicCount = 0; + this.globalPartitionCount = 0; this.activeControllerCount = registry.newGauge(ACTIVE_CONTROLLER_COUNT, new Gauge() { @Override public Integer value() { @@ -46,6 +56,18 @@ public Integer value() { }); this.eventQueueTime = registry.newHistogram(EVENT_QUEUE_TIME_MS, true); this.eventQueueProcessingTime = registry.newHistogram(EVENT_QUEUE_PROCESSING_TIME_MS, true); + this.globalTopicCountGauge = registry.newGauge(GLOBAL_TOPIC_COUNT, new Gauge() { + @Override + public Integer value() { + return globalTopicCount; + } + }); + this.globalPartitionCountGauge = registry.newGauge(GLOBAL_PARTITION_COUNT, new Gauge() { + @Override + public Integer value() { + return globalPartitionCount; + } + }); } @Override @@ -67,4 +89,24 @@ public void updateEventQueueTime(long durationMs) { public void updateEventQueueProcessingTime(long durationMs) { eventQueueTime.update(durationMs); } + + @Override + public void setGlobalTopicsCount(int topicCount) { + this.globalTopicCount = topicCount; + } + + @Override + public int globalTopicsCount() { + return this.globalTopicCount; + } + + @Override + public void setGlobalPartitionCount(int partitionCount) { + this.globalPartitionCount = partitionCount; + } + + @Override + public int globalPartitionCount() { + return this.globalPartitionCount; + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index b270e35205adb..48ab1c92a44a5 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -279,6 +279,11 @@ public String toString() { */ private final int defaultNumPartitions; + /** + * A count of the total number of partitions in the cluster. + */ + private int globalPartitionCount; + /** * A reference to the controller's configuration control manager. */ @@ -289,6 +294,11 @@ public String toString() { */ private final ClusterControlManager clusterControl; + /** + * A reference to the controller's metrics registry. + */ + private final ControllerMetrics controllerMetrics; + /** * Maps topic names to topic UUIDs. */ @@ -309,13 +319,16 @@ public String toString() { short defaultReplicationFactor, int defaultNumPartitions, ConfigurationControlManager configurationControl, - ClusterControlManager clusterControl) { + ClusterControlManager clusterControl, + ControllerMetrics controllerMetrics) { this.snapshotRegistry = snapshotRegistry; this.log = logContext.logger(ReplicationControlManager.class); this.defaultReplicationFactor = defaultReplicationFactor; this.defaultNumPartitions = defaultNumPartitions; this.configurationControl = configurationControl; + this.controllerMetrics = controllerMetrics; this.clusterControl = clusterControl; + this.globalPartitionCount = 0; this.topicsByName = new TimelineHashMap<>(snapshotRegistry, 0); this.topics = new TimelineHashMap<>(snapshotRegistry, 0); this.brokersToIsrs = new BrokersToIsrs(snapshotRegistry); @@ -325,6 +338,7 @@ public void replay(TopicRecord record) { topicsByName.put(record.name(), record.topicId()); topics.put(record.topicId(), new TopicControlInfo(record.name(), snapshotRegistry, record.topicId())); + controllerMetrics.setGlobalTopicsCount(topics.size()); log.info("Created topic {} with topic ID {}.", record.name(), record.topicId()); } @@ -343,6 +357,8 @@ public void replay(PartitionRecord record) { topicInfo.parts.put(record.partitionId(), newPartInfo); brokersToIsrs.update(record.topicId(), record.partitionId(), null, newPartInfo.isr, NO_LEADER, newPartInfo.leader); + globalPartitionCount++; + controllerMetrics.setGlobalPartitionCount(globalPartitionCount); } else if (!newPartInfo.equals(prevPartInfo)) { newPartInfo.maybeLogPartitionChange(log, description, prevPartInfo); topicInfo.parts.put(record.partitionId(), newPartInfo); @@ -389,9 +405,11 @@ public void replay(RemoveTopicRecord record) { for (int i = 0; i < partition.isr.length; i++) { brokersToIsrs.removeTopicEntryForBroker(topic.id, partition.isr[i]); } + globalPartitionCount--; } brokersToIsrs.removeTopicEntryForBroker(topic.id, NO_LEADER); - + controllerMetrics.setGlobalTopicsCount(topics.size()); + controllerMetrics.setGlobalPartitionCount(globalPartitionCount); log.info("Removed topic {} with ID {}.", topic.name, record.topicId()); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java index 4e6523e37f286..45a69d74c5573 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java +++ b/metadata/src/test/java/org/apache/kafka/controller/MockControllerMetrics.java @@ -17,12 +17,15 @@ package org.apache.kafka.controller; - public final class MockControllerMetrics implements ControllerMetrics { private volatile boolean active; + private volatile int topics; + private volatile int partitions; public MockControllerMetrics() { this.active = false; + this.topics = 0; + this.partitions = 0; } @Override @@ -44,4 +47,24 @@ public void updateEventQueueTime(long durationMs) { public void updateEventQueueProcessingTime(long durationMs) { // nothing to do } + + @Override + public void setGlobalTopicsCount(int topicCount) { + this.topics = topicCount; + } + + @Override + public int globalTopicsCount() { + return this.topics; + } + + @Override + public void setGlobalPartitionCount(int partitionCount) { + this.partitions = partitionCount; + } + + @Override + public int globalPartitionCount() { + return this.partitions; + } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index f7e0277eb87bf..934ca7a881a0a 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -88,6 +88,7 @@ private static class ReplicationControlTestContext { final ClusterControlManager clusterControl = new ClusterControlManager( logContext, time, snapshotRegistry, 1000, new SimpleReplicaPlacementPolicy(random)); + final ControllerMetrics metrics = new MockControllerMetrics(); final ConfigurationControlManager configurationControl = new ConfigurationControlManager( new LogContext(), snapshotRegistry, Collections.emptyMap()); final ReplicationControlManager replicationControl = new ReplicationControlManager(snapshotRegistry, @@ -95,7 +96,8 @@ private static class ReplicationControlTestContext { (short) 3, 1, configurationControl, - clusterControl); + clusterControl, + metrics); void replay(List records) throws Exception { ControllerTestUtils.replayAll(clusterControl, records); @@ -203,6 +205,53 @@ public void testCreateTopics() throws Exception { ctx.replicationControl.iterator(Long.MAX_VALUE)); } + @Test + public void testGlobalTopicAndPartitionMetrics() throws Exception { + ReplicationControlTestContext ctx = new ReplicationControlTestContext(); + ReplicationControlManager replicationControl = ctx.replicationControl; + CreateTopicsRequestData request = new CreateTopicsRequestData(); + request.topics().add(new CreatableTopic().setName("foo"). + setNumPartitions(1).setReplicationFactor((short) -1)); + + registerBroker(0, ctx); + unfenceBroker(0, ctx); + registerBroker(1, ctx); + unfenceBroker(1, ctx); + registerBroker(2, ctx); + unfenceBroker(2, ctx); + + List topicsToDelete = new ArrayList<>(); + + ControllerResult result = + replicationControl.createTopics(request); + topicsToDelete.add(result.response().topics().find("foo").topicId()); + + ControllerTestUtils.replayAll(replicationControl, result.records()); + assertEquals(1, ctx.metrics.globalTopicsCount()); + + request = new CreateTopicsRequestData(); + request.topics().add(new CreatableTopic().setName("bar"). + setNumPartitions(1).setReplicationFactor((short) -1)); + request.topics().add(new CreatableTopic().setName("baz"). + setNumPartitions(2).setReplicationFactor((short) -1)); + result = replicationControl.createTopics(request); + ControllerTestUtils.replayAll(replicationControl, result.records()); + assertEquals(3, ctx.metrics.globalTopicsCount()); + assertEquals(4, ctx.metrics.globalPartitionCount()); + + topicsToDelete.add(result.response().topics().find("baz").topicId()); + ControllerResult> deleteResult = replicationControl.deleteTopics(topicsToDelete); + ControllerTestUtils.replayAll(replicationControl, deleteResult.records()); + assertEquals(1, ctx.metrics.globalTopicsCount()); + assertEquals(1, ctx.metrics.globalPartitionCount()); + + Uuid topicToDelete = result.response().topics().find("bar").topicId(); + deleteResult = replicationControl.deleteTopics(Collections.singletonList(topicToDelete)); + ControllerTestUtils.replayAll(replicationControl, deleteResult.records()); + assertEquals(0, ctx.metrics.globalTopicsCount()); + assertEquals(0, ctx.metrics.globalPartitionCount()); + } + @Test public void testValidateNewTopicNames() { Map topicErrors = new HashMap<>(); From 6d1ae8bc007ffa1f3b3a9f949db6a99820dab75c Mon Sep 17 00:00:00 2001 From: Daniyar Yeralin Date: Thu, 13 May 2021 18:54:00 -0400 Subject: [PATCH 149/155] KAFKA-8326: Introduce List Serde (#6592) Introduce List serde for primitive types or custom serdes with a serializer and a deserializer according to KIP-466 Reviewers: Anna Sophie Blee-Goldman , Matthias J. Sax , John Roesler , Michael Noll --- checkstyle/import-control.xml | 1 + .../kafka/clients/CommonClientConfigs.java | 20 ++ .../serialization/ListDeserializer.java | 191 +++++++++++++ .../common/serialization/ListSerializer.java | 142 ++++++++++ .../kafka/common/serialization/Serdes.java | 30 +++ .../serialization/ListDeserializerTest.java | 251 ++++++++++++++++++ .../serialization/ListSerializerTest.java | 153 +++++++++++ .../serialization/SerializationTest.java | 188 +++++++++++++ docs/streams/developer-guide/datatypes.html | 4 +- docs/streams/upgrade-guide.html | 5 + .../apache/kafka/streams/StreamsConfig.java | 20 ++ 11 files changed, 1004 insertions(+), 1 deletion(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java create mode 100644 clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 318384fca9c0b..c98cfab787d9f 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -189,6 +189,7 @@ + diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 150dac1b73cb0..58075d628927e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -118,6 +118,26 @@ public class CommonClientConfigs { + "elapses the client will resend the request if necessary or fail the request if " + "retries are exhausted."; + public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS = "default.list.key.serde.inner"; + public static final String DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for key that implements the org.apache.kafka.common.serialization.Serde interface. " + + "This configuration will be read if and only if default.key.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde"; + + public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS = "default.list.value.serde.inner"; + public static final String DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC = "Default inner class of list serde for value that implements the org.apache.kafka.common.serialization.Serde interface. " + + "This configuration will be read if and only if default.value.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde"; + + public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS = "default.list.key.serde.type"; + public static final String DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC = "Default class for key that implements the java.util.List interface. " + + "This configuration will be read if and only if default.key.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde " + + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "'"; + + public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS = "default.list.value.serde.type"; + public static final String DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC = "Default class for value that implements the java.util.List interface. " + + "This configuration will be read if and only if default.value.serde configuration is set to org.apache.kafka.common.serialization.Serdes.ListSerde " + + "Note when list serde class is used, one needs to set the inner serde class that implements the org.apache.kafka.common.serialization.Serde interface via '" + + DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "'"; + public static final String GROUP_ID_CONFIG = "group.id"; public static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java new file mode 100644 index 0000000000000..272cbad52a74c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListDeserializer.java @@ -0,0 +1,191 @@ +/* + * 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.common.serialization; + +import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; + +import java.io.ByteArrayInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Serdes.ListSerde; +import org.apache.kafka.common.utils.Utils; + +public class ListDeserializer implements Deserializer> { + + private static final Map>, Integer> FIXED_LENGTH_DESERIALIZERS = mkMap( + mkEntry(ShortDeserializer.class, Short.BYTES), + mkEntry(IntegerDeserializer.class, Integer.BYTES), + mkEntry(FloatDeserializer.class, Float.BYTES), + mkEntry(LongDeserializer.class, Long.BYTES), + mkEntry(DoubleDeserializer.class, Double.BYTES), + mkEntry(UUIDDeserializer.class, 36) + ); + + private Deserializer inner; + private Class listClass; + private Integer primitiveSize; + + public ListDeserializer() {} + + public > ListDeserializer(Class listClass, Deserializer inner) { + if (listClass == null || inner == null) { + throw new IllegalArgumentException("ListDeserializer requires both \"listClass\" and \"innerDeserializer\" parameters to be provided during initialization"); + } + this.listClass = listClass; + this.inner = inner; + this.primitiveSize = FIXED_LENGTH_DESERIALIZERS.get(inner.getClass()); + } + + public Deserializer innerDeserializer() { + return inner; + } + + @Override + public void configure(Map configs, boolean isKey) { + if (listClass != null || inner != null) { + throw new ConfigException("List deserializer was already initialized using a non-default constructor"); + } + configureListClass(configs, isKey); + configureInnerSerde(configs, isKey); + } + + private void configureListClass(Map configs, boolean isKey) { + String listTypePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS; + final Object listClassOrName = configs.get(listTypePropertyName); + if (listClassOrName == null) { + throw new ConfigException("Not able to determine the list class because it was neither passed via the constructor nor set in the config."); + } + try { + if (listClassOrName instanceof String) { + listClass = Utils.loadClass((String) listClassOrName, Object.class); + } else if (listClassOrName instanceof Class) { + listClass = (Class) listClassOrName; + } else { + throw new KafkaException("Could not determine the list class instance using \"" + listTypePropertyName + "\" property."); + } + } catch (final ClassNotFoundException e) { + throw new ConfigException(listTypePropertyName, listClassOrName, "Deserializer's list class \"" + listClassOrName + "\" could not be found."); + } + } + + @SuppressWarnings("unchecked") + private void configureInnerSerde(Map configs, boolean isKey) { + String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + final Object innerSerdeClassOrName = configs.get(innerSerdePropertyName); + if (innerSerdeClassOrName == null) { + throw new ConfigException("Not able to determine the inner serde class because it was neither passed via the constructor nor set in the config."); + } + try { + if (innerSerdeClassOrName instanceof String) { + inner = Utils.newInstance((String) innerSerdeClassOrName, Serde.class).deserializer(); + } else if (innerSerdeClassOrName instanceof Class) { + inner = (Deserializer) ((Serde) Utils.newInstance((Class) innerSerdeClassOrName)).deserializer(); + } else { + throw new KafkaException("Could not determine the inner serde class instance using \"" + innerSerdePropertyName + "\" property."); + } + inner.configure(configs, isKey); + primitiveSize = FIXED_LENGTH_DESERIALIZERS.get(inner.getClass()); + } catch (final ClassNotFoundException e) { + throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Deserializer's inner serde class \"" + innerSerdeClassOrName + "\" could not be found."); + } + } + + @SuppressWarnings("unchecked") + private List createListInstance(int listSize) { + try { + Constructor> listConstructor; + try { + listConstructor = (Constructor>) listClass.getConstructor(Integer.TYPE); + return listConstructor.newInstance(listSize); + } catch (NoSuchMethodException e) { + listConstructor = (Constructor>) listClass.getConstructor(); + return listConstructor.newInstance(); + } + } catch (InstantiationException | IllegalAccessException | NoSuchMethodException | + IllegalArgumentException | InvocationTargetException e) { + throw new KafkaException("Could not construct a list instance of \"" + listClass.getCanonicalName() + "\"", e); + } + } + + private SerializationStrategy parseSerializationStrategyFlag(final int serializationStrategyFlag) throws IOException { + if (serializationStrategyFlag < 0 || serializationStrategyFlag >= SerializationStrategy.VALUES.length) { + throw new SerializationException("Invalid serialization strategy flag value"); + } + return SerializationStrategy.VALUES[serializationStrategyFlag]; + } + + private List deserializeNullIndexList(final DataInputStream dis) throws IOException { + int nullIndexListSize = dis.readInt(); + List nullIndexList = new ArrayList<>(nullIndexListSize); + while (nullIndexListSize != 0) { + nullIndexList.add(dis.readInt()); + nullIndexListSize--; + } + return nullIndexList; + } + + @Override + public List deserialize(String topic, byte[] data) { + if (data == null) { + return null; + } + try (final DataInputStream dis = new DataInputStream(new ByteArrayInputStream(data))) { + SerializationStrategy serStrategy = parseSerializationStrategyFlag(dis.readByte()); + List nullIndexList = null; + if (serStrategy == SerializationStrategy.CONSTANT_SIZE) { + // In CONSTANT_SIZE strategy, indexes of null entries are decoded from a null index list + nullIndexList = deserializeNullIndexList(dis); + } + final int size = dis.readInt(); + List deserializedList = createListInstance(size); + for (int i = 0; i < size; i++) { + int entrySize = serStrategy == SerializationStrategy.CONSTANT_SIZE ? primitiveSize : dis.readInt(); + if (entrySize == ListSerde.NULL_ENTRY_VALUE || (nullIndexList != null && nullIndexList.contains(i))) { + deserializedList.add(null); + continue; + } + byte[] payload = new byte[entrySize]; + if (dis.read(payload) == -1) { + throw new SerializationException("End of the stream was reached prematurely"); + } + deserializedList.add(inner.deserialize(topic, payload)); + } + return deserializedList; + } catch (IOException e) { + throw new KafkaException("Unable to deserialize into a List", e); + } + } + + @Override + public void close() { + if (inner != null) { + inner.close(); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java new file mode 100644 index 0000000000000..6274c9d3f7b53 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/ListSerializer.java @@ -0,0 +1,142 @@ +/* + * 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.common.serialization; + +import java.util.ArrayList; +import java.util.Iterator; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.utils.Utils; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.common.serialization.Serdes.ListSerde.SerializationStrategy; + +public class ListSerializer implements Serializer> { + + private static final List>> FIXED_LENGTH_SERIALIZERS = Arrays.asList( + ShortSerializer.class, + IntegerSerializer.class, + FloatSerializer.class, + LongSerializer.class, + DoubleSerializer.class, + UUIDSerializer.class); + + private Serializer inner; + private SerializationStrategy serStrategy; + + public ListSerializer() {} + + public ListSerializer(Serializer inner) { + if (inner == null) { + throw new IllegalArgumentException("ListSerializer requires \"serializer\" parameter to be provided during initialization"); + } + this.inner = inner; + this.serStrategy = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()) ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; + } + + public Serializer getInnerSerializer() { + return inner; + } + + @SuppressWarnings("unchecked") + @Override + public void configure(Map configs, boolean isKey) { + if (inner != null) { + throw new ConfigException("List serializer was already initialized using a non-default constructor"); + } + final String innerSerdePropertyName = isKey ? CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS : CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS; + final Object innerSerdeClassOrName = configs.get(innerSerdePropertyName); + if (innerSerdeClassOrName == null) { + throw new ConfigException("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config."); + } + try { + if (innerSerdeClassOrName instanceof String) { + inner = Utils.newInstance((String) innerSerdeClassOrName, Serde.class).serializer(); + } else if (innerSerdeClassOrName instanceof Class) { + inner = (Serializer) ((Serde) Utils.newInstance((Class) innerSerdeClassOrName)).serializer(); + } else { + throw new KafkaException("Could not create a serializer class instance using \"" + innerSerdePropertyName + "\" property."); + } + inner.configure(configs, isKey); + serStrategy = FIXED_LENGTH_SERIALIZERS.contains(inner.getClass()) ? SerializationStrategy.CONSTANT_SIZE : SerializationStrategy.VARIABLE_SIZE; + } catch (final ClassNotFoundException e) { + throw new ConfigException(innerSerdePropertyName, innerSerdeClassOrName, "Serializer class " + innerSerdeClassOrName + " could not be found."); + } + } + + private void serializeNullIndexList(final DataOutputStream out, List data) throws IOException { + int i = 0; + List nullIndexList = new ArrayList<>(); + for (Iterator it = data.listIterator(); it.hasNext(); i++) { + if (it.next() == null) { + nullIndexList.add(i); + } + } + out.writeInt(nullIndexList.size()); + for (int nullIndex : nullIndexList) { + out.writeInt(nullIndex); + } + } + + @Override + public byte[] serialize(String topic, List data) { + if (data == null) { + return null; + } + try (final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final DataOutputStream out = new DataOutputStream(baos)) { + out.writeByte(serStrategy.ordinal()); // write serialization strategy flag + if (serStrategy == SerializationStrategy.CONSTANT_SIZE) { + // In CONSTANT_SIZE strategy, indexes of null entries are encoded in a null index list + serializeNullIndexList(out, data); + } + final int size = data.size(); + out.writeInt(size); + for (Inner entry : data) { + if (entry == null) { + if (serStrategy == SerializationStrategy.VARIABLE_SIZE) { + out.writeInt(Serdes.ListSerde.NULL_ENTRY_VALUE); + } + } else { + final byte[] bytes = inner.serialize(topic, entry); + if (serStrategy == SerializationStrategy.VARIABLE_SIZE) { + out.writeInt(bytes.length); + } + out.write(bytes); + } + } + return baos.toByteArray(); + } catch (IOException e) { + throw new KafkaException("Failed to serialize List", e); + } + } + + @Override + public void close() { + if (inner != null) { + inner.close(); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index 347bf8713ece8..4a150e0c022ba 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.utils.Bytes; import java.nio.ByteBuffer; +import java.util.List; import java.util.Map; import java.util.UUID; @@ -125,6 +126,27 @@ public UUIDSerde() { } } + static public final class ListSerde extends WrapperSerde> { + + final static int NULL_ENTRY_VALUE = -1; + + enum SerializationStrategy { + CONSTANT_SIZE, + VARIABLE_SIZE; + + public static final SerializationStrategy[] VALUES = SerializationStrategy.values(); + } + + public ListSerde() { + super(new ListSerializer<>(), new ListDeserializer<>()); + } + + public > ListSerde(Class listClass, Serde serde) { + super(new ListSerializer<>(serde.serializer()), new ListDeserializer<>(listClass, serde.deserializer())); + } + + } + @SuppressWarnings("unchecked") static public Serde serdeFrom(Class type) { if (String.class.isAssignableFrom(type)) { @@ -265,4 +287,12 @@ static public Serde ByteArray() { static public Serde Void() { return new VoidSerde(); } + + /* + * A serde for {@code List} type + */ + static public , Inner> Serde> ListSerde(Class listClass, Serde innerSerde) { + return new ListSerde<>(listClass, innerSerde); + } + } diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java new file mode 100644 index 0000000000000..aff01e3fe8a89 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListDeserializerTest.java @@ -0,0 +1,251 @@ +/* + * 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.common.serialization; + +import static org.junit.jupiter.api.Assertions.assertEquals; +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; +import org.apache.kafka.common.config.ConfigException; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Test; + +@SuppressWarnings("unchecked") +public class ListDeserializerTest { + private final ListDeserializer listDeserializer = new ListDeserializer<>(); + private final Map props = new HashMap<>(); + private final String nonExistingClass = "non.existing.class"; + private static class FakeObject { + } + + @Test + public void testListKeyDeserializerNoArgConstructorsWithClassNames() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class.getName()); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); + 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"); + } + + @Test + public void testListValueDeserializerNoArgConstructorsWithClassNames() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class.getName()); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.IntegerSerde.class.getName()); + 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"); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsWithClassObjects() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + 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"); + } + + @Test + public void testListValueDeserializerNoArgConstructorsWithClassObjects() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + 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"); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingInnerClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertEquals("Not able to determine the inner serde class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingInnerClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Not able to determine the inner serde class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingTypeClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertEquals("Not able to determine the list class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueMissingTypeClassProp() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Not able to determine the list class because " + + "it was neither passed via the constructor nor set in the config.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidTypeClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, new FakeObject()); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, true) + ); + assertEquals("Could not determine the list class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + "\" property.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidTypeClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, new FakeObject()); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Could not determine the list class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + "\" property.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidInnerClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, true) + ); + assertEquals("Could not determine the inner serde class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidInnerClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Could not determine the inner serde class instance using " + + "\"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueListClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, nonExistingClass); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertEquals("Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS + ": Deserializer's list class " + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueListClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, nonExistingClass); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS + ": Deserializer's list class " + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerNoArgConstructorsShouldThrowConfigExceptionDueInnerSerdeClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, nonExistingClass); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, true) + ); + assertEquals("Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Deserializer's inner serde class " + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); + } + + @Test + public void testListValueDeserializerNoArgConstructorsShouldThrowConfigExceptionDueInnerSerdeClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, nonExistingClass); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> listDeserializer.configure(props, false) + ); + assertEquals("Invalid value " + nonExistingClass + " for configuration " + + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Deserializer's inner serde class " + + "\"" + nonExistingClass + "\" could not be found.", exception.getMessage()); + } + + @Test + public void testListKeyDeserializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListDeserializer initializedListDeserializer = new ListDeserializer<>(ArrayList.class, + Serdes.Integer().deserializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListDeserializer.configure(props, true) + ); + assertEquals("List deserializer was already initialized using a non-default constructor", exception.getMessage()); + } + + @Test + public void testListValueDeserializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, ArrayList.class); + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListDeserializer initializedListDeserializer = new ListDeserializer<>(ArrayList.class, + Serdes.Integer().deserializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListDeserializer.configure(props, true) + ); + assertEquals("List deserializer was already initialized using a non-default constructor", exception.getMessage()); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java new file mode 100644 index 0000000000000..a8ab191cad203 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/serialization/ListSerializerTest.java @@ -0,0 +1,153 @@ +/* + * 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.common.serialization; + +import static org.junit.jupiter.api.Assertions.assertEquals; +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; +import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + + +public class ListSerializerTest { + private final ListSerializer listSerializer = new ListSerializer<>(); + private final Map props = new HashMap<>(); + private final String nonExistingClass = "non.existing.class"; + private static class FakeObject { + } + + @Test + public void testListKeySerializerNoArgConstructorsWithClassName() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); + 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"); + } + + @Test + public void testListValueSerializerNoArgConstructorsWithClassName() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class.getName()); + 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"); + } + + @Test + public void testListKeySerializerNoArgConstructorsWithClassObject() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + 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"); + } + + @Test + public void testListValueSerializerNoArgConstructorsWithClassObject() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + 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"); + } + + @Test + public void testListSerializerNoArgConstructorsShouldThrowConfigExceptionDueMissingProp() { + ConfigException exception = assertThrows( + ConfigException.class, + () -> listSerializer.configure(props, true) + ); + assertEquals("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.", exception.getMessage()); + + exception = assertThrows( + ConfigException.class, + () -> listSerializer.configure(props, false) + ); + assertEquals("Not able to determine the serializer class because it was neither passed via the constructor nor set in the config.", exception.getMessage()); + } + + @Test + public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, true) + ); + assertEquals("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); + } + + @Test + public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDueInvalidClass() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, new FakeObject()); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, false) + ); + assertEquals("Could not create a serializer class instance using \"" + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + "\" property.", exception.getMessage()); + } + + @Test + public void testListKeySerializerNoArgConstructorsShouldThrowKafkaExceptionDueClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, nonExistingClass); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, true) + ); + assertEquals("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.", exception.getMessage()); + } + + @Test + public void testListValueSerializerNoArgConstructorsShouldThrowKafkaExceptionDueClassNotFound() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, nonExistingClass); + final KafkaException exception = assertThrows( + KafkaException.class, + () -> listSerializer.configure(props, false) + ); + assertEquals("Invalid value non.existing.class for configuration " + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS + ": Serializer class " + nonExistingClass + " could not be found.", exception.getMessage()); + } + + @Test + public void testListKeySerializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListSerializer initializedListSerializer = new ListSerializer<>(Serdes.Integer().serializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListSerializer.configure(props, true) + ); + assertEquals("List serializer was already initialized using a non-default constructor", exception.getMessage()); + } + + @Test + public void testListValueSerializerShouldThrowConfigExceptionDueAlreadyInitialized() { + props.put(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, Serdes.StringSerde.class); + final ListSerializer initializedListSerializer = new ListSerializer<>(Serdes.Integer().serializer()); + final ConfigException exception = assertThrows( + ConfigException.class, + () -> initializedListSerializer.configure(props, false) + ); + assertEquals("List serializer was already initialized using a non-default constructor", exception.getMessage()); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 0446bafc2fbe4..85c09dd17ae09 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -27,7 +27,11 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.Stack; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -106,6 +110,190 @@ public void stringSerdeShouldSupportDifferentEncodings() { } } + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldReturnEmptyCollection() { + List testData = Arrays.asList(); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get empty collection after serialization and deserialization on an empty list"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldReturnNull() { + List testData = null; + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get null after serialization and deserialization on an empty list"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripIntPrimitiveInput() { + List testData = Arrays.asList(1, 2, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of integer primitives after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForIntPrimitiveInput() { + List testData = Arrays.asList(1, 2, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals(21, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 21 bytes after serialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripShortPrimitiveInput() { + List testData = Arrays.asList((short) 1, (short) 2, (short) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of short primitives after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForShortPrimitiveInput() { + List testData = Arrays.asList((short) 1, (short) 2, (short) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Short()); + assertEquals(15, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 15 bytes after serialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripFloatPrimitiveInput() { + List testData = Arrays.asList((float) 1, (float) 2, (float) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Float()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of float primitives after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForFloatPrimitiveInput() { + List testData = Arrays.asList((float) 1, (float) 2, (float) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Float()); + assertEquals(21, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 21 bytes after serialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripLongPrimitiveInput() { + List testData = Arrays.asList((long) 1, (long) 2, (long) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Long()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of long primitives after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForLongPrimitiveInput() { + List testData = Arrays.asList((long) 1, (long) 2, (long) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Long()); + assertEquals(33, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 33 bytes after serialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripDoublePrimitiveInput() { + List testData = Arrays.asList((double) 1, (double) 2, (double) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Double()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of double primitives after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForDoublePrimitiveInput() { + List testData = Arrays.asList((double) 1, (double) 2, (double) 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Double()); + assertEquals(33, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 33 bytes after serialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripUUIDInput() { + List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.UUID()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of UUID after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeSerializerShouldReturnByteArrayOfFixedSizeForUUIDInput() { + List testData = Arrays.asList(UUID.randomUUID(), UUID.randomUUID(), UUID.randomUUID()); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.UUID()); + assertEquals(117, listSerde.serializer().serialize(topic, testData).length, + "Should get length of 117 bytes after serialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripNonPrimitiveInput() { + List testData = Arrays.asList("A", "B", "C"); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of strings list after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripPrimitiveInputWithNullEntries() { + List testData = Arrays.asList(1, null, 3); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.Integer()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of integer primitives with null entries " + + "after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldRoundtripNonPrimitiveInputWithNullEntries() { + List testData = Arrays.asList("A", null, "C"); + Serde> listSerde = Serdes.ListSerde(ArrayList.class, Serdes.String()); + assertEquals(testData, + listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)), + "Should get the original collection of strings list with null entries " + + "after serialization and deserialization"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldReturnLinkedList() { + List testData = new LinkedList<>(); + Serde> listSerde = Serdes.ListSerde(LinkedList.class, Serdes.Integer()); + assertTrue(listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) + instanceof LinkedList, "Should return List instance of type LinkedList"); + } + + @SuppressWarnings("unchecked") + @Test + public void listSerdeShouldReturnStack() { + List testData = new Stack<>(); + Serde> listSerde = Serdes.ListSerde(Stack.class, Serdes.Integer()); + assertTrue(listSerde.deserializer().deserialize(topic, listSerde.serializer().serialize(topic, testData)) + instanceof Stack, "Should return List instance of type Stack"); + } + @Test public void floatDeserializerShouldThrowSerializationExceptionOnZeroBytes() { try (Serde serde = Serdes.Float()) { diff --git a/docs/streams/developer-guide/datatypes.html b/docs/streams/developer-guide/datatypes.html index e6930bc31ce83..2201b5b69d35a 100644 --- a/docs/streams/developer-guide/datatypes.html +++ b/docs/streams/developer-guide/datatypes.html @@ -140,10 +140,12 @@

      Primitive and basic types

      - + + +
      replication.factorHighMedium The replication factor for changelog topics and repartition topics created by the application. - If your broker cluster is on version 2.4 or newer, you can set -1 to use the broker default replication factor.1-1
      retry.backoff.ms MediumUUID Serdes.UUID()
      Void Serdes.Void()
      ListSerdes.ListSerde()
      diff --git a/docs/streams/upgrade-guide.html b/docs/streams/upgrade-guide.html index de747bb862f33..3ae4573d2071f 100644 --- a/docs/streams/upgrade-guide.html +++ b/docs/streams/upgrade-guide.html @@ -150,6 +150,11 @@

      Streams API (meaning: use broker default replication factor). The replication.factor value of -1 requires broker version 2.4 or newer.

      +

      The new serde type was introduced ListSerde:

      +
        +
      • Added class ListSerde to (de)serialize List-based objects
      • +
      • Introduced ListSerializer and ListDeserializer to power the new functionality
      • +

      Streams API changes in 2.8.0

      diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 0e9e6a149f9b6..274322ee218d4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -643,6 +643,26 @@ public class StreamsConfig extends AbstractConfig { Serdes.ByteArraySerde.class.getName(), Importance.MEDIUM, DEFAULT_KEY_SERDE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_INNER_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_KEY_SERDE_TYPE_CLASS_DOC) + .define(CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS, + Type.CLASS, + null, + Importance.MEDIUM, + CommonClientConfigs.DEFAULT_LIST_VALUE_SERDE_TYPE_CLASS_DOC) .define(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, Type.CLASS, DefaultProductionExceptionHandler.class.getName(), From 4153e754f1a4ebbd9a3d10be8bf75a7057c82f1d Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Thu, 13 May 2021 16:16:35 -0700 Subject: [PATCH 150/155] MINOR: prevent cleanup() from being called while Streams is still shutting down (#10666) Currently KafkaStreams#cleanUp only throw an IllegalStateException if the state is RUNNING or REBALANCING, however the application could be in the process of shutting down in which case StreamThreads may still be running. We should also throw if the state is PENDING_ERROR or PENDING_SHUTDOWN Reviewers: Walker Carlson , Guozhang Wang --- .../apache/kafka/streams/KafkaStreams.java | 2 +- .../processor/internals/StateDirectory.java | 54 ++++++---------- .../kafka/streams/KafkaStreamsTest.java | 64 ++++++++++++------- .../StandbyTaskEOSIntegrationTest.java | 4 +- 4 files changed, 66 insertions(+), 58 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index f3f8e8ea64536..8446c35240964 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1444,7 +1444,7 @@ public synchronized boolean close(final Duration timeout) throws IllegalArgument * @throws StreamsException if cleanup failed */ public void cleanUp() { - if (isRunningOrRebalancing()) { + if (!(state == State.CREATED || state == State.NOT_RUNNING || state == State.ERROR)) { throw new IllegalStateException("Cannot clean up while running."); } stateDirectory.clean(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index 7f01264bbdf38..5f83b6a00546f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -351,7 +351,7 @@ public void close() { public synchronized void clean() { try { - cleanRemovedTasksCalledByUser(); + cleanStateAndTaskDirectoriesCalledByUser(); } catch (final Exception e) { throw new StreamsException(e); } @@ -413,43 +413,31 @@ private void cleanRemovedTasksCalledByCleanerThread(final long cleanupDelayMs) { } } - private void cleanRemovedTasksCalledByUser() throws Exception { + private void cleanStateAndTaskDirectoriesCalledByUser() throws Exception { + if (!lockedTasksToOwner.isEmpty()) { + log.warn("Found some still-locked task directories when user requested to cleaning up the state, " + + "since Streams is not running any more these will be ignored to complete the cleanup"); + } final AtomicReference firstException = new AtomicReference<>(); for (final File taskDir : listAllTaskDirectories()) { final String dirName = taskDir.getName(); final TaskId id = TaskId.parseTaskDirectoryName(dirName, null); - if (!lockedTasksToOwner.containsKey(id)) { - try { - if (lock(id)) { - log.info("{} Deleting state directory {} for task {} as user calling cleanup.", - logPrefix(), dirName, id); - Utils.delete(taskDir); - } else { - log.warn("{} Could not get lock for state directory {} for task {} as user calling cleanup.", - logPrefix(), dirName, id); - } - } catch (final OverlappingFileLockException | IOException exception) { - log.error( - String.format("%s Failed to delete state directory %s for task %s with exception:", - logPrefix(), dirName, id), - exception - ); - firstException.compareAndSet(null, exception); - } finally { - try { - unlock(id); - // for manual user call, stream threads are not running so it is safe to delete - // the whole directory - Utils.delete(taskDir); - } catch (final IOException exception) { - log.error( - String.format("%s Failed to release lock on state directory %s for task %s with exception:", - logPrefix(), dirName, id), - exception - ); - firstException.compareAndSet(null, exception); - } + try { + log.info("{} Deleting state directory {} for task {} as user calling cleanup.", + logPrefix(), dirName, id); + + if (lockedTasksToOwner.containsKey(id)) { + log.warn("{} Task {} in state directory {} was still locked by {}", + logPrefix(), dirName, id, lockedTasksToOwner.get(id)); } + Utils.delete(taskDir); + } catch (final IOException exception) { + log.error( + String.format("%s Failed to delete state directory %s for task %s with exception:", + logPrefix(), dirName, id), + exception + ); + firstException.compareAndSet(null, exception); } } final Exception exception = firstException.get(); diff --git a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java index 63fa5e7e0c378..492a145d33ba2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -34,6 +34,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.KafkaStreams.State; import org.apache.kafka.streams.errors.StreamsNotStartedException; import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler; import org.apache.kafka.streams.errors.TopologyException; @@ -96,6 +97,7 @@ import static org.apache.kafka.streams.state.QueryableStoreTypes.keyValueStore; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; +import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.easymock.EasyMock.anyInt; import static org.easymock.EasyMock.anyLong; import static org.easymock.EasyMock.anyObject; @@ -235,8 +237,8 @@ private void prepareStreams() throws Exception { EasyMock.expect(StreamThread.eosEnabled(anyObject(StreamsConfig.class))).andReturn(false).anyTimes(); EasyMock.expect(StreamThread.processingMode(anyObject(StreamsConfig.class))).andReturn(StreamThread.ProcessingMode.AT_LEAST_ONCE).anyTimes(); - EasyMock.expect(streamThreadOne.getId()).andReturn(0L).anyTimes(); - EasyMock.expect(streamThreadTwo.getId()).andReturn(1L).anyTimes(); + EasyMock.expect(streamThreadOne.getId()).andReturn(1L).anyTimes(); + EasyMock.expect(streamThreadTwo.getId()).andReturn(2L).anyTimes(); prepareStreamThread(streamThreadOne, 1, true); prepareStreamThread(streamThreadTwo, 2, false); @@ -298,7 +300,9 @@ private void prepareStreams() throws Exception { ); } - private void prepareStreamThread(final StreamThread thread, final int threadId, final boolean terminable) throws Exception { + private void prepareStreamThread(final StreamThread thread, + final int threadId, + final boolean terminable) throws Exception { final AtomicReference state = new AtomicReference<>(StreamThread.State.CREATED); EasyMock.expect(thread.state()).andAnswer(state::get).anyTimes(); @@ -351,19 +355,21 @@ private void prepareStreamThread(final StreamThread thread, final int threadId, producer.close(); } state.set(StreamThread.State.DEAD); + threadStatelistenerCapture.getValue().onChange(thread, StreamThread.State.PENDING_SHUTDOWN, StreamThread.State.RUNNING); threadStatelistenerCapture.getValue().onChange(thread, StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN); return null; }).anyTimes(); EasyMock.expect(thread.isRunning()).andReturn(state.get() == StreamThread.State.RUNNING).anyTimes(); thread.join(); - if (terminable) + if (terminable) { EasyMock.expectLastCall().anyTimes(); - else + } else { EasyMock.expectLastCall().andAnswer(() -> { - Thread.sleep(50L); + Thread.sleep(2000L); return null; }).anyTimes(); + } EasyMock.expect(thread.activeTasks()).andStubReturn(emptyList()); EasyMock.expect(thread.allTasks()).andStubReturn(Collections.emptyMap()); @@ -387,7 +393,7 @@ public void stateShouldTransitToRunningIfNonDeadThreadsBackToRunning() throws In streams.start(); - TestUtils.waitForCondition( + waitForCondition( () -> streamsStateListener.numChanges == 2, "Streams never started."); Assert.assertEquals(KafkaStreams.State.RUNNING, streams.state()); @@ -439,7 +445,7 @@ public void stateShouldTransitToRunningIfNonDeadThreadsBackToRunning() throws In streams.close(); - TestUtils.waitForCondition( + waitForCondition( () -> streamsStateListener.numChanges == 6, "Streams never closed."); Assert.assertEquals(KafkaStreams.State.NOT_RUNNING, streams.state()); @@ -453,7 +459,7 @@ public void shouldCleanupResourcesOnCloseWithoutPreviousStart() throws Exception final KafkaStreams streams = new KafkaStreams(builder.build(), props, supplier, time); streams.close(); - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.NOT_RUNNING, "Streams never stopped."); @@ -476,18 +482,18 @@ public void testStateThreadClose() throws Exception { assertEquals(streams.state(), KafkaStreams.State.CREATED); streams.start(); - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.RUNNING, "Streams never started."); for (int i = 0; i < NUM_THREADS; i++) { final StreamThread tmpThread = streams.threads.get(i); tmpThread.shutdown(); - TestUtils.waitForCondition(() -> tmpThread.state() == StreamThread.State.DEAD, + waitForCondition(() -> tmpThread.state() == StreamThread.State.DEAD, "Thread never stopped."); streams.threads.get(i).join(); } - TestUtils.waitForCondition( + waitForCondition( () -> streams.localThreadsMetadata().stream().allMatch(t -> t.threadState().equals("DEAD")), "Streams never stopped" ); @@ -495,7 +501,7 @@ public void testStateThreadClose() throws Exception { streams.close(); } - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.NOT_RUNNING, "Streams never stopped."); @@ -511,17 +517,17 @@ public void testStateGlobalThreadClose() throws Exception { try { streams.start(); - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.RUNNING, "Streams never started."); final GlobalStreamThread globalStreamThread = streams.globalStreamThread; globalStreamThread.shutdown(); - TestUtils.waitForCondition( + waitForCondition( () -> globalStreamThread.state() == GlobalStreamThread.State.DEAD, "Thread never stopped."); globalStreamThread.join(); - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.PENDING_ERROR, "Thread never stopped." ); @@ -529,7 +535,7 @@ public void testStateGlobalThreadClose() throws Exception { streams.close(); } - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.ERROR, "Thread never stopped." ); @@ -568,7 +574,7 @@ public void shouldAddThreadWhenRunning() throws InterruptedException { final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); streams.start(); final int oldSize = streams.threads.size(); - TestUtils.waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, 15L, "wait until running"); + waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, 15L, "wait until running"); assertThat(streams.addStreamThread(), equalTo(Optional.of("processId-StreamThread-" + 2))); assertThat(streams.threads.size(), equalTo(oldSize + 1)); } @@ -616,7 +622,7 @@ public void shouldRemoveThread() throws InterruptedException { final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); streams.start(); final int oldSize = streams.threads.size(); - TestUtils.waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, 15L, + waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, 15L, "Kafka Streams client did not reach state RUNNING"); assertThat(streams.removeStreamThread(), equalTo(Optional.of("processId-StreamThread-" + 1))); assertThat(streams.threads.size(), equalTo(oldSize - 1)); @@ -707,7 +713,7 @@ public void shouldAllowCleanupBeforeStartAndAfterClose() { public void shouldThrowOnCleanupWhileRunning() throws InterruptedException { final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); streams.start(); - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.RUNNING, "Streams never started."); @@ -719,6 +725,20 @@ public void shouldThrowOnCleanupWhileRunning() throws InterruptedException { } } + @Test + public void shouldThrowOnCleanupWhileShuttingDown() throws InterruptedException { + final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); + streams.start(); + waitForCondition( + () -> streams.state() == KafkaStreams.State.RUNNING, + "Streams never started."); + + streams.close(Duration.ZERO); + assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true)); + assertThrows(IllegalStateException.class, streams::cleanUp); + assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true)); + } + @Test public void shouldNotGetAllTasksWhenNotRunning() throws InterruptedException { try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) { @@ -1027,13 +1047,13 @@ public void shouldTransitToRunningWithGlobalOnlyTopology() throws InterruptedExc assertEquals(streams.state(), KafkaStreams.State.CREATED); streams.start(); - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.RUNNING, "Streams never started, state is " + streams.state()); streams.close(); - TestUtils.waitForCondition( + waitForCondition( () -> streams.state() == KafkaStreams.State.NOT_RUNNING, "Streams never stopped."); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java index 5ba607528faca..d762370489c51 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StandbyTaskEOSIntegrationTest.java @@ -170,8 +170,8 @@ public void shouldSurviveWithOneTaskAsStandby() throws Exception { // Wait for the record to be processed assertTrue(instanceLatch.await(15, TimeUnit.SECONDS)); - streamInstanceOne.close(Duration.ZERO); - streamInstanceTwo.close(Duration.ZERO); + streamInstanceOne.close(); + streamInstanceTwo.close(); streamInstanceOne.cleanUp(); streamInstanceTwo.cleanUp(); From 29c55fdbbc331bbede17908ccc878953a1b15d87 Mon Sep 17 00:00:00 2001 From: Chia-Ping Tsai Date: Fri, 14 May 2021 13:51:31 +0800 Subject: [PATCH 151/155] MINOR: set replication.factor to 1 to make StreamsBrokerCompatibilityService work with old broker (#10673) Reviewers: Matthias J. Sax , A. Sophie Blee-Goldman --- tests/kafkatest/services/streams.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index dc6facca26841..4342c96d1a218 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -466,6 +466,15 @@ def __init__(self, test_context, kafka, processingMode): "org.apache.kafka.streams.tests.BrokerCompatibilityTest", processingMode) + def prop_file(self): + properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT, + streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(), + # the old broker (< 2.4) does not support configuration replication.factor=-1 + "replication.factor": 1} + + cfg = KafkaConfig(**properties) + return cfg.render() + class StreamsBrokerDownResilienceService(StreamsTestBaseService): def __init__(self, test_context, kafka, configs): From db3e5e2c0de367ffcfe4078359d6d208ba722581 Mon Sep 17 00:00:00 2001 From: Cong Ding Date: Fri, 14 May 2021 11:25:24 -0500 Subject: [PATCH 152/155] Rework on KAFKA-3968: fsync the parent directory of a segment file when the file is created (#10680) (reverted #10405). #10405 has several issues, for example: It fails to create a topic with 9000 partitions. It flushes in several unnecessary places. If multiple segments of the same partition are flushed at roughly the same time, we may end up doing multiple unnecessary flushes: the logic of handling the flush in LogSegments.scala is weird. Kafka does not call fsync() on the directory when a new log segment is created and flushed to disk. The problem is that following sequence of calls doesn't guarantee file durability: fd = open("log", O_RDWR | O_CREATE); // suppose open creates "log" write(fd); fsync(fd); If the system crashes after fsync() but before the parent directory has been flushed to disk, the log file can disappear. This PR is to flush the directory when flush() is called for the first time. Did performance test which shows this PR has a minimal performance impact on Kafka clusters. Reviewers: Jun Rao --- .../kafka/common/record/FileRecords.java | 7 ------ .../org/apache/kafka/common/utils/Utils.java | 23 +++++++++---------- core/src/main/scala/kafka/log/Log.scala | 10 +++++--- .../src/main/scala/kafka/log/LogManager.scala | 5 ++-- .../src/main/scala/kafka/log/LogSegment.scala | 22 ++++-------------- 5 files changed, 25 insertions(+), 42 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index 72d2d6183ffd3..17a41e2a74479 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -197,13 +197,6 @@ public void flush() throws IOException { channel.force(true); } - /** - * Flush the parent directory of a file to the physical disk, which makes sure the file is accessible after crashing. - */ - public void flushParentDir() throws IOException { - Utils.flushParentDir(file.toPath()); - } - /** * Close this record set */ diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index b56ad155b1f21..09dcae4b1848b 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -902,27 +902,26 @@ public static void atomicMoveWithFallback(Path source, Path target, boolean need } } finally { if (needFlushParentDir) { - flushParentDir(target); + flushDir(target.toAbsolutePath().normalize().getParent()); } } } /** - * Flushes the parent directory to guarantee crash consistency. + * Flushes dirty directories to guarantee crash consistency. * - * @throws IOException if flushing the parent directory fails. + * @throws IOException if flushing the directory fails. */ - public static void flushParentDir(Path path) throws IOException { - FileChannel dir = null; - try { - Path parent = path.toAbsolutePath().getParent(); - if (parent != null) { - dir = FileChannel.open(parent, StandardOpenOption.READ); + public static void flushDir(Path path) throws IOException { + if (path != null) { + FileChannel dir = null; + try { + dir = FileChannel.open(path, StandardOpenOption.READ); dir.force(true); + } finally { + if (dir != null) + dir.close(); } - } finally { - if (dir != null) - dir.close(); } } diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index edbabb88feef6..6c306101b1474 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -1676,7 +1676,10 @@ class Log(@volatile private var _dir: File, if (offset > this.recoveryPoint) { debug(s"Flushing log up to offset $offset, last flushed: $lastFlushTime, current time: ${time.milliseconds()}, " + s"unflushed: $unflushedMessages") - logSegments(this.recoveryPoint, offset).foreach(_.flush()) + val segments = logSegments(this.recoveryPoint, offset) + segments.foreach(_.flush()) + // if there are any new segments, we need to flush the parent directory for crash consistency + segments.lastOption.filter(_.baseOffset >= this.recoveryPoint).foreach(_ => Utils.flushDir(dir.toPath)) lock synchronized { checkIfMemoryMappedBufferClosed() @@ -2312,7 +2315,7 @@ object Log extends Logging { // need to do this in two phases to be crash safe AND do the delete asynchronously // if we crash in the middle of this we complete the swap in loadSegments() if (!isRecoveredSwapFile) - sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix, false)) + sortedNewSegments.reverse.foreach(_.changeFileSuffixes(Log.CleanedFileSuffix, Log.SwapFileSuffix)) sortedNewSegments.reverse.foreach(existingSegments.add(_)) val newSegmentBaseOffsets = sortedNewSegments.map(_.baseOffset).toSet @@ -2335,6 +2338,7 @@ object Log extends Logging { } // okay we are safe now, remove the swap suffix sortedNewSegments.foreach(_.changeFileSuffixes(Log.SwapFileSuffix, "")) + Utils.flushDir(dir.toPath) } /** @@ -2369,7 +2373,7 @@ object Log extends Logging { scheduler: Scheduler, logDirFailureChannel: LogDirFailureChannel, producerStateManager: ProducerStateManager): Unit = { - segmentsToDelete.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix, false)) + segmentsToDelete.foreach(_.changeFileSuffixes("", Log.DeletedFileSuffix)) def deleteSegments(): Unit = { info(s"Deleting segment files ${segmentsToDelete.mkString(",")}") diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index a13a7d484e95f..2a4d9539babfc 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -150,7 +150,7 @@ class LogManager(logDirs: Seq[File], val created = dir.mkdirs() if (!created) throw new IOException(s"Failed to create data directory ${dir.getAbsolutePath}") - Utils.flushParentDir(dir.toPath) + Utils.flushDir(dir.toPath.toAbsolutePath.normalize.getParent) } if (!dir.isDirectory || !dir.canRead) throw new IOException(s"${dir.getAbsolutePath} is not a readable log directory.") @@ -640,6 +640,8 @@ class LogManager(logDirs: Seq[File], try { recoveryPointCheckpoints.get(logDir).foreach { checkpoint => val recoveryOffsets = logsToCheckpoint.map { case (tp, log) => tp -> log.recoveryPoint } + // checkpoint.write calls Utils.atomicMoveWithFallback, which flushes the parent + // directory and guarantees crash consistency. checkpoint.write(recoveryOffsets) } } catch { @@ -867,7 +869,6 @@ class LogManager(logDirs: Seq[File], val dir = new File(logDirPath, logDirName) try { Files.createDirectories(dir.toPath) - Utils.flushParentDir(dir.toPath) Success(dir) } catch { case e: IOException => diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index e3b09d4b90f8b..37882ffa52592 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -20,7 +20,6 @@ import java.io.{File, IOException} import java.nio.file.{Files, NoSuchFileException} import java.nio.file.attribute.FileTime import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicBoolean import kafka.common.LogSegmentOffsetOverflowException import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.server.epoch.LeaderEpochFileCache @@ -51,7 +50,6 @@ import scala.math._ * @param indexIntervalBytes The approximate number of bytes between entries in the index * @param rollJitterMs The maximum random jitter subtracted from the scheduled segment roll time * @param time The time instance - * @param needsFlushParentDir Whether or not we need to flush the parent directory during the first flush */ @nonthreadsafe class LogSegment private[log] (val log: FileRecords, @@ -61,8 +59,7 @@ class LogSegment private[log] (val log: FileRecords, val baseOffset: Long, val indexIntervalBytes: Int, val rollJitterMs: Long, - val time: Time, - val needsFlushParentDir: Boolean = false) extends Logging { + val time: Time) extends Logging { def offsetIndex: OffsetIndex = lazyOffsetIndex.get @@ -98,9 +95,6 @@ class LogSegment private[log] (val log: FileRecords, /* the number of bytes since we last added an entry in the offset index */ private var bytesSinceLastIndexEntry = 0 - /* whether or not we need to flush the parent dir during the next flush */ - private val atomicNeedsFlushParentDir = new AtomicBoolean(needsFlushParentDir) - // The timestamp we used for time based log rolling and for ensuring max compaction delay // volatile for LogCleaner to see the update @volatile private var rollingBasedTimestamp: Option[Long] = None @@ -478,9 +472,6 @@ class LogSegment private[log] (val log: FileRecords, offsetIndex.flush() timeIndex.flush() txnIndex.flush() - // We only need to flush the parent of the log file because all other files share the same parent - if (atomicNeedsFlushParentDir.getAndSet(false)) - log.flushParentDir() } } @@ -499,14 +490,11 @@ class LogSegment private[log] (val log: FileRecords, * Change the suffix for the index and log files for this log segment * IOException from this method should be handled by the caller */ - def changeFileSuffixes(oldSuffix: String, newSuffix: String, needsFlushParentDir: Boolean = true): Unit = { + def changeFileSuffixes(oldSuffix: String, newSuffix: String): Unit = { log.renameTo(new File(CoreUtils.replaceSuffix(log.file.getPath, oldSuffix, newSuffix))) lazyOffsetIndex.renameTo(new File(CoreUtils.replaceSuffix(lazyOffsetIndex.file.getPath, oldSuffix, newSuffix))) lazyTimeIndex.renameTo(new File(CoreUtils.replaceSuffix(lazyTimeIndex.file.getPath, oldSuffix, newSuffix))) txnIndex.renameTo(new File(CoreUtils.replaceSuffix(txnIndex.file.getPath, oldSuffix, newSuffix))) - // We only need to flush the parent of the log file because all other files share the same parent - if (needsFlushParentDir) - log.flushParentDir() } /** @@ -669,8 +657,7 @@ class LogSegment private[log] (val log: FileRecords, object LogSegment { def open(dir: File, baseOffset: Long, config: LogConfig, time: Time, fileAlreadyExists: Boolean = false, - initFileSize: Int = 0, preallocate: Boolean = false, fileSuffix: String = "", - needsRecovery: Boolean = false): LogSegment = { + initFileSize: Int = 0, preallocate: Boolean = false, fileSuffix: String = ""): LogSegment = { val maxIndexSize = config.maxIndexSize new LogSegment( FileRecords.open(Log.logFile(dir, baseOffset, fileSuffix), fileAlreadyExists, initFileSize, preallocate), @@ -680,8 +667,7 @@ object LogSegment { baseOffset, indexIntervalBytes = config.indexInterval, rollJitterMs = config.randomSegmentJitter, - time, - needsFlushParentDir = needsRecovery || !fileAlreadyExists) + time) } def deleteIfExists(dir: File, baseOffset: Long, fileSuffix: String = ""): Unit = { From f2785f3c4f83bbc3caef48993018896bf73224df Mon Sep 17 00:00:00 2001 From: Walker Carlson <18128741+wcarlson5@users.noreply.github.com> Date: Fri, 14 May 2021 12:17:31 -0700 Subject: [PATCH 153/155] KAFKA-12754: Improve endOffsets for TaskMetadata (#10634) Improve endOffsets for TaskMetadata by updating immediately after polling a new batch Reviewers: Anna Sophie Blee-Goldman --- .../kafka/streams/processor/TaskMetadata.java | 9 + .../processor/internals/StandbyTask.java | 5 - .../processor/internals/StreamTask.java | 16 +- .../processor/internals/StreamThread.java | 10 + .../streams/processor/internals/Task.java | 7 - .../processor/internals/TaskManager.java | 26 ++- .../TaskMetadataIntegrationTest.java | 200 ++++++++++++++++++ .../processor/internals/StreamTaskTest.java | 9 +- .../processor/internals/TaskManagerTest.java | 7 - 9 files changed, 254 insertions(+), 35 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java index 91b06eacc8ecb..63129dacc203e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskMetadata.java @@ -60,14 +60,23 @@ public Set topicPartitions() { return topicPartitions; } + /** + * This function will return a map of TopicPartitions and the highest committed offset seen so far + */ public Map committedOffsets() { return committedOffsets; } + /** + * This function will return a map of TopicPartitions and the highest offset seen so far in the Topic + */ public Map endOffsets() { return endOffsets; } + /** + * This function will return the time task idling started, if the task is not currently idling it will return empty + */ public Optional timeCurrentIdlingStarted() { return timeCurrentIdlingStarted; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index b386be859b8df..802bca1624827 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -297,11 +297,6 @@ public Optional timeCurrentIdlingStarted() { return Optional.empty(); } - @Override - public void updateCommittedOffsets(final TopicPartition topicPartition, final Long offset) { - - } - @Override public void addRecords(final TopicPartition partition, final Iterable> records) { throw new IllegalStateException("Attempted to add records to task " + id() + " for invalid input partition " + partition); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index d2f7926d46d9f..129407af5436b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -196,9 +196,13 @@ public StreamTask(final TaskId id, ); stateMgr.registerGlobalStateStores(topology.globalStateStores()); - this.committedOffsets = new HashMap<>(); - this.highWatermark = new HashMap<>(); - this.timeCurrentIdlingStarted = Optional.empty(); + committedOffsets = new HashMap<>(); + highWatermark = new HashMap<>(); + for (final TopicPartition topicPartition: inputPartitions) { + committedOffsets.put(topicPartition, -1L); + highWatermark.put(topicPartition, -1L); + } + timeCurrentIdlingStarted = Optional.empty(); } // create queues for each assigned partition and associate them @@ -1173,7 +1177,6 @@ public Map committedOffsets() { @Override public Map highWaterMark() { - highWatermark.putAll(recordCollector.offsets()); return Collections.unmodifiableMap(highWatermark); } @@ -1188,11 +1191,14 @@ public Optional timeCurrentIdlingStarted() { return timeCurrentIdlingStarted; } - @Override public void updateCommittedOffsets(final TopicPartition topicPartition, final Long offset) { committedOffsets.put(topicPartition, offset); } + public void updateEndOffsets(final TopicPartition topicPartition, final Long offset) { + highWatermark.put(topicPartition, offset); + } + public boolean hasRecordsQueued() { return numBuffered() > 0; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index fd6f9f87f2d05..fed8015521327 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -20,6 +20,7 @@ import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.InvalidOffsetException; import org.apache.kafka.common.KafkaException; @@ -52,6 +53,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -900,6 +902,14 @@ private long pollPhase() { final int numRecords = records.count(); + for (final TopicPartition topicPartition: records.partitions()) { + records + .records(topicPartition) + .stream() + .max(Comparator.comparing(ConsumerRecord::offset)) + .ifPresent(t -> taskManager.updateTaskEndMetadata(topicPartition, t.offset())); + } + log.debug("Main Consumer poll completed in {} ms and fetched {} records", pollLatency, numRecords); pollSensor.record(pollLatency, now); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java index feedeea0480a1..fdec5ec7b2d22 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java @@ -248,11 +248,4 @@ default boolean commitRequested() { * @return This returns the time the task started idling. If it is not idling it returns empty. */ Optional timeCurrentIdlingStarted(); - - /** - * Update the committed offsets in the Task - * @param topicPartition - * @param offset - */ - void updateCommittedOffsets(final TopicPartition topicPartition, final Long offset); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index 0de51e56efe81..7fa624a0ac1bd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -1091,7 +1091,7 @@ private void commitOffsetsOrTransaction(final Map allOffsets) { + private void updateTaskCommitMetadata(final Map allOffsets) { for (final Task task: tasks.activeTasks()) { - for (final TopicPartition topicPartition: task.inputPartitions()) { - if (allOffsets.containsKey(topicPartition)) { - task.updateCommittedOffsets(topicPartition, allOffsets.get(topicPartition).offset()); + if (task instanceof StreamTask) { + for (final TopicPartition topicPartition : task.inputPartitions()) { + if (allOffsets.containsKey(topicPartition)) { + ((StreamTask) task).updateCommittedOffsets(topicPartition, allOffsets.get(topicPartition).offset()); + } + } + } + } + } + + public void updateTaskEndMetadata(final TopicPartition topicPartition, final Long offset) { + for (final Task task: tasks.activeTasks()) { + if (task instanceof StreamTask) { + if (task.inputPartitions().contains(topicPartition)) { + ((StreamTask) task).updateEndOffsets(topicPartition, offset); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java new file mode 100644 index 0000000000000..b11779211829b --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java @@ -0,0 +1,200 @@ +/* + * 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.streams.integration; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.TaskMetadata; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import java.io.IOException; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +@Category(IntegrationTest.class) +public class TaskMetadataIntegrationTest { + + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1, new Properties(), 0L, 0L); + + @BeforeClass + public static void startCluster() throws IOException { + CLUSTER.start(); + } + + @AfterClass + public static void closeCluster() { + CLUSTER.stop(); + } + public static final Duration DEFAULT_DURATION = Duration.ofSeconds(30); + + @Rule + public TestName testName = new TestName(); + + private String inputTopic; + private static StreamsBuilder builder; + private static Properties properties; + private static String appId = "TaskMetadataTest_"; + private AtomicBoolean process; + private AtomicBoolean commit; + + @Before + public void setup() { + final String testId = safeUniqueTestName(getClass(), testName); + appId = appId + testId; + inputTopic = "input" + testId; + IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic); + + builder = new StreamsBuilder(); + + process = new AtomicBoolean(true); + commit = new AtomicBoolean(true); + + final KStream stream = builder.stream(inputTopic); + stream.process(PauseProcessor::new); + + properties = mkObjectProperties( + mkMap( + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId), + mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()), + mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2), + mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), + mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), + mkEntry(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1L) + ) + ); + } + + @Test + public void shouldReportCorrectCommittedOffsetInformation() { + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { + IntegrationTestUtils.startApplicationAndWaitUntilRunning(Collections.singletonList(kafkaStreams), DEFAULT_DURATION); + final TaskMetadata taskMetadata = getTaskMetadata(kafkaStreams); + assertThat(taskMetadata.committedOffsets().size(), equalTo(1)); + final TopicPartition topicPartition = new TopicPartition(inputTopic, 0); + + produceMessages(0L, inputTopic, "test"); + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); + TestUtils.waitForCondition(() -> taskMetadata.committedOffsets().get(topicPartition) == 1L, "the record was processed"); + process.set(true); + + produceMessages(0L, inputTopic, "test1"); + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); + TestUtils.waitForCondition(() -> taskMetadata.committedOffsets().get(topicPartition) == 2L, "the record was processed"); + process.set(true); + + produceMessages(0L, inputTopic, "test1"); + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); + TestUtils.waitForCondition(() -> taskMetadata.committedOffsets().get(topicPartition) == 3L, "the record was processed"); + } catch (final Exception e) { + e.printStackTrace(); + } + } + + @Test + public void shouldReportCorrectEndOffsetInformation() { + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { + IntegrationTestUtils.startApplicationAndWaitUntilRunning(Collections.singletonList(kafkaStreams), DEFAULT_DURATION); + final TaskMetadata taskMetadata = getTaskMetadata(kafkaStreams); + assertThat(taskMetadata.endOffsets().size(), equalTo(1)); + final TopicPartition topicPartition = new TopicPartition(inputTopic, 0); + commit.set(false); + + for (int i = 0; i < 10; i++) { + produceMessages(0L, inputTopic, "test"); + TestUtils.waitForCondition(() -> !process.get(), "The record was not processed"); + process.set(true); + } + assertThat(taskMetadata.endOffsets().get(topicPartition), equalTo(9L)); + + } catch (final Exception e) { + e.printStackTrace(); + } + } + + private TaskMetadata getTaskMetadata(final KafkaStreams kafkaStreams) { + final List taskMetadataList = kafkaStreams.localThreadsMetadata().stream().flatMap(t -> t.activeTasks().stream()).collect(Collectors.toList()); + assertThat("only one task", taskMetadataList.size() == 1); + return taskMetadataList.get(0); + } + + @After + public void teardown() throws IOException { + purgeLocalStreamsState(properties); + } + + private void produceMessages(final long timestamp, final String streamOneInput, final String msg) { + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( + streamOneInput, + Collections.singletonList(new KeyValue<>("1", msg)), + TestUtils.producerConfig( + CLUSTER.bootstrapServers(), + StringSerializer.class, + StringSerializer.class, + new Properties()), + timestamp); + } + + private class PauseProcessor extends AbstractProcessor { + @Override + public void process(final String key, final String value) { + while (!process.get()) { + try { + wait(100); + } catch (final InterruptedException e) { + + } + } + context().forward(key, value); + if (commit.get()) { + context().commit(); + } + process.set(false); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 5eeaaa24524f1..9763c4f0d9113 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -1388,7 +1388,8 @@ public void shouldWrapKafkaExceptionWithStreamsExceptionWhenProcess() { getConsumerRecordWithOffsetAsTimestamp(partition2, 45) )); - assertThat("Map was not empty", task.highWaterMark().isEmpty()); + assertThat("Map did not contain the partitions", task.highWaterMark().containsKey(partition1) + && task.highWaterMark().containsKey(partition2)); assertThrows(StreamsException.class, () -> task.process(0L)); } @@ -1444,7 +1445,7 @@ public void shouldReInitializeTopologyWhenResuming() throws IOException { EasyMock.reset(recordCollector); EasyMock.expect(recordCollector.offsets()).andReturn(emptyMap()); EasyMock.replay(recordCollector); - assertThat("Map was not empty", task.highWaterMark().isEmpty()); + assertThat("Map did not contain the partition", task.highWaterMark().containsKey(partition1)); } @Test @@ -1472,7 +1473,7 @@ public void shouldNotCheckpointOffsetsAgainOnCommitIfSnapshotNotChangedMuch() { task.postCommit(false); // should not checkpoint EasyMock.verify(stateManager, recordCollector); - assertThat("Map was not empty", task.highWaterMark().containsValue(offset)); + assertThat("Map was empty", task.highWaterMark().size() == 2); } @Test @@ -1502,7 +1503,7 @@ public void shouldCheckpointOffsetsOnCommitIfSnapshotMuchChanged() { task.postCommit(false); EasyMock.verify(recordCollector); - assertThat("Map was not empty", task.highWaterMark().containsValue(offset)); + assertThat("Map was empty", task.highWaterMark().size() == 2); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 07fc378e18514..7c27f74ab44dd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -69,7 +69,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -3466,12 +3465,6 @@ public Optional timeCurrentIdlingStarted() { return Optional.empty(); } - @Override - public void updateCommittedOffsets(final TopicPartition topicPartition, final Long offset) { - Objects.requireNonNull(topicPartition); - assertThat("It must be from an owned topic", inputPartitions.contains(topicPartition)); - } - @Override public void addRecords(final TopicPartition partition, final Iterable> records) { if (isActive()) { From f20fdbd839ea1275fa4b26e9eb3c49ef5d9af1ca Mon Sep 17 00:00:00 2001 From: Colin Patrick McCabe Date: Fri, 14 May 2021 12:44:16 -0700 Subject: [PATCH 154/155] KAFKA-12778: Fix QuorumController request timeouts and electLeaders (#10688) The QuorumController should honor the timeout for RPC requests which feature a timeout. For electLeaders, attempt to trigger a leader election for all partitions when the request specifies null for the topics argument. Reviewers: David Arthur --- checkstyle/suppressions.xml | 2 +- .../scala/kafka/server/ControllerApis.scala | 15 ++- .../test/java/kafka/test/MockController.java | 8 +- .../apache/kafka/controller/Controller.java | 23 +++- .../kafka/controller/QuorumController.java | 87 ++++++++++---- .../controller/ReplicationControlManager.java | 44 +++++-- .../controller/QuorumControllerTest.java | 110 ++++++++++++++++++ 7 files changed, 241 insertions(+), 48 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 03ba807e6d025..21cd5f4cc5ebc 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -270,7 +270,7 @@ + files="(QuorumControllerTest|ReplicationControlManager|ReplicationControlManagerTest).java"/> + controller.findTopicNames(deadlineNs, providedIds).thenCompose { topicNames => topicNames.forEach { (id, nameOrError) => if (nameOrError.isError) { appendResponse(null, id, nameOrError.error()) @@ -291,7 +293,7 @@ class ControllerApis(val requestChannel: RequestChannel, } // For each topic that was provided by name, check if authentication failed. // If so, create an error response for it. Otherwise, add it to the idToName map. - controller.findTopicIds(providedNames).thenCompose { topicIds => + controller.findTopicIds(deadlineNs, providedNames).thenCompose { topicIds => topicIds.forEach { (name, idOrError) => if (!describeable.contains(name)) { appendResponse(name, ZERO_UUID, new ApiError(TOPIC_AUTHORIZATION_FAILED)) @@ -315,7 +317,7 @@ class ControllerApis(val requestChannel: RequestChannel, } // Finally, the idToName map contains all the topics that we are authorized to delete. // Perform the deletion and create responses for each one. - controller.deleteTopics(idToName.keySet).thenApply { idToError => + controller.deleteTopics(deadlineNs, idToName.keySet).thenApply { idToError => idToError.forEach { (id, error) => appendResponse(idToName.get(id), id, error) } @@ -706,6 +708,7 @@ class ControllerApis(val requestChannel: RequestChannel, hasClusterAuth: Boolean, getCreatableTopics: Iterable[String] => Set[String]) : CompletableFuture[util.List[CreatePartitionsTopicResult]] = { + val deadlineNs = time.nanoseconds() + NANOSECONDS.convert(request.timeoutMs, MILLISECONDS); val responses = new util.ArrayList[CreatePartitionsTopicResult]() val duplicateTopicNames = new util.HashSet[String]() val topicNames = new util.HashSet[String]() @@ -739,7 +742,7 @@ class ControllerApis(val requestChannel: RequestChannel, setErrorCode(TOPIC_AUTHORIZATION_FAILED.code)) } } - controller.createPartitions(topics).thenApply { results => + controller.createPartitions(deadlineNs, topics).thenApply { results => results.forEach(response => responses.add(response)) responses } @@ -750,7 +753,7 @@ class ControllerApis(val requestChannel: RequestChannel, authHelper.authorizeClusterOperation(request, ALTER) val response = controller.alterPartitionReassignments(alterRequest.data()).get() requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new AlterPartitionReassignmentsResponse(response)) + new AlterPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) } def handleListPartitionReassignments(request: RequestChannel.Request): Unit = { @@ -758,6 +761,6 @@ class ControllerApis(val requestChannel: RequestChannel, authHelper.authorizeClusterOperation(request, DESCRIBE) val response = controller.listPartitionReassignments(listRequest.data()).get() requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new ListPartitionReassignmentsResponse(response)) + new ListPartitionReassignmentsResponse(response.setThrottleTimeMs(requestThrottleMs))) } } diff --git a/core/src/test/java/kafka/test/MockController.java b/core/src/test/java/kafka/test/MockController.java index fc14145e28037..1fba2952165b0 100644 --- a/core/src/test/java/kafka/test/MockController.java +++ b/core/src/test/java/kafka/test/MockController.java @@ -144,7 +144,7 @@ static class MockTopic { @Override synchronized public CompletableFuture>> - findTopicIds(Collection topicNames) { + findTopicIds(long deadlineNs, Collection topicNames) { Map> results = new HashMap<>(); for (String topicName : topicNames) { if (!topicNameToId.containsKey(topicName)) { @@ -158,7 +158,7 @@ static class MockTopic { @Override synchronized public CompletableFuture>> - findTopicNames(Collection topicIds) { + findTopicNames(long deadlineNs, Collection topicIds) { Map> results = new HashMap<>(); for (Uuid topicId : topicIds) { MockTopic topic = topics.get(topicId); @@ -173,7 +173,7 @@ static class MockTopic { @Override synchronized public CompletableFuture> - deleteTopics(Collection topicIds) { + deleteTopics(long deadlineNs, Collection topicIds) { if (!active) { CompletableFuture> future = new CompletableFuture<>(); future.completeExceptionally(NOT_CONTROLLER_EXCEPTION); @@ -303,7 +303,7 @@ public CompletableFuture waitForReadyBrokers(int minBrokers) { @Override synchronized public CompletableFuture> - createPartitions(List topicList) { + createPartitions(long deadlineNs, List topicList) { if (!active) { CompletableFuture> future = new CompletableFuture<>(); future.completeExceptionally(NOT_CONTROLLER_EXCEPTION); diff --git a/metadata/src/main/java/org/apache/kafka/controller/Controller.java b/metadata/src/main/java/org/apache/kafka/controller/Controller.java index 071076474dfea..a34b084ea1302 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/Controller.java +++ b/metadata/src/main/java/org/apache/kafka/controller/Controller.java @@ -80,27 +80,36 @@ public interface Controller extends AutoCloseable { /** * Find the ids for topic names. * + * @param deadlineNs The time by which this operation needs to be complete, before + * we will complete this operation with a timeout. * @param topicNames The topic names to resolve. * @return A future yielding a map from topic name to id. */ - CompletableFuture>> findTopicIds(Collection topicNames); + CompletableFuture>> findTopicIds(long deadlineNs, + Collection topicNames); /** * Find the names for topic ids. * + * @param deadlineNs The time by which this operation needs to be complete, before + * we will complete this operation with a timeout. * @param topicIds The topic ids to resolve. * @return A future yielding a map from topic id to name. */ - CompletableFuture>> findTopicNames(Collection topicIds); + CompletableFuture>> findTopicNames(long deadlineNs, + Collection topicIds); /** * Delete a batch of topics. * + * @param deadlineNs The time by which this operation needs to be complete, before + * we will complete this operation with a timeout. * @param topicIds The IDs of the topics to delete. * * @return A future yielding the response. */ - CompletableFuture> deleteTopics(Collection topicIds); + CompletableFuture> deleteTopics(long deadlineNs, + Collection topicIds); /** * Describe the current configuration of various resources. @@ -225,11 +234,13 @@ CompletableFuture> alterClientQuotas( /** * Create partitions on certain topics. * - * @param topics The list of topics to create partitions for. - * @return A future yielding per-topic results. + * @param deadlineNs The time by which this operation needs to be complete, before + * we will complete this operation with a timeout. + * @param topics The list of topics to create partitions for. + * @return A future yielding per-topic results. */ CompletableFuture> - createPartitions(List topics); + createPartitions(long deadlineNs, List topics); /** * Begin shutting down, but don't block. You must still call close to clean up all diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index 7995b4b5134ea..5cb5efc20f345 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -27,6 +27,7 @@ import java.util.OptionalLong; import java.util.Random; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -89,6 +90,7 @@ import org.slf4j.Logger; import static java.util.concurrent.TimeUnit.MICROSECONDS; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -475,6 +477,12 @@ CompletableFuture appendReadEvent(String name, Supplier handler) { return event.future(); } + CompletableFuture appendReadEvent(String name, long deadlineNs, Supplier handler) { + ControllerReadEvent event = new ControllerReadEvent(name, handler); + queue.appendWithDeadline(deadlineNs, event); + return event.future(); + } + interface ControllerWriteOperation { /** * Generate the metadata records needed to implement this controller write @@ -602,11 +610,10 @@ public String toString() { } private CompletableFuture appendWriteEvent(String name, - long timeoutMs, + long deadlineNs, ControllerWriteOperation op) { ControllerWriteEvent event = new ControllerWriteEvent<>(name, op); - queue.appendWithDeadline(time.nanoseconds() + - NANOSECONDS.convert(timeoutMs, TimeUnit.MILLISECONDS), event); + queue.appendWithDeadline(deadlineNs, event); return event.future(); } @@ -961,8 +968,9 @@ public CompletableFuture alterIsr(AlterIsrRequestData requ if (request.topics().isEmpty()) { return CompletableFuture.completedFuture(new CreateTopicsResponseData()); } - return appendWriteEvent("createTopics", () -> - replicationControl.createTopics(request)); + return appendWriteEvent("createTopics", + time.nanoseconds() + NANOSECONDS.convert(request.timeoutMs(), MILLISECONDS), + () -> replicationControl.createTopics(request)); } @Override @@ -972,23 +980,26 @@ public CompletableFuture unregisterBroker(int brokerId) { } @Override - public CompletableFuture>> findTopicIds(Collection names) { + public CompletableFuture>> findTopicIds(long deadlineNs, + Collection names) { if (names.isEmpty()) return CompletableFuture.completedFuture(Collections.emptyMap()); - return appendReadEvent("findTopicIds", + return appendReadEvent("findTopicIds", deadlineNs, () -> replicationControl.findTopicIds(lastCommittedOffset, names)); } @Override - public CompletableFuture>> findTopicNames(Collection ids) { + public CompletableFuture>> findTopicNames(long deadlineNs, + Collection ids) { if (ids.isEmpty()) return CompletableFuture.completedFuture(Collections.emptyMap()); - return appendReadEvent("findTopicNames", + return appendReadEvent("findTopicNames", deadlineNs, () -> replicationControl.findTopicNames(lastCommittedOffset, ids)); } @Override - public CompletableFuture> deleteTopics(Collection ids) { + public CompletableFuture> deleteTopics(long deadlineNs, + Collection ids) { if (ids.isEmpty()) return CompletableFuture.completedFuture(Collections.emptyMap()); - return appendWriteEvent("deleteTopics", + return appendWriteEvent("deleteTopics", deadlineNs, () -> replicationControl.deleteTopics(ids)); } @@ -1002,7 +1013,13 @@ public CompletableFuture> deleteTopics(Collection ids) @Override public CompletableFuture electLeaders(ElectLeadersRequestData request) { - return appendWriteEvent("electLeaders", request.timeoutMs(), + // If topicPartitions is null, we will try to trigger a new leader election on + // all partitions (!). But if it's empty, there is nothing to do. + if (request.topicPartitions() != null && request.topicPartitions().isEmpty()) { + return CompletableFuture.completedFuture(new ElectLeadersResponseData()); + } + return appendWriteEvent("electLeaders", + time.nanoseconds() + NANOSECONDS.convert(request.timeoutMs(), MILLISECONDS), () -> replicationControl.electLeaders(request)); } @@ -1016,6 +1033,9 @@ public CompletableFuture finalizedFeatures() { public CompletableFuture> incrementalAlterConfigs( Map>> configChanges, boolean validateOnly) { + if (configChanges.isEmpty()) { + return CompletableFuture.completedFuture(Collections.emptyMap()); + } return appendWriteEvent("incrementalAlterConfigs", () -> { ControllerResult> result = configurationControl.incrementalAlterConfigs(configChanges); @@ -1030,17 +1050,24 @@ public CompletableFuture> incrementalAlterConfigs( @Override public CompletableFuture alterPartitionReassignments(AlterPartitionReassignmentsRequestData request) { - CompletableFuture future = new CompletableFuture<>(); - future.completeExceptionally(new UnsupportedOperationException()); - return future; + if (request.topics().isEmpty()) { + return CompletableFuture.completedFuture(new AlterPartitionReassignmentsResponseData()); + } + return appendWriteEvent("alterPartitionReassignments", + time.nanoseconds() + NANOSECONDS.convert(request.timeoutMs(), MILLISECONDS), + () -> { + throw new UnsupportedOperationException(); + }); } @Override public CompletableFuture listPartitionReassignments(ListPartitionReassignmentsRequestData request) { - CompletableFuture future = new CompletableFuture<>(); - future.completeExceptionally(new UnsupportedOperationException()); - return future; + return appendReadEvent("listPartitionReassignments", + time.nanoseconds() + NANOSECONDS.convert(request.timeoutMs(), MILLISECONDS), + () -> { + throw new UnsupportedOperationException(); + }); } @Override @@ -1118,12 +1145,12 @@ public CompletableFuture> alterClientQuotas( @Override public CompletableFuture> - createPartitions(List topics) { + createPartitions(long deadlineNs, List topics) { if (topics.isEmpty()) { return CompletableFuture.completedFuture(Collections.emptyList()); } - return appendWriteEvent("createPartitions", () -> - replicationControl.createPartitions(topics)); + return appendWriteEvent("createPartitions", deadlineNs, + () -> replicationControl.createPartitions(topics)); } @Override @@ -1165,4 +1192,22 @@ public long curClaimEpoch() { public void close() throws InterruptedException { queue.close(); } + + // VisibleForTesting + CountDownLatch pause() { + final CountDownLatch latch = new CountDownLatch(1); + appendControlEvent("pause", () -> { + try { + latch.await(); + } catch (InterruptedException e) { + log.info("Interrupted while waiting for unpause.", e); + } + }); + return latch; + } + + // VisibleForTesting + Time time() { + return time; + } } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index 48ab1c92a44a5..cba9ae5c01b6a 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -800,16 +800,40 @@ ControllerResult electLeaders(ElectLeadersRequestData boolean uncleanOk = electionTypeIsUnclean(request.electionType()); List records = new ArrayList<>(); ElectLeadersResponseData response = new ElectLeadersResponseData(); - for (TopicPartitions topic : request.topicPartitions()) { - ReplicaElectionResult topicResults = - new ReplicaElectionResult().setTopic(topic.topic()); - response.replicaElectionResults().add(topicResults); - for (int partitionId : topic.partitions()) { - ApiError error = electLeader(topic.topic(), partitionId, uncleanOk, records); - topicResults.partitionResult().add(new PartitionResult(). - setPartitionId(partitionId). - setErrorCode(error.error().code()). - setErrorMessage(error.message())); + if (request.topicPartitions() == null) { + // If topicPartitions is null, we try to elect a new leader for every partition. There + // are some obvious issues with this wire protocol. For example, what if we have too + // many partitions to fit the results in a single RPC? This behavior should probably be + // removed from the protocol. For now, however, we have to implement this for + // compatibility with the old controller. + for (Entry topicEntry : topicsByName.entrySet()) { + String topicName = topicEntry.getKey(); + ReplicaElectionResult topicResults = + new ReplicaElectionResult().setTopic(topicName); + response.replicaElectionResults().add(topicResults); + TopicControlInfo topic = topics.get(topicEntry.getValue()); + if (topic != null) { + for (int partitionId : topic.parts.keySet()) { + ApiError error = electLeader(topicName, partitionId, uncleanOk, records); + topicResults.partitionResult().add(new PartitionResult(). + setPartitionId(partitionId). + setErrorCode(error.error().code()). + setErrorMessage(error.message())); + } + } + } + } else { + for (TopicPartitions topic : request.topicPartitions()) { + ReplicaElectionResult topicResults = + new ReplicaElectionResult().setTopic(topic.topic()); + response.replicaElectionResults().add(topicResults); + for (int partitionId : topic.partitions()) { + ApiError error = electLeader(topic.topic(), partitionId, uncleanOk, records); + topicResults.partitionResult().add(new PartitionResult(). + setPartitionId(partitionId). + setErrorCode(error.error().code()). + setErrorMessage(error.message())); + } } } return ControllerResult.of(records, response); diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index b6eea362c9024..5e5ca01707ddd 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -24,21 +24,34 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingDeque; import java.util.function.Function; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.AlterPartitionReassignmentsRequestData.ReassignableTopic; +import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.message.BrokerHeartbeatRequestData; import org.apache.kafka.common.message.BrokerRegistrationRequestData.Listener; import org.apache.kafka.common.message.BrokerRegistrationRequestData.ListenerCollection; import org.apache.kafka.common.message.BrokerRegistrationRequestData; +import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic; +import org.apache.kafka.common.message.CreatePartitionsResponseData.CreatePartitionsTopicResult; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignment; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableReplicaAssignmentCollection; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCollection; import org.apache.kafka.common.message.CreateTopicsRequestData; import org.apache.kafka.common.message.CreateTopicsResponseData; +import org.apache.kafka.common.message.ElectLeadersRequestData; +import org.apache.kafka.common.message.ElectLeadersResponseData; +import org.apache.kafka.common.message.ListPartitionReassignmentsRequestData; +import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord; import org.apache.kafka.common.metadata.RegisterBrokerRecord.BrokerEndpoint; @@ -57,12 +70,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static java.util.concurrent.TimeUnit.HOURS; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; import static org.apache.kafka.controller.ConfigurationControlManagerTest.BROKER0; import static org.apache.kafka.controller.ConfigurationControlManagerTest.CONFIGS; import static org.apache.kafka.controller.ConfigurationControlManagerTest.entry; 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.assertTrue; @@ -323,4 +338,99 @@ private void checkSnapshotContents(Uuid fooId, setRack(null), (short) 0))), iterator); } + + /** + * Test that certain controller operations time out if they stay on the controller + * queue for too long. + */ + @Test + public void testTimeouts() throws Throwable { + try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) { + try (QuorumControllerTestEnv controlEnv = + new QuorumControllerTestEnv(logEnv, b -> b.setConfigDefs(CONFIGS))) { + QuorumController controller = controlEnv.activeController(); + CountDownLatch countDownLatch = controller.pause(); + CompletableFuture createFuture = + controller.createTopics(new CreateTopicsRequestData().setTimeoutMs(0). + setTopics(new CreatableTopicCollection(Collections.singleton( + new CreatableTopic().setName("foo")).iterator()))); + long now = controller.time().nanoseconds(); + CompletableFuture> deleteFuture = + controller.deleteTopics(now, Collections.singletonList(Uuid.ZERO_UUID)); + CompletableFuture>> findTopicIdsFuture = + controller.findTopicIds(now, Collections.singletonList("foo")); + CompletableFuture>> findTopicNamesFuture = + controller.findTopicNames(now, Collections.singletonList(Uuid.ZERO_UUID)); + CompletableFuture> createPartitionsFuture = + controller.createPartitions(now, Collections.singletonList( + new CreatePartitionsTopic())); + CompletableFuture electLeadersFuture = + controller.electLeaders(new ElectLeadersRequestData().setTimeoutMs(0). + setTopicPartitions(null)); + CompletableFuture alterReassignmentsFuture = + controller.alterPartitionReassignments( + new AlterPartitionReassignmentsRequestData().setTimeoutMs(0). + setTopics(Collections.singletonList(new ReassignableTopic()))); + CompletableFuture listReassignmentsFuture = + controller.listPartitionReassignments( + new ListPartitionReassignmentsRequestData().setTimeoutMs(0)); + while (controller.time().nanoseconds() == now) { + Thread.sleep(0, 10); + } + countDownLatch.countDown(); + assertYieldsTimeout(createFuture); + assertYieldsTimeout(deleteFuture); + assertYieldsTimeout(findTopicIdsFuture); + assertYieldsTimeout(findTopicNamesFuture); + assertYieldsTimeout(createPartitionsFuture); + assertYieldsTimeout(electLeadersFuture); + assertYieldsTimeout(alterReassignmentsFuture); + assertYieldsTimeout(listReassignmentsFuture); + } + } + } + + private static void assertYieldsTimeout(Future future) { + assertEquals(TimeoutException.class, assertThrows(ExecutionException.class, + () -> future.get()).getCause().getClass()); + } + + /** + * Test that certain controller operations finish immediately without putting an event + * on the controller queue, if there is nothing to do. + */ + @Test + public void testEarlyControllerResults() throws Throwable { + try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv(1)) { + try (QuorumControllerTestEnv controlEnv = + new QuorumControllerTestEnv(logEnv, b -> b.setConfigDefs(CONFIGS))) { + QuorumController controller = controlEnv.activeController(); + CountDownLatch countDownLatch = controller.pause(); + CompletableFuture createFuture = + controller.createTopics(new CreateTopicsRequestData().setTimeoutMs(120000)); + long deadlineMs = controller.time().nanoseconds() + HOURS.toNanos(1); + CompletableFuture> deleteFuture = + controller.deleteTopics(deadlineMs, Collections.emptyList()); + CompletableFuture>> findTopicIdsFuture = + controller.findTopicIds(deadlineMs, Collections.emptyList()); + CompletableFuture>> findTopicNamesFuture = + controller.findTopicNames(deadlineMs, Collections.emptyList()); + CompletableFuture> createPartitionsFuture = + controller.createPartitions(deadlineMs, Collections.emptyList()); + CompletableFuture electLeadersFuture = + controller.electLeaders(new ElectLeadersRequestData().setTimeoutMs(120000)); + CompletableFuture alterReassignmentsFuture = + controller.alterPartitionReassignments( + new AlterPartitionReassignmentsRequestData().setTimeoutMs(12000)); + createFuture.get(); + deleteFuture.get(); + findTopicIdsFuture.get(); + findTopicNamesFuture.get(); + createPartitionsFuture.get(); + electLeadersFuture.get(); + alterReassignmentsFuture.get(); + countDownLatch.countDown(); + } + } + } } From ad91c5edf09af83cf44e2be71f19407ade1896bb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Dejan=20Stojadinovi=C4=87?= Date: Sat, 15 May 2021 15:10:04 +0200 Subject: [PATCH 155/155] KAFKA-12728: Upgrade gradle to 7.0.2 and shadow to 7.0.0 (#10606) Details: * gradle upgrade: 6.8.3 -> 7.0.2 https://github.com/gradle/gradle/releases/tag/v7.0.0 https://github.com/gradle/gradle/releases/tag/v7.0.1 https://github.com/gradle/gradle/releases/tag/v7.0.2 * 'distributionSha256Sum' gradle property is included into 'gradle-wrapper.properties' file * gradle shadow plugin upgrade: 6.1.0 -> 7.0.0 https://github.com/johnrengelman/shadow/releases/tag/7.0.0 * Remaining configurations obsoleted in Gradle 6 (and removed in Gradle 7) are replaced: `compile` -> `implementation` `testCompile` -> `testImplementation` Reviewers: Ismael Juma --- build.gradle | 16 ++++++++-------- gradle/dependencies.gradle | 2 +- gradle/wrapper/gradle-wrapper.properties | 3 ++- gradlew | 5 ++++- 4 files changed, 15 insertions(+), 11 deletions(-) diff --git a/build.gradle b/build.gradle index b10fc53eab3c6..94e521b9a3e17 100644 --- a/build.gradle +++ b/build.gradle @@ -40,7 +40,7 @@ plugins { id "com.github.spotbugs" version '4.7.1' apply false id 'org.gradle.test-retry' version '1.2.1' apply false id 'org.scoverage' version '5.0.0' apply false - id 'com.github.johnrengelman.shadow' version '6.1.0' apply false + id 'com.github.johnrengelman.shadow' version '7.0.0' apply false } spotless { @@ -842,15 +842,15 @@ project(':core') { configurations { // manually excludes some unnecessary dependencies - compile.exclude module: 'javax' - compile.exclude module: 'jline' - compile.exclude module: 'jms' - compile.exclude module: 'jmxri' - compile.exclude module: 'jmxtools' - compile.exclude module: 'mail' + implementation.exclude module: 'javax' + implementation.exclude module: 'jline' + implementation.exclude module: 'jms' + implementation.exclude module: 'jmxri' + implementation.exclude module: 'jmxtools' + implementation.exclude module: 'mail' // To prevent a UniqueResourceException due the same resource existing in both // org.apache.directory.api/api-all and org.apache.directory.api/api-ldap-schema-data - testCompile.exclude module: 'api-ldap-schema-data' + testImplementation.exclude module: 'api-ldap-schema-data' } tasks.create(name: "copyDependantLibs", type: Copy) { diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 2c67d137865f7..e8e5a79556736 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -61,7 +61,7 @@ versions += [ bcpkix: "1.66", checkstyle: "8.36.2", commonsCli: "1.4", - gradle: "6.8.3", + gradle: "7.0.2", grgit: "4.1.0", httpclient: "4.5.13", easymock: "4.3", diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 8cf6eb5ad222e..e1e2fd2c75e69 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,5 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-6.8.3-all.zip +distributionSha256Sum=13bf8d3cf8eeeb5770d19741a59bde9bd966dd78d17f1bbad787a05ef19d1c2d +distributionUrl=https\://services.gradle.org/distributions/gradle-7.0.2-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew index 61d317e696d1d..852728359c3cf 100755 --- a/gradlew +++ b/gradlew @@ -1,5 +1,8 @@ #!/usr/bin/env sh +# +# Copyright 2015 the original author or authors. +# # Licensed 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 @@ -81,7 +84,7 @@ esac # Loop in case we encounter an error. for attempt in 1 2 3; do if [ ! -e "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" ]; then - if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v6.8.3/gradle/wrapper/gradle-wrapper.jar"; then + if ! curl -s -S --retry 3 -L -o "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" "https://raw.githubusercontent.com/gradle/gradle/v7.0.2/gradle/wrapper/gradle-wrapper.jar"; then rm -f "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" # Pause for a bit before looping in case the server throttled us. sleep 5