mirror of https://github.com/apache/kafka.git
Compare commits
9 Commits
233cca8d95
...
96194a7aeb
Author | SHA1 | Date |
---|---|---|
|
96194a7aeb | |
|
4a5aa37169 | |
|
2938c4242e | |
|
ebae768bd8 | |
|
fa2496bb91 | |
|
c6bbbbe24d | |
|
aa0f4ea808 | |
|
d56ccebafc | |
|
9ef493983c |
|
@ -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:
|
||||
|
|
|
@ -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
|
||||
|
|
23
build.gradle
23
build.gradle
|
@ -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'
|
||||
|
|
|
@ -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"/>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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—{@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—{@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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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.");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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):
|
||||
|
|
|
@ -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))
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue