ktl 1.0.0-java

Sign up to get free protection for your applications and to get access to all the features.
@@ -0,0 +1,145 @@
1
+ # encoding: utf-8
2
+
3
+ module Ktl
4
+ class ShufflePlan
5
+ def initialize(zk_client, options = {})
6
+ @zk_client = zk_client
7
+ @options = options
8
+ @logger = options[:logger] || NullLogger.new
9
+ @log_plan = !!options[:log_plan]
10
+ end
11
+
12
+ def generate
13
+ topics = @zk_client.all_topics
14
+ if (filter = @options[:filter])
15
+ topics = topics.filter { |t| !!t.match(filter) }
16
+ end
17
+ topics_partitions = ScalaEnumerable.new(@zk_client.partitions_for_topics(topics))
18
+ topics_partitions = topics_partitions.sort_by(&:first)
19
+ replica_assignments = @zk_client.replica_assignment_for_topics(topics)
20
+ brokers = select_brokers
21
+ reassignment_plan = Scala::Collection::Map.empty
22
+ topics_partitions.each do |tp|
23
+ topic, partitions = tp.elements
24
+ nr_replicas = @options[:replication_factor] || replica_assignments.apply(Kafka::TopicAndPartition.new(topic, 0)).size
25
+ assignment = assign_replicas_to_brokers(topic, brokers, partitions.size, nr_replicas)
26
+ assignment.each do |pr|
27
+ partition, replicas = pr.elements
28
+ topic_partition = Kafka::TopicAndPartition.new(topic, partition)
29
+ current_assignment = replica_assignments.apply(topic_partition)
30
+ unless current_assignment == replicas
31
+ @logger.info "Moving #{topic_partition.topic},#{topic_partition.partition} from #{current_assignment} to #{replicas}" if @log_plan
32
+ reassignment_plan += Scala::Tuple.new(topic_partition, replicas)
33
+ end
34
+ end
35
+ end
36
+ reassignment_plan
37
+ end
38
+
39
+ def generate_for_new_topic(topic, partition_count)
40
+ brokers = select_brokers
41
+ nr_replicas = @options[:replication_factor] || 1
42
+ assignment = assign_replicas_to_brokers(topic, brokers, partition_count, nr_replicas)
43
+ assignment.map do |pr|
44
+ partition, replicas = pr.elements
45
+ Scala::Collection::JavaConversions.as_java_iterable(replicas).to_a
46
+ end
47
+ end
48
+
49
+ private
50
+
51
+ def select_brokers
52
+ brokers = @options[:brokers] ? Array(@options[:brokers]).map(&:to_i) : ScalaEnumerable.new(@zk_client.broker_ids).to_a
53
+ brokers -= Array(@options[:blacklist]).map(&:to_i) if @options[:blacklist]
54
+ brokers
55
+ end
56
+
57
+ def assign_replicas_to_brokers(topic, brokers, partition_count, replica_count)
58
+ broker_metadatas = brokers.map { |x| Kafka::Admin::BrokerMetadata.new(x.to_java(:int), Scala::Option[nil]) }
59
+ broker_metadatas = Scala::Collection::JavaConversions.as_scala_iterable(broker_metadatas).to_seq
60
+ Kafka::Admin.assign_replicas_to_brokers(broker_metadatas, partition_count, replica_count)
61
+ rescue Kafka::Admin::AdminOperationException => e
62
+ raise ArgumentError, sprintf('%s (%s)', e.message, e.class.name), e.backtrace
63
+ end
64
+ end
65
+
66
+ class RendezvousShufflePlan < ShufflePlan
67
+ def assign_replicas_to_brokers(topic, brokers, partition_count, replica_count)
68
+ if replica_count > brokers.size
69
+ raise ArgumentError, sprintf('replication factor: %i larger than available brokers: %i', replica_count, brokers.size)
70
+ end
71
+ result = []
72
+ partition_count.times do |partition|
73
+ sorted = brokers.sort_by do |broker|
74
+ key = [partition, topic, broker].pack('l<a*l<')
75
+ Java::OrgJrubyUtil::MurmurHash.hash32(key.to_java_bytes, 0, key.bytesize, SEED)
76
+ end
77
+ selected = sorted.take(replica_count)
78
+ result.push(Scala::Tuple.new(partition, Scala::Collection::JavaConversions.as_scala_iterable(selected).to_list))
79
+ end
80
+ result
81
+ end
82
+
83
+ private
84
+
85
+ SEED = 1683520333
86
+ end
87
+
88
+ class RackAwareShufflePlan < RendezvousShufflePlan
89
+ def initialize(*args)
90
+ super
91
+ @rack_mappings = {}
92
+ end
93
+
94
+ def assign_replicas_to_brokers(topic, brokers, partition_count, replica_count)
95
+ if replica_count > brokers.size
96
+ raise ArgumentError, sprintf('replication factor: %i larger than available brokers: %i', replica_count, brokers.size)
97
+ end
98
+ result = []
99
+ racks = brokers.each_with_object({}) do |broker, acc|
100
+ rack = rack_for(broker)
101
+ acc[rack] ||= []
102
+ acc[rack] << broker
103
+ end
104
+ partition_count.times do |partition|
105
+ first_sorted = racks.flat_map do |rack, rack_brokers|
106
+ hashed_brokers = rack_brokers.map do |broker|
107
+ key = [partition, topic, broker].pack('l<a*l<')
108
+ {id: broker, hash: Java::OrgJrubyUtil::MurmurHash.hash32(key.to_java_bytes, 0, key.bytesize, SEED)}
109
+ end.sort_by do |broker|
110
+ broker[:hash]
111
+ end
112
+ hashed_brokers.each_with_index do |broker, index|
113
+ broker[:index] = index
114
+ end
115
+ end
116
+ sorted = first_sorted.sort_by do |broker|
117
+ [broker[:index], broker[:hash], broker[:id]]
118
+ end
119
+ selected = sorted.take(replica_count).map {|broker| broker[:id]}
120
+ result.push(Scala::Tuple.new(partition, Scala::Collection::JavaConversions.as_scala_iterable(selected).to_list))
121
+ end
122
+ result
123
+ end
124
+
125
+ private
126
+
127
+ def rack_for(broker_id)
128
+ unless @rack_mappings[broker_id]
129
+ broker_metadata = Kafka::Admin.get_broker_metadatas(@zk_client, [broker_id]).first
130
+ rack = broker_metadata.rack
131
+ unless rack.isDefined
132
+ raise "Broker #{broker_metadata.id} is missing rack information, unable to create rack aware shuffle plan."
133
+ end
134
+ @rack_mappings[broker_id] = rack.get
135
+ end
136
+ @rack_mappings[broker_id]
137
+ rescue Java::KafkaAdmin::AdminOperationException => e
138
+ if e.message.match '--disable-rack-aware'
139
+ raise "Not all brokers have rack information. Unable to create rack aware shuffle plan."
140
+ else
141
+ raise e
142
+ end
143
+ end
144
+ end
145
+ end
data/lib/ktl/topic.rb ADDED
@@ -0,0 +1,123 @@
1
+ # encoding: utf-8
2
+
3
+ module Ktl
4
+ class Topic < Command
5
+ desc 'list', 'List current topics'
6
+ option :zookeeper, aliases: %w[-z], required: true, desc: 'ZooKeeper URI'
7
+ def list
8
+ with_zk_client do |zk_client|
9
+ topic_options = Kafka::Admin.to_topic_options(options.merge(list: nil))
10
+ Kafka::Admin::TopicCommand.list_topics(zk_client.raw_client, topic_options)
11
+ end
12
+ end
13
+
14
+ desc 'describe', 'Describe (optionally filtered) topics'
15
+ option :unavailable, aliases: '-u', desc: 'Describe unavailable partitions for topic(s)'
16
+ option :with_overrides, aliases: '-w', desc: 'Describe topics with config. overrides'
17
+ option :under_replicated, aliases: '-r', desc: 'Describe under-replicated partitions for topic(s)'
18
+ option :zookeeper, aliases: %w[-z], required: true, desc: 'ZooKeeper URI'
19
+ def describe(regexp=nil)
20
+ with_zk_client do |zk_client|
21
+ opts = {describe: nil}
22
+ opts[:topic] = regexp if regexp
23
+ opts[:topics_with_overrides] = nil if options.with_overrides?
24
+ opts[:unavailable_partitions] = nil if options.unavailable?
25
+ opts[:under_replicated_partitions] = nil if options.under_replicated?
26
+ topic_options = Kafka::Admin.to_topic_options(opts)
27
+ Kafka::Admin::TopicCommand.describe_topic(zk_client.raw_client, topic_options)
28
+ end
29
+ end
30
+
31
+ desc 'create NAMES..', 'Create one or more new topics'
32
+ option :partitions, aliases: %w[-p], default: '1', desc: 'Partitions for new topic(s)'
33
+ option :replication_factor, aliases: %w[-r], default: '1', desc: 'Replication factor for new topic(s)'
34
+ option :replica_assignment, aliases: %w[-a], desc: 'Manual replica assignment'
35
+ option :disable_rack_aware, desc: 'Disable rack awareness'
36
+ option :rack_aware_allocation, desc: 'Create partitions with Rack aware + Rendezvous-hashing based shuffle'
37
+ option :rendezvous_allocation, desc: 'Create partitions with Rendezvous-hashing based shuffle'
38
+ option :config, aliases: %w[-c], desc: 'Key-value pairs of configuration options', type: :hash, default: {}
39
+ option :zookeeper, aliases: %w[-z], required: true, desc: 'ZooKeeper URI'
40
+ def create(*names)
41
+ with_zk_client do |zk_client|
42
+ names.each do |name|
43
+ opts = options.merge(create: nil, topic: name)
44
+ if options.rack_aware_allocation || options.rendezvous_allocation
45
+ plan_factory = if options.rack_aware_allocation
46
+ RackAwareShufflePlan
47
+ else
48
+ RendezvousShufflePlan
49
+ end
50
+
51
+ plan = plan_factory.new(zk_client, replication_factor: options.replication_factor.to_i)
52
+ zk_utils = Kafka::Utils::ZkUtils.new(nil, nil, false)
53
+ opts.delete(:rack_aware_allocation)
54
+ opts.delete(:rendezvous_allocation)
55
+ plan = plan.generate_for_new_topic(name, options.partitions.to_i)
56
+ opts[:replica_assignment] = plan.map {|broker_list| broker_list.join(':')}.join(',')
57
+ end
58
+ topic_options = Kafka::Admin.to_topic_options(opts)
59
+ silence_scala do
60
+ Kafka::Admin::TopicCommand.create_topic(zk_client.raw_client, topic_options)
61
+ end
62
+ message = %(created topic "#{name}" with #{opts[:partitions]} partition(s))
63
+ message << %(, and replication factor #{opts[:replication_factor]})
64
+ message << %(, with replica assignment: #{opts[:replica_assignment]}) if opts[:replica_assignment]
65
+ message << %(, with config: #{opts[:config]}) unless opts[:config].empty?
66
+ logger.info(message)
67
+ end
68
+ end
69
+ end
70
+
71
+ desc 'add-partitions NAMES..', 'Add partitions to one or more existing topics'
72
+ option :partitions, aliases: %w[-p], required: true, desc: 'New number of partitions'
73
+ option :zookeeper, aliases: %w[-z], required: true, desc: 'ZooKeeper URI'
74
+ def add_partitions(*names)
75
+ with_zk_client do |zk_client|
76
+ names.each do |name|
77
+ opts = options.merge(alter: nil, topic: name)
78
+ topic_options = Kafka::Admin.to_topic_options(opts)
79
+ logger.warn %(if "#{name}" uses keyed messages, the partition logic or ordering of the messages will be affected)
80
+ silence_scala do
81
+ Kafka::Admin::TopicCommand.alter_topic(zk_client.raw_client, topic_options)
82
+ end
83
+ logger.info %(increased partitions to #{options.partitions} for "#{name}")
84
+ end
85
+ end
86
+ end
87
+
88
+ desc 'delete REGEXP', 'Delete topics matching given regexp'
89
+ option :zookeeper, aliases: %w[-z], required: true, desc: 'ZooKeeper URI'
90
+ def delete(regexp)
91
+ with_zk_client do |zk_client|
92
+ topics = zk_client.all_topics
93
+ topics = topics.filter { |t| !!t.match(regexp) }
94
+ logger.info %(about to mark #{topics.size} topics for deletion)
95
+ topics.foreach do |topic|
96
+ Kafka::Utils.delete_topic(zk_client.raw_client, topic)
97
+ logger.debug %(successfully marked "#{topic}" for deletion)
98
+ end
99
+ end
100
+ end
101
+
102
+ desc 'alter REGEXP', 'Alter topic configuration'
103
+ option :add, aliases: %w[-a], desc: 'Key-value pairs of config options to add', type: :hash, default: {}
104
+ option :remove, aliases: %w[-r], desc: 'Key-value pairs of config options to remove', type: :array, default: []
105
+ option :zookeeper, aliases: %w[-z], required: true, desc: 'ZooKeeper URI'
106
+ def alter(regexp)
107
+ with_zk_client do |zk_client|
108
+ opts = {zookeeper: options.zookeeper, topic: regexp}
109
+ opts[:config] = options.add.dup unless options.add.empty?
110
+ opts[:delete_config] = options.remove.dup unless options.remove.empty?
111
+ if opts[:config] || opts[:delete_config]
112
+ topic_options = Kafka::Admin.to_topic_options(opts)
113
+ silence_scala do
114
+ Kafka::Admin::TopicCommand.alter_topic(zk_client.raw_client, topic_options)
115
+ end
116
+ logger.info %(updated configuration for topics matching "#{regexp}")
117
+ else
118
+ raise ArgumentError, 'missing --add or --remove option'
119
+ end
120
+ end
121
+ end
122
+ end
123
+ end
@@ -0,0 +1,5 @@
1
+ # encoding: utf-8
2
+
3
+ module Ktl
4
+ VERSION = '1.0.0'.freeze
5
+ end
@@ -0,0 +1,111 @@
1
+ # encoding: utf-8
2
+
3
+ module Ktl
4
+ class ZookeeperClient
5
+ attr_reader :utils
6
+
7
+ def initialize(uri, options={})
8
+ @uri = uri
9
+ @threadpool = options[:threadpool] || JavaConcurrent::Executors.new_fixed_thread_pool(CONCURRENCY)
10
+ @utils = options[:utils] || Kafka::Utils::ZkUtils.apply(@uri, 5000, 5000, false)
11
+ end
12
+
13
+ def setup
14
+ @submit = @threadpool.java_method(:submit, [java.lang.Class.for_name('java.util.concurrent.Callable')])
15
+ self
16
+ end
17
+
18
+ def close
19
+ @threadpool.shutdown_now if @threadpool
20
+ @utils.close
21
+ end
22
+
23
+ def raw_client
24
+ @utils
25
+ end
26
+
27
+ def all_partitions
28
+ @utils.get_all_partitions
29
+ end
30
+
31
+ def all_topics
32
+ @utils.get_all_topics
33
+ end
34
+
35
+ def brokers
36
+ @utils.get_all_brokers_in_cluster
37
+ end
38
+
39
+ def broker_ids
40
+ @utils.get_sorted_broker_list
41
+ end
42
+
43
+ def leader_and_isr_for(partitions)
44
+ @utils.get_partition_leader_and_isr_for_topics(@utils.class.create_zk_client(@uri, 5_000, 5_000), partitions)
45
+ end
46
+
47
+ def partitions_for_topics(topics)
48
+ request(:get_partitions_for_topics, topics)
49
+ end
50
+
51
+ def replica_assignment_for_topics(topics)
52
+ request(:get_replica_assignment_for_topics, topics)
53
+ end
54
+
55
+ def partitions_being_reassigned
56
+ @utils.get_partitions_being_reassigned
57
+ end
58
+
59
+ def reassign_partitions(json)
60
+ @utils.create_persistent_path(@utils.class.reassign_partitions_path, json, no_acl)
61
+ end
62
+
63
+ def create_znode(path, data='')
64
+ @utils.create_persistent_path(path, data, no_acl)
65
+ end
66
+
67
+ def delete_znode(path, options={})
68
+ if options[:recursive]
69
+ @utils.delete_path_recursive(path)
70
+ else
71
+ @utils.delete_path(path)
72
+ end
73
+ end
74
+
75
+ def read_data(path)
76
+ @utils.read_data(path)
77
+ end
78
+
79
+ def get_children(path)
80
+ @utils.get_children(path)
81
+ end
82
+
83
+ def exists?(path)
84
+ @utils.path_exists(path)
85
+ end
86
+
87
+ private
88
+
89
+ CONCURRENCY = 8
90
+
91
+ def no_acl
92
+ Kafka::Utils::ZkUtils::DefaultAcls(false)
93
+ end
94
+
95
+ def request(method, input)
96
+ chunk_size = [(input.size.to_f / CONCURRENCY).round, 1].max
97
+ groups = ScalaEnumerable.new(input.grouped(chunk_size).to_seq)
98
+ futures = groups.map do |slice|
99
+ @submit.call { @utils.send(method, slice) }
100
+ end
101
+ merge(futures.map(&:get))
102
+ end
103
+
104
+ def merge(results)
105
+ result = Scala::Collection::Map.empty
106
+ results.reduce(result) do |acc, v|
107
+ acc.send('++', v)
108
+ end
109
+ end
110
+ end
111
+ end
metadata ADDED
@@ -0,0 +1,101 @@
1
+ --- !ruby/object:Gem::Specification
2
+ name: ktl
3
+ version: !ruby/object:Gem::Version
4
+ version: 1.0.0
5
+ platform: java
6
+ authors:
7
+ - Burt Platform Team
8
+ autorequire:
9
+ bindir: bin
10
+ cert_chain: []
11
+ date: 2017-02-15 00:00:00.000000000 Z
12
+ dependencies:
13
+ - !ruby/object:Gem::Dependency
14
+ requirement: !ruby/object:Gem::Requirement
15
+ requirements:
16
+ - - '='
17
+ - !ruby/object:Gem::Version
18
+ version: 0.10.0.1
19
+ name: kafka-jars
20
+ prerelease: false
21
+ type: :runtime
22
+ version_requirements: !ruby/object:Gem::Requirement
23
+ requirements:
24
+ - - '='
25
+ - !ruby/object:Gem::Version
26
+ version: 0.10.0.1
27
+ - !ruby/object:Gem::Dependency
28
+ requirement: !ruby/object:Gem::Requirement
29
+ requirements:
30
+ - - ~>
31
+ - !ruby/object:Gem::Version
32
+ version: '0'
33
+ - - <
34
+ - !ruby/object:Gem::Version
35
+ version: '1.0'
36
+ name: thor
37
+ prerelease: false
38
+ type: :runtime
39
+ version_requirements: !ruby/object:Gem::Requirement
40
+ requirements:
41
+ - - ~>
42
+ - !ruby/object:Gem::Version
43
+ version: '0'
44
+ - - <
45
+ - !ruby/object:Gem::Version
46
+ version: '1.0'
47
+ description: |-
48
+ ktl is a tool that attempts to make it easier
49
+ to manage Kafka clusers that host a lot of topics
50
+ email:
51
+ - david@burtcorp.com
52
+ executables:
53
+ - ktl
54
+ extensions: []
55
+ extra_rdoc_files: []
56
+ files:
57
+ - LICENSE.txt
58
+ - README.md
59
+ - bin/ktl
60
+ - lib/ext/kafka.rb
61
+ - lib/ext/thor.rb
62
+ - lib/ktl.rb
63
+ - lib/ktl/cli.rb
64
+ - lib/ktl/cluster.rb
65
+ - lib/ktl/cluster_stats_task.rb
66
+ - lib/ktl/command.rb
67
+ - lib/ktl/decommission_plan.rb
68
+ - lib/ktl/migration_plan.rb
69
+ - lib/ktl/reassigner.rb
70
+ - lib/ktl/reassignment_progress.rb
71
+ - lib/ktl/reassignment_task.rb
72
+ - lib/ktl/shell_formatter.rb
73
+ - lib/ktl/shuffle_plan.rb
74
+ - lib/ktl/topic.rb
75
+ - lib/ktl/version.rb
76
+ - lib/ktl/zookeeper_client.rb
77
+ homepage: http://github.com/burtcorp/ktl
78
+ licenses:
79
+ - BSD-3-Clause
80
+ metadata: {}
81
+ post_install_message:
82
+ rdoc_options: []
83
+ require_paths:
84
+ - lib
85
+ required_ruby_version: !ruby/object:Gem::Requirement
86
+ requirements:
87
+ - - '>='
88
+ - !ruby/object:Gem::Version
89
+ version: '0'
90
+ required_rubygems_version: !ruby/object:Gem::Requirement
91
+ requirements:
92
+ - - '>='
93
+ - !ruby/object:Gem::Version
94
+ version: '0'
95
+ requirements: []
96
+ rubyforge_project:
97
+ rubygems_version: 2.4.5
98
+ signing_key:
99
+ specification_version: 4
100
+ summary: Management tool for Kafka clusters
101
+ test_files: []