Compare commits

...

9 Commits

Author SHA1 Message Date
Jinhe Zhang 96194a7aeb
Merge aa0f4ea808 into 4a5aa37169 2025-10-07 10:30:25 -07:00
Chang-Chi Hsu 4a5aa37169
MINOR: Move ReconfigurableQuorumIntegrationTest from core module to server module (#20636)
CI / build (push) Waiting to run Details
It moves the `ReconfigurableQuorumIntegrationTest` class to the
`org.apache.kafka.server` package and consolidates two related tests,
`RemoveAndAddVoterWithValidClusterId` and
`RemoveAndAddVoterWithInconsistentClusterId`, into a single file. This
improves code organization and reduces redundancy.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-08 01:10:58 +08:00
lucliu1108 2938c4242e
KAFKA-19754: Add RPC-level integration test for StreamsGroupDescribeRequest (#20632)
CI / build (push) Waiting to run Details
Test the `StreamsGroupDescribeRequest` RPC and corresponding responses
for situations where
- `streams.version` not upgraded to 1
- `streams.version` enabled, multiple groups listening to the same
topic.

Reviewers: Lucas Brutschy <lucasbru@apache.org>
2025-10-07 15:47:32 +02:00
Ken Huang ebae768bd8
KAFKA-18193 Refactor Kafka Streams CloseOptions to Fluent API Style (#19955)
In Kafka Streams, configuration classes typically follow a fluent API
pattern to ensure a consistent and intuitive developer experience.
However, the current implementation of
`org.apache.kafka.streams.KafkaStreams$CloseOptions` deviates from this
convention by exposing a public constructor, breaking the uniformity
expected across the API.

To address this inconsistency, we propose introducing a new
`CloseOptions` class that adheres to the fluent API style, replacing the
existing implementation. The new class will retain the existing
`timeout(Duration)` and `leaveGroup(boolean)` methods but will enforce
fluent instantiation and configuration. Given the design shift, we will
not maintain backward compatibility with the current class.

This change aligns with the goal of standardizing configuration objects
across Kafka Streams, offering developers a more cohesive and
predictable API.

Reviewers: Bill Bejeck<bbejeck@apache.org>
2025-10-07 08:50:18 -04:00
Jhen-Yung Hsu fa2496bb91
MINOR: skip ci-complete on 4.0 (#20644)
Since that 4.0 branch does not include
[KAFKA-18748](https://issues.apache.org/jira/browse/KAFKA-18748), it is
unable to find the related scan reports, but the ci-complete workflow is
still being triggered on the 4.0 branch. Disable this on the 4.0 branch,
as its reports can be safely ignored.

See https://github.com/apache/kafka/pull/20616#issuecomment-3370876779.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-07 20:27:29 +08:00
Dejan Stojadinović c6bbbbe24d
KAFKA-19174 Gradle version upgrade 8 -->> 9 (#19513)
List of changes:
- prerequisite Jira ticket:
  - [KAFKA-19591](https://issues.apache.org/jira/browse/KAFKA-19591)
- mandatory version upgrades:
  - Gradle version: 8.14.3 -->> 9.1.0
  - Gradle Shadow plugin: 8.3.6 -->> 8.3.9
  - Gradle dependencycheck plugin: 8.2.1 -->> 12.1.3
  - Gradle spotbugs plugin: 6.2.3 -->> 6.2.5
  - Gradle spotless plugin: 6.25.0 -->> 7.2.1
- build logic will be refactored to accommodate Gradle 9 breaking
changes
- (optional): a dozen of Gradle plugins versions will also be upgraded
- other JIRA tickets that had to be solved all along:
  - [KAFKA-16801](https://issues.apache.org/jira/browse/KAFKA-16801)
  - [KAFKA-19654](https://issues.apache.org/jira/browse/KAFKA-19654)

 **Related links:**
- https://gradle.org/whats-new/gradle-9
- https://github.com/gradle/gradle/releases/tag/v9.0.0
- https://docs.gradle.org/9.0.0/release-notes.html
- https://docs.gradle.org/9.0.0/userguide/upgrading_major_version_9.html
- https://docs.gradle.org/9.1.0/release-notes.html

Notes:
- new Gradle version brings up some breaking changes, as always 😃
- Kafka build with Gradle 9 has same issues as other projects:
  - https://github.com/redhat-developer/vscode-java/issues/4018
  - https://github.com/gradle/gradle/pull/32597

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-10-07 18:20:04 +08:00
Jinhe Zhang aa0f4ea808 remove eos tests 2025-08-25 15:27:58 -04:00
Jinhe Zhang d56ccebafc recfactor to reduce the complexity 2025-08-25 14:48:26 -04:00
Jinhe Zhang 9ef493983c verify all transaction finished 2025-08-13 21:23:34 -04:00
28 changed files with 1073 additions and 1622 deletions

View File

@ -42,7 +42,7 @@ runs:
distribution: temurin
java-version: ${{ inputs.java-version }}
- name: Setup Gradle
uses: gradle/actions/setup-gradle@94baf225fe0a508e581a564467443d0e2379123b # v4.3.0
uses: gradle/actions/setup-gradle@748248ddd2a24f49513d8f472f81c3a07d4d50e1 # v4.4.4
env:
GRADLE_BUILD_ACTION_CACHE_DEBUG_ENABLED: true
with:

View File

@ -38,7 +38,7 @@ run-name: Build Scans for ${{ github.event.workflow_run.display_title}}
jobs:
upload-build-scan:
# Skip this workflow if the CI run was skipped or cancelled
if: (github.event.workflow_run.conclusion == 'success' || github.event.workflow_run.conclusion == 'failure')
if: (github.event.workflow_run.conclusion == 'success' || github.event.workflow_run.conclusion == 'failure') && github.event.workflow_run.head_branch != '4.0'
runs-on: ubuntu-latest
strategy:
fail-fast: false

View File

@ -29,22 +29,21 @@ buildscript {
}
plugins {
id 'com.github.ben-manes.versions' version '0.48.0'
id 'com.github.ben-manes.versions' version '0.52.0'
id 'idea'
id 'jacoco'
id 'java-library'
id 'org.owasp.dependencycheck' version '8.2.1'
id 'org.owasp.dependencycheck' version '12.1.3'
id 'org.nosphere.apache.rat' version "0.8.1"
id "io.swagger.core.v3.swagger-gradle-plugin" version "${swaggerVersion}"
id "com.github.spotbugs" version '6.2.3' apply false
id "com.github.spotbugs" version '6.2.5' apply false
id 'org.scoverage' version '8.0.3' apply false
id 'com.gradleup.shadow' version '8.3.6' apply false
id 'com.diffplug.spotless' version "6.25.0"
id 'com.gradleup.shadow' version '8.3.9' apply false
id 'com.diffplug.spotless' version "7.2.1"
}
ext {
gradleVersion = versions.gradle
minClientJavaVersion = 11
minNonClientJavaVersion = 17
modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams:examples", ":streams-scala", ":test-common:test-common-util"]
@ -297,7 +296,7 @@ if (repo != null) {
} else {
rat.enabled = false
}
println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $gradleVersion, Java ${JavaVersion.current()} and Scala ${versions.scala}")
println("Starting build with version $version (commit id ${commitId == null ? "null" : commitId.take(8)}) using Gradle $versions.gradle, Java ${JavaVersion.current()} and Scala ${versions.scala}")
println("Build properties: ignoreFailures=$userIgnoreFailures, maxParallelForks=$maxTestForks, maxScalacThreads=$maxScalacThreads, maxTestRetries=$userMaxTestRetries")
subprojects {
@ -328,6 +327,16 @@ subprojects {
tasks.register('uploadArchives').configure { dependsOn(publish) }
}
tasks.withType(AbstractArchiveTask).configureEach {
reproducibleFileOrder = false
preserveFileTimestamps = true
useFileSystemPermissions()
}
tasks.withType(AbstractTestTask).configureEach {
failOnNoDiscoveredTests = false
}
// apply the eclipse plugin only to subprojects that hold code. 'connect' is just a folder.
if (!project.name.equals('connect')) {
apply plugin: 'eclipse'

View File

@ -250,7 +250,7 @@
files="(EosV2UpgradeIntegrationTest|KStreamKStreamJoinTest|StreamThreadTest|TaskManagerTest|StreamTaskTest).java"/>
<suppress checks="NPathComplexity"
files="(EosV2UpgradeIntegrationTest|EosTestDriver|KStreamKStreamJoinTest|KTableKTableForeignKeyJoinIntegrationTest|RelationalSmokeTest|MockProcessorContextStateStoreTest|TopologyTestDriverTest|IQv2StoreIntegrationTest).java"/>
files="(EosV2UpgradeIntegrationTest|KStreamKStreamJoinTest|KTableKTableForeignKeyJoinIntegrationTest|RelationalSmokeTest|MockProcessorContextStateStoreTest|TopologyTestDriverTest|IQv2StoreIntegrationTest).java"/>
<suppress checks="(FinalLocalVariable|WhitespaceAround|LocalVariableName|ImportControl|AvoidStarImport)"
files="Murmur3Test.java"/>

View File

@ -1,132 +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.admin;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.InconsistentClusterIdException;
import org.apache.kafka.common.test.KafkaClusterTestKit;
import org.apache.kafka.common.test.TestKitNodes;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals;
@Tag("integration")
public class ReconfigurableQuorumIntegrationTest {
static Map<Integer, Uuid> descVoterDirs(Admin admin) throws ExecutionException, InterruptedException {
var quorumInfo = admin.describeMetadataQuorum().quorumInfo().get();
return quorumInfo.voters().stream().collect(Collectors.toMap(QuorumInfo.ReplicaState::replicaId, QuorumInfo.ReplicaState::replicaDirectoryId));
}
@Test
public void testRemoveAndAddVoterWithValidClusterId() throws Exception {
final var nodes = new TestKitNodes.Builder()
.setClusterId("test-cluster")
.setNumBrokerNodes(1)
.setNumControllerNodes(3)
.build();
final Map<Integer, Uuid> initialVoters = new HashMap<>();
for (final var controllerNode : nodes.controllerNodes().values()) {
initialVoters.put(
controllerNode.id(),
controllerNode.metadataDirectoryId()
);
}
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
TestUtils.waitForCondition(() -> {
Map<Integer, Uuid> voters = descVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID));
}, "Initial quorum voters should be {3000, 3001, 3002} and all should have non-zero directory IDs");
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
admin.removeRaftVoter(
3000,
dirId,
new RemoveRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
).all().get();
TestUtils.waitForCondition(() -> {
Map<Integer, Uuid> voters = descVoterDirs(admin);
assertEquals(Set.of(3001, 3002), voters.keySet());
return voters.values().stream().noneMatch(directory -> directory.equals(Uuid.ZERO_UUID));
}, "After removing voter 3000, remaining voters should be {3001, 3002} with non-zero directory IDs");
admin.addRaftVoter(
3000,
dirId,
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
new AddRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
).all().get();
}
}
}
@Test
public void testRemoveAndAddVoterWithInconsistentClusterId() throws Exception {
final var nodes = new TestKitNodes.Builder()
.setClusterId("test-cluster")
.setNumBrokerNodes(1)
.setNumControllerNodes(3)
.build();
final Map<Integer, Uuid> initialVoters = new HashMap<>();
for (final var controllerNode : nodes.controllerNodes().values()) {
initialVoters.put(
controllerNode.id(),
controllerNode.metadataDirectoryId()
);
}
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
var removeFuture = admin.removeRaftVoter(
3000,
dirId,
new RemoveRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
).all();
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, removeFuture);
var addFuture = admin.addRaftVoter(
3000,
dirId,
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
new AddRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
).all();
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, addFuture);
}
}
}
}

View File

@ -26,9 +26,9 @@ import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse
import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment
import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, StreamsGroupDescribeRequestData, StreamsGroupDescribeResponseData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, StreamsGroupDescribeRequest, StreamsGroupDescribeResponse, StreamsGroupHeartbeatRequest, StreamsGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
import org.apache.kafka.common.serialization.StringSerializer
import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.common.utils.ProducerIdAndEpoch
@ -768,6 +768,21 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
shareGroupDescribeResponse.data.groups.asScala.toList
}
protected def streamsGroupDescribe(
groupIds: List[String],
includeAuthorizedOperations: Boolean = false,
version: Short = ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)
): List[StreamsGroupDescribeResponseData.DescribedGroup] = {
val streamsGroupDescribeRequest = new StreamsGroupDescribeRequest.Builder(
new StreamsGroupDescribeRequestData()
.setGroupIds(groupIds.asJava)
.setIncludeAuthorizedOperations(includeAuthorizedOperations)
).build(version)
val streamsGroupDescribeResponse = connectAndReceive[StreamsGroupDescribeResponse](streamsGroupDescribeRequest)
streamsGroupDescribeResponse.data.groups.asScala.toList
}
protected def heartbeat(
groupId: String,
generationId: Int,
@ -855,6 +870,41 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
shareGroupHeartbeatResponse.data
}
protected def streamsGroupHeartbeat(
groupId: String,
memberId: String = "",
memberEpoch: Int = 0,
rebalanceTimeoutMs: Int = -1,
activeTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
standbyTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
warmupTasks: List[StreamsGroupHeartbeatRequestData.TaskIds] = null,
topology: StreamsGroupHeartbeatRequestData.Topology = null,
expectedError: Errors = Errors.NONE,
version: Short = ApiKeys.STREAMS_GROUP_HEARTBEAT.latestVersion(isUnstableApiEnabled)
): StreamsGroupHeartbeatResponseData = {
val streamsGroupHeartbeatRequest = new StreamsGroupHeartbeatRequest.Builder(
new StreamsGroupHeartbeatRequestData()
.setGroupId(groupId)
.setMemberId(memberId)
.setMemberEpoch(memberEpoch)
.setRebalanceTimeoutMs(rebalanceTimeoutMs)
.setActiveTasks(activeTasks.asJava)
.setStandbyTasks(standbyTasks.asJava)
.setWarmupTasks(warmupTasks.asJava)
.setTopology(topology)
).build(version)
// Send the request until receiving a successful response. There is a delay
// here because the group coordinator is loaded in the background.
var streamsGroupHeartbeatResponse: StreamsGroupHeartbeatResponse = null
TestUtils.waitUntilTrue(() => {
streamsGroupHeartbeatResponse = connectAndReceive[StreamsGroupHeartbeatResponse](streamsGroupHeartbeatRequest)
streamsGroupHeartbeatResponse.data.errorCode == expectedError.code
}, msg = s"Could not heartbeat successfully. Last response $streamsGroupHeartbeatResponse.")
streamsGroupHeartbeatResponse.data
}
protected def leaveGroupWithNewProtocol(
groupId: String,
memberId: String

View File

@ -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 kafka.server
import kafka.utils.TestUtils
import org.apache.kafka.common.message.{StreamsGroupDescribeRequestData, StreamsGroupDescribeResponseData, StreamsGroupHeartbeatRequestData, StreamsGroupHeartbeatResponseData}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{StreamsGroupDescribeRequest, StreamsGroupDescribeResponse}
import org.apache.kafka.common.resource.ResourceType
import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.common.test.api._
import scala.jdk.CollectionConverters._
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.common.Feature
import org.junit.Assert.{assertEquals, assertTrue}
import java.lang.{Byte => JByte}
@ClusterTestDefaults(
types = Array(Type.KRAFT),
brokers = 1,
serverProperties = Array(
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1")
)
)
class StreamsGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) {
@ClusterTest(
features = Array(
new ClusterFeature(feature = Feature.STREAMS_VERSION, version = 0)
)
)
def testStreamsGroupDescribeWhenFeatureFlagNotEnabled(): Unit = {
val streamsGroupDescribeRequest = new StreamsGroupDescribeRequest.Builder(
new StreamsGroupDescribeRequestData().setGroupIds(List("grp-mock-1", "grp-mock-2").asJava)
).build(ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled))
val streamsGroupDescribeResponse = connectAndReceive[StreamsGroupDescribeResponse](streamsGroupDescribeRequest)
val expectedResponse = new StreamsGroupDescribeResponseData()
expectedResponse.groups().add(
new StreamsGroupDescribeResponseData.DescribedGroup()
.setGroupId("grp-mock-1")
.setErrorCode(Errors.UNSUPPORTED_VERSION.code)
)
expectedResponse.groups().add(
new StreamsGroupDescribeResponseData.DescribedGroup()
.setGroupId("grp-mock-2")
.setErrorCode(Errors.UNSUPPORTED_VERSION.code)
)
assertEquals(expectedResponse, streamsGroupDescribeResponse.data)
}
@ClusterTest(
serverProperties = Array(
new ClusterConfigProperty(key = GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, value = "classic,consumer,streams"),
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1")
)
)
def testStreamsGroupDescribeGroupsWithNewGroupCoordinator(): Unit = {
// Creates the __consumer_offsets topics because it won't be created automatically
// in this test because it does not use FindCoordinator API.
createOffsetsTopic()
val admin = cluster.admin()
val topicName = "foo"
try {
TestUtils.createTopicWithAdminRaw(
admin = admin,
topic = topicName,
numPartitions = 3
)
TestUtils.waitUntilTrue(() => {
admin.listTopics().names().get().contains(topicName)
}, msg = s"Topic $topicName is not available to the group coordinator")
val timeoutMs = 5 * 60 * 1000
val clientId = "client-id"
val clientHost = "/127.0.0.1"
val authorizedOperationsInt = Utils.to32BitField(
AclEntry.supportedOperations(ResourceType.GROUP).asScala
.map(_.code.asInstanceOf[JByte]).asJava)
var grp1Member1Response: StreamsGroupHeartbeatResponseData = null
var grp1Member2Response: StreamsGroupHeartbeatResponseData = null
var grp2Member1Response: StreamsGroupHeartbeatResponseData = null
var grp2Member2Response: StreamsGroupHeartbeatResponseData = null
// grp-1 with 2 members
TestUtils.waitUntilTrue(() => {
grp1Member1Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = "member-1",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
grp1Member2Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = "member-2",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
val groupsDescription1 = streamsGroupDescribe(
groupIds = List("grp-1"),
includeAuthorizedOperations = true
)
grp1Member1Response.errorCode == Errors.NONE.code && grp1Member2Response.errorCode == Errors.NONE.code &&
groupsDescription1.size == 1 && groupsDescription1.head.members.size == 2
}, msg = s"Could not create grp-1 with 2 members successfully")
// grp-2 with 2 members
TestUtils.waitUntilTrue(() => {
grp2Member1Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = "member-3",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
grp2Member2Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = "member-4",
rebalanceTimeoutMs = timeoutMs,
activeTasks = List.empty,
standbyTasks = List.empty,
warmupTasks = List.empty,
topology = new StreamsGroupHeartbeatRequestData.Topology()
.setEpoch(1)
.setSubtopologies(List(
new StreamsGroupHeartbeatRequestData.Subtopology()
.setSubtopologyId("subtopology-1")
.setSourceTopics(List(topicName).asJava)
.setRepartitionSinkTopics(List.empty.asJava)
.setRepartitionSourceTopics(List.empty.asJava)
.setStateChangelogTopics(List.empty.asJava)
).asJava)
)
val groupsDescription2 = streamsGroupDescribe(
groupIds = List("grp-2"),
includeAuthorizedOperations = true,
version = ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled).toShort
)
grp2Member1Response.errorCode == Errors.NONE.code && grp2Member2Response.errorCode == Errors.NONE.code &&
groupsDescription2.size == 1 && groupsDescription2.head.members.size == 2
}, msg = s"Could not create grp-2 with 2 members successfully")
// Send follow-up heartbeats until both groups are stable
TestUtils.waitUntilTrue(() => {
grp1Member1Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = grp1Member1Response.memberId,
memberEpoch = grp1Member1Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp1Member1Response.activeTasks),
standbyTasks = convertTaskIds(grp1Member1Response.standbyTasks),
warmupTasks = convertTaskIds(grp1Member1Response.warmupTasks),
topology = null
)
grp1Member2Response = streamsGroupHeartbeat(
groupId = "grp-1",
memberId = grp1Member2Response.memberId,
memberEpoch = grp1Member2Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp1Member2Response.activeTasks),
standbyTasks = convertTaskIds(grp1Member2Response.standbyTasks),
warmupTasks = convertTaskIds(grp1Member2Response.warmupTasks),
topology = null
)
grp2Member1Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = grp2Member1Response.memberId,
memberEpoch = grp2Member1Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp2Member1Response.activeTasks),
standbyTasks = convertTaskIds(grp2Member1Response.standbyTasks),
warmupTasks = convertTaskIds(grp2Member1Response.warmupTasks),
topology = null
)
grp2Member2Response = streamsGroupHeartbeat(
groupId = "grp-2",
memberId = grp2Member2Response.memberId,
memberEpoch = grp2Member2Response.memberEpoch,
rebalanceTimeoutMs = timeoutMs,
activeTasks = convertTaskIds(grp2Member2Response.activeTasks),
standbyTasks = convertTaskIds(grp2Member2Response.standbyTasks),
warmupTasks = convertTaskIds(grp2Member2Response.warmupTasks),
topology = null
)
val actual = streamsGroupDescribe(
groupIds = List("grp-1","grp-2"),
includeAuthorizedOperations = true,
version = ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled).toShort
)
actual.head.groupState() == "Stable" && actual(1).groupState() == "Stable" &&
actual.head.members.size == 2 && actual(1).members.size == 2
}, "Two groups did not stabilize with 2 members each in time")
// Test the describe request for both groups in stable state
for (version <- ApiKeys.STREAMS_GROUP_DESCRIBE.oldestVersion() to ApiKeys.STREAMS_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)) {
val actual = streamsGroupDescribe(
groupIds = List("grp-1","grp-2"),
includeAuthorizedOperations = true,
version = version.toShort
)
assertEquals(2, actual.size)
assertEquals(actual.map(_.groupId).toSet, Set("grp-1", "grp-2"))
for (describedGroup <- actual) {
assertEquals("Stable", describedGroup.groupState)
assertTrue("Group epoch is not equal to the assignment epoch", describedGroup.groupEpoch == describedGroup.assignmentEpoch)
// Verify topology
assertEquals(1, describedGroup.topology.epoch)
assertEquals(1, describedGroup.topology.subtopologies.size)
assertEquals("subtopology-1", describedGroup.topology.subtopologies.get(0).subtopologyId)
assertEquals(List(topicName).asJava, describedGroup.topology.subtopologies.get(0).sourceTopics)
// Verify members
assertEquals(2, describedGroup.members.size)
val expectedMemberIds = describedGroup.groupId match {
case "grp-1" => Set(grp1Member1Response.memberId, grp1Member2Response.memberId)
case "grp-2" => Set(grp2Member1Response.memberId, grp2Member2Response.memberId)
case unexpected => throw new AssertionError(s"Unexpected group ID: $unexpected")
}
val actualMemberIds = describedGroup.members.asScala.map(_.memberId).toSet
assertEquals(expectedMemberIds, actualMemberIds)
assertEquals(authorizedOperationsInt, describedGroup.authorizedOperations)
describedGroup.members.asScala.foreach { member =>
assertTrue("Group epoch is not equal to the member epoch", member.memberEpoch == describedGroup.assignmentEpoch)
assertEquals(1, member.topologyEpoch)
assertEquals(member.targetAssignment, member.assignment)
assertEquals(clientId, member.clientId())
assertEquals(clientHost, member.clientHost())
}
// Verify all partitions 0, 1, 2 are assigned exactly once
val allAssignedPartitions = describedGroup.members.asScala.flatMap { member =>
member.assignment.activeTasks.asScala.flatMap(_.partitions.asScala)
}.toList
assertEquals(List(0, 1, 2).sorted, allAssignedPartitions.sorted)
}
}
} finally{
admin.close()
}
}
private def convertTaskIds(responseTasks: java.util.List[StreamsGroupHeartbeatResponseData.TaskIds]): List[StreamsGroupHeartbeatRequestData.TaskIds] = {
if (responseTasks == null) {
List.empty
} else {
responseTasks.asScala.map { responseTask =>
new StreamsGroupHeartbeatRequestData.TaskIds()
.setSubtopologyId(responseTask.subtopologyId)
.setPartitions(responseTask.partitions)
}.toList
}
}
}

