ruby-kafka-aws-iam 1.4.1

Sign up to get free protection for your applications and to get access to all the features.
Files changed (145) hide show
  1. checksums.yaml +7 -0
  2. data/.circleci/config.yml +393 -0
  3. data/.github/workflows/stale.yml +19 -0
  4. data/.gitignore +13 -0
  5. data/.readygo +1 -0
  6. data/.rspec +3 -0
  7. data/.rubocop.yml +44 -0
  8. data/.ruby-version +1 -0
  9. data/.yardopts +3 -0
  10. data/CHANGELOG.md +314 -0
  11. data/Gemfile +5 -0
  12. data/ISSUE_TEMPLATE.md +23 -0
  13. data/LICENSE.txt +176 -0
  14. data/Procfile +2 -0
  15. data/README.md +1356 -0
  16. data/Rakefile +8 -0
  17. data/benchmarks/message_encoding.rb +23 -0
  18. data/bin/console +8 -0
  19. data/bin/setup +5 -0
  20. data/docker-compose.yml +39 -0
  21. data/examples/consumer-group.rb +35 -0
  22. data/examples/firehose-consumer.rb +64 -0
  23. data/examples/firehose-producer.rb +54 -0
  24. data/examples/simple-consumer.rb +34 -0
  25. data/examples/simple-producer.rb +42 -0
  26. data/examples/ssl-producer.rb +44 -0
  27. data/lib/kafka/async_producer.rb +297 -0
  28. data/lib/kafka/broker.rb +217 -0
  29. data/lib/kafka/broker_info.rb +16 -0
  30. data/lib/kafka/broker_pool.rb +41 -0
  31. data/lib/kafka/broker_uri.rb +43 -0
  32. data/lib/kafka/client.rb +838 -0
  33. data/lib/kafka/cluster.rb +513 -0
  34. data/lib/kafka/compression.rb +45 -0
  35. data/lib/kafka/compressor.rb +86 -0
  36. data/lib/kafka/connection.rb +228 -0
  37. data/lib/kafka/connection_builder.rb +33 -0
  38. data/lib/kafka/consumer.rb +642 -0
  39. data/lib/kafka/consumer_group/assignor.rb +63 -0
  40. data/lib/kafka/consumer_group.rb +231 -0
  41. data/lib/kafka/crc32_hash.rb +15 -0
  42. data/lib/kafka/datadog.rb +420 -0
  43. data/lib/kafka/digest.rb +22 -0
  44. data/lib/kafka/fetch_operation.rb +115 -0
  45. data/lib/kafka/fetched_batch.rb +58 -0
  46. data/lib/kafka/fetched_batch_generator.rb +120 -0
  47. data/lib/kafka/fetched_message.rb +48 -0
  48. data/lib/kafka/fetched_offset_resolver.rb +48 -0
  49. data/lib/kafka/fetcher.rb +224 -0
  50. data/lib/kafka/gzip_codec.rb +34 -0
  51. data/lib/kafka/heartbeat.rb +25 -0
  52. data/lib/kafka/instrumenter.rb +38 -0
  53. data/lib/kafka/interceptors.rb +33 -0
  54. data/lib/kafka/lz4_codec.rb +27 -0
  55. data/lib/kafka/message_buffer.rb +87 -0
  56. data/lib/kafka/murmur2_hash.rb +17 -0
  57. data/lib/kafka/offset_manager.rb +259 -0
  58. data/lib/kafka/partitioner.rb +40 -0
  59. data/lib/kafka/pause.rb +92 -0
  60. data/lib/kafka/pending_message.rb +29 -0
  61. data/lib/kafka/pending_message_queue.rb +41 -0
  62. data/lib/kafka/produce_operation.rb +205 -0
  63. data/lib/kafka/producer.rb +528 -0
  64. data/lib/kafka/prometheus.rb +316 -0
  65. data/lib/kafka/protocol/add_offsets_to_txn_request.rb +29 -0
  66. data/lib/kafka/protocol/add_offsets_to_txn_response.rb +21 -0
  67. data/lib/kafka/protocol/add_partitions_to_txn_request.rb +34 -0
  68. data/lib/kafka/protocol/add_partitions_to_txn_response.rb +47 -0
  69. data/lib/kafka/protocol/alter_configs_request.rb +44 -0
  70. data/lib/kafka/protocol/alter_configs_response.rb +49 -0
  71. data/lib/kafka/protocol/api_versions_request.rb +21 -0
  72. data/lib/kafka/protocol/api_versions_response.rb +53 -0
  73. data/lib/kafka/protocol/consumer_group_protocol.rb +19 -0
  74. data/lib/kafka/protocol/create_partitions_request.rb +42 -0
  75. data/lib/kafka/protocol/create_partitions_response.rb +28 -0
  76. data/lib/kafka/protocol/create_topics_request.rb +45 -0
  77. data/lib/kafka/protocol/create_topics_response.rb +26 -0
  78. data/lib/kafka/protocol/decoder.rb +175 -0
  79. data/lib/kafka/protocol/delete_topics_request.rb +33 -0
  80. data/lib/kafka/protocol/delete_topics_response.rb +26 -0
  81. data/lib/kafka/protocol/describe_configs_request.rb +35 -0
  82. data/lib/kafka/protocol/describe_configs_response.rb +73 -0
  83. data/lib/kafka/protocol/describe_groups_request.rb +27 -0
  84. data/lib/kafka/protocol/describe_groups_response.rb +73 -0
  85. data/lib/kafka/protocol/encoder.rb +184 -0
  86. data/lib/kafka/protocol/end_txn_request.rb +29 -0
  87. data/lib/kafka/protocol/end_txn_response.rb +19 -0
  88. data/lib/kafka/protocol/fetch_request.rb +70 -0
  89. data/lib/kafka/protocol/fetch_response.rb +136 -0
  90. data/lib/kafka/protocol/find_coordinator_request.rb +29 -0
  91. data/lib/kafka/protocol/find_coordinator_response.rb +29 -0
  92. data/lib/kafka/protocol/heartbeat_request.rb +27 -0
  93. data/lib/kafka/protocol/heartbeat_response.rb +17 -0
  94. data/lib/kafka/protocol/init_producer_id_request.rb +26 -0
  95. data/lib/kafka/protocol/init_producer_id_response.rb +27 -0
  96. data/lib/kafka/protocol/join_group_request.rb +47 -0
  97. data/lib/kafka/protocol/join_group_response.rb +41 -0
  98. data/lib/kafka/protocol/leave_group_request.rb +25 -0
  99. data/lib/kafka/protocol/leave_group_response.rb +17 -0
  100. data/lib/kafka/protocol/list_groups_request.rb +23 -0
  101. data/lib/kafka/protocol/list_groups_response.rb +35 -0
  102. data/lib/kafka/protocol/list_offset_request.rb +53 -0
  103. data/lib/kafka/protocol/list_offset_response.rb +89 -0
  104. data/lib/kafka/protocol/member_assignment.rb +42 -0
  105. data/lib/kafka/protocol/message.rb +172 -0
  106. data/lib/kafka/protocol/message_set.rb +55 -0
  107. data/lib/kafka/protocol/metadata_request.rb +31 -0
  108. data/lib/kafka/protocol/metadata_response.rb +185 -0
  109. data/lib/kafka/protocol/offset_commit_request.rb +47 -0
  110. data/lib/kafka/protocol/offset_commit_response.rb +29 -0
  111. data/lib/kafka/protocol/offset_fetch_request.rb +38 -0
  112. data/lib/kafka/protocol/offset_fetch_response.rb +56 -0
  113. data/lib/kafka/protocol/produce_request.rb +94 -0
  114. data/lib/kafka/protocol/produce_response.rb +63 -0
  115. data/lib/kafka/protocol/record.rb +88 -0
  116. data/lib/kafka/protocol/record_batch.rb +223 -0
  117. data/lib/kafka/protocol/request_message.rb +26 -0
  118. data/lib/kafka/protocol/sasl_handshake_request.rb +33 -0
  119. data/lib/kafka/protocol/sasl_handshake_response.rb +28 -0
  120. data/lib/kafka/protocol/sync_group_request.rb +33 -0
  121. data/lib/kafka/protocol/sync_group_response.rb +26 -0
  122. data/lib/kafka/protocol/txn_offset_commit_request.rb +46 -0
  123. data/lib/kafka/protocol/txn_offset_commit_response.rb +47 -0
  124. data/lib/kafka/protocol.rb +225 -0
  125. data/lib/kafka/round_robin_assignment_strategy.rb +52 -0
  126. data/lib/kafka/sasl/awsmskiam.rb +128 -0
  127. data/lib/kafka/sasl/gssapi.rb +76 -0
  128. data/lib/kafka/sasl/oauth.rb +64 -0
  129. data/lib/kafka/sasl/plain.rb +39 -0
  130. data/lib/kafka/sasl/scram.rb +180 -0
  131. data/lib/kafka/sasl_authenticator.rb +73 -0
  132. data/lib/kafka/snappy_codec.rb +29 -0
  133. data/lib/kafka/socket_with_timeout.rb +96 -0
  134. data/lib/kafka/ssl_context.rb +66 -0
  135. data/lib/kafka/ssl_socket_with_timeout.rb +192 -0
  136. data/lib/kafka/statsd.rb +296 -0
  137. data/lib/kafka/tagged_logger.rb +77 -0
  138. data/lib/kafka/transaction_manager.rb +306 -0
  139. data/lib/kafka/transaction_state_machine.rb +72 -0
  140. data/lib/kafka/version.rb +5 -0
  141. data/lib/kafka/zstd_codec.rb +27 -0
  142. data/lib/kafka.rb +373 -0
  143. data/lib/ruby-kafka.rb +5 -0
  144. data/ruby-kafka.gemspec +54 -0
  145. metadata +520 -0
