1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109 110 111 112 113 114 115 116 117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152 153 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189 190 191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223 224 225 226 227 228 229 230 231 232 233 234 235 236 237 238 239 240 241 242 243 244 245 246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701 702 703 704 705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032
|
#!/usr/bin/env python
#
# Copyright 2018 Confluent Inc.
#
# Licensed 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.
# Example use of AdminClient operations.
from confluent_kafka import (KafkaException, ConsumerGroupTopicPartitions,
TopicPartition, ConsumerGroupState,
TopicCollection, IsolationLevel,
ConsumerGroupType, ElectionType)
from confluent_kafka.admin import (AdminClient, NewTopic, NewPartitions, ConfigResource,
ConfigEntry, ConfigSource, AclBinding,
AclBindingFilter, ResourceType, ResourcePatternType,
AclOperation, AclPermissionType, AlterConfigOpType,
ScramMechanism, ScramCredentialInfo,
UserScramCredentialUpsertion, UserScramCredentialDeletion,
OffsetSpec)
import sys
import threading
import logging
import argparse
logging.basicConfig()
def parse_nullable_string(s):
if s == "None":
return None
else:
return s
def example_create_topics(a, topics):
""" Create topics """
new_topics = [NewTopic(topic, num_partitions=3, replication_factor=1) for topic in topics]
# Call create_topics to asynchronously create topics, a dict
# of <topic,future> is returned.
fs = a.create_topics(new_topics)
# Wait for operation to finish.
# Timeouts are preferably controlled by passing request_timeout=15.0
# to the create_topics() call.
# All futures will finish at the same time.
for topic, f in fs.items():
try:
f.result() # The result itself is None
print("Topic {} created".format(topic))
except Exception as e:
print("Failed to create topic {}: {}".format(topic, e))
def example_delete_topics(a, topics):
""" delete topics """
# Call delete_topics to asynchronously delete topics, a future is returned.
# By default this operation on the broker returns immediately while
# topics are deleted in the background. But here we give it some time (30s)
# to propagate in the cluster before returning.
#
# Returns a dict of <topic,future>.
fs = a.delete_topics(topics, operation_timeout=30)
# Wait for operation to finish.
for topic, f in fs.items():
try:
f.result() # The result itself is None
print("Topic {} deleted".format(topic))
except Exception as e:
print("Failed to delete topic {}: {}".format(topic, e))
def example_create_partitions(a, topics):
""" create partitions """
new_parts = [NewPartitions(topic, int(new_total_count)) for
topic, new_total_count in zip(topics[0::2], topics[1::2])]
# Try switching validate_only to True to only validate the operation
# on the broker but not actually perform it.
fs = a.create_partitions(new_parts, validate_only=False)
# Wait for operation to finish.
for topic, f in fs.items():
try:
f.result() # The result itself is None
print("Additional partitions created for topic {}".format(topic))
except Exception as e:
print("Failed to add partitions to topic {}: {}".format(topic, e))
def print_config(config, depth):
print('%40s = %-50s [%s,is:read-only=%r,default=%r,sensitive=%r,synonym=%r,synonyms=%s]' %
((' ' * depth) + config.name, config.value, ConfigSource(config.source),
config.is_read_only, config.is_default,
config.is_sensitive, config.is_synonym,
["%s:%s" % (x.name, ConfigSource(x.source))
for x in iter(config.synonyms.values())]))
def example_describe_configs(a, args):
""" describe configs """
resources = [ConfigResource(restype, resname) for
restype, resname in zip(args[0::2], args[1::2])]
fs = a.describe_configs(resources)
# Wait for operation to finish.
for res, f in fs.items():
try:
configs = f.result()
for config in iter(configs.values()):
print_config(config, 1)
except KafkaException as e:
print("Failed to describe {}: {}".format(res, e))
except Exception:
raise
def example_create_acls(a, args):
""" create acls """
acl_bindings = [
AclBinding(
ResourceType[restype],
parse_nullable_string(resname),
ResourcePatternType[resource_pattern_type],
parse_nullable_string(principal),
parse_nullable_string(host),
AclOperation[operation],
AclPermissionType[permission_type]
)
for restype, resname, resource_pattern_type,
principal, host, operation, permission_type
in zip(
args[0::7],
args[1::7],
args[2::7],
args[3::7],
args[4::7],
args[5::7],
args[6::7],
)
]
try:
fs = a.create_acls(acl_bindings, request_timeout=10)
except ValueError as e:
print(f"create_acls() failed: {e}")
return
# Wait for operation to finish.
for res, f in fs.items():
try:
result = f.result()
if result is None:
print("Created {}".format(res))
except KafkaException as e:
print("Failed to create ACL {}: {}".format(res, e))
except Exception:
raise
def example_describe_acls(a, args):
""" describe acls """
acl_binding_filters = [
AclBindingFilter(
ResourceType[restype],
parse_nullable_string(resname),
ResourcePatternType[resource_pattern_type],
parse_nullable_string(principal),
parse_nullable_string(host),
AclOperation[operation],
AclPermissionType[permission_type]
)
for restype, resname, resource_pattern_type,
principal, host, operation, permission_type
in zip(
args[0::7],
args[1::7],
args[2::7],
args[3::7],
args[4::7],
args[5::7],
args[6::7],
)
]
fs = [
a.describe_acls(acl_binding_filter, request_timeout=10)
for acl_binding_filter in acl_binding_filters
]
# Wait for operations to finish.
for acl_binding_filter, f in zip(acl_binding_filters, fs):
try:
print("Acls matching filter: {}".format(acl_binding_filter))
acl_bindings = f.result()
for acl_binding in acl_bindings:
print(acl_binding)
except KafkaException as e:
print("Failed to describe {}: {}".format(acl_binding_filter, e))
except Exception:
raise
def example_delete_acls(a, args):
""" delete acls """
acl_binding_filters = [
AclBindingFilter(
ResourceType[restype],
parse_nullable_string(resname),
ResourcePatternType[resource_pattern_type],
parse_nullable_string(principal),
parse_nullable_string(host),
AclOperation[operation],
AclPermissionType[permission_type]
)
for restype, resname, resource_pattern_type,
principal, host, operation, permission_type
in zip(
args[0::7],
args[1::7],
args[2::7],
args[3::7],
args[4::7],
args[5::7],
args[6::7],
)
]
try:
fs = a.delete_acls(acl_binding_filters, request_timeout=10)
except ValueError as e:
print(f"delete_acls() failed: {e}")
return
# Wait for operation to finish.
for res, f in fs.items():
try:
acl_bindings = f.result()
print("Deleted acls matching filter: {}".format(res))
for acl_binding in acl_bindings:
print(" ", acl_binding)
except KafkaException as e:
print("Failed to delete {}: {}".format(res, e))
except Exception:
raise
def example_incremental_alter_configs(a, args):
""" Incrementally alter configs, keeping non-specified
configuration properties with their previous values.
Input Format : ResourceType1 ResourceName1 Key=Operation:Value;Key2=Operation2:Value2;Key3=DELETE
ResourceType2 ResourceName2 ...
Example: TOPIC T1 compression.type=SET:lz4;cleanup.policy=ADD:compact;
retention.ms=DELETE TOPIC T2 compression.type=SET:gzip ...
"""
resources = []
for restype, resname, configs in zip(args[0::3], args[1::3], args[2::3]):
incremental_configs = []
for name, operation_and_value in [conf.split('=') for conf in configs.split(';')]:
if operation_and_value == "DELETE":
operation, value = operation_and_value, None
else:
operation, value = operation_and_value.split(':')
operation = AlterConfigOpType[operation]
incremental_configs.append(ConfigEntry(name, value,
incremental_operation=operation))
resources.append(ConfigResource(restype, resname,
incremental_configs=incremental_configs))
fs = a.incremental_alter_configs(resources)
# Wait for operation to finish.
for res, f in fs.items():
try:
f.result() # empty, but raises exception on failure
print("{} configuration successfully altered".format(res))
except Exception:
raise
def example_alter_configs(a, args):
""" Alter configs atomically, replacing non-specified
configuration properties with their default values.
"""
resources = []
for restype, resname, configs in zip(args[0::3], args[1::3], args[2::3]):
resource = ConfigResource(restype, resname)
resources.append(resource)
for k, v in [conf.split('=') for conf in configs.split(',')]:
resource.set_config(k, v)
fs = a.alter_configs(resources)
# Wait for operation to finish.
for res, f in fs.items():
try:
f.result() # empty, but raises exception on failure
print("{} configuration successfully altered".format(res))
except Exception:
raise
def example_delta_alter_configs(a, args):
"""
The AlterConfigs Kafka API requires all configuration to be passed,
any left out configuration properties will revert to their default settings.
This example shows how to just modify the supplied configuration entries
by first reading the configuration from the broker, updating the supplied
configuration with the broker configuration (without overwriting), and
then writing it all back.
The async nature of futures is also show-cased, which makes this example
a bit more complex than it needs to be in the synchronous case.
"""
# Convert supplied config to resources.
# We can reuse the same resources both for describe_configs and
# alter_configs.
resources = []
for restype, resname, configs in zip(args[0::3], args[1::3], args[2::3]):
resource = ConfigResource(restype, resname)
resources.append(resource)
for k, v in [conf.split('=') for conf in configs.split(',')]:
resource.set_config(k, v)
# Set up a locked counter and an Event (for signaling) to track when the
# second level of futures are done. This is a bit of contrived example
# due to no other asynchronous mechanism being used, so we'll need
# to wait on something to signal completion.
class WaitZero(object):
def __init__(self, waitcnt):
self.cnt = waitcnt
self.lock = threading.Lock()
self.event = threading.Event()
def decr(self):
""" Decrement cnt by 1"""
with self.lock:
assert self.cnt > 0
self.cnt -= 1
self.event.set()
def wait(self):
""" Wait until cnt reaches 0 """
self.lock.acquire()
while self.cnt > 0:
self.lock.release()
self.event.wait()
self.event.clear()
self.lock.acquire()
self.lock.release()
def __len__(self):
with self.lock:
return self.cnt
wait_zero = WaitZero(len(resources))
# Read existing configuration from cluster
fs = a.describe_configs(resources)
def delta_alter_configs_done(fut, resource):
e = fut.exception()
if e is not None:
print("Config update for {} failed: {}".format(resource, e))
else:
print("Config for {} updated".format(resource))
wait_zero.decr()
def delta_alter_configs(resource, remote_config):
print("Updating {} supplied config entries {} with {} config entries read from cluster".format(
len(resource), resource, len(remote_config)))
# Only set configuration that is not default
for k, entry in [(k, v) for k, v in remote_config.items() if not v.is_default]:
resource.set_config(k, entry.value, overwrite=False)
fs = a.alter_configs([resource])
fs[resource].add_done_callback(lambda fut: delta_alter_configs_done(fut, resource))
# For each resource's future set up a completion callback
# that in turn calls alter_configs() on that single resource.
# This is ineffective since the resources can usually go in
# one single alter_configs() call, but we're also show-casing
# the futures here.
for res, f in fs.items():
f.add_done_callback(lambda fut, resource=res: delta_alter_configs(resource, fut.result()))
# Wait for done callbacks to be triggered and operations to complete.
print("Waiting for {} resource updates to finish".format(len(wait_zero)))
wait_zero.wait()
def example_list(a, args):
""" list topics, groups and cluster metadata """
if len(args) == 0:
what = "all"
else:
what = args[0]
md = a.list_topics(timeout=10)
print("Cluster {} metadata (response from broker {}):".format(md.cluster_id, md.orig_broker_name))
if what in ("all", "brokers"):
print(" {} brokers:".format(len(md.brokers)))
for b in iter(md.brokers.values()):
if b.id == md.controller_id:
print(" {} (controller)".format(b))
else:
print(" {}".format(b))
if what in ("all", "topics"):
print(" {} topics:".format(len(md.topics)))
for t in iter(md.topics.values()):
if t.error is not None:
errstr = ": {}".format(t.error)
else:
errstr = ""
print(" \"{}\" with {} partition(s){}".format(t, len(t.partitions), errstr))
for p in iter(t.partitions.values()):
if p.error is not None:
errstr = ": {}".format(p.error)
else:
errstr = ""
print("partition {} leader: {}, replicas: {},"
" isrs: {} errstr: {}".format(p.id, p.leader, p.replicas,
p.isrs, errstr))
if what in ("all", "groups"):
groups = a.list_groups(timeout=10)
print(" {} consumer groups".format(len(groups)))
for g in groups:
if g.error is not None:
errstr = ": {}".format(g.error)
else:
errstr = ""
print(" \"{}\" with {} member(s), protocol: {}, protocol_type: {}{}".format(
g, len(g.members), g.protocol, g.protocol_type, errstr))
for m in g.members:
print("id {} client_id: {} client_host: {}".format(m.id, m.client_id, m.client_host))
def parse_list_consumer_groups_args(args, states, types):
parser = argparse.ArgumentParser(prog='list_consumer_groups')
parser.add_argument('-states')
parser.add_argument('-types')
parsed_args = parser.parse_args(args)
def usage(message):
print(message)
parser.print_usage()
sys.exit(1)
if parsed_args.states:
for arg in parsed_args.states.split(","):
try:
states.add(ConsumerGroupState[arg])
except KeyError:
usage(f"Invalid state: {arg}")
if parsed_args.types:
for arg in parsed_args.types.split(","):
try:
types.add(ConsumerGroupType[arg])
except KeyError:
usage(f"Invalid type: {arg}")
def example_list_consumer_groups(a, args):
"""
List Consumer Groups
"""
states = set()
types = set()
parse_list_consumer_groups_args(args, states, types)
future = a.list_consumer_groups(request_timeout=10, states=states, types=types)
try:
list_consumer_groups_result = future.result()
print("{} consumer groups".format(len(list_consumer_groups_result.valid)))
for valid in list_consumer_groups_result.valid:
print(" id: {} is_simple: {} state: {} type: {}".format(
valid.group_id, valid.is_simple_consumer_group, valid.state, valid.type))
print("{} errors".format(len(list_consumer_groups_result.errors)))
for error in list_consumer_groups_result.errors:
print(" error: {}".format(error))
except Exception:
raise
def example_describe_consumer_groups(a, args):
"""
Describe Consumer Groups
"""
include_auth_ops = bool(int(args[0]))
args = args[1:]
futureMap = a.describe_consumer_groups(args, include_authorized_operations=include_auth_ops, request_timeout=10)
for group_id, future in futureMap.items():
try:
g = future.result()
print("Group Id: {}".format(g.group_id))
print(" Is Simple : {}".format(g.is_simple_consumer_group))
print(" State : {}".format(g.state))
print(" Type : {}".format(g.type))
print(" Partition Assignor : {}".format(g.partition_assignor))
print(
f" Coordinator : {g.coordinator}")
print(" Members: ")
for member in g.members:
print(" Id : {}".format(member.member_id))
print(" Host : {}".format(member.host))
print(" Client Id : {}".format(member.client_id))
print(" Group Instance Id : {}".format(member.group_instance_id))
if member.assignment:
print(" Assignments :")
for toppar in member.assignment.topic_partitions:
print(" {} [{}]".format(toppar.topic, toppar.partition))
if member.target_assignment:
print(" Target Assignments:")
for toppar in member.target_assignment.topic_partitions:
print(f" {toppar.topic} [{toppar.partition}]")
if (include_auth_ops):
print(" Authorized operations: ")
op_string = ""
for acl_op in g.authorized_operations:
op_string += acl_op.name + " "
print(" {}".format(op_string))
except KafkaException as e:
print("Error while describing group id '{}': {}".format(group_id, e))
except Exception:
raise
def example_describe_topics(a, args):
"""
Describe Topics
"""
include_auth_ops = bool(int(args[0]))
args = args[1:]
topics = TopicCollection(topic_names=args)
futureMap = a.describe_topics(topics, request_timeout=10, include_authorized_operations=include_auth_ops)
for topic_name, future in futureMap.items():
try:
t = future.result()
print("Topic name : {}".format(t.name))
print("Topic id : {}".format(t.topic_id))
if (t.is_internal):
print("Topic is Internal")
if (include_auth_ops):
print("Authorized operations : ")
op_string = ""
for acl_op in t.authorized_operations:
op_string += acl_op.name + " "
print(" {}".format(op_string))
print("Partition Information")
for partition in t.partitions:
print(" Id : {}".format(partition.id))
leader = partition.leader
print(f" Leader : {leader}")
print(" Replicas : {}".format(len(partition.replicas)))
for replica in partition.replicas:
print(f" Replica : {replica}")
print(" In-Sync Replicas : {}".format(len(partition.isr)))
for isr in partition.isr:
print(f" In-Sync Replica : {isr}")
print("")
print("")
except KafkaException as e:
print("Error while describing topic '{}': {}".format(topic_name, e))
except Exception:
raise
def example_describe_cluster(a, args):
"""
Describe Cluster
"""
include_auth_ops = bool(int(args[0]))
args = args[1:]
future = a.describe_cluster(request_timeout=10, include_authorized_operations=include_auth_ops)
try:
c = future.result()
print("Cluster_id : {}".format(c.cluster_id))
if (c.controller):
print(f"Controller: {c.controller}")
else:
print("No Controller Information Available")
print("Nodes :")
for node in c.nodes:
print(f" Node: {node}")
if (include_auth_ops):
print("Authorized operations: ")
op_string = ""
for acl_op in c.authorized_operations:
op_string += acl_op.name + " "
print(" {}".format(op_string))
except KafkaException as e:
print("Error while describing cluster: {}".format(e))
except Exception:
raise
def example_delete_consumer_groups(a, args):
"""
Delete Consumer Groups
"""
groups = a.delete_consumer_groups(args, request_timeout=10)
for group_id, future in groups.items():
try:
future.result() # The result itself is None
print("Deleted group with id '" + group_id + "' successfully")
except KafkaException as e:
print("Error deleting group id '{}': {}".format(group_id, e))
except Exception:
raise
def example_list_consumer_group_offsets(a, args):
"""
List consumer group offsets
"""
topic_partitions = []
for topic, partition in zip(args[1::2], args[2::2]):
topic_partitions.append(TopicPartition(topic, int(partition)))
if len(topic_partitions) == 0:
topic_partitions = None
groups = [ConsumerGroupTopicPartitions(args[0], topic_partitions)]
futureMap = a.list_consumer_group_offsets(groups)
for group_id, future in futureMap.items():
try:
response_offset_info = future.result()
print("Group: " + response_offset_info.group_id)
for topic_partition in response_offset_info.topic_partitions:
if topic_partition.error:
print(" Error: " + topic_partition.error.str() + " occurred with " +
topic_partition.topic + " [" + str(topic_partition.partition) + "]")
else:
print(" " + topic_partition.topic +
" [" + str(topic_partition.partition) + "]: " + str(topic_partition.offset))
except KafkaException as e:
print("Failed to list {}: {}".format(group_id, e))
except Exception:
raise
def example_alter_consumer_group_offsets(a, args):
"""
Alter consumer group offsets
"""
topic_partitions = []
for topic, partition, offset in zip(args[1::3], args[2::3], args[3::3]):
topic_partitions.append(TopicPartition(topic, int(partition), int(offset)))
if len(topic_partitions) == 0:
topic_partitions = None
groups = [ConsumerGroupTopicPartitions(args[0], topic_partitions)]
futureMap = a.alter_consumer_group_offsets(groups)
for group_id, future in futureMap.items():
try:
response_offset_info = future.result()
print("Group: " + response_offset_info.group_id)
for topic_partition in response_offset_info.topic_partitions:
if topic_partition.error:
print(" Error: " + topic_partition.error.str() + " occurred with " +
topic_partition.topic + " [" + str(topic_partition.partition) + "]")
else:
print(" " + topic_partition.topic +
" [" + str(topic_partition.partition) + "]: " + str(topic_partition.offset))
except KafkaException as e:
print("Failed to alter {}: {}".format(group_id, e))
except Exception:
raise
def example_describe_user_scram_credentials(a, args):
"""
Describe User Scram Credentials
"""
if len(args) == 0:
"""
Case: Describes all user scram credentials
Input: no argument passed or None
Gets a future which result will give a
dict[str, UserScramCredentialsDescription]
or will throw a KafkaException
"""
f = a.describe_user_scram_credentials()
try:
results = f.result()
for username, response in results.items():
print("Username : {}".format(username))
for scram_credential_info in response.scram_credential_infos:
print(f" Mechanism: {scram_credential_info.mechanism} " +
f"Iterations: {scram_credential_info.iterations}")
except KafkaException as e:
print("Failed to describe all user scram credentials : {}".format(e))
except Exception:
raise
else:
"""
Case: Describe specified user scram credentials
Input: users is a list
Gets a dict[str, future] where the result() of
each future will give a UserScramCredentialsDescription
or a KafkaException
"""
futmap = a.describe_user_scram_credentials(args)
for username, fut in futmap.items():
print("Username: {}".format(username))
try:
response = fut.result()
for scram_credential_info in response.scram_credential_infos:
print(f" Mechanism: {scram_credential_info.mechanism} " +
f"Iterations: {scram_credential_info.iterations}")
except KafkaException as e:
print(" Error: {}".format(e))
except Exception:
raise
def example_alter_user_scram_credentials(a, args):
"""
AlterUserScramCredentials
"""
alterations_args = []
alterations = []
i = 0
op_cnt = 0
while i < len(args):
op = args[i]
if op == "UPSERT":
if i + 5 >= len(args):
raise ValueError(
f"Invalid number of arguments for alteration {op_cnt}, expected 5, got {len(args) - i - 1}")
user = args[i + 1]
mechanism = ScramMechanism[args[i + 2]]
iterations = int(args[i + 3])
password = bytes(args[i + 4], 'utf8')
# if salt is an empty string,
# set it to None to generate it randomly.
salt = args[i + 5]
if not salt:
salt = None
else:
salt = bytes(salt, 'utf8')
alterations_args.append([op, user, mechanism, iterations,
iterations, password, salt])
i += 6
elif op == "DELETE":
if i + 2 >= len(args):
raise ValueError(
f"Invalid number of arguments for alteration {op_cnt}, expected 2, got {len(args) - i - 1}")
user = args[i + 1]
mechanism = ScramMechanism[args[i + 2]]
alterations_args.append([op, user, mechanism])
i += 3
else:
raise ValueError(f"Invalid alteration {op}, must be UPSERT or DELETE")
op_cnt += 1
for alteration_arg in alterations_args:
op = alteration_arg[0]
if op == "UPSERT":
[_, user, mechanism, iterations,
iterations, password, salt] = alteration_arg
scram_credential_info = ScramCredentialInfo(mechanism, iterations)
upsertion = UserScramCredentialUpsertion(user, scram_credential_info,
password, salt)
alterations.append(upsertion)
elif op == "DELETE":
[_, user, mechanism] = alteration_arg
deletion = UserScramCredentialDeletion(user, mechanism)
alterations.append(deletion)
futmap = a.alter_user_scram_credentials(alterations)
for username, fut in futmap.items():
try:
fut.result()
print("{}: Success".format(username))
except KafkaException as e:
print("{}: Error: {}".format(username, e))
def example_list_offsets(a, args):
topic_partition_offsets = {}
if len(args) == 0:
raise ValueError(
"Invalid number of arguments for list offsets, expected at least 1, got 0")
i = 1
partition_i = 1
isolation_level = IsolationLevel[args[0]]
while i < len(args):
if i + 3 > len(args):
raise ValueError(
f"Invalid number of arguments for list offsets, partition {partition_i}, expected 3," +
f" got {len(args) - i}")
topic = args[i]
partition = int(args[i + 1])
topic_partition = TopicPartition(topic, partition)
if "EARLIEST" == args[i + 2]:
offset_spec = OffsetSpec.earliest()
elif "LATEST" == args[i + 2]:
offset_spec = OffsetSpec.latest()
elif "MAX_TIMESTAMP" == args[i + 2]:
offset_spec = OffsetSpec.max_timestamp()
elif "TIMESTAMP" == args[i + 2]:
if i + 4 > len(args):
raise ValueError(
f"Invalid number of arguments for list offsets, partition {partition_i}, expected 4" +
f", got {len(args) - i}")
offset_spec = OffsetSpec.for_timestamp(int(args[i + 3]))
i += 1
else:
raise ValueError("Invalid OffsetSpec, must be EARLIEST, LATEST, MAX_TIMESTAMP or TIMESTAMP")
topic_partition_offsets[topic_partition] = offset_spec
i = i + 3
partition_i += 1
futmap = a.list_offsets(topic_partition_offsets, isolation_level=isolation_level, request_timeout=30)
for partition, fut in futmap.items():
try:
result = fut.result()
print("Topicname : {} Partition_Index : {} Offset : {} Timestamp : {}"
.format(partition.topic, partition.partition, result.offset,
result.timestamp))
except KafkaException as e:
print("Topicname : {} Partition_Index : {} Error : {}"
.format(partition.topic, partition.partition, e))
def example_delete_records(a, args):
if len(args) == 0:
raise ValueError(
"Invalid number of arguments for delete_records, expected at least 3 " +
"(Usage: delete_records <topic1> <partition1> <offset1> [<topic2> <partition2> <offset2> ..])")
if len(args) % 3 != 0:
raise ValueError(
"Invalid number of arguments for delete_records " +
"(Usage: delete_records <topic1> <partition1> <offset1> [<topic2> <partition2> <offset2> ..])")
topic_partition_offsets = [
TopicPartition(topic, int(partition), int(offset))
for topic, partition, offset in zip(args[::3], args[1::3], args[2::3])
]
futmap = a.delete_records(topic_partition_offsets)
for partition, fut in futmap.items():
try:
result = fut.result()
if partition.offset == -1:
print(f"All records deleted in topic {partition.topic} partition {partition.partition}." +
f"The minimum offset in this partition is now {result.low_watermark}")
else:
print(
f"All records deleted before offset {partition.offset} in topic {partition.topic}" +
f" partition {partition.partition}. The minimum offset in this partition" +
f" is now {result.low_watermark}")
except KafkaException as e:
print(
f"Error deleting records in topic {partition.topic} partition {partition.partition}" +
f" before offset {partition.offset}: {e}")
def example_elect_leaders(a, args):
partitions = []
if (len(args) - 1) % 2 != 0:
raise ValueError("Invalid number of arguments for elect_leaders, Expected format: " +
"elect_leaders <election_type> [<topic1> <partition1>" +
" <topic2> <partition2> ..]")
try:
election_type = ElectionType[args[0]]
except KeyError:
raise ValueError(f"Invalid election_type: {args[0]}, expected 'PREFERRED' or 'UNCLEAN'")
for topic, partition in zip(args[1::2], args[2::2]):
partitions.append(TopicPartition(topic, int(partition)))
if len(partitions) == 0:
# When passing None as partitions, election is triggered for
# all partitions in the cluster
partitions = None
f = a.elect_leaders(election_type, partitions)
try:
results = f.result()
print(f"Elect leaders call returned {len(results)} result(s):")
for partition, error in results.items():
if error is None:
print(f"Leader Election Successful for topic: '{partition.topic}'" +
f" partition: '{partition.partition}'")
else:
print(
"Leader Election Failed for topic: " +
f"'{partition.topic}' partition: '{partition.partition}' " +
f"error code: {error.code()} error message: {error.str()}")
except KafkaException as e:
print(f"Error electing leaders: {e}")
if __name__ == '__main__':
if len(sys.argv) < 3:
sys.stderr.write('Usage: %s <bootstrap-brokers> <operation> <args..>\n\n' % sys.argv[0])
sys.stderr.write('operations:\n')
sys.stderr.write(' create_topics <topic1> <topic2> ..\n')
sys.stderr.write(' delete_topics <topic1> <topic2> ..\n')
sys.stderr.write(' create_partitions <topic1> <new_total_count1> <topic2> <new_total_count2> ..\n')
sys.stderr.write(' describe_configs <resource_type1> <resource_name1> <resource2> <resource_name2> ..\n')
sys.stderr.write(' alter_configs <resource_type1> <resource_name1> ' +
'<config=val,config2=val2> <resource_type2> <resource_name2> <config..> ..\n')
sys.stderr.write(' incremental_alter_configs <resource_type1> <resource_name1> ' +
'<config1=op1:val1;config2=op2:val2;config3=DELETE> ' +
'<resource_type2> <resource_name2> <config1=op1:..> ..\n')
sys.stderr.write(' delta_alter_configs <resource_type1> <resource_name1> ' +
'<config=val,config2=val2> <resource_type2> <resource_name2> <config..> ..\n')
sys.stderr.write(' create_acls <resource_type1> <resource_name1> <resource_patter_type1> ' +
'<principal1> <host1> <operation1> <permission_type1> ..\n')
sys.stderr.write(' describe_acls <resource_type1 <resource_name1> <resource_patter_type1> ' +
'<principal1> <host1> <operation1> <permission_type1> ..\n')
sys.stderr.write(' delete_acls <resource_type1> <resource_name1> <resource_patter_type1> ' +
'<principal1> <host1> <operation1> <permission_type1> ..\n')
sys.stderr.write(' list [<all|topics|brokers|groups>]\n')
sys.stderr.write(' list_consumer_groups [-states <state1>,<state2>,..] ' +
'[-types <type1>,<type2>,..]\n')
sys.stderr.write(' describe_consumer_groups <include_authorized_operations> <group1> <group2> ..\n')
sys.stderr.write(' describe_topics <include_authorized_operations> <topic1> <topic2> ..\n')
sys.stderr.write(' describe_cluster <include_authorized_operations>\n')
sys.stderr.write(' delete_consumer_groups <group1> <group2> ..\n')
sys.stderr.write(' list_consumer_group_offsets <group> [<topic1> <partition1> <topic2> <partition2> ..]\n')
sys.stderr.write(
' alter_consumer_group_offsets <group> <topic1> <partition1> <offset1> ' +
'<topic2> <partition2> <offset2> ..\n')
sys.stderr.write(' describe_user_scram_credentials [<user1> <user2> ..]\n')
sys.stderr.write(' alter_user_scram_credentials UPSERT <user1> <mechanism1> ' +
'<iterations1> <password1> <salt1> ' +
'[UPSERT <user2> <mechanism2> <iterations2> ' +
' <password2> <salt2> DELETE <user3> <mechanism3> ..]\n')
sys.stderr.write(' list_offsets <isolation_level> <topic1> <partition1> <offset_spec1> ' +
'[<topic2> <partition2> <offset_spec2> ..]\n')
sys.stderr.write(' delete_records <topic1> <partition1> <offset1> [<topic2> <partition2> <offset2> ..]\n')
sys.stderr.write(' elect_leaders <election_type> [<topic1> <partition1> <topic2> <partition2> ..]\n')
sys.exit(1)
broker = sys.argv[1]
operation = sys.argv[2]
args = sys.argv[3:]
# Create Admin client
a = AdminClient({'bootstrap.servers': broker})
opsmap = {'create_topics': example_create_topics,
'delete_topics': example_delete_topics,
'create_partitions': example_create_partitions,
'describe_configs': example_describe_configs,
'alter_configs': example_alter_configs,
'incremental_alter_configs': example_incremental_alter_configs,
'delta_alter_configs': example_delta_alter_configs,
'create_acls': example_create_acls,
'describe_acls': example_describe_acls,
'delete_acls': example_delete_acls,
'list': example_list,
'list_consumer_groups': example_list_consumer_groups,
'describe_consumer_groups': example_describe_consumer_groups,
'describe_topics': example_describe_topics,
'describe_cluster': example_describe_cluster,
'delete_consumer_groups': example_delete_consumer_groups,
'list_consumer_group_offsets': example_list_consumer_group_offsets,
'alter_consumer_group_offsets': example_alter_consumer_group_offsets,
'describe_user_scram_credentials': example_describe_user_scram_credentials,
'alter_user_scram_credentials': example_alter_user_scram_credentials,
'list_offsets': example_list_offsets,
'delete_records': example_delete_records,
'elect_leaders': example_elect_leaders}
if operation not in opsmap:
sys.stderr.write('Unknown operation: %s\n' % operation)
sys.exit(1)
opsmap[operation](a, args)
|