View File

@ -189,6 +189,13 @@
A new metric <code>AvgIdleRatio</code> has been added to the <code>ControllerEventManager</code> group. This metric measures the average idle ratio of the controller event queue thread,
providing visibility into how much time the controller spends waiting for events versus processing them. The metric value ranges from 0.0 (always busy) to 1.0 (always idle).
</li>
<li>
Deprecated <code>org.apache.kafka.streams.KafkaStreams$CloseOptions</code> and its related methods, such as
<code>KafkaStreams#close(org.apache.kafka.streams.KafkaStreams$CloseOptions)</code>.
As a replacement, please use <code>org.apache.kafka.streams.CloseOptions</code> and
<code>KafkaStreams#close(org.apache.kafka.streams.CloseOptions)</code>.
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/QAq9F">KIP-1153</a>.
</li>
</ul>
<h4><a id="upgrade_4_1_0" href="#upgrade_4_1_0">Upgrading to 4.1.0</a></h4>

View File

@ -60,7 +60,7 @@ versions += [
commonsLang: "3.18.0",
commonsValidator: "1.10.0",
classgraph: "4.8.179",
gradle: "8.14.3",
gradle: "9.1.0",
grgit: "4.1.1",
httpclient: "4.5.14",
jackson: "2.19.0",
@ -125,7 +125,7 @@ versions += [
snappy: "1.1.10.7",
spotbugs: "4.9.4",
mockOAuth2Server: "2.2.1",
zinc: "1.9.2",
zinc: "1.10.8",
// When updating the zstd version, please do as well in docker/native/native-image-configs/resource-config.json
// Also make sure the compression levels in org.apache.kafka.common.record.CompressionType are still valid
zstd: "1.5.6-10",

View File

@ -1,7 +1,7 @@
distributionBase=GRADLE_USER_HOME
distributionPath=wrapper/dists
distributionSha256Sum=bd71102213493060956ec229d946beee57158dbd89d0e62b91bca0fa2c5f3531
distributionUrl=https\://services.gradle.org/distributions/gradle-8.14.3-bin.zip
distributionSha256Sum=a17ddd85a26b6a7f5ddb71ff8b05fc5104c0202c6e64782429790c933686c806
distributionUrl=https\://services.gradle.org/distributions/gradle-9.1.0-bin.zip
networkTimeout=10000
validateDistributionUrl=true
zipStoreBase=GRADLE_USER_HOME

41
gradlew vendored
View File

@ -1,7 +1,7 @@
#!/bin/sh
#
# Copyright © 2015-2021 the original authors.
# Copyright © 2015 the original authors.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
@ -15,6 +15,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
#
# SPDX-License-Identifier: Apache-2.0
#
##############################################################################
#
@ -55,7 +57,7 @@
# Darwin, MinGW, and NonStop.
#
# (3) This script is generated from the Groovy template
# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt
# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt
# within the Gradle project.
#
# You can find Gradle at https://github.com/gradle/gradle/.
@ -84,7 +86,7 @@ done
# shellcheck disable=SC2034
APP_BASE_NAME=${0##*/}
# Discard cd standard output in case $CDPATH is set (https://github.com/gradle/gradle/issues/25036)
APP_HOME=$( cd "${APP_HOME:-./}" > /dev/null && pwd -P ) || exit
APP_HOME=$( cd -P "${APP_HOME:-./}" > /dev/null && printf '%s\n' "$PWD" ) || exit
# Use the maximum available, or set MAX_FD != -1 to use that value.
MAX_FD=maximum
@ -113,20 +115,6 @@ case "$( uname )" in #(
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/v8.14.3/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
continue
fi
fi
done
CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
# Determine the Java command to use to start the JVM.
if [ -n "$JAVA_HOME" ] ; then
@ -183,7 +171,6 @@ fi
# For Cygwin or MSYS, switch paths to Windows format before running java
if "$cygwin" || "$msys" ; then
APP_HOME=$( cygpath --path --mixed "$APP_HOME" )
CLASSPATH=$( cygpath --path --mixed "$CLASSPATH" )
JAVACMD=$( cygpath --unix "$JAVACMD" )
@ -212,19 +199,31 @@ if "$cygwin" || "$msys" ; then
fi
# 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/v9.1.0/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
continue
fi
fi
done
# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"'
# Collect all arguments for the java command:
# * DEFAULT_JVM_OPTS, JAVA_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments,
# * DEFAULT_JVM_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments,
# and any embedded shellness will be escaped.
# * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be
# treated as '${Hostname}' itself on the command line.
set -- \
"-Dorg.gradle.appname=$APP_BASE_NAME" \
-classpath "$CLASSPATH" \
org.gradle.wrapper.GradleWrapperMain \
-jar "$APP_HOME/gradle/wrapper/gradle-wrapper.jar" \
"$@"
# Stop when "xargs" is not available.

View File

@ -15,13 +15,16 @@
* limitations under the License.
*/
package kafka.server;
package org.apache.kafka.server;
import org.apache.kafka.clients.admin.AddRaftVoterOptions;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.FeatureMetadata;
import org.apache.kafka.clients.admin.QuorumInfo;
import org.apache.kafka.clients.admin.RaftVoterEndpoint;
import org.apache.kafka.clients.admin.RemoveRaftVoterOptions;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.InconsistentClusterIdException;
import org.apache.kafka.common.test.KafkaClusterTestKit;
import org.apache.kafka.common.test.TestKitNodes;
import org.apache.kafka.common.test.api.TestKitDefaults;
@ -29,10 +32,12 @@ import org.apache.kafka.raft.QuorumConfig;
import org.apache.kafka.server.common.KRaftVersion;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TreeMap;
@ -41,6 +46,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
@Tag("integration")
public class ReconfigurableQuorumIntegrationTest {
static void checkKRaftVersions(Admin admin, short finalized) throws Exception {
FeatureMetadata featureMetadata = admin.describeFeatures().featureMetadata().get();
@ -70,7 +76,7 @@ public class ReconfigurableQuorumIntegrationTest {
).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_0.featureLevel());
});
@ -88,7 +94,7 @@ public class ReconfigurableQuorumIntegrationTest {
).setStandalone(true).build()) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, () -> {
checkKRaftVersions(admin, KRaftVersion.KRAFT_VERSION_1.featureLevel());
});
@ -126,7 +132,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
@ -161,7 +167,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002, 3003), voters.keySet());
@ -200,7 +206,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
@ -238,7 +244,7 @@ public class ReconfigurableQuorumIntegrationTest {
) {
cluster.format();
cluster.startup();
try (Admin admin = Admin.create(cluster.clientProperties())) {
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
@ -249,4 +255,95 @@ public class ReconfigurableQuorumIntegrationTest {
}
}
}
}
@Test
public void testRemoveAndAddVoterWithValidClusterId() throws Exception {
final var nodes = new TestKitNodes.Builder()
.setClusterId("test-cluster")
.setNumBrokerNodes(1)
.setNumControllerNodes(3)
.build();
final Map<Integer, Uuid> initialVoters = new HashMap<>();
for (final var controllerNode : nodes.controllerNodes().values()) {
initialVoters.put(
controllerNode.id(),
controllerNode.metadataDirectoryId()
);
}
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
cluster.format();
cluster.startup();
try (var admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
for (int replicaId : new int[] {3000, 3001, 3002}) {
assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));
}
});
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
admin.removeRaftVoter(
3000,
dirId,
new RemoveRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
).all().get();
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(Set.of(3001, 3002), voters.keySet());
for (int replicaId : new int[] {3001, 3002}) {
assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));
}
});
admin.addRaftVoter(
3000,
dirId,
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
new AddRaftVoterOptions().setClusterId(Optional.of("test-cluster"))
).all().get();
}
}
}
@Test
public void testRemoveAndAddVoterWithInconsistentClusterId() throws Exception {
final var nodes = new TestKitNodes.Builder()
.setClusterId("test-cluster")
.setNumBrokerNodes(1)
.setNumControllerNodes(3)
.build();
final Map<Integer, Uuid> initialVoters = new HashMap<>();
for (final var controllerNode : nodes.controllerNodes().values()) {
initialVoters.put(
controllerNode.id(),
controllerNode.metadataDirectoryId()
);
}
try (var cluster = new KafkaClusterTestKit.Builder(nodes).setInitialVoterSet(initialVoters).build()) {
cluster.format();
cluster.startup();
try (var admin = Admin.create(cluster.clientProperties())) {
Uuid dirId = cluster.nodes().controllerNodes().get(3000).metadataDirectoryId();
var removeFuture = admin.removeRaftVoter(
3000,
dirId,
new RemoveRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
).all();
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, removeFuture);
var addFuture = admin.addRaftVoter(
3000,
dirId,
Set.of(new RaftVoterEndpoint("CONTROLLER", "example.com", 8080)),
new AddRaftVoterOptions().setClusterId(Optional.of("inconsistent"))
).all();
TestUtils.assertFutureThrows(InconsistentClusterIdException.class, addFuture);
}
}
}
}

View File

@ -28,8 +28,8 @@ import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KafkaStreams.CloseOptions;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
@ -159,7 +159,7 @@ public class KafkaStreamsCloseOptionsIntegrationTest {
IntegrationTestUtils.startApplicationAndWaitUntilRunning(streams);
IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(resultConsumerConfig, OUTPUT_TOPIC, 10);
streams.close(new CloseOptions().leaveGroup(true).timeout(Duration.ofSeconds(30)));
streams.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP).withTimeout(Duration.ofSeconds(30)));
waitForEmptyConsumerGroup(adminClient, streamsConfig.getProperty(StreamsConfig.APPLICATION_ID_CONFIG), 0);
}

View File

@ -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.streams;
import java.time.Duration;
import java.util.Objects;
import java.util.Optional;
public class CloseOptions {
/**
* Enum to specify the group membership operation upon closing the Kafka Streams application.
*
* <ul>
* <li><b>{@code LEAVE_GROUP}</b>: means the consumer leave the group.</li>
* <li><b>{@code REMAIN_IN_GROUP}</b>: means the consumer will remain in the group.</li>
* </ul>
*/
public enum GroupMembershipOperation {
LEAVE_GROUP,
REMAIN_IN_GROUP
}
/**
* Specifies the group membership operation upon shutdown.
* By default, {@code GroupMembershipOperation.REMAIN_IN_GROUP} will be applied, which follows the KafkaStreams default behavior.
*/
protected GroupMembershipOperation operation = GroupMembershipOperation.REMAIN_IN_GROUP;
/**
* Specifies the maximum amount of time to wait for the close process to complete.
* This allows users to define a custom timeout for gracefully stopping the KafkaStreams.
*/
protected Optional<Duration> timeout = Optional.of(Duration.ofMillis(Long.MAX_VALUE));
private CloseOptions() {
}
protected CloseOptions(final CloseOptions closeOptions) {
this.operation = closeOptions.operation;
this.timeout = closeOptions.timeout;
}
/**
* Static method to create a {@code CloseOptions} with a custom timeout.
*
* @param timeout the maximum time to wait for the KafkaStreams to close.
* @return a new {@code CloseOptions} instance with the specified timeout.
*/
public static CloseOptions timeout(final Duration timeout) {
return new CloseOptions().withTimeout(timeout);
}
/**
* Static method to create a {@code CloseOptions} with a specified group membership operation.
*
* @param operation the group membership operation to apply. Must be one of {@code LEAVE_GROUP}, {@code REMAIN_IN_GROUP}.
* @return a new {@code CloseOptions} instance with the specified group membership operation.
*/
public static CloseOptions groupMembershipOperation(final GroupMembershipOperation operation) {
return new CloseOptions().withGroupMembershipOperation(operation);
}
/**
* Fluent method to set the timeout for the close process.
*
* @param timeout the maximum time to wait for the KafkaStreams to close. If {@code null}, the default timeout will be used.
* @return this {@code CloseOptions} instance.
*/
public CloseOptions withTimeout(final Duration timeout) {
this.timeout = Optional.ofNullable(timeout);
return this;
}
/**
* Fluent method to set the group membership operation upon shutdown.
*
* @param operation the group membership operation to apply. Must be one of {@code LEAVE_GROUP}, {@code REMAIN_IN_GROUP}.
* @return this {@code CloseOptions} instance.
*/
public CloseOptions withGroupMembershipOperation(final GroupMembershipOperation operation) {
this.operation = Objects.requireNonNull(operation, "operation should not be null");
return this;
}
}

View File

