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_COORDINATOR_STATE_TOPIC_MIN_ISR = "share.coordinator.state.topic.min.isr"
|
||||||
SHARE_GROUP_ENABLE = "group.share.enable"
|
SHARE_GROUP_ENABLE = "group.share.enable"
|
||||||
|
|
||||||
|
UNSTABLE_API_VERSIONS_ENABLE = "unstable.api.versions.enable"
|
||||||
|
|
||||||
"""
|
"""
|
||||||
From KafkaConfig.scala
|
From KafkaConfig.scala
|
||||||
|
|
||||||
|
|
|
@ -206,7 +206,8 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
||||||
consumer_group_migration_policy=None,
|
consumer_group_migration_policy=None,
|
||||||
dynamicRaftQuorum=False,
|
dynamicRaftQuorum=False,
|
||||||
use_transactions_v2=False,
|
use_transactions_v2=False,
|
||||||
use_share_groups=None
|
use_share_groups=None,
|
||||||
|
use_streams_groups=False
|
||||||
):
|
):
|
||||||
"""
|
"""
|
||||||
:param context: test context
|
: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 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_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_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
|
self.zk = zk
|
||||||
|
@ -296,6 +298,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service):
|
||||||
# Assign the determined value.
|
# Assign the determined value.
|
||||||
self.use_transactions_v2 = use_transactions_v2
|
self.use_transactions_v2 = use_transactions_v2
|
||||||
self.use_share_groups = use_share_groups
|
self.use_share_groups = use_share_groups
|
||||||
|
self.use_streams_groups = use_streams_groups
|
||||||
|
|
||||||
# Set consumer_group_migration_policy based on context and arguments.
|
# Set consumer_group_migration_policy based on context and arguments.
|
||||||
if consumer_group_migration_policy is None:
|
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), []):
|
for prop in self.per_node_server_prop_overrides.get(self.idx(node), []):
|
||||||
override_configs[prop[0]] = prop[1]
|
override_configs[prop[0]] = prop[1]
|
||||||
|
|
||||||
|
enabledProtocols = 'classic,consumer'
|
||||||
|
|
||||||
if self.use_share_groups is not None and self.use_share_groups is True:
|
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.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
|
#update template configs with test override configs
|
||||||
configs.update(override_configs)
|
configs.update(override_configs)
|
||||||
|
|
|
@ -320,13 +320,14 @@ class StreamsTestBaseService(KafkaPathResolverMixin, JmxMixin, Service):
|
||||||
class StreamsSmokeTestBaseService(StreamsTestBaseService):
|
class StreamsSmokeTestBaseService(StreamsTestBaseService):
|
||||||
"""Base class for Streams Smoke Test services providing some common settings and functionality"""
|
"""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,
|
super(StreamsSmokeTestBaseService, self).__init__(test_context,
|
||||||
kafka,
|
kafka,
|
||||||
"org.apache.kafka.streams.tests.StreamsSmokeTest",
|
"org.apache.kafka.streams.tests.StreamsSmokeTest",
|
||||||
command)
|
command)
|
||||||
self.NUM_THREADS = num_threads
|
self.NUM_THREADS = num_threads
|
||||||
self.PROCESSING_GUARANTEE = processing_guarantee
|
self.PROCESSING_GUARANTEE = processing_guarantee
|
||||||
|
self.GROUP_PROTOCOL = group_protocol
|
||||||
self.KAFKA_STREAMS_VERSION = ""
|
self.KAFKA_STREAMS_VERSION = ""
|
||||||
self.UPGRADE_FROM = None
|
self.UPGRADE_FROM = None
|
||||||
self.REPLICATION_FACTOR = replication_factor
|
self.REPLICATION_FACTOR = replication_factor
|
||||||
|
@ -341,6 +342,7 @@ class StreamsSmokeTestBaseService(StreamsTestBaseService):
|
||||||
properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT,
|
properties = {streams_property.STATE_DIR: self.PERSISTENT_ROOT,
|
||||||
streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(),
|
streams_property.KAFKA_SERVERS: self.kafka.bootstrap_servers(),
|
||||||
streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE,
|
streams_property.PROCESSING_GUARANTEE: self.PROCESSING_GUARANTEE,
|
||||||
|
streams_property.GROUP_PROTOCOL: self.GROUP_PROTOCOL,
|
||||||
streams_property.NUM_THREADS: self.NUM_THREADS,
|
streams_property.NUM_THREADS: self.NUM_THREADS,
|
||||||
"replication.factor": self.REPLICATION_FACTOR,
|
"replication.factor": self.REPLICATION_FACTOR,
|
||||||
"num.standby.replicas": 2,
|
"num.standby.replicas": 2,
|
||||||
|
@ -436,8 +438,8 @@ class StreamsSmokeTestDriverService(StreamsSmokeTestBaseService):
|
||||||
return cmd
|
return cmd
|
||||||
|
|
||||||
class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService):
|
class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService):
|
||||||
def __init__(self, test_context, kafka, processing_guarantee, num_threads = 3, replication_factor = 3):
|
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, num_threads, replication_factor)
|
super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process", processing_guarantee, group_protocol, num_threads, replication_factor)
|
||||||
|
|
||||||
class StreamsEosTestDriverService(StreamsEosTestBaseService):
|
class StreamsEosTestDriverService(StreamsEosTestBaseService):
|
||||||
def __init__(self, test_context, kafka):
|
def __init__(self, test_context, kafka):
|
||||||
|
|
|
@ -21,3 +21,4 @@ STATE_DIR = "state.dir"
|
||||||
KAFKA_SERVERS = "bootstrap.servers"
|
KAFKA_SERVERS = "bootstrap.servers"
|
||||||
NUM_THREADS = "num.stream.threads"
|
NUM_THREADS = "num.stream.threads"
|
||||||
PROCESSING_GUARANTEE = "processing.guarantee"
|
PROCESSING_GUARANTEE = "processing.guarantee"
|
||||||
|
GROUP_PROTOCOL = "group.protocol"
|
||||||
|
|
|
@ -13,13 +13,14 @@
|
||||||
# See the License for the specific language governing permissions and
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
|
|
||||||
|
from ducktape.tests.test import Test
|
||||||
from ducktape.utils.util import wait_until
|
from ducktape.utils.util import wait_until
|
||||||
from kafkatest.services.verifiable_consumer import VerifiableConsumer
|
from kafkatest.services.verifiable_consumer import VerifiableConsumer
|
||||||
from kafkatest.services.verifiable_producer import VerifiableProducer
|
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
|
Helper class that contains methods for producing and consuming
|
||||||
messages and verification of results from log files
|
messages and verification of results from log files
|
||||||
|
@ -28,7 +29,23 @@ class BaseStreamsTest(KafkaTest):
|
||||||
see tests/kafkatest/tests/kafka_test.py for more info
|
see tests/kafkatest/tests/kafka_test.py for more info
|
||||||
"""
|
"""
|
||||||
def __init__(self, test_context, topics, num_controllers=1, num_brokers=3):
|
def __init__(self, test_context, topics, num_controllers=1, num_brokers=3):
|
||||||
super(BaseStreamsTest, self).__init__(test_context, num_controllers, num_brokers, topics)
|
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):
|
def get_consumer(self, client_id, topic, num_messages):
|
||||||
return VerifiableConsumer(self.test_context,
|
return VerifiableConsumer(self.test_context,
|
||||||
|
|
|
@ -41,8 +41,8 @@ class StreamsBrokerDownResilience(BaseStreamsTest):
|
||||||
num_brokers=1)
|
num_brokers=1)
|
||||||
|
|
||||||
def setUp(self):
|
def setUp(self):
|
||||||
if self.zk:
|
# do not start kafka
|
||||||
self.zk.start()
|
pass
|
||||||
|
|
||||||
@cluster(num_nodes=7)
|
@cluster(num_nodes=7)
|
||||||
@matrix(metadata_quorum=[quorum.combined_kraft])
|
@matrix(metadata_quorum=[quorum.combined_kraft])
|
||||||
|
|
|
@ -17,16 +17,16 @@ from ducktape.mark import matrix
|
||||||
from ducktape.mark.resource import cluster
|
from ducktape.mark.resource import cluster
|
||||||
|
|
||||||
from kafkatest.services.kafka import quorum
|
from kafkatest.services.kafka import quorum
|
||||||
from kafkatest.tests.kafka_test import KafkaTest
|
|
||||||
from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService
|
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.
|
Simple test of Kafka Streams.
|
||||||
"""
|
"""
|
||||||
|
|
||||||
def __init__(self, test_context):
|
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 },
|
'echo' : { 'partitions': 5, 'replication-factor': 1 },
|
||||||
'data' : { 'partitions': 5, 'replication-factor': 1 },
|
'data' : { 'partitions': 5, 'replication-factor': 1 },
|
||||||
'min' : { 'partitions': 5, 'replication-factor': 1 },
|
'min' : { 'partitions': 5, 'replication-factor': 1 },
|
||||||
|
@ -49,11 +49,12 @@ class StreamsSmokeTest(KafkaTest):
|
||||||
@cluster(num_nodes=8)
|
@cluster(num_nodes=8)
|
||||||
@matrix(processing_guarantee=['exactly_once_v2', 'at_least_once'],
|
@matrix(processing_guarantee=['exactly_once_v2', 'at_least_once'],
|
||||||
crash=[True, False],
|
crash=[True, False],
|
||||||
metadata_quorum=[quorum.combined_kraft])
|
metadata_quorum=[quorum.combined_kraft],
|
||||||
def test_streams(self, processing_guarantee, crash, metadata_quorum):
|
group_protocol=["classic", "streams"])
|
||||||
processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee)
|
def test_streams(self, processing_guarantee, crash, metadata_quorum, group_protocol):
|
||||||
processor2 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee)
|
processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee, group_protocol)
|
||||||
processor3 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka, processing_guarantee)
|
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:
|
with processor1.node.account.monitor_log(processor1.STDOUT_FILE) as monitor1:
|
||||||
processor1.start()
|
processor1.start()
|
||||||
|
|
Loading…
Reference in New Issue