mirror of https://github.com/apache/kafka.git
KAFKA-19202: Enable KIP-1071 in streams_smoke_test.py (#19560)
CI / build (push) Waiting to run
Details
CI / build (push) Waiting to run
Details
Enables KIP-1071 (`group.protocol=streams`) in the first streams system test `streams_smoke_test.py`. All tests using KIP-1071 cannot use `KafkaTest` anymore, since we need to customize the broker configuration. The corresponding functionality is added to `BaseStreamsTest`, which all streams tests will have to extend from now on. There are some left-overs from ZK in the tests that I copied from 'KafkaTest'. They need to be cleaned up, but this should be done in a separate PR.
This commit is contained in:
parent
019459e950
commit
5c63b4569b
|
@ -81,6 +81,8 @@ SHARE_COORDINATOR_STATE_TOPIC_REPLICATION_FACTOR ="share.coordinator.state.topic
|
|||
SHARE_COORDINATOR_STATE_TOPIC_MIN_ISR = "share.coordinator.state.topic.min.isr"
|
||||
SHARE_GROUP_ENABLE = "group.share.enable"
|
||||
|
||||
UNSTABLE_API_VERSIONS_ENABLE = "unstable.api.versions.enable"
|
||||
|
||||
"""
|
||||
From KafkaConfig.scala
|
||||
|
||||
|
|
|
@ -206,7 +206,8 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
consumer_group_migration_policy=None,
|
||||
dynamicRaftQuorum=False,
|
||||
use_transactions_v2=False,
|
||||
use_share_groups=None
|
||||
use_share_groups=None,
|
||||
use_streams_groups=False
|
||||
):
|
||||
"""
|
||||
:param context: test context
|
||||
|
@ -271,6 +272,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
:param dynamicRaftQuorum: When true, controller_quorum_bootstrap_servers, and bootstraps the first controller using the standalone flag
|
||||
:param use_transactions_v2: When true, uses transaction.version=2 which utilizes the new transaction protocol introduced in KIP-890
|
||||
:param use_share_groups: When true, enables the use of share groups introduced in KIP-932
|
||||
:param use_streams_groups: When true, enables the use of streams groups introduced in KIP-1071
|
||||
"""
|
||||
|
||||
self.zk = zk
|
||||
|
@ -296,6 +298,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
# Assign the determined value.
|
||||
self.use_transactions_v2 = use_transactions_v2
|
||||
self.use_share_groups = use_share_groups
|
||||
self.use_streams_groups = use_streams_groups
|
||||
|
||||
# Set consumer_group_migration_policy based on context and arguments.
|
||||
if consumer_group_migration_policy is None:
|
||||
|
@ -776,9 +779,17 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
for prop in self.per_node_server_prop_overrides.get(self.idx(node), []):
|
||||
override_configs[prop[0]] = prop[1]
|
||||
|
||||
enabledProtocols = 'classic,consumer'
|
||||
|
||||
if self.use_share_groups is not None and self.use_share_groups is True:
|
||||
override_configs[config_property.SHARE_GROUP_ENABLE] = str(self.use_share_groups)
|
||||
override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = 'classic,consumer,share'
|
||||
enabledProtocols += ',share'
|
||||
|
||||
if self.use_streams_groups is True:
|
||||
override_configs[config_property.UNSTABLE_API_VERSIONS_ENABLE] = str(True)
|
||||
enabledProtocols += ',streams'
|
||||
|
||||
override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = enabledProtocols
|
||||
|
||||
#update template configs with test override configs
|
||||
configs.update(override_configs)
|
||||
|
|
|
@ -320,13 +320,14 @@ class StreamsTestBaseService(KafkaPathResolverMixin, JmxMixin, Service):
|
|||
class StreamsSmokeTestBaseService(StreamsTestBaseService):
|
||||
"""Base class for Streams Smoke Test services providing some common settings and functionality"""
|
||||
|
||||
def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', num_threads = 3, replication_factor = 3):
|
||||
def __init__(self, test_context, kafka, command, processing_guarantee = 'at_least_once', group_protocol = 'classic', num_threads = 3, replication_factor = 3):
|
||||
super(StreamsSmokeTestBaseService, self).__init__(test_context,
|
||||
kafka,
|
||||
"org.apache.kafka.streams.tests.StreamsSmokeTest",
|
||||
command)
|
||||
self.NUM_THREADS = num_threads
|
||||
self.PROCESSING_GUARANTEE = processing_guarantee
|
||||
self.GROUP_PROTOCOL = group_protocol
|
||||
self.KAFKA_STREAMS_VERSION = ""
|
||||
self.UPGRADE_FROM = None
|
||||
self.REPLICATION_FACTOR = replication_factor
|
||||
|
@ -341,6 +342,7 @@ class StreamsSmokeTestBaseService(StreamsTestBaseService):
|
|||
properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT,
|
||||
streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(),
|
||||
streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE,
|
||||
streams_property.GROUP_PROTOCOL: self.GROUP_PROTOCOL,
|
||||
streams_property.NUM_THREADS: self.NUM_THREADS,
|
||||
"replication.factor": self.REPLICATION_FACTOR,
|
||||
"num.standby.replicas": 2,
|
||||
|
@ -436,8 +438,8 @@ class StreamsSmokeTestDriverService(StreamsSmokeTestBaseService):
|
|||
return cmd
|
||||
|
||||
class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService):
|
||||
def __init__(self, test_context, kafka, processing_guarantee, num_threads = 3, replication_factor = 3):
|
||||
super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, num_threads, replication_factor)
|
||||
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):
|
||||
|
|
|
@ -21,3 +21,4 @@ STATE_DIR = "state.dir"
|
|||
KAFKA_SERVERS = "bootstrap.servers"
|
||||
NUM_THREADS = "num.stream.threads"
|
||||
PROCESSING_GUARANTEE = "processing.guarantee"
|
||||
GROUP_PROTOCOL = "group.protocol"
|
||||
|
|
|
@ -13,13 +13,14 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
from ducktape.tests.test import Test
|
||||
from ducktape.utils.util import wait_until
|
||||
from kafkatest.services.verifiable_consumer import VerifiableConsumer
|
||||
from kafkatest.services.verifiable_producer import VerifiableProducer
|
||||
from kafkatest.tests.kafka_test import KafkaTest
|
||||
from kafkatest.services.kafka import KafkaService
|
||||
|
||||
|
||||
class BaseStreamsTest(KafkaTest):
|
||||
class BaseStreamsTest(Test):
|
||||
"""
|
||||
Helper class that contains methods for producing and consuming
|
||||
messages and verification of results from log files
|
||||
|
@ -27,8 +28,24 @@ class BaseStreamsTest(KafkaTest):
|
|||
Extends KafkaTest which manages setting up Kafka Cluster and Zookeeper
|
||||
see tests/kafkatest/tests/kafka_test.py for more info
|
||||
"""
|
||||
def __init__(self, test_context, topics, num_controllers=1, num_brokers=3):
|
||||
super(BaseStreamsTest, self).__init__(test_context, num_controllers, num_brokers, topics)
|
||||
def __init__(self, test_context, topics, num_controllers=1, num_brokers=3):
|
||||
self.num_controllers = num_controllers
|
||||
self.num_brokers = num_brokers
|
||||
self.topics = topics
|
||||
|
||||
self.kafka = KafkaService(
|
||||
test_context, self.num_brokers,
|
||||
None, topics=self.topics,
|
||||
controller_num_nodes_override=self.num_controllers,
|
||||
use_streams_groups=True,
|
||||
server_prop_overrides=[
|
||||
[ "group.streams.min.session.timeout.ms", "10000" ], # Need to up the lower bound
|
||||
[ "group.streams.session.timeout.ms", "10000" ] # As in classic groups, set this to 10s
|
||||
]
|
||||
)
|
||||
|
||||
def setUp(self):
|
||||
self.kafka.start()
|
||||
|
||||
def get_consumer(self, client_id, topic, num_messages):
|
||||
return VerifiableConsumer(self.test_context,
|
||||
|
|
|
@ -41,8 +41,8 @@ class StreamsBrokerDownResilience(BaseStreamsTest):
|
|||
num_brokers=1)
|
||||
|
||||
def setUp(self):
|
||||
if self.zk:
|
||||
self.zk.start()
|
||||
# do not start kafka
|
||||
pass
|
||||
|
||||
@cluster(num_nodes=7)
|
||||
@matrix(metadata_quorum=[quorum.combined_kraft])
|
||||
|
|
|
@ -17,16 +17,16 @@ from ducktape.mark import matrix
|
|||
from ducktape.mark.resource import cluster
|
||||
|
||||
from kafkatest.services.kafka import quorum
|
||||
from kafkatest.tests.kafka_test import KafkaTest
|
||||
from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService
|
||||
from kafkatest.tests.streams.base_streams_test import BaseStreamsTest
|
||||
|
||||
class StreamsSmokeTest(KafkaTest):
|
||||
class StreamsSmokeTest(BaseStreamsTest):
|
||||
"""
|
||||
Simple test of Kafka Streams.
|
||||
"""
|
||||
|
||||
def __init__(self, test_context):
|
||||
super(StreamsSmokeTest, self).__init__(test_context, num_zk=1, num_brokers=3, topics={
|
||||
super(StreamsSmokeTest, self).__init__(test_context, topics={
|
||||
'echo' : { 'partitions': 5, 'replication-factor': 1 },
|
||||
'data' : { 'partitions': 5, 'replication-factor': 1 },
|
||||
'min' : { 'partitions': 5, 'replication-factor': 1 },
|
||||
|
@ -49,11 +49,12 @@ class StreamsSmokeTest(KafkaTest):
|
|||
@cluster(num_nodes=8)
|
||||
@matrix(processing_guarantee=['exactly_once_v2', 'at_least_once'],
|
||||
crash=[True, False],
|
||||
metadata_quorum=[quorum.combined_kraft])
|
||||
def test_streams(self, processing_guarantee, crash, metadata_quorum):
|
||||
processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee)
|
||||
processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee)
|
||||
processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee)
|
||||
metadata_quorum=[quorum.combined_kraft],
|
||||
group_protocol=["classic", "streams"])
|
||||
def test_streams(self, processing_guarantee, crash, metadata_quorum, group_protocol):
|
||||
processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee, group_protocol)
|
||||
processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee, group_protocol)
|
||||
processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee, group_protocol)
|
||||
|
||||
with processor1.node.account.monitor_log(processor1.STDOUT_FILE) as monitor1:
|
||||
processor1.start()
|
||||
|
|
Loading…
Reference in New Issue