@ -49,6 +49,7 @@ import org.apache.kafka.streams.errors.StreamsStoppedException;
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
import org.apache.kafka.streams.errors.UnknownStateStoreException;
import org.apache.kafka.streams.internals.ClientInstanceIdsImpl;
import org.apache.kafka.streams.internals.CloseOptionsInternal;
import org.apache.kafka.streams.internals.metrics.ClientMetrics;
import org.apache.kafka.streams.internals.metrics.StreamsClientMetricsDelegatingReporter;
import org.apache.kafka.streams.processor.StandbyUpdateListener;
@ -488,7 +489,7 @@ public class KafkaStreams implements AutoCloseable {
closeToError();
}
final StreamThread deadThread = (StreamThread) Thread.currentThread();
deadThread.shutdown(false);
deadThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
addStreamThread();
if (throwable instanceof RuntimeException) {
throw (RuntimeException) throwable;
@ -765,7 +766,7 @@ public class KafkaStreams implements AutoCloseable {
@Override
public void onUpdateStart(final TopicPartition topicPartition,
final String storeName,
final String storeName,
final long startingOffset) {
if (userStandbyListener != null) {
try {
@ -1136,7 +1137,7 @@ public class KafkaStreams implements AutoCloseable {
return Optional.of(streamThread.getName());
} else {
log.warn("Terminating the new thread because the Kafka Streams client is in state {}", state);
streamThread.shutdown(true);
streamThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
threads.remove(streamThread);
final long cacheSizePerThread = cacheSizePerThread(numLiveStreamThreads());
log.info("Resizing thread cache due to terminating added thread, new cache size per thread is {}", cacheSizePerThread);
@ -1200,7 +1201,7 @@ public class KafkaStreams implements AutoCloseable {
final boolean callingThreadIsNotCurrentStreamThread = !streamThread.getName().equals(Thread.currentThread().getName());
if (streamThread.isThreadAlive() && (callingThreadIsNotCurrentStreamThread || numLiveStreamThreads() == 1)) {
log.info("Removing StreamThread {}", streamThread.getName());
streamThread.shutdown(true);
streamThread.shutdown(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
if (callingThreadIsNotCurrentStreamThread) {
final long remainingTimeMs = timeoutMs - (time.milliseconds() - startMs);
if (remainingTimeMs <= 0 || !streamThread.waitOnThreadState(StreamThread.State.DEAD, remainingTimeMs)) {
@ -1418,15 +1419,18 @@ public class KafkaStreams implements AutoCloseable {
/**
* Class that handles options passed in case of {@code KafkaStreams} instance scale down
*/
@Deprecated(since = "4.2")
public static class CloseOptions {
private Duration timeout = Duration.ofMillis(Long.MAX_VALUE);
private boolean leaveGroup = false;
@Deprecated(since = "4.2")
public CloseOptions timeout(final Duration timeout) {
this.timeout = timeout;
return this;
}
@Deprecated(since = "4.2")
public CloseOptions leaveGroup(final boolean leaveGroup) {
this.leaveGroup = leaveGroup;
return this;
@ -1438,10 +1442,14 @@ public class KafkaStreams implements AutoCloseable {
* This will block until all threads have stopped.
*/
public void close() {
close(Optional.empty(), false);
close(Optional.empty(), org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
}
private Thread shutdownHelper(final boolean error, final long timeoutMs, final boolean leaveGroup) {
private Thread shutdownHelper(
final boolean error,
final long timeoutMs,
final org.apache.kafka.streams.CloseOptions.GroupMembershipOperation operation
) {
stateDirCleaner.shutdownNow();
if (rocksDBMetricsRecordingService != null) {
rocksDBMetricsRecordingService.shutdownNow();
@ -1453,7 +1461,9 @@ public class KafkaStreams implements AutoCloseable {
return new Thread(() -> {
// notify all the threads to stop; avoid deadlocks by stopping any
// further state reports from the thread since we're shutting down
int numStreamThreads = processStreamThread(streamThread -> streamThread.shutdown(leaveGroup));
int numStreamThreads = processStreamThread(
streamThread -> streamThread.shutdown(operation)
);
log.info("Shutting down {} stream threads", numStreamThreads);
@ -1513,7 +1523,7 @@ public class KafkaStreams implements AutoCloseable {
}, clientId + "-CloseThread");
}
private boolean close(final Optional<Long> timeout, final boolean leaveGroup) {
private boolean close(final Optional<Long> timeout, final org.apache.kafka.streams.CloseOptions.GroupMembershipOperation operation) {
final long timeoutMs;
if (timeout.isPresent()) {
timeoutMs = timeout.get();
@ -1544,7 +1554,7 @@ public class KafkaStreams implements AutoCloseable {
+ "PENDING_SHUTDOWN, PENDING_ERROR, ERROR, or NOT_RUNNING");
}
final Thread shutdownThread = shutdownHelper(false, timeoutMs, leaveGroup);
final Thread shutdownThread = shutdownHelper(false, timeoutMs, operation);
shutdownThread.setDaemon(true);
shutdownThread.start();
@ -1562,7 +1572,7 @@ public class KafkaStreams implements AutoCloseable {
if (!setState(State.PENDING_ERROR)) {
log.info("Skipping shutdown since we are already in {}", state());
} else {
final Thread shutdownThread = shutdownHelper(true, -1, false);
final Thread shutdownThread = shutdownHelper(true, -1, org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
shutdownThread.setDaemon(true);
shutdownThread.start();
@ -1588,12 +1598,13 @@ public class KafkaStreams implements AutoCloseable {
throw new IllegalArgumentException("Timeout can't be negative.");
}
return close(Optional.of(timeoutMs), false);
return close(Optional.of(timeoutMs), org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
}
/**
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
* threads to join.
* This method is deprecated and replaced by {@link #close(org.apache.kafka.streams.CloseOptions)}.
* @param options contains timeout to specify how long to wait for the threads to shut down, and a flag leaveGroup to
* trigger consumer leave call
* @return {@code true} if all threads were successfully stopped&mdash;{@code false} if the timeout was reached
@ -1601,15 +1612,36 @@ public class KafkaStreams implements AutoCloseable {
* Note that this method must not be called in the {@link StateListener#onChange(KafkaStreams.State, KafkaStreams.State)} callback of {@link StateListener}.
* @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds}
*/
@Deprecated(since = "4.2")
public synchronized boolean close(final CloseOptions options) throws IllegalArgumentException {
final org.apache.kafka.streams.CloseOptions closeOptions = org.apache.kafka.streams.CloseOptions.timeout(options.timeout)
.withGroupMembershipOperation(options.leaveGroup ?
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP :
org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
return close(closeOptions);
}
/**
* Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the
* threads to join.
* @param options contains timeout to specify how long to wait for the threads to shut down,
* and a {@link org.apache.kafka.streams.CloseOptions.GroupMembershipOperation}
* to trigger consumer leave call or remain in the group
* @return {@code true} if all threads were successfully stopped&mdash;{@code false} if the timeout was reached
* before all threads stopped
* Note that this method must not be called in the {@link StateListener#onChange(KafkaStreams.State, KafkaStreams.State)} callback of {@link StateListener}.
* @throws IllegalArgumentException if {@code timeout} can't be represented as {@code long milliseconds}
*/
public synchronized boolean close(final org.apache.kafka.streams.CloseOptions options) throws IllegalArgumentException {
Objects.requireNonNull(options, "options cannot be null");
final String msgPrefix = prepareMillisCheckFailMsgPrefix(options.timeout, "timeout");
final long timeoutMs = validateMillisecondDuration(options.timeout, msgPrefix);
final CloseOptionsInternal optionsInternal = new CloseOptionsInternal(options);
final String msgPrefix = prepareMillisCheckFailMsgPrefix(optionsInternal.timeout(), "timeout");
final long timeoutMs = validateMillisecondDuration(optionsInternal.timeout().get(), msgPrefix);
if (timeoutMs < 0) {
throw new IllegalArgumentException("Timeout can't be negative.");
}
return close(Optional.of(timeoutMs), options.leaveGroup);
return close(Optional.of(timeoutMs), optionsInternal.operation());
}
/**

View File

@ -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.streams.internals;
import org.apache.kafka.streams.CloseOptions;
import java.time.Duration;
import java.util.Optional;
public class CloseOptionsInternal extends CloseOptions {
public CloseOptionsInternal(final CloseOptions options) {
super(options);
}
public GroupMembershipOperation operation() {
return operation;
}
public Optional<Duration> timeout() {
return timeout;
}
}

View File

@ -91,9 +91,9 @@ import java.util.Optional;
import java.util.Queue;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
@ -367,7 +367,8 @@ public class StreamThread extends Thread implements ProcessingThread {
// These are used to signal from outside the stream thread, but the variables themselves are internal to the thread
private final AtomicLong cacheResizeSize = new AtomicLong(-1L);
private final AtomicBoolean leaveGroupRequested = new AtomicBoolean(false);
private final AtomicReference<org.apache.kafka.streams.CloseOptions.GroupMembershipOperation> leaveGroupRequested =
new AtomicReference<>(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
private final AtomicLong lastShutdownWarningTimestamp = new AtomicLong(0L);
private final boolean eosEnabled;
private final boolean stateUpdaterEnabled;
@ -898,7 +899,7 @@ public class StreamThread extends Thread implements ProcessingThread {
cleanRun = runLoop();
} catch (final Throwable e) {
failedStreamThreadSensor.record();
leaveGroupRequested.set(true);
leaveGroupRequested.set(org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streamsUncaughtExceptionHandler.accept(e, false);
// Note: the above call currently rethrows the exception, so nothing below this line will be executed
} finally {
@ -1547,7 +1548,7 @@ public class StreamThread extends Thread implements ProcessingThread {
if (streamsRebalanceData.isPresent()) {
boolean hasMissingSourceTopics = false;
String missingTopicsDetail = null;
for (final StreamsGroupHeartbeatResponseData.Status status : streamsRebalanceData.get().statuses()) {
if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.SHUTDOWN_APPLICATION.code()) {
shutdownErrorHook.run();
@ -1560,7 +1561,7 @@ public class StreamThread extends Thread implements ProcessingThread {
throw new TopologyException(errorMsg);
}
}
if (hasMissingSourceTopics) {
handleMissingSourceTopicsWithTimeout(missingTopicsDetail);
} else {
@ -1589,25 +1590,25 @@ public class StreamThread extends Thread implements ProcessingThread {
// Start timeout tracking on first encounter with missing topics
if (topicsReadyTimer == null) {
topicsReadyTimer = time.timer(maxPollTimeMs);
log.info("Missing source topics detected: {}. Will wait up to {}ms before failing.",
log.info("Missing source topics detected: {}. Will wait up to {}ms before failing.",
missingTopicsDetail, maxPollTimeMs);
} else {
topicsReadyTimer.update();
}
if (topicsReadyTimer.isExpired()) {
final long elapsedTime = topicsReadyTimer.elapsedMs();
final String errorMsg = String.format("Missing source topics: %s. Timeout exceeded after %dms.",
final String errorMsg = String.format("Missing source topics: %s. Timeout exceeded after %dms.",
missingTopicsDetail, elapsedTime);
log.error(errorMsg);
throw new MissingSourceTopicException(errorMsg);
} else {
log.debug("Missing source topics: {}. Elapsed time: {}ms, timeout in: {}ms",
log.debug("Missing source topics: {}. Elapsed time: {}ms, timeout in: {}ms",
missingTopicsDetail, topicsReadyTimer.elapsedMs(), topicsReadyTimer.remainingMs());
}
}
static Map<TopicPartition, PartitionInfo> getTopicPartitionInfo(final Map<HostInfo, Set<TopicPartition>> partitionsByHost) {
final Map<TopicPartition, PartitionInfo> topicToPartitionInfo = new HashMap<>();
@ -1879,12 +1880,12 @@ public class StreamThread extends Thread implements ProcessingThread {
* Note that there is nothing to prevent this function from being called multiple times
* (e.g., in testing), hence the state is set only the first time
*
* @param leaveGroup this flag will control whether the consumer will leave the group on close or not
* @param operation the group membership operation to apply on shutdown. Must be one of LEAVE_GROUP or REMAIN_IN_GROUP.
*/
public void shutdown(final boolean leaveGroup) {
public void shutdown(final org.apache.kafka.streams.CloseOptions.GroupMembershipOperation operation) {
log.info("Informed to shut down");
final State oldState = setState(State.PENDING_SHUTDOWN);
leaveGroupRequested.set(leaveGroup);
leaveGroupRequested.set(operation);
if (oldState == State.CREATED) {
// The thread may not have been started. Take responsibility for shutting down
completeShutdown(true);
@ -1917,7 +1918,8 @@ public class StreamThread extends Thread implements ProcessingThread {
log.error("Failed to close changelog reader due to the following error:", e);
}
try {
final GroupMembershipOperation membershipOperation = leaveGroupRequested.get() ? LEAVE_GROUP : REMAIN_IN_GROUP;
final GroupMembershipOperation membershipOperation =
leaveGroupRequested.get() == org.apache.kafka.streams.CloseOptions.GroupMembershipOperation.LEAVE_GROUP ? LEAVE_GROUP : REMAIN_IN_GROUP;
mainConsumer.close(CloseOptions.groupMembershipOperation(membershipOperation));
} catch (final Throwable e) {
log.error("Failed to close consumer due to the following error:", e);

View File

@ -19,7 +19,6 @@ package org.apache.kafka.streams;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo;
import org.apache.kafka.clients.admin.MockAdminClient;
import org.apache.kafka.clients.consumer.CloseOptions;
import org.apache.kafka.clients.producer.MockProducer;
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.KafkaFuture;
@ -310,8 +309,12 @@ public class KafkaStreamsTest {
private void prepareConsumer(final StreamThread thread, final AtomicReference<StreamThread.State> state) {
doAnswer(invocation -> {
supplier.consumer.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP));
supplier.restoreConsumer.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP));
supplier.consumer.close(
org.apache.kafka.clients.consumer.CloseOptions.groupMembershipOperation(org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP)
);
supplier.restoreConsumer.close(
org.apache.kafka.clients.consumer.CloseOptions.groupMembershipOperation(org.apache.kafka.clients.consumer.CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP)
);
for (final MockProducer<byte[], byte[]> producer : supplier.producers) {
producer.close();
}
@ -320,7 +323,7 @@ public class KafkaStreamsTest {
threadStateListenerCapture.getValue().onChange(thread, StreamThread.State.PENDING_SHUTDOWN, StreamThread.State.RUNNING);
threadStateListenerCapture.getValue().onChange(thread, StreamThread.State.DEAD, StreamThread.State.PENDING_SHUTDOWN);
return null;
}).when(thread).shutdown(false);
}).when(thread).shutdown(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
}
private void prepareThreadLock(final StreamThread thread) {
@ -571,7 +574,7 @@ public class KafkaStreamsTest {
for (int i = 0; i < NUM_THREADS; i++) {
final StreamThread tmpThread = streams.threads.get(i);
tmpThread.shutdown(false);
tmpThread.shutdown(CloseOptions.GroupMembershipOperation.REMAIN_IN_GROUP);
waitForCondition(() -> tmpThread.state() == StreamThread.State.DEAD,
"Thread never stopped.");
streams.threads.get(i).join();
@ -790,7 +793,7 @@ public class KafkaStreamsTest {
prepareThreadLock(streamThreadTwo);
try (final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time)) {
streams.start();
streamThreadOne.shutdown(true);
streamThreadOne.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
final Set<ThreadMetadata> threads = streams.metadataForLocalThreads();
assertThat(threads.size(), equalTo(1));
assertThat(threads, hasItem(streamThreadTwo.threadMetadata()));
@ -1016,9 +1019,8 @@ public class KafkaStreamsTest {
() -> streams.state() == KafkaStreams.State.RUNNING,
"Streams never started.");
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO)
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streams.close(closeOptions);
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
@ -1041,8 +1043,7 @@ public class KafkaStreamsTest {
() -> streams.state() == KafkaStreams.State.RUNNING,
"Streams never started.");
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO);
streams.close(closeOptions);
assertThat(streams.state() == State.PENDING_SHUTDOWN, equalTo(true));
@ -1229,8 +1230,7 @@ public class KafkaStreamsTest {
prepareStreamThread(streamThreadTwo, 2);
prepareTerminableThread(streamThreadOne);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(10L));
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(10L));
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1243,8 +1243,7 @@ public class KafkaStreamsTest {
prepareStreamThread(streamThreadTwo, 2);
prepareTerminableThread(streamThreadOne);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(-1L));
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(-1L));
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier, time)) {
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
}
@ -1257,8 +1256,7 @@ public class KafkaStreamsTest {
prepareStreamThread(streamThreadTwo, 2);
prepareTerminableThread(streamThreadOne);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, supplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1275,9 +1273,8 @@ public class KafkaStreamsTest {
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(10L));
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(10L))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1293,9 +1290,8 @@ public class KafkaStreamsTest {
final MockClientSupplier mockClientSupplier = spy(MockClientSupplier.class);
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofMillis(-1L));
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ofMillis(-1L))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier, time)) {
assertThrows(IllegalArgumentException.class, () -> streams.close(closeOptions));
}
@ -1312,9 +1308,8 @@ public class KafkaStreamsTest {
when(mockClientSupplier.getAdmin(any())).thenReturn(adminClient);
final KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ZERO);
closeOptions.leaveGroup(true);
final CloseOptions closeOptions = CloseOptions.timeout(Duration.ZERO)
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
try (final KafkaStreams streams = new KafkaStreamsWithTerminableThread(getBuilderWithSource().build(), props, mockClientSupplier)) {
assertFalse(streams.close(closeOptions));
}
@ -1720,7 +1715,7 @@ public class KafkaStreamsTest {
producerFuture.complete(producerInstanceId);
final Uuid adminInstanceId = Uuid.randomUuid();
adminClient.setClientInstanceId(adminInstanceId);
final Map<String, KafkaFuture<Uuid>> expectedClientIds = Map.of("main-consumer", consumerFuture, "some-thread-producer", producerFuture);
when(streamThreadOne.clientInstanceIds(any())).thenReturn(expectedClientIds);

View File

@ -60,6 +60,7 @@ import org.apache.kafka.common.utils.LogCaptureAppender;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.GroupProtocol;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.StreamsConfig.InternalConfig;
@ -247,7 +248,7 @@ public class StreamThreadTest {
if (thread.state() != State.CREATED) {
thread.taskManager().shutdown(false);
}
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
thread = null;
}
final Set<Thread> t = Collections.unmodifiableSet(Thread.getAllStackTraces().keySet());
@ -409,12 +410,12 @@ public class StreamThreadTest {
assertEquals(4, stateListener.numChanges);
assertEquals(StreamThread.State.PARTITIONS_ASSIGNED, stateListener.oldState);
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
assertSame(StreamThread.State.PENDING_SHUTDOWN, thread.state());
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldChangeStateAtStartClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
@ -427,18 +428,18 @@ public class StreamThreadTest {
10 * 1000,
"Thread never started.");
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
TestUtils.waitForCondition(
() -> thread.state() == StreamThread.State.DEAD,
10 * 1000,
"Thread never shut down.");
thread.shutdown(true);
assertEquals(thread.state(), StreamThread.State.DEAD);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
assertEquals(State.DEAD, thread.state());
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCreateMetricsAtStartup(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread(CLIENT_ID, new MockTime(1), stateUpdaterEnabled, processingThreadsEnabled);
final String defaultGroupName = "stream-thread-metrics";
@ -538,7 +539,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCommitBeforeTheCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 1000L;
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -565,7 +566,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotPurgeBeforeThePurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 1000L;
final long purgeInterval = 2000L;
@ -593,7 +594,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldAlsoPurgeWhenNothingGetsCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long purgeInterval = 1000L;
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -658,7 +659,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotProcessWhenPartitionRevoked(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(processingThreadsEnabled);
@ -682,7 +683,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldProcessWhenRunning(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(processingThreadsEnabled);
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -707,7 +708,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldProcessWhenPartitionAssigned(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
assumeFalse(processingThreadsEnabled);
@ -732,7 +733,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldProcessWhenStarting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
assumeFalse(processingThreadsEnabled);
@ -757,7 +758,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldEnforceRebalanceWhenScheduledAndNotCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
final Time mockTime = new MockTime(1);
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -812,7 +813,7 @@ public class StreamThreadTest {
10 * 1000,
"Thread never started.");
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
TestUtils.waitForCondition(
() -> thread.state() == StreamThread.State.DEAD,
10 * 1000,
@ -822,7 +823,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotEnforceRebalanceWhenCurrentlyRebalancing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
final Time mockTime = new MockTime(1);
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -880,7 +881,7 @@ public class StreamThreadTest {
() -> { }
);
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
// Validate that the scheduled rebalance wasn't reset then set to MAX_VALUE so we
// don't trigger one before we can shut down, since the rebalance must be ended
@ -918,7 +919,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRespectNumIterationsInMainLoopWithoutProcessingThreads(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
// With processing threads, there is no guarantee how many iterations will be performed
assumeFalse(processingThreadsEnabled);
@ -1047,7 +1048,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCauseExceptionIfNothingCommitted(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 1000L;
final Properties props = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -1076,7 +1077,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCommitAfterCommitInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 100L;
final long commitLatency = 10L;
@ -1137,7 +1138,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldPurgeAfterPurgeInterval(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final long commitInterval = 100L;
final long purgeInterval = 200L;
@ -1170,7 +1171,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRecordCommitLatency(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1279,7 +1280,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldInjectSharedProducerForAllTasksUsingClientSupplierOnCreateIfEosDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
internalStreamsBuilder.buildAndOptimizeTopology();
@ -1319,7 +1320,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldInjectProducerPerThreadUsingClientSupplierOnCreateIfEosV2Enabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1);
@ -1357,7 +1358,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldOnlyCompleteShutdownAfterRebalanceNotInProgress(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws InterruptedException {
// The state updater is disabled for this test because this test relies on the fact the mainConsumer.resume()
// is not called. This is not true when the state updater is enabled which leads to
@ -1390,7 +1391,7 @@ public class StreamThreadTest {
10 * 1000,
"Thread never started.");
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
// even if thread is no longer running, it should still be polling
// as long as the rebalance is still ongoing
@ -1411,7 +1412,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldShutdownTaskManagerOnClose(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1426,7 +1427,7 @@ public class StreamThreadTest {
thread.setStateListener(
(t, newState, oldState) -> {
if (oldState == StreamThread.State.CREATED && newState == StreamThread.State.STARTING) {
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
}
});
thread.run();
@ -1435,7 +1436,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotReturnDataAfterTaskMigrated(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final TaskManager taskManager = mock(TaskManager.class);
final InternalTopologyBuilder internalTopologyBuilder = mock(InternalTopologyBuilder.class);
@ -1512,7 +1513,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldShutdownTaskManagerOnCloseWithoutStart(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1524,13 +1525,13 @@ public class StreamThreadTest {
topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
.updateThreadMetadata(adminClientId(CLIENT_ID));
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
verify(taskManager).shutdown(true);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldOnlyShutdownOnce(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
when(consumer.groupMetadata()).thenReturn(consumerGroupMetadata);
@ -1542,7 +1543,7 @@ public class StreamThreadTest {
topologyMetadata.buildAndRewriteTopology();
thread = buildStreamThread(consumer, taskManager, config, topologyMetadata)
.updateThreadMetadata(adminClientId(CLIENT_ID));
thread.shutdown(true);
thread.shutdown(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
// Execute the run method. Verification of the mock will check that shutdown was only done once
thread.run();
@ -1550,7 +1551,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotThrowWhenStandbyTasksAssignedAndNoStateStoresForTopology(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
internalTopologyBuilder.addSource(null, "name", null, null, null, "topic");
internalTopologyBuilder.addSink("out", "output", null, null, null, "name");
@ -1572,7 +1573,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerWasFencedWhileProcessing(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
internalTopologyBuilder.addSink("sink", "dummyTopic", null, null, null, "source");
@ -1688,18 +1689,18 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
testThrowingDurringCommitTransactionException(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfInvalidPidMappingOccurredInCommitTransactionWhenSuspendingTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
testThrowingDurringCommitTransactionException(new InvalidPidMappingException("PidMapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReinitializeRevivedTasksInAnyState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
thread = createStreamThread(CLIENT_ID, config, new MockTime(1));
@ -1873,19 +1874,19 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfProducerGotFencedInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new ProducerFencedException("Producer is fenced"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskAndRemoveFromTaskManagerIfPidMappingIsInvalidInCommitTransactionWhenCommitting(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
testNotCloseTaskAndRemoveFromTaskManagerInCommitTransactionWhenCommitting(new InvalidPidMappingException("PID Mapping is invalid"), stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCloseTaskProducerWhenSuspending(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
thread = createStreamThread(CLIENT_ID, config);
@ -1933,7 +1934,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnActiveTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
internalTopologyBuilder.addSource(null, "source", null, null, null, topic1);
@ -2011,7 +2012,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnStandbyTaskMetadataWhileRunningState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
internalStreamsBuilder.stream(Collections.singleton(topic1), consumed)
@ -2059,7 +2060,7 @@ public class StreamThreadTest {
@SuppressWarnings("unchecked")
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldUpdateStandbyTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
// Updating standby tasks on the stream thread only happens when the state updater is disabled
assumeFalse(stateUpdaterEnabled);
@ -2183,7 +2184,7 @@ public class StreamThreadTest {
@SuppressWarnings("unchecked")
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotUpdateStandbyTaskWhenPaused(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
// Updating standby tasks on the stream thread only happens when the state updater is disabled
assumeFalse(stateUpdaterEnabled);
@ -2243,7 +2244,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCreateStandbyTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
setupInternalTopologyWithoutState(config);
@ -2253,7 +2254,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCreateStandbyTaskWithoutStateStores(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
setupInternalTopologyWithoutState(config);
@ -2262,7 +2263,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
setupInternalTopologyWithoutState(config);
@ -2275,7 +2276,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("deprecation")
public void shouldPunctuateActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(processingThreadsEnabled);
@ -2426,7 +2427,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldAlwaysUpdateTasksMetadataAfterChangingState(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
thread = createStreamThread(CLIENT_ID, config);
@ -2442,7 +2443,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRecoverFromInvalidOffsetExceptionOnRestoreAndFinishRestore(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
internalStreamsBuilder.stream(Collections.singleton("topic"), consumed)
.groupByKey()
@ -2632,7 +2633,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldThrowTaskMigratedExceptionHandlingTaskLost(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
@ -2660,7 +2661,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldThrowTaskMigratedExceptionHandlingRevocation(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final Set<TopicPartition> assignedPartitions = Collections.singleton(t1p1);
@ -2688,7 +2689,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldCatchHandleCorruptionOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -2749,7 +2750,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldCatchTimeoutExceptionFromHandleCorruptionAndInvokeExceptionHandler(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -2815,7 +2816,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldCatchTaskMigratedExceptionOnOnTaskCorruptedExceptionPath(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
@ -2881,7 +2882,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnActiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
@ -2946,7 +2947,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
@SuppressWarnings("unchecked")
public void shouldNotEnforceRebalanceWhenTaskCorruptedExceptionIsThrownForAnInactiveTask(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(true, stateUpdaterEnabled, processingThreadsEnabled));
@ -3009,7 +3010,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotCommitNonRunningNonRestoringTasks(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final TaskManager taskManager = mock(TaskManager.class);
@ -3048,7 +3049,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldLogAndRecordSkippedRecordsForInvalidTimestamps(
final boolean stateUpdaterEnabled,
final boolean processingThreadsEnabled
@ -3155,7 +3156,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTransmitTaskManagerMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class);
@ -3182,7 +3183,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldConstructAdminMetrics(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final StreamsConfig config = new StreamsConfig(configProps(false, stateUpdaterEnabled, processingThreadsEnabled));
final Node broker1 = new Node(0, "dummyHost-1", 1234);
@ -3239,13 +3240,13 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
runAndVerifyFailedStreamThreadRecording(false, stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRecordFailedStreamThread(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
runAndVerifyFailedStreamThreadRecording(true, stateUpdaterEnabled, processingThreadsEnabled);
}
@ -3308,7 +3309,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCheckStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -3326,7 +3327,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldCheckStateUpdaterInBetweenProcessCalls(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
assumeFalse(processingThreadsEnabled);
@ -3344,7 +3345,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldUpdateLagsAfterPolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
thread = setUpThread(streamsConfigProps);
@ -3362,7 +3363,7 @@ public class StreamThreadTest {
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldResumePollingForPartitionsWithAvailableSpaceBeforePolling(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
thread = setUpThread(streamsConfigProps);
@ -3377,7 +3378,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldRespectPollTimeInPartitionsAssignedStateWithStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeTrue(stateUpdaterEnabled);
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -3393,7 +3394,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldNotBlockWhenPollingInPartitionsAssignedStateWithoutStateUpdater(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
assumeFalse(stateUpdaterEnabled);
final Properties streamsConfigProps = configProps(false, stateUpdaterEnabled, processingThreadsEnabled);
@ -3407,13 +3408,13 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldGetMainAndRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
getClientInstanceId(false, stateUpdaterEnabled, processingThreadsEnabled);
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldGetMainAndRestoreConsumerInstanceIdWithInternalTimeout(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
getClientInstanceId(true, stateUpdaterEnabled, processingThreadsEnabled);
}
@ -3460,7 +3461,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnErrorIfMainConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
thread.setState(State.STARTING);
@ -3477,7 +3478,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnErrorIfRestoreConsumerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
thread.setState(State.STARTING);
@ -3494,7 +3495,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnErrorIfProducerInstanceIdNotInitialized(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
thread.setState(State.STARTING);
@ -3511,7 +3512,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnNullIfMainConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
clientSupplier.consumer.disableTelemetry();
thread = createStreamThread("clientId", stateUpdaterEnabled, processingThreadsEnabled);
@ -3528,7 +3529,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnNullIfRestoreConsumerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
clientSupplier.restoreConsumer.disableTelemetry();
@ -3546,7 +3547,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldReturnNullIfProducerTelemetryDisabled(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) throws Exception {
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
producer.disableTelemetry();
@ -3566,7 +3567,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTimeOutOnMainConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
clientSupplier.consumer.setClientInstanceId(Uuid.randomUuid());
clientSupplier.consumer.injectTimeoutException(-1);
@ -3591,7 +3592,7 @@ public class StreamThreadTest {
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTimeOutOnRestoreConsumerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
clientSupplier.restoreConsumer.setClientInstanceId(Uuid.randomUuid());
clientSupplier.restoreConsumer.injectTimeoutException(-1);
@ -3616,7 +3617,7 @@ public class StreamThreadTest {
}
@ParameterizedTest
@MethodSource("data")
@MethodSource("data")
public void shouldTimeOutOnProducerInstanceId(final boolean stateUpdaterEnabled, final boolean processingThreadsEnabled) {
final MockProducer<byte[], byte[]> producer = new MockProducer<>();
producer.setClientInstanceId(Uuid.randomUuid());
@ -3964,13 +3965,13 @@ public class StreamThreadTest {
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Missing source topics")
));
// First call should not throw exception (within timeout)
thread.runOnceWithoutProcessingThreads();
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
mockTime.sleep(300001);
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithoutProcessingThreads());
assertTrue(exception.getMessage().contains("Missing source topics"));
assertTrue(exception.getMessage().contains("Timeout exceeded"));
@ -4032,7 +4033,7 @@ public class StreamThreadTest {
));
// Should immediately throw TopologyException (no timeout like MISSING_SOURCE_TOPICS)
final TopologyException exception = assertThrows(TopologyException.class,
final TopologyException exception = assertThrows(TopologyException.class,
() -> thread.runOnceWithoutProcessingThreads());
assertTrue(exception.getMessage().contains("Topics are incorrectly partitioned"));
}
@ -4151,13 +4152,13 @@ public class StreamThreadTest {
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Missing source topics")
));
// First call should not throw exception (within timeout)
thread.runOnceWithProcessingThreads();
// Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout
mockTime.sleep(300001);
final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithProcessingThreads());
assertTrue(exception.getMessage().contains("Missing source topics"));
assertTrue(exception.getMessage().contains("Timeout exceeded"));
@ -4219,35 +4220,35 @@ public class StreamThreadTest {
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Missing source topics")
));
// First call should not throw exception (within timeout)
thread.runOnceWithoutProcessingThreads();
// Advance time but not beyond timeout
mockTime.sleep(150000); // Half of max.poll.interval.ms
// Should still not throw exception
thread.runOnceWithoutProcessingThreads();
// Clear the missing source topics (simulate recovery)
streamsRebalanceData.setStatuses(List.of());
// Should complete without exception (recovery successful)
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
// Set missing topics again - should reset the timeout
streamsRebalanceData.setStatuses(List.of(
new StreamsGroupHeartbeatResponseData.Status()
.setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code())
.setStatusDetail("Different missing topics")
));
// Advance time by 250 seconds to test if timer was reset
// Total time from beginning: 150000 + 250000 = 400000ms (400s)
// If timer was NOT reset: elapsed time = 400s > 300s should throw
// If timer WAS reset: elapsed time = 250s < 300s should NOT throw
mockTime.sleep(250000); // Advance by 250 seconds
// Should not throw because timer was reset - only 250s elapsed from reset point
assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads());
}
@ -4427,7 +4428,7 @@ public class StreamThreadTest {
null
);
}
private void runOnce(final boolean processingThreadsEnabled) {
if (processingThreadsEnabled) {
thread.runOnceWithProcessingThreads();

View File

@ -1,225 +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.tests;
import org.apache.kafka.clients.consumer.ConsumerGroupMetadata;
import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer;
import org.apache.kafka.clients.consumer.internals.StreamsRebalanceData;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsBuilder;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler;
import org.apache.kafka.streams.internals.ConsumerWrapper;
import org.apache.kafka.streams.kstream.KGroupedStream;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.Produced;
import java.time.Duration;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
public class EosTestClient extends SmokeTestUtil {
static final String APP_ID = "EosTest";
private final Properties properties;
private final boolean withRepartitioning;
private final AtomicBoolean notRunningCallbackReceived = new AtomicBoolean(false);
private static final List<CapturingConsumerWrapper> CAPTURING_CONSUMER_WRAPPERS = new ArrayList<>();
private int minGroupEpoch = 0;
private KafkaStreams streams;
private boolean uncaughtException;
EosTestClient(final Properties properties, final boolean withRepartitioning) {
super();
this.properties = properties;
this.withRepartitioning = withRepartitioning;
this.properties.put(StreamsConfig.InternalConfig.INTERNAL_CONSUMER_WRAPPER, CapturingConsumerWrapper.class);
CAPTURING_CONSUMER_WRAPPERS.clear();
}
private volatile boolean isRunning = true;
public void start() {
Exit.addShutdownHook("streams-shutdown-hook", () -> {
isRunning = false;
streams.close(Duration.ofSeconds(300));
// need to wait for callback to avoid race condition
// -> make sure the callback printout to stdout is there as it is expected test output
waitForStateTransitionCallback();
// do not remove these printouts since they are needed for health scripts
if (!uncaughtException) {
System.out.println(System.currentTimeMillis());
System.out.println("EOS-TEST-CLIENT-CLOSED");
System.out.flush();
}
});
while (isRunning) {
if (streams == null) {
uncaughtException = false;
streams = createKafkaStreams(properties);
streams.setUncaughtExceptionHandler(e -> {
System.out.println(System.currentTimeMillis());
System.out.println("EOS-TEST-CLIENT-EXCEPTION");
e.printStackTrace();
System.out.flush();
uncaughtException = true;
return StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
});
streams.setStateListener((newState, oldState) -> {
// don't remove this -- it's required test output
System.out.println(System.currentTimeMillis());
System.out.println("StateChange: " + oldState + " -> " + newState);
System.out.flush();
if (newState == KafkaStreams.State.NOT_RUNNING) {
notRunningCallbackReceived.set(true);
}
});
streams.start();
}
if (uncaughtException) {
streams.close(Duration.ofSeconds(60_000L));
streams = null;
}
logGroupEpochBump();
sleep(100);
}
}
private KafkaStreams createKafkaStreams(final Properties props) {
props.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID);
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2);
props.put(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, Duration.ofMinutes(1).toMillis());
props.put(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, Integer.MAX_VALUE);
props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3);
props.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 5000L); // increase commit interval to make sure a client is killed having an open transaction
props.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
props.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass());
final StreamsBuilder builder = new StreamsBuilder();
final KStream<String, Integer> data = builder.stream("data");
data.to("echo");
data.process(SmokeTestUtil.printProcessorSupplier("data"));
final KGroupedStream<String, Integer> groupedData = data.groupByKey();
// min
groupedData
.aggregate(
() -> Integer.MAX_VALUE,
(aggKey, value, aggregate) -> (value < aggregate) ? value : aggregate,
Materialized.with(null, intSerde))
.toStream()
.to("min", Produced.with(stringSerde, intSerde));
// sum
groupedData.aggregate(
() -> 0L,
(aggKey, value, aggregate) -> (long) value + aggregate,
Materialized.with(null, longSerde))
.toStream()
.to("sum", Produced.with(stringSerde, longSerde));
if (withRepartitioning) {
data.to("repartition");
final KStream<String, Integer> repartitionedData = builder.stream("repartition");
repartitionedData.process(SmokeTestUtil.printProcessorSupplier("repartition"));
final KGroupedStream<String, Integer> groupedDataAfterRepartitioning = repartitionedData.groupByKey();
// max
groupedDataAfterRepartitioning
.aggregate(
() -> Integer.MIN_VALUE,
(aggKey, value, aggregate) -> (value > aggregate) ? value : aggregate,
Materialized.with(null, intSerde))
.toStream()
.to("max", Produced.with(stringSerde, intSerde));
// count
groupedDataAfterRepartitioning.count()
.toStream()
.to("cnt", Produced.with(stringSerde, longSerde));
}
return new KafkaStreams(builder.build(), props);
}
private void waitForStateTransitionCallback() {
final long maxWaitTime = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(300);
while (!notRunningCallbackReceived.get() && System.currentTimeMillis() < maxWaitTime) {
try {
Thread.sleep(500);
} catch (final InterruptedException ignoreAndSwallow) { /* just keep waiting */ }
}
if (!notRunningCallbackReceived.get()) {
System.err.println("State transition callback to NOT_RUNNING never received. Timed out after 5 minutes.");
System.err.flush();
}
}
// Used in the streams group protocol
// Detect a completed rebalance by checking if the group epoch has been bumped for all threads.
private void logGroupEpochBump() {
int currentMin = Integer.MAX_VALUE;
for (final CapturingConsumerWrapper consumer : CAPTURING_CONSUMER_WRAPPERS) {
final int groupEpoch = consumer.lastSeenGroupEpoch;
if (groupEpoch < currentMin) {
currentMin = groupEpoch;
}
}
if (currentMin > minGroupEpoch) {
System.out.println("MemberEpochBump");
}
if (currentMin != Integer.MAX_VALUE) {
minGroupEpoch = currentMin;
}
}
public static class CapturingConsumerWrapper extends ConsumerWrapper {
public volatile int lastSeenGroupEpoch = 0;
@Override
public void wrapConsumer(final AsyncKafkaConsumer<byte[], byte[]> delegate, final Map<String, Object> config, final Optional<StreamsRebalanceData> streamsRebalanceData) {
CAPTURING_CONSUMER_WRAPPERS.add(this);
super.wrapConsumer(delegate, config, streamsRebalanceData);
}
@Override
public ConsumerGroupMetadata groupMetadata() {
final ConsumerGroupMetadata consumerGroupMetadata = delegate.groupMetadata();
lastSeenGroupEpoch = consumerGroupMetadata.generationId();
return consumerGroupMetadata;
}
}
}

View File

@ -1,678 +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.tests;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.ConsumerGroupDescription;
import org.apache.kafka.clients.admin.ListConsumerGroupOffsetsResult;
import org.apache.kafka.clients.admin.StreamsGroupDescription;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.IntegerDeserializer;
import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.LongDeserializer;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.Random;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
public class EosTestDriver extends SmokeTestUtil {
private static final int MAX_NUMBER_OF_KEYS = 20000;
private static final long MAX_IDLE_TIME_MS = 600000L;
private static volatile boolean isRunning = true;
private static final CountDownLatch TERMINATED = new CountDownLatch(1);
private static int numRecordsProduced = 0;
private static synchronized void updateNumRecordsProduces(final int delta) {
numRecordsProduced += delta;
}
static void generate(final String kafka) {
Exit.addShutdownHook("streams-eos-test-driver-shutdown-hook", () -> {
System.out.println("Terminating");
isRunning = false;
try {
if (TERMINATED.await(5L, TimeUnit.MINUTES)) {
System.out.println("Terminated");
} else {
System.out.println("Terminated with timeout");
}
} catch (final InterruptedException swallow) {
swallow.printStackTrace(System.err);
System.out.println("Terminated with error");
}
System.err.flush();
System.out.flush();
});
final Properties producerProps = new Properties();
producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "EosTest");
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true);
final Map<Integer, List<Long>> offsets = new HashMap<>();
try {
try (final KafkaProducer<String, Integer> producer = new KafkaProducer<>(producerProps)) {
final Random rand = new Random(System.currentTimeMillis());
while (isRunning) {
final String key = "" + rand.nextInt(MAX_NUMBER_OF_KEYS);
final int value = rand.nextInt(10000);
final ProducerRecord<String, Integer> record = new ProducerRecord<>("data", key, value);
producer.send(record, (metadata, exception) -> {
if (exception != null) {
exception.printStackTrace(System.err);
System.err.flush();
if (exception instanceof TimeoutException) {
try {
// message == org.apache.kafka.common.errors.TimeoutException: Expiring 4 record(s) for data-0: 30004 ms has passed since last attempt plus backoff time
final int expired = Integer.parseInt(exception.getMessage().split(" ")[2]);
updateNumRecordsProduces(-expired);
} catch (final Exception ignore) {
}
}
} else {
offsets.getOrDefault(metadata.partition(), new LinkedList<>()).add(metadata.offset());
}
});
updateNumRecordsProduces(1);
if (numRecordsProduced % 1000 == 0) {
System.out.println(numRecordsProduced + " records produced");
System.out.flush();
}
Utils.sleep(rand.nextInt(10));
}
}
System.out.println("Producer closed: " + numRecordsProduced + " records produced");
System.out.flush();
// verify offsets
for (final Map.Entry<Integer, List<Long>> offsetsOfPartition : offsets.entrySet()) {
offsetsOfPartition.getValue().sort(Long::compareTo);
for (int i = 0; i < offsetsOfPartition.getValue().size() - 1; ++i) {
if (offsetsOfPartition.getValue().get(i) != i) {
System.err.println("Offset for partition " + offsetsOfPartition.getKey() + " is not " + i + " as expected but " + offsetsOfPartition.getValue().get(i));
System.err.flush();
}
}
System.out.println("Max offset of partition " + offsetsOfPartition.getKey() + " is " + offsetsOfPartition.getValue().get(offsetsOfPartition.getValue().size() - 1));
}
final Properties props = new Properties();
props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier");
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString());
try (final KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(props)) {
final List<TopicPartition> partitions = getAllPartitions(consumer, "data");
System.out.println("Partitions: " + partitions);
System.out.flush();
consumer.assign(partitions);
consumer.seekToEnd(partitions);
for (final TopicPartition tp : partitions) {
System.out.println("End-offset for " + tp + " is " + consumer.position(tp));
System.out.flush();
}
}
System.out.flush();
} finally {
TERMINATED.countDown();
}
}
public static void verify(final String kafka, final boolean withRepartitioning, final String groupProtocol) {
final Properties props = new Properties();
props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier");
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString());
try (final KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(props)) {
verifyAllTransactionFinished(consumer, kafka, withRepartitioning);
} catch (final Exception e) {
e.printStackTrace(System.err);
System.out.println("FAILED");
return;
}
final Map<TopicPartition, Long> committedOffsets;
try (final Admin adminClient = Admin.create(props)) {
ensureStreamsApplicationDown(adminClient, groupProtocol);
committedOffsets = getCommittedOffsets(adminClient, withRepartitioning);
}
final String[] allInputTopics;
final String[] allOutputTopics;
if (withRepartitioning) {
allInputTopics = new String[] {"data", "repartition"};
allOutputTopics = new String[] {"echo", "min", "sum", "repartition", "max", "cnt"};
} else {
allInputTopics = new String[] {"data"};
allOutputTopics = new String[] {"echo", "min", "sum"};
}
final Map<String, Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>> inputRecordsPerTopicPerPartition;
try (final KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(props)) {
final List<TopicPartition> partitions = getAllPartitions(consumer, allInputTopics);
consumer.assign(partitions);
consumer.seekToBeginning(partitions);
inputRecordsPerTopicPerPartition = getRecords(consumer, committedOffsets, withRepartitioning, true);
} catch (final Exception e) {
e.printStackTrace(System.err);
System.out.println("FAILED");
return;
}
final Map<String, Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>> outputRecordsPerTopicPerPartition;
try (final KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(props)) {
final List<TopicPartition> partitions = getAllPartitions(consumer, allOutputTopics);
consumer.assign(partitions);
consumer.seekToBeginning(partitions);
outputRecordsPerTopicPerPartition = getRecords(consumer, consumer.endOffsets(partitions), withRepartitioning, false);
} catch (final Exception e) {
e.printStackTrace(System.err);
System.out.println("FAILED");
return;
}
verifyReceivedAllRecords(inputRecordsPerTopicPerPartition.get("data"), outputRecordsPerTopicPerPartition.get("echo"));
if (withRepartitioning) {
verifyReceivedAllRecords(inputRecordsPerTopicPerPartition.get("data"), outputRecordsPerTopicPerPartition.get("repartition"));
}
verifyMin(inputRecordsPerTopicPerPartition.get("data"), outputRecordsPerTopicPerPartition.get("min"));
verifySum(inputRecordsPerTopicPerPartition.get("data"), outputRecordsPerTopicPerPartition.get("sum"));
if (withRepartitioning) {
verifyMax(inputRecordsPerTopicPerPartition.get("repartition"), outputRecordsPerTopicPerPartition.get("max"));
verifyCnt(inputRecordsPerTopicPerPartition.get("repartition"), outputRecordsPerTopicPerPartition.get("cnt"));
}
// do not modify: required test output
System.out.println("ALL-RECORDS-DELIVERED");
System.out.flush();
}
private static void ensureStreamsApplicationDown(final Admin adminClient, final String groupProtocol) {
final long maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
boolean isEmpty;
do {
if (Objects.equals(groupProtocol, "streams")) {
final StreamsGroupDescription description = getStreamsGroupDescription(adminClient);
isEmpty = description.members().isEmpty();
if (System.currentTimeMillis() > maxWaitTime && !isEmpty) {
throwNotDownException(description);
}
} else {
final ConsumerGroupDescription description = getConsumerGroupDescription(adminClient);
isEmpty = description.members().isEmpty();
if (System.currentTimeMillis() > maxWaitTime && !isEmpty) {
throwNotDownException(description);
}
}
sleep(1000L);
} while (!isEmpty);
}
private static void throwNotDownException(final Object description) {
throw new RuntimeException(
"Streams application not down after " + MAX_IDLE_TIME_MS / 1000L + " seconds. " +
"Group: " + description
);
}
private static Map<TopicPartition, Long> getCommittedOffsets(final Admin adminClient,
final boolean withRepartitioning) {
final Map<TopicPartition, OffsetAndMetadata> topicPartitionOffsetAndMetadataMap;
try {
final ListConsumerGroupOffsetsResult listConsumerGroupOffsetsResult = adminClient.listConsumerGroupOffsets(EosTestClient.APP_ID);
topicPartitionOffsetAndMetadataMap = listConsumerGroupOffsetsResult.partitionsToOffsetAndMetadata().get(10, TimeUnit.SECONDS);
} catch (final Exception e) {
e.printStackTrace();
throw new RuntimeException(e);
}
final Map<TopicPartition, Long> committedOffsets = new HashMap<>();
for (final Map.Entry<TopicPartition, OffsetAndMetadata> entry : topicPartitionOffsetAndMetadataMap.entrySet()) {
final String topic = entry.getKey().topic();
if (topic.equals("data") || withRepartitioning && topic.equals("repartition")) {
committedOffsets.put(entry.getKey(), entry.getValue().offset());
}
}
return committedOffsets;
}
private static Map<String, Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>> getRecords(final KafkaConsumer<byte[], byte[]> consumer,
final Map<TopicPartition, Long> readEndOffsets,
final boolean withRepartitioning,
final boolean isInputTopic) {
System.out.println("read end offset: " + readEndOffsets);
final Map<String, Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>> recordPerTopicPerPartition = new HashMap<>();
final Map<TopicPartition, Long> maxReceivedOffsetPerPartition = new HashMap<>();
final Map<TopicPartition, Long> maxConsumerPositionPerPartition = new HashMap<>();
long maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
boolean allRecordsReceived = false;
while (!allRecordsReceived && System.currentTimeMillis() < maxWaitTime) {
final ConsumerRecords<byte[], byte[]> receivedRecords = consumer.poll(Duration.ofSeconds(1L));
for (final ConsumerRecord<byte[], byte[]> record : receivedRecords) {
maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
final TopicPartition tp = new TopicPartition(record.topic(), record.partition());
maxReceivedOffsetPerPartition.put(tp, record.offset());
final long readEndOffset = readEndOffsets.get(tp);
if (record.offset() < readEndOffset) {
addRecord(record, recordPerTopicPerPartition, withRepartitioning);
} else if (!isInputTopic) {
throw new RuntimeException("FAIL: did receive more records than expected for " + tp
+ " (expected EOL offset: " + readEndOffset + "; current offset: " + record.offset());
}
}
for (final TopicPartition tp : readEndOffsets.keySet()) {
maxConsumerPositionPerPartition.put(tp, consumer.position(tp));
if (consumer.position(tp) >= readEndOffsets.get(tp)) {
consumer.pause(Collections.singletonList(tp));
}
}
allRecordsReceived = consumer.paused().size() == readEndOffsets.keySet().size();
}
if (!allRecordsReceived) {
System.err.println("Pause partitions (ie, received all data): " + consumer.paused());
System.err.println("Max received offset per partition: " + maxReceivedOffsetPerPartition);
System.err.println("Max consumer position per partition: " + maxConsumerPositionPerPartition);
throw new RuntimeException("FAIL: did not receive all records after " + (MAX_IDLE_TIME_MS / 1000L) + " sec idle time.");
}
return recordPerTopicPerPartition;
}
private static void addRecord(final ConsumerRecord<byte[], byte[]> record,
final Map<String, Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>>> recordPerTopicPerPartition,
final boolean withRepartitioning) {
final String topic = record.topic();
final TopicPartition partition = new TopicPartition(topic, record.partition());
if (verifyTopic(topic, withRepartitioning)) {
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> topicRecordsPerPartition =
recordPerTopicPerPartition.computeIfAbsent(topic, k -> new HashMap<>());
final List<ConsumerRecord<byte[], byte[]>> records =
topicRecordsPerPartition.computeIfAbsent(partition, k -> new ArrayList<>());
records.add(record);
} else {
throw new RuntimeException("FAIL: received data from unexpected topic: " + record);
}
}
private static boolean verifyTopic(final String topic,
final boolean withRepartitioning) {
final boolean validTopic = "data".equals(topic) || "echo".equals(topic) || "min".equals(topic) || "sum".equals(topic);
if (withRepartitioning) {
return validTopic || "repartition".equals(topic) || "max".equals(topic) || "cnt".equals(topic);
}
return validTopic;
}
private static void verifyReceivedAllRecords(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> expectedRecords,
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> receivedRecords) {
if (expectedRecords.size() != receivedRecords.size()) {
throw new RuntimeException("Result verification failed. Received " + receivedRecords.size() + " records but expected " + expectedRecords.size());
}
final StringDeserializer stringDeserializer = new StringDeserializer();
final IntegerDeserializer integerDeserializer = new IntegerDeserializer();
for (final Map.Entry<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords : receivedRecords.entrySet()) {
final TopicPartition inputTopicPartition = new TopicPartition("data", partitionRecords.getKey().partition());
final List<ConsumerRecord<byte[], byte[]>> receivedRecordsForPartition = partitionRecords.getValue();
final List<ConsumerRecord<byte[], byte[]>> expectedRecordsForPartition = expectedRecords.get(inputTopicPartition);
System.out.println(partitionRecords.getKey() + " with " + receivedRecordsForPartition.size() + ", " +
inputTopicPartition + " with " + expectedRecordsForPartition.size());
final Iterator<ConsumerRecord<byte[], byte[]>> expectedRecord = expectedRecordsForPartition.iterator();
RuntimeException exception = null;
for (final ConsumerRecord<byte[], byte[]> receivedRecord : receivedRecordsForPartition) {
if (!expectedRecord.hasNext()) {
exception = new RuntimeException("Result verification failed for " + receivedRecord + " since there's no more expected record");
}
final ConsumerRecord<byte[], byte[]> expected = expectedRecord.next();
final String receivedKey = stringDeserializer.deserialize(receivedRecord.topic(), receivedRecord.key());
final int receivedValue = integerDeserializer.deserialize(receivedRecord.topic(), receivedRecord.value());
final String expectedKey = stringDeserializer.deserialize(expected.topic(), expected.key());
final int expectedValue = integerDeserializer.deserialize(expected.topic(), expected.value());
if (!receivedKey.equals(expectedKey) || receivedValue != expectedValue) {
exception = new RuntimeException("Result verification failed for " + receivedRecord + " expected <" + expectedKey + "," + expectedValue + "> but was <" + receivedKey + "," + receivedValue + ">");
}
}
if (exception != null) {
throw exception;
}
}
}
private static void verifyMin(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> inputPerTopicPerPartition,
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> minPerTopicPerPartition) {
final StringDeserializer stringDeserializer = new StringDeserializer();
final IntegerDeserializer integerDeserializer = new IntegerDeserializer();
final HashMap<String, Integer> currentMinPerKey = new HashMap<>();
for (final Map.Entry<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords : minPerTopicPerPartition.entrySet()) {
final TopicPartition inputTopicPartition = new TopicPartition("data", partitionRecords.getKey().partition());
final List<ConsumerRecord<byte[], byte[]>> partitionInput = inputPerTopicPerPartition.get(inputTopicPartition);
final List<ConsumerRecord<byte[], byte[]>> partitionMin = partitionRecords.getValue();
if (partitionInput.size() != partitionMin.size()) {
throw new RuntimeException("Result verification failed: expected " + partitionInput.size() + " records for "
+ partitionRecords.getKey() + " but received " + partitionMin.size());
}
final Iterator<ConsumerRecord<byte[], byte[]>> inputRecords = partitionInput.iterator();
for (final ConsumerRecord<byte[], byte[]> receivedRecord : partitionMin) {
final ConsumerRecord<byte[], byte[]> input = inputRecords.next();
final String receivedKey = stringDeserializer.deserialize(receivedRecord.topic(), receivedRecord.key());
final int receivedValue = integerDeserializer.deserialize(receivedRecord.topic(), receivedRecord.value());
final String key = stringDeserializer.deserialize(input.topic(), input.key());
final int value = integerDeserializer.deserialize(input.topic(), input.value());
Integer min = currentMinPerKey.get(key);
if (min == null) {
min = value;
} else {
min = Math.min(min, value);
}
currentMinPerKey.put(key, min);
if (!receivedKey.equals(key) || receivedValue != min) {
throw new RuntimeException("Result verification failed for " + receivedRecord + " expected <" + key + "," + min + "> but was <" + receivedKey + "," + receivedValue + ">");
}
}
}
}
private static void verifySum(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> inputPerTopicPerPartition,
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> minPerTopicPerPartition) {
final StringDeserializer stringDeserializer = new StringDeserializer();
final IntegerDeserializer integerDeserializer = new IntegerDeserializer();
final LongDeserializer longDeserializer = new LongDeserializer();
final HashMap<String, Long> currentSumPerKey = new HashMap<>();
for (final Map.Entry<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords : minPerTopicPerPartition.entrySet()) {
final TopicPartition inputTopicPartition = new TopicPartition("data", partitionRecords.getKey().partition());
final List<ConsumerRecord<byte[], byte[]>> partitionInput = inputPerTopicPerPartition.get(inputTopicPartition);
final List<ConsumerRecord<byte[], byte[]>> partitionSum = partitionRecords.getValue();
if (partitionInput.size() != partitionSum.size()) {
throw new RuntimeException("Result verification failed: expected " + partitionInput.size() + " records for "
+ partitionRecords.getKey() + " but received " + partitionSum.size());
}
final Iterator<ConsumerRecord<byte[], byte[]>> inputRecords = partitionInput.iterator();
for (final ConsumerRecord<byte[], byte[]> receivedRecord : partitionSum) {
final ConsumerRecord<byte[], byte[]> input = inputRecords.next();
final String receivedKey = stringDeserializer.deserialize(receivedRecord.topic(), receivedRecord.key());
final long receivedValue = longDeserializer.deserialize(receivedRecord.topic(), receivedRecord.value());
final String key = stringDeserializer.deserialize(input.topic(), input.key());
final int value = integerDeserializer.deserialize(input.topic(), input.value());
Long sum = currentSumPerKey.get(key);
if (sum == null) {
sum = (long) value;
} else {
sum += value;
}
currentSumPerKey.put(key, sum);
if (!receivedKey.equals(key) || receivedValue != sum) {
throw new RuntimeException("Result verification failed for " + receivedRecord + " expected <" + key + "," + sum + "> but was <" + receivedKey + "," + receivedValue + ">");
}
}
}
}
private static void verifyMax(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> inputPerTopicPerPartition,
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> maxPerTopicPerPartition) {
final StringDeserializer stringDeserializer = new StringDeserializer();
final IntegerDeserializer integerDeserializer = new IntegerDeserializer();
final HashMap<String, Integer> currentMinPerKey = new HashMap<>();
for (final Map.Entry<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords : maxPerTopicPerPartition.entrySet()) {
final TopicPartition inputTopicPartition = new TopicPartition("repartition", partitionRecords.getKey().partition());
final List<ConsumerRecord<byte[], byte[]>> partitionInput = inputPerTopicPerPartition.get(inputTopicPartition);
final List<ConsumerRecord<byte[], byte[]>> partitionMax = partitionRecords.getValue();
if (partitionInput.size() != partitionMax.size()) {
throw new RuntimeException("Result verification failed: expected " + partitionInput.size() + " records for "
+ partitionRecords.getKey() + " but received " + partitionMax.size());
}
final Iterator<ConsumerRecord<byte[], byte[]>> inputRecords = partitionInput.iterator();
for (final ConsumerRecord<byte[], byte[]> receivedRecord : partitionMax) {
final ConsumerRecord<byte[], byte[]> input = inputRecords.next();
final String receivedKey = stringDeserializer.deserialize(receivedRecord.topic(), receivedRecord.key());
final int receivedValue = integerDeserializer.deserialize(receivedRecord.topic(), receivedRecord.value());
final String key = stringDeserializer.deserialize(input.topic(), input.key());
final int value = integerDeserializer.deserialize(input.topic(), input.value());
Integer max = currentMinPerKey.get(key);
if (max == null) {
max = Integer.MIN_VALUE;
}
max = Math.max(max, value);
currentMinPerKey.put(key, max);
if (!receivedKey.equals(key) || receivedValue != max) {
throw new RuntimeException("Result verification failed for " + receivedRecord + " expected <" + key + "," + max + "> but was <" + receivedKey + "," + receivedValue + ">");
}
}
}
}
private static void verifyCnt(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> inputPerTopicPerPartition,
final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> cntPerTopicPerPartition) {
final StringDeserializer stringDeserializer = new StringDeserializer();
final LongDeserializer longDeserializer = new LongDeserializer();
final HashMap<String, Long> currentSumPerKey = new HashMap<>();
for (final Map.Entry<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecords : cntPerTopicPerPartition.entrySet()) {
final TopicPartition inputTopicPartition = new TopicPartition("repartition", partitionRecords.getKey().partition());
final List<ConsumerRecord<byte[], byte[]>> partitionInput = inputPerTopicPerPartition.get(inputTopicPartition);
final List<ConsumerRecord<byte[], byte[]>> partitionCnt = partitionRecords.getValue();
if (partitionInput.size() != partitionCnt.size()) {
throw new RuntimeException("Result verification failed: expected " + partitionInput.size() + " records for "
+ partitionRecords.getKey() + " but received " + partitionCnt.size());
}
final Iterator<ConsumerRecord<byte[], byte[]>> inputRecords = partitionInput.iterator();
for (final ConsumerRecord<byte[], byte[]> receivedRecord : partitionCnt) {
final ConsumerRecord<byte[], byte[]> input = inputRecords.next();
final String receivedKey = stringDeserializer.deserialize(receivedRecord.topic(), receivedRecord.key());
final long receivedValue = longDeserializer.deserialize(receivedRecord.topic(), receivedRecord.value());
final String key = stringDeserializer.deserialize(input.topic(), input.key());
Long cnt = currentSumPerKey.get(key);
if (cnt == null) {
cnt = 0L;
}
currentSumPerKey.put(key, ++cnt);
if (!receivedKey.equals(key) || receivedValue != cnt) {
throw new RuntimeException("Result verification failed for " + receivedRecord + " expected <" + key + "," + cnt + "> but was <" + receivedKey + "," + receivedValue + ">");
}
}
}
}
private static void verifyAllTransactionFinished(final KafkaConsumer<byte[], byte[]> consumer,
final String kafka,
final boolean withRepartitioning) {
final String[] topics;
if (withRepartitioning) {
topics = new String[] {"echo", "min", "sum", "repartition", "max", "cnt"};
} else {
topics = new String[] {"echo", "min", "sum"};
}
final List<TopicPartition> partitions = getAllPartitions(consumer, topics);
consumer.assign(partitions);
consumer.seekToEnd(partitions);
for (final TopicPartition tp : partitions) {
System.out.println(tp + " at position " + consumer.position(tp));
}
final Properties consumerProps = new Properties();
consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "consumer-uncommitted");
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
final long maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
try (final KafkaConsumer<byte[], byte[]> consumerUncommitted = new KafkaConsumer<>(consumerProps)) {
while (!partitions.isEmpty() && System.currentTimeMillis() < maxWaitTime) {
consumer.seekToEnd(partitions);
final Map<TopicPartition, Long> topicEndOffsets = consumerUncommitted.endOffsets(partitions);
final Iterator<TopicPartition> iterator = partitions.iterator();
while (iterator.hasNext()) {
final TopicPartition topicPartition = iterator.next();
final long position = consumer.position(topicPartition);
if (position == topicEndOffsets.get(topicPartition)) {
iterator.remove();
System.out.println("Removing " + topicPartition + " at position " + position);
} else if (consumer.position(topicPartition) > topicEndOffsets.get(topicPartition)) {
throw new IllegalStateException("Offset for partition " + topicPartition + " is larger than topic endOffset: " + position + " > " + topicEndOffsets.get(topicPartition));
} else {
System.out.println("Retry " + topicPartition + " at position " + position);
}
}
sleep(1000L);
}
}
if (!partitions.isEmpty()) {
throw new RuntimeException("Could not read all verification records. Did not receive any new record within the last " + (MAX_IDLE_TIME_MS / 1000L) + " sec.");
}
}
private static List<TopicPartition> getAllPartitions(final KafkaConsumer<?, ?> consumer,
final String... topics) {
final ArrayList<TopicPartition> partitions = new ArrayList<>();
for (final String topic : topics) {
for (final PartitionInfo info : consumer.partitionsFor(topic)) {
partitions.add(new TopicPartition(info.topic(), info.partition()));
}
}
return partitions;
}
private static ConsumerGroupDescription getConsumerGroupDescription(final Admin adminClient) {
final ConsumerGroupDescription description;
try {
description = adminClient.describeConsumerGroups(Collections.singleton(EosTestClient.APP_ID))
.describedGroups()
.get(EosTestClient.APP_ID)
.get(10, TimeUnit.SECONDS);
} catch (final InterruptedException | ExecutionException | java.util.concurrent.TimeoutException e) {
e.printStackTrace();
throw new RuntimeException("Unexpected Exception getting group description", e);
}
return description;
}
private static StreamsGroupDescription getStreamsGroupDescription(final Admin adminClient) {
final StreamsGroupDescription description;
try {
description = adminClient.describeStreamsGroups(Collections.singleton(EosTestClient.APP_ID))
.describedGroups()
.get(EosTestClient.APP_ID)
.get(10, TimeUnit.SECONDS);
} catch (final InterruptedException | ExecutionException | java.util.concurrent.TimeoutException e) {
e.printStackTrace();
throw new RuntimeException("Unexpected Exception getting group description", e);
}
return description;
}
}

View File

@ -16,25 +16,6 @@
*/
package org.apache.kafka.streams.tests;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
import java.nio.charset.StandardCharsets;
@ -43,8 +24,10 @@ import java.time.Instant;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import static java.util.Collections.emptyMap;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
@ -59,7 +42,26 @@ import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static java.util.Collections.emptyMap;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.StringDeserializer;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import static org.apache.kafka.common.utils.Utils.mkEntry;
public class SmokeTestDriver extends SmokeTestUtil {
@ -69,6 +71,8 @@ public class SmokeTestDriver extends SmokeTestUtil {
*/
private static final long CREATE_TIME_SHIFT_MS = Duration.ofDays(2).toMillis();
private static final long MAX_IDLE_TIME_MS = 600000L;
private static final String[] NUMERIC_VALUE_TOPICS = {
"data",
"echo",
@ -376,84 +380,150 @@ public class SmokeTestDriver extends SmokeTestUtil {
final Map<String, Set<Integer>> inputs,
final int maxRecordsPerKey,
final boolean eosEnabled) {
final Properties props = new Properties();
props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier");
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class);
props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed");
final Properties props = createConsumerProperties(kafka);
final KafkaConsumer<String, Number> consumer = new KafkaConsumer<>(props);
final List<TopicPartition> partitions = getAllPartitions(consumer, NUMERIC_VALUE_TOPICS);
consumer.assign(partitions);
consumer.seekToBeginning(partitions);
final int recordsGenerated = inputs.size() * maxRecordsPerKey;
int recordsProcessed = 0;
final Map<String, AtomicInteger> processed =
Stream.of(NUMERIC_VALUE_TOPICS)
.collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0)));
final RecordProcessingState state = new RecordProcessingState(recordsGenerated);
final Map<String, Map<String, LinkedList<ConsumerRecord<String, Number>>>> events = new HashMap<>();
VerificationResult verificationResult = new VerificationResult(false, "no results yet");
int retry = 0;
final long start = System.currentTimeMillis();
while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) {
final ConsumerRecords<String, Number> records = consumer.poll(Duration.ofSeconds(5));
if (records.isEmpty() && recordsProcessed >= recordsGenerated) {
verificationResult = verifyAll(inputs, events, false, eosEnabled);
if (verificationResult.passed()) {
break;
} else if (retry++ > MAX_RECORD_EMPTY_RETRIES) {
System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries.");
break;
} else {
System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + retry);
}
} else {
System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry.");
retry = 0;
for (final ConsumerRecord<String, Number> record : records) {
final String key = record.key();
final String topic = record.topic();
processed.get(topic).incrementAndGet();
if (topic.equals("echo")) {
recordsProcessed++;
if (recordsProcessed % 100 == 0) {
System.out.println("Echo records processed = " + recordsProcessed);
}
}
events.computeIfAbsent(topic, t -> new HashMap<>())
.computeIfAbsent(key, k -> new LinkedList<>())
.add(record);
}
System.out.println(processed);
}
}
final VerificationResult verificationResult = consumeAndProcessRecords(consumer, inputs, events, state, start, eosEnabled);
consumer.close();
final VerificationResult eosResult = performEosVerification(eosEnabled, kafka);
if (!eosResult.passed()) {
return eosResult;
}
return validateAndReportResults(inputs, events, state, verificationResult, start, eosEnabled);
}
private static Properties createConsumerProperties(final String kafka) {
final Properties props = new Properties();
props.put(ConsumerConfig.CLIENT_ID_CONFIG, "verifier");
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class);
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, NumberDeserializer.class);
props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed");
return props;
}
private static class RecordProcessingState {
final int recordsGenerated;
int recordsProcessed = 0;
int retry = 0;
final Map<String, AtomicInteger> processed;
RecordProcessingState(final int recordsGenerated) {
this.recordsGenerated = recordsGenerated;
this.processed = Stream.of(NUMERIC_VALUE_TOPICS)
.collect(Collectors.toMap(t -> t, t -> new AtomicInteger(0)));
}
}
private static VerificationResult consumeAndProcessRecords(
final KafkaConsumer<String, Number> consumer,
final Map<String, Set<Integer>> inputs,
final Map<String, Map<String, LinkedList<ConsumerRecord<String, Number>>>> events,
final RecordProcessingState state,
final long start,
final boolean eosEnabled) {
VerificationResult verificationResult = new VerificationResult(false, "no results yet");
while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) {
final ConsumerRecords<String, Number> records = consumer.poll(Duration.ofSeconds(5));
if (records.isEmpty() && state.recordsProcessed >= state.recordsGenerated) {
verificationResult = handleEmptyRecords(inputs, events, state, eosEnabled);
if (verificationResult.passed() || state.retry > MAX_RECORD_EMPTY_RETRIES) {
break;
}
} else {
processRecords(records, events, state);
}
}
return verificationResult;
}
private static VerificationResult handleEmptyRecords(
final Map<String, Set<Integer>> inputs,
final Map<String, Map<String, LinkedList<ConsumerRecord<String, Number>>>> events,
final RecordProcessingState state,
final boolean eosEnabled) {
final VerificationResult result = verifyAll(inputs, events, false, eosEnabled);
if (result.passed()) {
return result;
}
state.retry++;
if (state.retry > MAX_RECORD_EMPTY_RETRIES) {
System.out.println(Instant.now() + " Didn't get any more results, verification hasn't passed, and out of retries.");
} else {
System.out.println(Instant.now() + " Didn't get any more results, but verification hasn't passed (yet). Retrying..." + state.retry);
}
return result;
}
private static void processRecords(
final ConsumerRecords<String, Number> records,
final Map<String, Map<String, LinkedList<ConsumerRecord<String, Number>>>> events,
final RecordProcessingState state) {
System.out.println(Instant.now() + " Get some more results from " + records.partitions() + ", resetting retry.");
state.retry = 0;
for (final ConsumerRecord<String, Number> record : records) {
final String key = record.key();
final String topic = record.topic();
state.processed.get(topic).incrementAndGet();
if (topic.equals("echo")) {
state.recordsProcessed++;
if (state.recordsProcessed % 100 == 0) {
System.out.println("Echo records processed = " + state.recordsProcessed);
}
}
events.computeIfAbsent(topic, t -> new HashMap<>())
.computeIfAbsent(key, k -> new LinkedList<>())
.add(record);
}
System.out.println(state.processed);
}
private static VerificationResult validateAndReportResults(
final Map<String, Set<Integer>> inputs,
final Map<String, Map<String, LinkedList<ConsumerRecord<String, Number>>>> events,
final RecordProcessingState state,
VerificationResult verificationResult,
final long start,
final boolean eosEnabled) {
final long finished = System.currentTimeMillis() - start;
System.out.println("Verification time=" + finished);
System.out.println("-------------------");
System.out.println("Result Verification");
System.out.println("-------------------");
System.out.println("recordGenerated=" + recordsGenerated);
System.out.println("recordProcessed=" + recordsProcessed);
System.out.println("recordGenerated=" + state.recordsGenerated);
System.out.println("recordProcessed=" + state.recordsProcessed);
if (recordsProcessed > recordsGenerated) {
if (state.recordsProcessed > state.recordsGenerated) {
System.out.println("PROCESSED-MORE-THAN-GENERATED");
} else if (recordsProcessed < recordsGenerated) {
} else if (state.recordsProcessed < state.recordsGenerated) {
System.out.println("PROCESSED-LESS-THAN-GENERATED");
}
final Map<String, Set<Number>> received = parseRecordsForEchoTopic(events);
boolean success = inputs.equals(received);
if (success) {
@ -461,19 +531,17 @@ public class SmokeTestDriver extends SmokeTestUtil {
} else {
int missedCount = 0;
for (final Map.Entry<String, Set<Integer>> entry : inputs.entrySet()) {
missedCount += received.get(entry.getKey()).size();
missedCount += received.getOrDefault(entry.getKey(), Collections.emptySet()).size();
}
System.out.println("missedRecords=" + missedCount);
}
// give it one more try if it's not already passing.
if (!verificationResult.passed()) {
verificationResult = verifyAll(inputs, events, true, eosEnabled);
}
success &= verificationResult.passed();
System.out.println(verificationResult.result());
System.out.println(success ? "SUCCESS" : "FAILURE");
return verificationResult;
}
@ -563,7 +631,7 @@ public class SmokeTestDriver extends SmokeTestUtil {
} else {
if (outputEvents.size() != inputData.size()) {
if (outputEvents.size() < inputData.size()) {
resultStream.println("fail: missing result data; got " + inputData.size() + " keys, expected: " + outputEvents.size() + " keys");
resultStream.println("fail: missing result data; got " + outputEvents.size() + " keys, expected: " + inputData.size() + " keys");
return false;
} else {
if (eosEnabled) {
@ -721,6 +789,28 @@ public class SmokeTestDriver extends SmokeTestUtil {
return Integer.parseInt(key.split("-")[1]);
}
private static VerificationResult performEosVerification(final boolean eosEnabled, final String kafka) {
if (!eosEnabled) {
return new VerificationResult(true, "EOS verification skipped");
}
final Properties eosProps = new Properties();
eosProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "transaction-verifier");
eosProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
eosProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
eosProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
eosProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString());
try (final KafkaConsumer<byte[], byte[]> eosConsumer = new KafkaConsumer<>(eosProps)) {
verifyAllTransactionFinished(eosConsumer, kafka);
return new VerificationResult(true, "EOS verification passed");
} catch (final Exception e) {
e.printStackTrace(System.err);
System.out.println("FAILED");
return new VerificationResult(false, "eos verification failed");
}
}
private static List<TopicPartition> getAllPartitions(final KafkaConsumer<?, ?> consumer, final String... topics) {
final List<TopicPartition> partitions = new ArrayList<>();
@ -732,4 +822,52 @@ public class SmokeTestDriver extends SmokeTestUtil {
return partitions;
}
private static void verifyAllTransactionFinished(final KafkaConsumer<byte[], byte[]> consumer,
final String kafka) {
final String[] topics;
topics = new String[] {"echo", "min", "sum", "max", "cnt"};
final List<TopicPartition> partitions = getAllPartitions(consumer, topics);
consumer.assign(partitions);
consumer.seekToEnd(partitions);
for (final TopicPartition tp : partitions) {
System.out.println(tp + " at position " + consumer.position(tp));
}
final Properties consumerProps = new Properties();
consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, "consumer-uncommitted");
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class);
final long maxWaitTime = System.currentTimeMillis() + MAX_IDLE_TIME_MS;
try (final KafkaConsumer<byte[], byte[]> consumerUncommitted = new KafkaConsumer<>(consumerProps)) {
while (!partitions.isEmpty() && System.currentTimeMillis() < maxWaitTime) {
consumer.seekToEnd(partitions);
final Map<TopicPartition, Long> topicEndOffsets = consumerUncommitted.endOffsets(partitions);
final Iterator<TopicPartition> iterator = partitions.iterator();
while (iterator.hasNext()) {
final TopicPartition topicPartition = iterator.next();
final long position = consumer.position(topicPartition);
if (position == topicEndOffsets.get(topicPartition)) {
iterator.remove();
System.out.println("Removing " + topicPartition + " at position " + position);
} else if (consumer.position(topicPartition) > topicEndOffsets.get(topicPartition)) {
throw new IllegalStateException("Offset for partition " + topicPartition + " is larger than topic endOffset: " + position + " > " + topicEndOffsets.get(topicPartition));
} else {
System.out.println("Retry " + topicPartition + " at position " + position);
}
}
Utils.sleep(1000L);
}
}
if (!partitions.isEmpty()) {
throw new RuntimeException("Could not read all verification records. Did not receive any new record within the last " + (MAX_IDLE_TIME_MS / 1000L) + " sec.");
}
}
}

View File

@ -1,91 +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.tests;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.StreamsConfig;
import java.io.IOException;
import java.util.Properties;
public class StreamsEosTest {
/**
* args ::= kafka propFileName command
* command := "run" | "process" | "verify"
*/
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");
Exit.exit(1);
}
final String propFileName = args[0];
final String command = args[1];
final Properties streamsProperties = Utils.loadProps(propFileName);
final String kafka = streamsProperties.getProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
final String processingGuarantee = streamsProperties.getProperty(StreamsConfig.PROCESSING_GUARANTEE_CONFIG);
if (kafka == null) {
System.err.println("No bootstrap kafka servers specified in " + StreamsConfig.BOOTSTRAP_SERVERS_CONFIG);
Exit.exit(1);
}
if ("process".equals(command) || "process-complex".equals(command)) {
if (!StreamsConfig.EXACTLY_ONCE_V2.equals(processingGuarantee)) {
System.err.println("processingGuarantee must be " + StreamsConfig.EXACTLY_ONCE_V2);
Exit.exit(1);
}
}
System.out.println("StreamsTest instance started");
System.out.println("kafka=" + kafka);
System.out.println("props=" + streamsProperties);
System.out.println("command=" + command);
System.out.flush();
if (command == null || propFileName == null) {
Exit.exit(-1);
}
switch (command) {
case "run":
EosTestDriver.generate(kafka);
break;
case "process":
new EosTestClient(streamsProperties, false).start();
break;
case "process-complex":
new EosTestClient(streamsProperties, true).start();
break;
case "verify":
EosTestDriver.verify(kafka, false, streamsProperties.getProperty("group.protocol"));
break;
case "verify-complex":
EosTestDriver.verify(kafka, true, streamsProperties.getProperty("group.protocol"));
break;
default:
System.out.println("unknown command: " + command);
System.out.flush();
Exit.exit(-1);
}
}
}

View File

@ -443,27 +443,6 @@ class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService):
def __init__(self, test_context, kafka, processing_guarantee, group_protocol = 'classic', num_threads = 3, replication_factor = 3):
super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, group_protocol, num_threads, replication_factor)
class StreamsEosTestDriverService(StreamsEosTestBaseService):
def __init__(self, test_context, kafka):
super(StreamsEosTestDriverService, self).__init__(test_context, kafka, "run", "classic")
class StreamsEosTestJobRunnerService(StreamsEosTestBaseService):
def __init__(self, test_context, kafka, group_protocol):
super(StreamsEosTestJobRunnerService, self).__init__(test_context, kafka, "process", group_protocol)
class StreamsComplexEosTestJobRunnerService(StreamsEosTestBaseService):
def __init__(self, test_context, kafka, group_protocol):
super(StreamsComplexEosTestJobRunnerService, self).__init__(test_context, kafka, "process-complex", group_protocol)
class StreamsEosTestVerifyRunnerService(StreamsEosTestBaseService):
def __init__(self, test_context, kafka, group_protocol):
super(StreamsEosTestVerifyRunnerService, self).__init__(test_context, kafka, "verify", group_protocol)
class StreamsComplexEosTestVerifyRunnerService(StreamsEosTestBaseService):
def __init__(self, test_context, kafka, group_protocol):
super(StreamsComplexEosTestVerifyRunnerService, self).__init__(test_context, kafka, "verify-complex", group_protocol)
class StreamsSmokeTestShutdownDeadlockService(StreamsSmokeTestBaseService):
def __init__(self, test_context, kafka):

View File

@ -1,183 +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.
from ducktape.mark import matrix
from ducktape.mark.resource import cluster
from kafkatest.services.kafka import quorum
from kafkatest.services.streams import StreamsEosTestDriverService, StreamsEosTestJobRunnerService, \
StreamsComplexEosTestJobRunnerService, StreamsEosTestVerifyRunnerService, StreamsComplexEosTestVerifyRunnerService
from kafkatest.tests.streams.base_streams_test import BaseStreamsTest
class StreamsEosTest(BaseStreamsTest):
"""
Test of Kafka Streams exactly-once semantics
"""
def __init__(self, test_context):
super(StreamsEosTest, self).__init__(test_context, num_controllers=1, num_brokers=3, topics={
'data': {'partitions': 5, 'replication-factor': 2},
'echo': {'partitions': 5, 'replication-factor': 2},
'min': {'partitions': 5, 'replication-factor': 2},
'sum': {'partitions': 5, 'replication-factor': 2},
'repartition': {'partitions': 5, 'replication-factor': 2},
'max': {'partitions': 5, 'replication-factor': 2},
'cnt': {'partitions': 5, 'replication-factor': 2}
})
self.driver = StreamsEosTestDriverService(test_context, self.kafka)
self.test_context = test_context
@cluster(num_nodes=8)
@matrix(metadata_quorum=[quorum.combined_kraft],
group_protocol=["classic", "streams"])
def test_rebalance_simple(self, metadata_quorum, group_protocol):
self.group_protocol = group_protocol
self.run_rebalance(StreamsEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsEosTestVerifyRunnerService(self.test_context, self.kafka, group_protocol))
@cluster(num_nodes=8)
@matrix(metadata_quorum=[quorum.combined_kraft],
group_protocol=["classic", "streams"])
def test_rebalance_complex(self, metadata_quorum, group_protocol):
self.group_protocol = group_protocol
self.run_rebalance(StreamsComplexEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsComplexEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsComplexEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsComplexEosTestVerifyRunnerService(self.test_context, self.kafka, group_protocol))
def run_rebalance(self, processor1, processor2, processor3, verifier):
"""
Starts and stops two test clients a few times.
Ensure that all records are delivered exactly-once.
"""
self.driver.start()
self.add_streams(processor1)
processor1.clean_node_enabled = False
self.add_streams2(processor1, processor2)
self.add_streams3(processor1, processor2, processor3)
self.stop_streams3(processor2, processor3, processor1)
self.add_streams3(processor2, processor3, processor1)
self.stop_streams3(processor1, processor3, processor2)
self.stop_streams2(processor1, processor3)
self.stop_streams(processor1)
processor1.clean_node_enabled = True
self.driver.stop()
verifier.start()
verifier.wait()
verifier.node.account.ssh("grep ALL-RECORDS-DELIVERED %s" % verifier.STDOUT_FILE, allow_fail=False)
@cluster(num_nodes=8)
@matrix(metadata_quorum=[quorum.combined_kraft],
group_protocol=["classic", "streams"])
def test_failure_and_recovery(self, metadata_quorum, group_protocol):
self.group_protocol = group_protocol
self.run_failure_and_recovery(StreamsEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsEosTestVerifyRunnerService(self.test_context, self.kafka, group_protocol))
@cluster(num_nodes=8)
@matrix(metadata_quorum=[quorum.combined_kraft],
group_protocol=["classic", "streams"])
def test_failure_and_recovery_complex(self, metadata_quorum, group_protocol):
self.group_protocol = group_protocol
self.run_failure_and_recovery(StreamsComplexEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsComplexEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsComplexEosTestJobRunnerService(self.test_context, self.kafka, group_protocol),
StreamsComplexEosTestVerifyRunnerService(self.test_context, self.kafka, group_protocol))
def run_failure_and_recovery(self, processor1, processor2, processor3, verifier):
"""
Starts two test clients, then abort (kill -9) and restart them a few times.
Ensure that all records are delivered exactly-once.
"""
self.driver.start()
self.add_streams(processor1)
processor1.clean_node_enabled = False
self.add_streams2(processor1, processor2)
self.add_streams3(processor1, processor2, processor3)
self.abort_streams(processor2, processor3, processor1)
self.add_streams3(processor2, processor3, processor1)
self.abort_streams(processor2, processor3, processor1)
self.add_streams3(processor2, processor3, processor1)
self.abort_streams(processor1, processor3, processor2)
self.stop_streams2(processor1, processor3)
self.stop_streams(processor1)
processor1.clean_node_enabled = True
self.driver.stop()
verifier.start()
verifier.wait()
verifier.node.account.ssh("grep ALL-RECORDS-DELIVERED %s" % verifier.STDOUT_FILE, allow_fail=False)
def add_streams(self, processor):
with processor.node.account.monitor_log(processor.STDOUT_FILE) as monitor:
processor.start()
self.wait_for_startup(monitor, processor)
def add_streams2(self, running_processor, processor_to_be_started):
with running_processor.node.account.monitor_log(running_processor.STDOUT_FILE) as monitor:
self.add_streams(processor_to_be_started)
self.wait_for_startup(monitor, running_processor)
def add_streams3(self, running_processor1, running_processor2, processor_to_be_started):
with running_processor1.node.account.monitor_log(running_processor1.STDOUT_FILE) as monitor:
self.add_streams2(running_processor2, processor_to_be_started)
self.wait_for_startup(monitor, running_processor1)
def stop_streams(self, processor_to_be_stopped):
with processor_to_be_stopped.node.account.monitor_log(processor_to_be_stopped.STDOUT_FILE) as monitor2:
processor_to_be_stopped.stop()
self.wait_for(monitor2, processor_to_be_stopped, "StateChange: PENDING_SHUTDOWN -> NOT_RUNNING")
def stop_streams2(self, keep_alive_processor, processor_to_be_stopped):
with keep_alive_processor.node.account.monitor_log(keep_alive_processor.STDOUT_FILE) as monitor:
self.stop_streams(processor_to_be_stopped)
self.wait_for_startup(monitor, keep_alive_processor)
def stop_streams3(self, keep_alive_processor1, keep_alive_processor2, processor_to_be_stopped):
with keep_alive_processor1.node.account.monitor_log(keep_alive_processor1.STDOUT_FILE) as monitor:
self.stop_streams2(keep_alive_processor2, processor_to_be_stopped)
self.wait_for_startup(monitor, keep_alive_processor1)
def abort_streams(self, keep_alive_processor1, keep_alive_processor2, processor_to_be_aborted):
with keep_alive_processor1.node.account.monitor_log(keep_alive_processor1.STDOUT_FILE) as monitor1:
with keep_alive_processor2.node.account.monitor_log(keep_alive_processor2.STDOUT_FILE) as monitor2:
processor_to_be_aborted.stop_nodes(False)
self.wait_for_startup(monitor2, keep_alive_processor2)
self.wait_for_startup(monitor1, keep_alive_processor1)
def wait_for_startup(self, monitor, processor):
if self.group_protocol == "classic":
self.wait_for(monitor, processor, "StateChange: REBALANCING -> RUNNING")
else:
# In the streams group protocol, not all members will take part in the rebalance.
# We can indirectly observe the progress of the group by seeing the member epoch being bumped.
self.wait_for(monitor, processor, "MemberEpochBump")
self.wait_for(monitor, processor, "processed [0-9]* records from topic")
@staticmethod
def wait_for(monitor, processor, output):
monitor.wait_until(output,
timeout_sec=480,
err_msg=("Never saw output '%s' on " % output) + str(processor.node.account))

View File

@ -27,6 +27,7 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.GroupProtocol;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValueTimestamp;
@ -341,9 +342,8 @@ public class DeleteStreamsGroupOffsetTest {
private void stopKSApp(String appId, String topic, KafkaStreams streams, StreamsGroupCommand.StreamsGroupService service) throws InterruptedException {
if (streams != null) {
KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofSeconds(30));
closeOptions.leaveGroup(true);
CloseOptions closeOptions = CloseOptions.timeout(Duration.ofSeconds(30))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streams.close(closeOptions);
streams.cleanUp();

View File

@ -31,6 +31,7 @@ import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.streams.CloseOptions;
import org.apache.kafka.streams.GroupProtocol;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValueTimestamp;
@ -512,9 +513,8 @@ public class DeleteStreamsGroupTest {
private void stopKSApp(String appId, KafkaStreams streams, StreamsGroupCommand.StreamsGroupService service) throws InterruptedException {
if (streams != null) {
KafkaStreams.CloseOptions closeOptions = new KafkaStreams.CloseOptions();
closeOptions.timeout(Duration.ofSeconds(30));
closeOptions.leaveGroup(true);
CloseOptions closeOptions = CloseOptions.timeout(Duration.ofSeconds(30))
.withGroupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP);
streams.close(closeOptions);
streams.cleanUp();

View File

@ -23,7 +23,7 @@
// and not the version installed on the machine running the task.
// Read more about the wrapper here: https://docs.gradle.org/current/userguide/gradle_wrapper.html
wrapper {
gradleVersion = project.gradleVersion
gradleVersion = versions.gradle
}
// Custom task to inject support for downloading the gradle wrapper jar if it doesn't exist.
@ -35,14 +35,12 @@ task bootstrapWrapper() {
def wrapperBasePath = "\$APP_HOME/gradle/wrapper"
def wrapperJarPath = wrapperBasePath + "/gradle-wrapper.jar"
// Add a trailing zero to the version if needed.
def fullVersion = project.gradleVersion.count(".") == 1 ? "${project.gradleVersion}.0" : versions.gradle
// Leverages the wrapper jar checked into the gradle project on github because the jar isn't
// available elsewhere. Using raw.githubusercontent.com instead of github.com because
// github.com servers deprecated TLSv1/TLSv1.1 support some time ago, so older versions
// of curl (built against OpenSSL library that doesn't support TLSv1.2) would fail to
// fetch the jar.
def wrapperBaseUrl = "https://raw.githubusercontent.com/gradle/gradle/v$fullVersion/gradle/wrapper"
def wrapperBaseUrl = "https://raw.githubusercontent.com/gradle/gradle/v$versions.gradle/gradle/wrapper"
def wrapperJarUrl = wrapperBaseUrl + "/gradle-wrapper.jar"
def bootstrapString = """
@ -59,13 +57,15 @@ task bootstrapWrapper() {
done
""".stripIndent()
String putBootstrapStringAbove = "# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script."
def wrapperScript = wrapper.scriptFile
def wrapperLines = wrapperScript.readLines()
wrapperScript.withPrintWriter { out ->
def bootstrapWritten = false
wrapperLines.each { line ->
// Print the wrapper bootstrap before the first usage of the wrapper jar.
if (!bootstrapWritten && line.contains("gradle-wrapper.jar")) {
if (!bootstrapWritten && line.contains(putBootstrapStringAbove)) {
out.println(bootstrapString)
bootstrapWritten = true
}