@@ -0,0 +1,223 @@
1
+ require 'bigdecimal'
2
+ require 'digest/crc32'
3
+ require 'kafka/protocol/record'
4
+
5
+ module Kafka
6
+ module Protocol
7
+ class RecordBatch
8
+ MAGIC_BYTE = 2
9
+ # The size of metadata before the real record data
10
+ RECORD_BATCH_OVERHEAD = 49
11
+ # Masks to extract information from attributes
12
+ CODEC_ID_MASK = 0b00000111
13
+ IN_TRANSACTION_MASK = 0b00010000
14
+ IS_CONTROL_BATCH_MASK = 0b00100000
15
+ TIMESTAMP_TYPE_MASK = 0b001000
16
+
17
+ attr_reader :records, :first_offset, :first_timestamp, :partition_leader_epoch, :in_transaction, :is_control_batch, :last_offset_delta, :max_timestamp, :producer_id, :producer_epoch, :first_sequence
18
+
19
+ attr_accessor :codec_id
20
+
21
+ def initialize(
22
+ records: [],
23
+ first_offset: 0,
24
+ first_timestamp: Time.now,
25
+ partition_leader_epoch: 0,
26
+ codec_id: 0,
27
+ in_transaction: false,
28
+ is_control_batch: false,
29
+ last_offset_delta: 0,
30
+ producer_id: -1,
31
+ producer_epoch: 0,
32
+ first_sequence: 0,
33
+ max_timestamp: Time.now
34
+ )
35
+ @records = Array(records)
36
+ @first_offset = first_offset
37
+ @first_timestamp = first_timestamp
38
+ @codec_id = codec_id
39
+
40
+ # Records verification
41
+ @last_offset_delta = last_offset_delta
42
+ @max_timestamp = max_timestamp
43
+
44
+ # Transaction information
45
+ @producer_id = producer_id
46
+ @producer_epoch = producer_epoch
47
+
48
+ @first_sequence = first_sequence
49
+ @partition_leader_epoch = partition_leader_epoch
50
+ @in_transaction = in_transaction
51
+ @is_control_batch = is_control_batch
52
+
53
+ mark_control_record
54
+ end
55
+
56
+ def size
57
+ @records.size
58
+ end
59
+
60
+ def last_offset
61
+ @first_offset + @last_offset_delta
62
+ end
63
+
64
+ def attributes
65
+ 0x0000 | @codec_id |
66
+ (@in_transaction ? IN_TRANSACTION_MASK : 0x0) |
67
+ (@is_control_batch ? IS_CONTROL_BATCH_MASK : 0x0)
68
+ end
69
+
70
+ def encode(encoder)
71
+ encoder.write_int64(@first_offset)
72
+
73
+ record_batch_buffer = StringIO.new
74
+ record_batch_encoder = Encoder.new(record_batch_buffer)
75
+
76
+ record_batch_encoder.write_int32(@partition_leader_epoch)
77
+ record_batch_encoder.write_int8(MAGIC_BYTE)
78
+
79
+ body = encode_record_batch_body
80
+ crc = ::Digest::CRC32c.checksum(body)
81
+
82
+ record_batch_encoder.write_int32(crc)
83
+ record_batch_encoder.write(body)
84
+
85
+ encoder.write_bytes(record_batch_buffer.string)
86
+ end
87
+
88
+ def encode_record_batch_body
89
+ buffer = StringIO.new
90
+ encoder = Encoder.new(buffer)
91
+
92
+ encoder.write_int16(attributes)
93
+ encoder.write_int32(@last_offset_delta)
94
+ encoder.write_int64((@first_timestamp.to_f * 1000).to_i)
95
+ encoder.write_int64((@max_timestamp.to_f * 1000).to_i)
96
+
97
+ encoder.write_int64(@producer_id)
98
+ encoder.write_int16(@producer_epoch)
99
+ encoder.write_int32(@first_sequence)
100
+
101
+ encoder.write_int32(@records.length)
102
+
103
+ records_array = encode_record_array
104
+ if compressed?
105
+ codec = Compression.find_codec_by_id(@codec_id)
106
+ records_array = codec.compress(records_array)
107
+ end
108
+ encoder.write(records_array)
109
+
110
+ buffer.string
111
+ end
112
+
113
+ def encode_record_array
114
+ buffer = StringIO.new
115
+ encoder = Encoder.new(buffer)
116
+ @records.each do |record|
117
+ record.encode(encoder)
118
+ end
119
+ buffer.string
120
+ end
121
+
122
+ def compressed?
123
+ @codec_id != 0
124
+ end
125
+
126
+ def fulfill_relative_data
127
+ first_record = records.min_by { |record| record.create_time }
128
+ @first_timestamp = first_record.nil? ? Time.now : first_record.create_time
129
+
130
+ last_record = records.max_by { |record| record.create_time }
131
+ @max_timestamp = last_record.nil? ? Time.now : last_record.create_time
132
+
133
+ records.each_with_index do |record, index|
134
+ record.offset_delta = index
135
+ record.timestamp_delta = ((record.create_time - first_timestamp) * 1000).to_i
136
+ end
137
+ @last_offset_delta = records.length - 1
138
+ end
139
+
140
+ def ==(other)
141
+ records == other.records &&
142
+ first_offset == other.first_offset &&
143
+ partition_leader_epoch == other.partition_leader_epoch &&
144
+ in_transaction == other.in_transaction &&
145
+ is_control_batch == other.is_control_batch &&
146
+ last_offset_delta == other.last_offset_delta &&
147
+ producer_id == other.producer_id &&
148
+ producer_epoch == other.producer_epoch &&
149
+ first_sequence == other.first_sequence
150
+ end
151
+
152
+ def self.decode(decoder)
153
+ first_offset = decoder.int64
154
+
155
+ record_batch_raw = decoder.bytes
156
+ record_batch_decoder = Decoder.from_string(record_batch_raw)
157
+
158
+ partition_leader_epoch = record_batch_decoder.int32
159
+ # Currently, the magic byte is used to distingush legacy MessageSet and
160
+ # RecordBatch. Therefore, we don't care about magic byte here yet.
161
+ _magic_byte = record_batch_decoder.int8
162
+ _crc = record_batch_decoder.int32
163
+
164
+ attributes = record_batch_decoder.int16
165
+ codec_id = attributes & CODEC_ID_MASK
166
+ in_transaction = (attributes & IN_TRANSACTION_MASK) > 0
167
+ is_control_batch = (attributes & IS_CONTROL_BATCH_MASK) > 0
168
+ log_append_time = (attributes & TIMESTAMP_TYPE_MASK) != 0
169
+
170
+ last_offset_delta = record_batch_decoder.int32
171
+ first_timestamp = Time.at(record_batch_decoder.int64 / BigDecimal(1000))
172
+ max_timestamp = Time.at(record_batch_decoder.int64 / BigDecimal(1000))
173
+
174
+ producer_id = record_batch_decoder.int64
175
+ producer_epoch = record_batch_decoder.int16
176
+ first_sequence = record_batch_decoder.int32
177
+
178
+ records_array_length = record_batch_decoder.int32
179
+ records_array_raw = record_batch_decoder.read(
180
+ record_batch_raw.size - RECORD_BATCH_OVERHEAD
181
+ )
182
+ if codec_id != 0
183
+ codec = Compression.find_codec_by_id(codec_id)
184
+ records_array_raw = codec.decompress(records_array_raw)
185
+ end
186
+
187
+ records_array_decoder = Decoder.from_string(records_array_raw)
188
+ records_array = []
189
+ until records_array_decoder.eof?
190
+ record = Record.decode(records_array_decoder)
191
+ record.offset = first_offset + record.offset_delta
192
+ record.create_time = log_append_time && max_timestamp ? max_timestamp : first_timestamp + record.timestamp_delta / BigDecimal(1000)
193
+ records_array << record
194
+ end
195
+
196
+ raise InsufficientDataMessage if records_array.length != records_array_length
197
+
198
+ new(
199
+ records: records_array,
200
+ first_offset: first_offset,
201
+ first_timestamp: first_timestamp,
202
+ partition_leader_epoch: partition_leader_epoch,
203
+ in_transaction: in_transaction,
204
+ is_control_batch: is_control_batch,
205
+ last_offset_delta: last_offset_delta,
206
+ producer_id: producer_id,
207
+ producer_epoch: producer_epoch,
208
+ first_sequence: first_sequence,
209
+ max_timestamp: max_timestamp
210
+ )
211
+ rescue EOFError
212
+ raise InsufficientDataMessage, 'Partial trailing record detected!'
213
+ end
214
+
215
+ def mark_control_record
216
+ if is_control_batch
217
+ record = @records.first
218
+ record.is_control_record = true unless record.nil?
219
+ end
220
+ end
221
+ end
222
+ end
223
+ end
@@ -0,0 +1,26 @@
1
+ # frozen_string_literal: true
2
+
3
+ module Kafka
4
+ module Protocol
5
+ class RequestMessage
6
+ API_VERSION = 0
7
+
8
+ def initialize(api_key:, api_version: API_VERSION, correlation_id:, client_id:, request:)
9
+ @api_key = api_key
10
+ @api_version = api_version
11
+ @correlation_id = correlation_id
12
+ @client_id = client_id
13
+ @request = request
14
+ end
15
+
16
+ def encode(encoder)
17
+ encoder.write_int16(@api_key)
18
+ encoder.write_int16(@api_version)
19
+ encoder.write_int32(@correlation_id)
20
+ encoder.write_string(@client_id)
21
+
22
+ @request.encode(encoder)
23
+ end
24
+ end
25
+ end
26
+ end
@@ -0,0 +1,33 @@
1
+ # frozen_string_literal: true
2
+
3
+ module Kafka
4
+ module Protocol
5
+
6
+ # SaslHandshake Request (Version: 0) => mechanism
7
+ # mechanism => string
8
+
9
+ class SaslHandshakeRequest
10
+
11
+ SUPPORTED_MECHANISMS = %w(AWS_MSK_IAM GSSAPI PLAIN SCRAM-SHA-256 SCRAM-SHA-512 OAUTHBEARER)
12
+
13
+ def initialize(mechanism)
14
+ unless SUPPORTED_MECHANISMS.include?(mechanism)
15
+ raise Kafka::Error, "Unsupported SASL mechanism #{mechanism}. Supported are #{SUPPORTED_MECHANISMS.join(', ')}"
16
+ end
17
+ @mechanism = mechanism
18
+ end
19
+
20
+ def api_key
21
+ SASL_HANDSHAKE_API
22
+ end
23
+
24
+ def response_class
25
+ SaslHandshakeResponse
26
+ end
27
+
28
+ def encode(encoder)
29
+ encoder.write_string(@mechanism)
30
+ end
31
+ end
32
+ end
33
+ end
@@ -0,0 +1,28 @@
1
+ # frozen_string_literal: true
2
+
3
+ module Kafka
4
+ module Protocol
5
+
6
+ # SaslHandshake Response (Version: 0) => error_code [enabled_mechanisms]
7
+ # error_code => int16
8
+ # enabled_mechanisms => array of strings
9
+
10
+ class SaslHandshakeResponse
11
+ attr_reader :error_code
12
+
13
+ attr_reader :enabled_mechanisms
14
+
15
+ def initialize(error_code:, enabled_mechanisms:)
16
+ @error_code = error_code
17
+ @enabled_mechanisms = enabled_mechanisms
18
+ end
19
+
20
+ def self.decode(decoder)
21
+ new(
22
+ error_code: decoder.int16,
23
+ enabled_mechanisms: decoder.array { decoder.string }
24
+ )
25
+ end
26
+ end
27
+ end
28
+ end
@@ -0,0 +1,33 @@
1
+ # frozen_string_literal: true
2
+
3
+ module Kafka
4
+ module Protocol
5
+ class SyncGroupRequest
6
+ def initialize(group_id:, generation_id:, member_id:, group_assignment: {})
7
+ @group_id = group_id
8
+ @generation_id = generation_id
9
+ @member_id = member_id
10
+ @group_assignment = group_assignment
11
+ end
12
+
13
+ def api_key
14
+ SYNC_GROUP_API
15
+ end
16
+
17
+ def response_class
18
+ SyncGroupResponse
19
+ end
20
+
21
+ def encode(encoder)
22
+ encoder.write_string(@group_id)
23
+ encoder.write_int32(@generation_id)
24
+ encoder.write_string(@member_id)
25
+
26
+ encoder.write_array(@group_assignment) do |member_id, member_assignment|
27
+ encoder.write_string(member_id)
28
+ encoder.write_bytes(Encoder.encode_with(member_assignment))
29
+ end
30
+ end
31
+ end
32
+ end
33
+ end
@@ -0,0 +1,26 @@
1
+ # frozen_string_literal: true
2
+
3
+ require "kafka/protocol/member_assignment"
4
+
5
+ module Kafka
6
+ module Protocol
7
+ class SyncGroupResponse
8
+ attr_reader :error_code, :member_assignment
9
+
10
+ def initialize(error_code:, member_assignment:)
11
+ @error_code = error_code
12
+ @member_assignment = member_assignment
13
+ end
14
+
15
+ def self.decode(decoder)
16
+ error_code = decoder.int16
17
+ member_assignment_bytes = decoder.bytes
18
+
19
+ new(
20
+ error_code: error_code,
21
+ member_assignment: member_assignment_bytes ? MemberAssignment.decode(Decoder.from_string(member_assignment_bytes)) : nil
22
+ )
23
+ end
24
+ end
25
+ end
26
+ end
@@ -0,0 +1,46 @@
1
+ # frozen_string_literal: true
2
+
3
+ module Kafka
4
+ module Protocol
5
+ class TxnOffsetCommitRequest
6
+
7
+ def api_key
8
+ TXN_OFFSET_COMMIT_API
9
+ end
10
+
11
+ def api_version
12
+ 2
13
+ end
14
+
15
+ def response_class
16
+ TxnOffsetCommitResponse
17
+ end
18
+
19
+ def initialize(transactional_id:, group_id:, producer_id:, producer_epoch:, offsets:)
20
+ @transactional_id = transactional_id
21
+ @producer_id = producer_id
22
+ @producer_epoch = producer_epoch
23
+ @group_id = group_id
24
+ @offsets = offsets
25
+ end
26
+
27
+ def encode(encoder)
28
+ encoder.write_string(@transactional_id.to_s)
29
+ encoder.write_string(@group_id)
30
+ encoder.write_int64(@producer_id)
31
+ encoder.write_int16(@producer_epoch)
32
+
33
+ encoder.write_array(@offsets) do |topic, partitions|
34
+ encoder.write_string(topic)
35
+ encoder.write_array(partitions) do |partition, offset|
36
+ encoder.write_int32(partition)
37
+ encoder.write_int64(offset[:offset])
38
+ encoder.write_string(nil) # metadata
39
+ encoder.write_int32(offset[:leader_epoch])
40
+ end
41
+ end
42
+ end
43
+
44
+ end
45
+ end
46
+ end
@@ -0,0 +1,47 @@
1
+ # frozen_string_literal: true
2
+
3
+ module Kafka
4
+ module Protocol
5
+ class TxnOffsetCommitResponse
6
+ class PartitionError
7
+ attr_reader :partition, :error_code
8
+
9
+ def initialize(partition:, error_code:)
10
+ @partition = partition
11
+ @error_code = error_code
12
+ end
13
+ end
14
+
15
+ class TopicPartitionsError
16
+ attr_reader :topic, :partitions
17
+
18
+ def initialize(topic:, partitions:)
19
+ @topic = topic
20
+ @partitions = partitions
21
+ end
22
+ end
23
+
24
+ attr_reader :errors
25
+
26
+ def initialize(errors:)
27
+ @errors = errors
28
+ end
29
+
30
+ def self.decode(decoder)
31
+ _throttle_time_ms = decoder.int32
32
+ errors = decoder.array do
33
+ TopicPartitionsError.new(
34
+ topic: decoder.string,
35
+ partitions: decoder.array do
36
+ PartitionError.new(
37
+ partition: decoder.int32,
38
+ error_code: decoder.int16
39
+ )
40
+ end
41
+ )
42
+ end
43
+ new(errors: errors)
44
+ end
45
+ end
46
+ end
47
+ end
@@ -0,0 +1,225 @@
1
+ # frozen_string_literal: true
2
+
3
+ module Kafka
4
+
5
+ # The protocol layer of the library.
6
+ #
7
+ # The Kafka protocol (https://kafka.apache.org/protocol) defines a set of API
8
+ # requests, each with a well-known numeric API key, as well as a set of error
9
+ # codes with specific meanings.
10
+ #
11
+ # This module, and the classes contained in it, implement the client side of
12
+ # the protocol.
13
+ module Protocol
14
+ # The replica id of non-brokers is always -1.
15
+ REPLICA_ID = -1
16
+
17
+ PRODUCE_API = 0
18
+ FETCH_API = 1
19
+ LIST_OFFSET_API = 2
20
+ TOPIC_METADATA_API = 3
21
+ OFFSET_COMMIT_API = 8
22
+ OFFSET_FETCH_API = 9
23
+ FIND_COORDINATOR_API = 10
24
+ JOIN_GROUP_API = 11
25
+ HEARTBEAT_API = 12
26
+ LEAVE_GROUP_API = 13
27
+ SYNC_GROUP_API = 14
28
+ DESCRIBE_GROUPS_API = 15
29
+ LIST_GROUPS_API = 16
30
+ SASL_HANDSHAKE_API = 17
31
+ API_VERSIONS_API = 18
32
+ CREATE_TOPICS_API = 19
33
+ DELETE_TOPICS_API = 20
34
+ INIT_PRODUCER_ID_API = 22
35
+ ADD_PARTITIONS_TO_TXN_API = 24
36
+ ADD_OFFSETS_TO_TXN_API = 25
37
+ END_TXN_API = 26
38
+ TXN_OFFSET_COMMIT_API = 28
39
+ DESCRIBE_CONFIGS_API = 32
40
+ ALTER_CONFIGS_API = 33
41
+ CREATE_PARTITIONS_API = 37
42
+
43
+ # A mapping from numeric API keys to symbolic API names.
44
+ APIS = {
45
+ PRODUCE_API => :produce,
46
+ FETCH_API => :fetch,
47
+ LIST_OFFSET_API => :list_offset,
48
+ TOPIC_METADATA_API => :topic_metadata,
49
+ OFFSET_COMMIT_API => :offset_commit,
50
+ OFFSET_FETCH_API => :offset_fetch,
51
+ FIND_COORDINATOR_API => :find_coordinator,
52
+ JOIN_GROUP_API => :join_group,
53
+ HEARTBEAT_API => :heartbeat,
54
+ LEAVE_GROUP_API => :leave_group,
55
+ SYNC_GROUP_API => :sync_group,
56
+ SASL_HANDSHAKE_API => :sasl_handshake,
57
+ API_VERSIONS_API => :api_versions,
58
+ CREATE_TOPICS_API => :create_topics,
59
+ DELETE_TOPICS_API => :delete_topics,
60
+ INIT_PRODUCER_ID_API => :init_producer_id_api,
61
+ ADD_PARTITIONS_TO_TXN_API => :add_partitions_to_txn_api,
62
+ ADD_OFFSETS_TO_TXN_API => :add_offsets_to_txn_api,
63
+ END_TXN_API => :end_txn_api,
64
+ TXN_OFFSET_COMMIT_API => :txn_offset_commit_api,
65
+ DESCRIBE_CONFIGS_API => :describe_configs_api,
66
+ CREATE_PARTITIONS_API => :create_partitions
67
+ }
68
+
69
+ # A mapping from numeric error codes to exception classes.
70
+ ERRORS = {
71
+ -1 => UnknownError,
72
+ 1 => OffsetOutOfRange,
73
+ 2 => CorruptMessage,
74
+ 3 => UnknownTopicOrPartition,
75
+ 4 => InvalidMessageSize,
76
+ 5 => LeaderNotAvailable,
77
+ 6 => NotLeaderForPartition,
78
+ 7 => RequestTimedOut,
79
+ 8 => BrokerNotAvailable,
80
+ 9 => ReplicaNotAvailable,
81
+ 10 => MessageSizeTooLarge,
82
+ 11 => StaleControllerEpoch,
83
+ 12 => OffsetMetadataTooLarge,
84
+ 13 => NetworkException,
85
+ 14 => CoordinatorLoadInProgress,
86
+ 15 => CoordinatorNotAvailable,
87
+ 16 => NotCoordinatorForGroup,
88
+ 17 => InvalidTopic,
89
+ 18 => RecordListTooLarge,
90
+ 19 => NotEnoughReplicas,
91
+ 20 => NotEnoughReplicasAfterAppend,
92
+ 21 => InvalidRequiredAcks,
93
+ 22 => IllegalGeneration,
94
+ 23 => InconsistentGroupProtocol,
95
+ 24 => InvalidGroupId,
96
+ 25 => UnknownMemberId,
97
+ 26 => InvalidSessionTimeout,
98
+ 27 => RebalanceInProgress,
99
+ 28 => InvalidCommitOffsetSize,
100
+ 29 => TopicAuthorizationFailed,
101
+ 30 => GroupAuthorizationFailed,
102
+ 31 => ClusterAuthorizationFailed,
103
+ 32 => InvalidTimestamp,
104
+ 33 => UnsupportedSaslMechanism,
105
+ 34 => InvalidSaslState,
106
+ 35 => UnsupportedVersion,
107
+ 36 => TopicAlreadyExists,
108
+ 37 => InvalidPartitions,
109
+ 38 => InvalidReplicationFactor,
110
+ 39 => InvalidReplicaAssignment,
111
+ 40 => InvalidConfig,
112
+ 41 => NotController,
113
+ 42 => InvalidRequest,
114
+ 43 => UnsupportedForMessageFormat,
115
+ 44 => PolicyViolation,
116
+ 45 => OutOfOrderSequenceNumberError,
117
+ 46 => DuplicateSequenceNumberError,
118
+ 47 => InvalidProducerEpochError,
119
+ 48 => InvalidTxnStateError,
120
+ 49 => InvalidProducerIDMappingError,
121
+ 50 => InvalidTransactionTimeoutError,
122
+ 51 => ConcurrentTransactionError,
123
+ 52 => TransactionCoordinatorFencedError
124
+ }
125
+
126
+ # A mapping from int to corresponding resource type in symbol.
127
+ # https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/resource/ResourceType.java
128
+ RESOURCE_TYPE_UNKNOWN = 0
129
+ RESOURCE_TYPE_ANY = 1
130
+ RESOURCE_TYPE_TOPIC = 2
131
+ RESOURCE_TYPE_GROUP = 3
132
+ RESOURCE_TYPE_CLUSTER = 4
133
+ RESOURCE_TYPE_TRANSACTIONAL_ID = 5
134
+ RESOURCE_TYPE_DELEGATION_TOKEN = 6
135
+ RESOURCE_TYPES = {
136
+ RESOURCE_TYPE_UNKNOWN => :unknown,
137
+ RESOURCE_TYPE_ANY => :any,
138
+ RESOURCE_TYPE_TOPIC => :topic,
139
+ RESOURCE_TYPE_GROUP => :group,
140
+ RESOURCE_TYPE_CLUSTER => :cluster,
141
+ RESOURCE_TYPE_TRANSACTIONAL_ID => :transactional_id,
142
+ RESOURCE_TYPE_DELEGATION_TOKEN => :delegation_token,
143
+ }
144
+
145
+ # Coordinator types. Since Kafka 0.11.0, there are types of coordinators:
146
+ # Group and Transaction
147
+ COORDINATOR_TYPE_GROUP = 0
148
+ COORDINATOR_TYPE_TRANSACTION = 1
149
+
150
+ # Handles an error code by either doing nothing (if there was no error) or
151
+ # by raising an appropriate exception.
152
+ #
153
+ # @param error_code Integer
154
+ # @raise [ProtocolError]
155
+ # @return [nil]
156
+ def self.handle_error(error_code, error_message = nil)
157
+ if error_code == 0
158
+ # No errors, yay!
159
+ elsif error = ERRORS[error_code]
160
+ raise error, error_message
161
+ else
162
+ raise UnknownError, "Unknown error with code #{error_code} #{error_message}"
163
+ end
164
+ end
165
+
166
+ # Returns the symbolic name for an API key.
167
+ #
168
+ # @param api_key Integer
169
+ # @return [Symbol]
170
+ def self.api_name(api_key)
171
+ APIS.fetch(api_key, :unknown)
172
+ end
173
+ end
174
+ end
175
+
176
+ require "kafka/protocol/metadata_request"
177
+ require "kafka/protocol/metadata_response"
178
+ require "kafka/protocol/produce_request"
179
+ require "kafka/protocol/produce_response"
180
+ require "kafka/protocol/fetch_request"
181
+ require "kafka/protocol/fetch_response"
182
+ require "kafka/protocol/list_offset_request"
183
+ require "kafka/protocol/list_offset_response"
184
+ require "kafka/protocol/add_offsets_to_txn_request"
185
+ require "kafka/protocol/add_offsets_to_txn_response"
186
+ require "kafka/protocol/txn_offset_commit_request"
187
+ require "kafka/protocol/txn_offset_commit_response"
188
+ require "kafka/protocol/find_coordinator_request"
189
+ require "kafka/protocol/find_coordinator_response"
190
+ require "kafka/protocol/join_group_request"
191
+ require "kafka/protocol/join_group_response"
192
+ require "kafka/protocol/sync_group_request"
193
+ require "kafka/protocol/sync_group_response"
194
+ require "kafka/protocol/leave_group_request"
195
+ require "kafka/protocol/leave_group_response"
196
+ require "kafka/protocol/heartbeat_request"
197
+ require "kafka/protocol/heartbeat_response"
198
+ require "kafka/protocol/offset_fetch_request"
199
+ require "kafka/protocol/offset_fetch_response"
200
+ require "kafka/protocol/offset_commit_request"
201
+ require "kafka/protocol/offset_commit_response"
202
+ require "kafka/protocol/api_versions_request"
203
+ require "kafka/protocol/api_versions_response"
204
+ require "kafka/protocol/sasl_handshake_request"
205
+ require "kafka/protocol/sasl_handshake_response"
206
+ require "kafka/protocol/create_topics_request"
207
+ require "kafka/protocol/create_topics_response"
208
+ require "kafka/protocol/delete_topics_request"
209
+ require "kafka/protocol/delete_topics_response"
210
+ require "kafka/protocol/describe_configs_request"
211
+ require "kafka/protocol/describe_configs_response"
212
+ require "kafka/protocol/alter_configs_request"
213
+ require "kafka/protocol/alter_configs_response"
214
+ require "kafka/protocol/create_partitions_request"
215
+ require "kafka/protocol/create_partitions_response"
216
+ require "kafka/protocol/list_groups_request"
217
+ require "kafka/protocol/list_groups_response"
218
+ require "kafka/protocol/describe_groups_request"
219
+ require "kafka/protocol/describe_groups_response"
220
+ require "kafka/protocol/init_producer_id_request"
221
+ require "kafka/protocol/init_producer_id_response"
222
+ require "kafka/protocol/add_partitions_to_txn_request"
223
+ require "kafka/protocol/add_partitions_to_txn_response"
224
+ require "kafka/protocol/end_txn_request"
225
+ require "kafka/protocol/end_txn_response"