|
13 | 13 | # See the License for the specific language governing permissions and
|
14 | 14 | # limitations under the License.
|
15 | 15 |
|
| 16 | +from ducktape.utils.util import wait_until |
16 | 17 | from ducktape.mark import matrix
|
17 | 18 | from ducktape.mark import parametrize
|
18 | 19 | from ducktape.mark.resource import cluster
|
19 | 20 | from ducktape.services.service import Service
|
20 | 21 | from ducktape.tests.test import Test
|
21 | 22 |
|
22 | 23 | from kafkatest.services.kafka import KafkaService, quorum
|
23 |
| -from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService, throughput, latency, compute_aggregate_throughput |
| 24 | +from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService, ShareConsumerPerformanceService, throughput, latency, compute_aggregate_throughput |
| 25 | +from kafkatest.services.security.security_config import SecurityConfig |
24 | 26 | from kafkatest.version import DEV_BRANCH, KafkaVersion
|
25 | 27 |
|
| 28 | +import os |
| 29 | + |
26 | 30 | TOPIC_REP_ONE = "topic-replication-factor-one"
|
27 | 31 | TOPIC_REP_THREE = "topic-replication-factor-three"
|
28 | 32 | DEFAULT_RECORD_SIZE = 100 # bytes
|
@@ -232,6 +236,71 @@ def test_producer_and_consumer(self, compression_type="none", security_protocol=
|
232 | 236 | str(data)]
|
233 | 237 | self.logger.info("\n".join(summary))
|
234 | 238 | return data
|
| 239 | + |
| 240 | + @cluster(num_nodes=8) |
| 241 | + @matrix(security_protocol=['SSL'], interbroker_security_protocol=['PLAINTEXT'], tls_version=['TLSv1.2', 'TLSv1.3'], |
| 242 | + compression_type=["none", "snappy"], metadata_quorum=[quorum.isolated_kraft], use_share_groups=[True]) |
| 243 | + @matrix(security_protocol=['PLAINTEXT'], compression_type=["none", "snappy"], metadata_quorum=[quorum.isolated_kraft], |
| 244 | + use_share_groups=[True]) |
| 245 | + def test_producer_and_share_consumer(self, compression_type="none", security_protocol="PLAINTEXT", tls_version=None, |
| 246 | + interbroker_security_protocol=None, client_version=str(DEV_BRANCH), broker_version=str(DEV_BRANCH), |
| 247 | + metadata_quorum=quorum.isolated_kraft, use_share_groups=True): |
| 248 | + """ |
| 249 | + Setup: 3 node kafka cluster |
| 250 | + Concurrently produce and consume 1e6 messages with a single producer and a single share consumer, |
| 251 | +
|
| 252 | + Return aggregate throughput statistics for both producer and share consumer. |
| 253 | +
|
| 254 | + (Under the hood, this runs ProducerPerformance.java, and ShareConsumerPerformance.java) |
| 255 | + """ |
| 256 | + client_version = KafkaVersion(client_version) |
| 257 | + broker_version = KafkaVersion(broker_version) |
| 258 | + self.validate_versions(client_version, broker_version) |
| 259 | + if interbroker_security_protocol is None: |
| 260 | + interbroker_security_protocol = security_protocol |
| 261 | + self.start_kafka(security_protocol, interbroker_security_protocol, broker_version, tls_version) |
| 262 | + num_records = 1000 * 1000 # 1e6 |
| 263 | + |
| 264 | + self.producer = ProducerPerformanceService( |
| 265 | + self.test_context, 1, self.kafka, |
| 266 | + topic=TOPIC_REP_THREE, |
| 267 | + num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version, |
| 268 | + settings={ |
| 269 | + 'acks': 1, |
| 270 | + 'compression.type': compression_type, |
| 271 | + 'batch.size': self.batch_size, |
| 272 | + 'buffer.memory': self.buffer_memory |
| 273 | + } |
| 274 | + ) |
| 275 | + |
| 276 | + share_group = "perf-share-consumer" |
| 277 | + |
| 278 | + kafka_node = self.kafka.nodes[0] |
| 279 | + PERSISTENT_ROOT = "/mnt/share_consumer_performance" |
| 280 | + COMMAND_CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "command.properties") |
| 281 | + |
| 282 | + if security_protocol is not SecurityConfig.PLAINTEXT: |
| 283 | + prop_file = str(self.kafka.security_config.client_config()) |
| 284 | + self.logger.debug(prop_file) |
| 285 | + kafka_node.account.ssh("mkdir -p %s" % PERSISTENT_ROOT, allow_fail=False) |
| 286 | + kafka_node.account.create_file(COMMAND_CONFIG_FILE, prop_file) |
| 287 | + |
| 288 | + wait_until(lambda: self.kafka.set_share_group_offset_reset_strategy(group=share_group, strategy="earliest", command_config=COMMAND_CONFIG_FILE), |
| 289 | + timeout_sec=20, backoff_sec=2, err_msg="share.auto.offset.reset not set to earliest") |
| 290 | + |
| 291 | + self.share_consumer = ShareConsumerPerformanceService( |
| 292 | + self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, messages=num_records, group=share_group, timeout=20000) |
| 293 | + Service.run_parallel(self.producer, self.share_consumer) |
| 294 | + |
| 295 | + data = { |
| 296 | + "producer": compute_aggregate_throughput(self.producer), |
| 297 | + "share_consumer": compute_aggregate_throughput(self.share_consumer) |
| 298 | + } |
| 299 | + summary = [ |
| 300 | + "Producer + share_consumer:", |
| 301 | + str(data)] |
| 302 | + self.logger.info("\n".join(summary)) |
| 303 | + return data |
235 | 304 |
|
236 | 305 | @cluster(num_nodes=8)
|
237 | 306 | @matrix(security_protocol=['SSL'], interbroker_security_protocol=['PLAINTEXT'], tls_version=['TLSv1.2', 'TLSv1.3'],
|
@@ -273,6 +342,62 @@ def test_consumer_throughput(self, compression_type="none", security_protocol="P
|
273 | 342 | self.consumer.group = "test-consumer-group"
|
274 | 343 | self.consumer.run()
|
275 | 344 | return compute_aggregate_throughput(self.consumer)
|
| 345 | + |
| 346 | + @cluster(num_nodes=8) |
| 347 | + @matrix(security_protocol=['SSL'], interbroker_security_protocol=['PLAINTEXT'], tls_version=['TLSv1.2', 'TLSv1.3'], |
| 348 | + compression_type=["none", "snappy"], metadata_quorum=[quorum.isolated_kraft], use_share_groups=[True]) |
| 349 | + @matrix(security_protocol=['PLAINTEXT'], compression_type=["none", "snappy"], metadata_quorum=[quorum.isolated_kraft], |
| 350 | + use_share_groups=[True]) |
| 351 | + def test_share_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT", tls_version=None, |
| 352 | + interbroker_security_protocol=None, num_consumers=1, client_version=str(DEV_BRANCH), |
| 353 | + broker_version=str(DEV_BRANCH), metadata_quorum=quorum.isolated_kraft, use_share_groups=True): |
| 354 | + """ |
| 355 | + Consume 1e6 100-byte messages with 1 or more consumers from a topic with 6 partitions |
| 356 | + and report throughput. |
| 357 | + """ |
| 358 | + client_version = KafkaVersion(client_version) |
| 359 | + broker_version = KafkaVersion(broker_version) |
| 360 | + self.validate_versions(client_version, broker_version) |
| 361 | + if interbroker_security_protocol is None: |
| 362 | + interbroker_security_protocol = security_protocol |
| 363 | + self.start_kafka(security_protocol, interbroker_security_protocol, broker_version, tls_version) |
| 364 | + num_records = 1000 * 1000 # 1e6 |
| 365 | + |
| 366 | + # seed kafka w/messages |
| 367 | + self.producer = ProducerPerformanceService( |
| 368 | + self.test_context, 1, self.kafka, |
| 369 | + topic=TOPIC_REP_THREE, |
| 370 | + num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version, |
| 371 | + settings={ |
| 372 | + 'acks': 1, |
| 373 | + 'compression.type': compression_type, |
| 374 | + 'batch.size': self.batch_size, |
| 375 | + 'buffer.memory': self.buffer_memory |
| 376 | + } |
| 377 | + ) |
| 378 | + self.producer.run() |
| 379 | + |
| 380 | + share_group = "test-share-consumer-group" |
| 381 | + |
| 382 | + kafka_node = self.kafka.nodes[0] |
| 383 | + PERSISTENT_ROOT = "/mnt/share_consumer_performance" |
| 384 | + COMMAND_CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "command.properties") |
| 385 | + |
| 386 | + if security_protocol is not SecurityConfig.PLAINTEXT: |
| 387 | + prop_file = str(self.kafka.security_config.client_config()) |
| 388 | + self.logger.debug(prop_file) |
| 389 | + kafka_node.account.ssh("mkdir -p %s" % PERSISTENT_ROOT, allow_fail=False) |
| 390 | + kafka_node.account.create_file(COMMAND_CONFIG_FILE, prop_file) |
| 391 | + |
| 392 | + wait_until(lambda: self.kafka.set_share_group_offset_reset_strategy(group=share_group, strategy="earliest", command_config=COMMAND_CONFIG_FILE), |
| 393 | + timeout_sec=20, backoff_sec=2, err_msg="share.auto.offset.reset not set to earliest") |
| 394 | + |
| 395 | + # consume |
| 396 | + self.share_consumer = ShareConsumerPerformanceService( |
| 397 | + self.test_context, num_consumers, self.kafka, |
| 398 | + topic=TOPIC_REP_THREE, messages=num_records, group=share_group, timeout=20000) |
| 399 | + self.share_consumer.run() |
| 400 | + return compute_aggregate_throughput(self.share_consumer) |
276 | 401 |
|
277 | 402 | def validate_versions(self, client_version, broker_version):
|
278 | 403 | assert client_version <= broker_version, "Client version %s should be <= than broker version %s" (client_version, broker_version)
|
0 commit comments