logstash-integration-kafka 10.1.0-java → 10.5.1-java
Sign up to get free protection for your applications and to get access to all the features.
- checksums.yaml +4 -4
- data/CHANGELOG.md +20 -0
- data/CONTRIBUTORS +1 -0
- data/docs/index.asciidoc +7 -2
- data/docs/input-kafka.asciidoc +124 -81
- data/docs/output-kafka.asciidoc +69 -27
- data/lib/logstash/inputs/kafka.rb +61 -51
- data/lib/logstash/outputs/kafka.rb +48 -31
- data/logstash-integration-kafka.gemspec +1 -1
- data/spec/unit/inputs/kafka_spec.rb +50 -0
- data/spec/unit/outputs/kafka_spec.rb +40 -8
- metadata +2 -2
data/docs/output-kafka.asciidoc
CHANGED
@@ -1,6 +1,9 @@
|
|
1
|
+
:integration: kafka
|
1
2
|
:plugin: kafka
|
2
3
|
:type: output
|
3
4
|
:default_codec: plain
|
5
|
+
:kafka_client: 2.4
|
6
|
+
:kafka_client_doc: 24
|
4
7
|
|
5
8
|
///////////////////////////////////////////
|
6
9
|
START - GENERATED VARIABLES, DO NOT EDIT!
|
@@ -17,15 +20,20 @@ END - GENERATED VARIABLES, DO NOT EDIT!
|
|
17
20
|
|
18
21
|
=== Kafka output plugin
|
19
22
|
|
20
|
-
include::{include_path}/plugin_header.asciidoc[]
|
23
|
+
include::{include_path}/plugin_header-integration.asciidoc[]
|
21
24
|
|
22
25
|
==== Description
|
23
26
|
|
24
27
|
Write events to a Kafka topic.
|
25
28
|
|
26
|
-
This plugin uses Kafka Client
|
29
|
+
This plugin uses Kafka Client {kafka_client}. For broker compatibility, see the
|
30
|
+
official
|
31
|
+
https://cwiki.apache.org/confluence/display/KAFKA/Compatibility+Matrix[Kafka
|
32
|
+
compatibility reference]. If the linked compatibility wiki is not up-to-date,
|
33
|
+
please contact Kafka support/community to confirm compatibility.
|
27
34
|
|
28
|
-
If you require features not yet available in this plugin (including client
|
35
|
+
If you require features not yet available in this plugin (including client
|
36
|
+
version upgrades), please file an issue with details about what you need.
|
29
37
|
|
30
38
|
This output supports connecting to Kafka over:
|
31
39
|
|
@@ -36,9 +44,12 @@ By default security is disabled but can be turned on as needed.
|
|
36
44
|
|
37
45
|
The only required configuration is the topic_id.
|
38
46
|
|
39
|
-
The default codec is plain. Logstash will encode your events with not only the
|
47
|
+
The default codec is plain. Logstash will encode your events with not only the
|
48
|
+
message field but also with a timestamp and hostname.
|
49
|
+
|
50
|
+
If you want the full content of your events to be sent as json, you should set
|
51
|
+
the codec in the output configuration like this:
|
40
52
|
|
41
|
-
If you want the full content of your events to be sent as json, you should set the codec in the output configuration like this:
|
42
53
|
[source,ruby]
|
43
54
|
output {
|
44
55
|
kafka {
|
@@ -47,15 +58,21 @@ If you want the full content of your events to be sent as json, you should set t
|
|
47
58
|
}
|
48
59
|
}
|
49
60
|
|
50
|
-
For more information see
|
61
|
+
For more information see
|
62
|
+
https://kafka.apache.org/{kafka_client_doc}/documentation.html#theproducer
|
51
63
|
|
52
|
-
Kafka producer configuration:
|
64
|
+
Kafka producer configuration:
|
65
|
+
https://kafka.apache.org/{kafka_client_doc}/documentation.html#producerconfigs
|
53
66
|
|
54
67
|
[id="plugins-{type}s-{plugin}-options"]
|
55
68
|
==== Kafka Output Configuration Options
|
56
69
|
|
57
70
|
This plugin supports the following configuration options plus the <<plugins-{type}s-{plugin}-common-options>> described later.
|
58
71
|
|
72
|
+
NOTE: Some of these options map to a Kafka option. Defaults usually reflect the Kafka default setting,
|
73
|
+
and might change if Kafka's producer defaults change.
|
74
|
+
See the https://kafka.apache.org/{kafka_client_doc}/documentation for more details.
|
75
|
+
|
59
76
|
[cols="<,<,<",options="header",]
|
60
77
|
|=======================================================================
|
61
78
|
|Setting |Input type|Required
|
@@ -63,6 +80,7 @@ This plugin supports the following configuration options plus the <<plugins-{typ
|
|
63
80
|
| <<plugins-{type}s-{plugin}-batch_size>> |<<number,number>>|No
|
64
81
|
| <<plugins-{type}s-{plugin}-bootstrap_servers>> |<<string,string>>|No
|
65
82
|
| <<plugins-{type}s-{plugin}-buffer_memory>> |<<number,number>>|No
|
83
|
+
| <<plugins-{type}s-{plugin}-client_dns_lookup>> |<<string,string>>|No
|
66
84
|
| <<plugins-{type}s-{plugin}-client_id>> |<<string,string>>|No
|
67
85
|
| <<plugins-{type}s-{plugin}-compression_type>> |<<string,string>>, one of `["none", "gzip", "snappy", "lz4"]`|No
|
68
86
|
| <<plugins-{type}s-{plugin}-jaas_path>> |a valid filesystem path|No
|
@@ -76,7 +94,7 @@ This plugin supports the following configuration options plus the <<plugins-{typ
|
|
76
94
|
| <<plugins-{type}s-{plugin}-partitioner>> |<<string,string>>|No
|
77
95
|
| <<plugins-{type}s-{plugin}-receive_buffer_bytes>> |<<number,number>>|No
|
78
96
|
| <<plugins-{type}s-{plugin}-reconnect_backoff_ms>> |<<number,number>>|No
|
79
|
-
| <<plugins-{type}s-{plugin}-request_timeout_ms>> |<<
|
97
|
+
| <<plugins-{type}s-{plugin}-request_timeout_ms>> |<<number,number>>|No
|
80
98
|
| <<plugins-{type}s-{plugin}-retries>> |<<number,number>>|No
|
81
99
|
| <<plugins-{type}s-{plugin}-retry_backoff_ms>> |<<number,number>>|No
|
82
100
|
| <<plugins-{type}s-{plugin}-sasl_jaas_config>> |<<string,string>>|No
|
@@ -110,16 +128,19 @@ output plugins.
|
|
110
128
|
The number of acknowledgments the producer requires the leader to have received
|
111
129
|
before considering a request complete.
|
112
130
|
|
113
|
-
acks=0
|
114
|
-
|
115
|
-
|
116
|
-
|
131
|
+
`acks=0`. The producer will not wait for any acknowledgment from the server.
|
132
|
+
|
133
|
+
`acks=1`. The leader will write the record to its local log, but will respond
|
134
|
+
without waiting for full acknowledgement from all followers.
|
135
|
+
|
136
|
+
`acks=all`. The leader will wait for the full set of in-sync replicas before
|
137
|
+
acknowledging the record.
|
117
138
|
|
118
139
|
[id="plugins-{type}s-{plugin}-batch_size"]
|
119
140
|
===== `batch_size`
|
120
141
|
|
121
142
|
* Value type is <<number,number>>
|
122
|
-
* Default value is `16384
|
143
|
+
* Default value is `16384`.
|
123
144
|
|
124
145
|
The producer will attempt to batch records together into fewer requests whenever multiple
|
125
146
|
records are being sent to the same partition. This helps performance on both the client
|
@@ -141,10 +162,22 @@ subset of brokers.
|
|
141
162
|
===== `buffer_memory`
|
142
163
|
|
143
164
|
* Value type is <<number,number>>
|
144
|
-
* Default value is `33554432`
|
165
|
+
* Default value is `33554432` (32MB).
|
145
166
|
|
146
167
|
The total bytes of memory the producer can use to buffer records waiting to be sent to the server.
|
147
168
|
|
169
|
+
[id="plugins-{type}s-{plugin}-client_dns_lookup"]
|
170
|
+
===== `client_dns_lookup`
|
171
|
+
|
172
|
+
* Value type is <<string,string>>
|
173
|
+
* Valid options are `use_all_dns_ips`, `resolve_canonical_bootstrap_servers_only`, `default`
|
174
|
+
* Default value is `"default"`
|
175
|
+
|
176
|
+
Controls how DNS lookups are done. If set to `use_all_dns_ips`, Logstash tries
|
177
|
+
all IP addresses returned for a hostname before failing the connection.
|
178
|
+
If set to `resolve_canonical_bootstrap_servers_only`, each entry will be
|
179
|
+
resolved and expanded into a list of canonical names.
|
180
|
+
|
148
181
|
[id="plugins-{type}s-{plugin}-client_id"]
|
149
182
|
===== `client_id`
|
150
183
|
|
@@ -162,7 +195,7 @@ ip/port by allowing a logical application name to be included with the request
|
|
162
195
|
* Default value is `"none"`
|
163
196
|
|
164
197
|
The compression type for all data generated by the producer.
|
165
|
-
The default is none (i.e. no compression). Valid values are none, gzip, or
|
198
|
+
The default is none (i.e. no compression). Valid values are none, gzip, snappy, or lz4.
|
166
199
|
|
167
200
|
[id="plugins-{type}s-{plugin}-jaas_path"]
|
168
201
|
===== `jaas_path`
|
@@ -221,7 +254,7 @@ to allow other records to be sent so that the sends can be batched together.
|
|
221
254
|
===== `max_request_size`
|
222
255
|
|
223
256
|
* Value type is <<number,number>>
|
224
|
-
* Default value is `1048576`
|
257
|
+
* Default value is `1048576` (1MB).
|
225
258
|
|
226
259
|
The maximum size of a request
|
227
260
|
|
@@ -231,23 +264,23 @@ The maximum size of a request
|
|
231
264
|
* Value type is <<string,string>>
|
232
265
|
* There is no default value for this setting.
|
233
266
|
|
234
|
-
The key for the message
|
267
|
+
The key for the message.
|
235
268
|
|
236
269
|
[id="plugins-{type}s-{plugin}-metadata_fetch_timeout_ms"]
|
237
270
|
===== `metadata_fetch_timeout_ms`
|
238
271
|
|
239
272
|
* Value type is <<number,number>>
|
240
|
-
* Default value is `60000`
|
273
|
+
* Default value is `60000` milliseconds (60 seconds).
|
241
274
|
|
242
|
-
|
275
|
+
The timeout setting for initial metadata request to fetch topic metadata.
|
243
276
|
|
244
277
|
[id="plugins-{type}s-{plugin}-metadata_max_age_ms"]
|
245
278
|
===== `metadata_max_age_ms`
|
246
279
|
|
247
280
|
* Value type is <<number,number>>
|
248
|
-
* Default value is `300000`
|
281
|
+
* Default value is `300000` milliseconds (5 minutes).
|
249
282
|
|
250
|
-
|
283
|
+
The max time in milliseconds before a metadata refresh is forced.
|
251
284
|
|
252
285
|
[id="plugins-{type}s-{plugin}-partitioner"]
|
253
286
|
===== `partitioner`
|
@@ -268,7 +301,7 @@ Available options for choosing a partitioning strategy are as follows:
|
|
268
301
|
===== `receive_buffer_bytes`
|
269
302
|
|
270
303
|
* Value type is <<number,number>>
|
271
|
-
* Default value is `32768`
|
304
|
+
* Default value is `32768` (32KB).
|
272
305
|
|
273
306
|
The size of the TCP receive buffer to use when reading data
|
274
307
|
|
@@ -276,15 +309,15 @@ The size of the TCP receive buffer to use when reading data
|
|
276
309
|
===== `reconnect_backoff_ms`
|
277
310
|
|
278
311
|
* Value type is <<number,number>>
|
279
|
-
* Default value is `
|
312
|
+
* Default value is `50`.
|
280
313
|
|
281
314
|
The amount of time to wait before attempting to reconnect to a given host when a connection fails.
|
282
315
|
|
283
316
|
[id="plugins-{type}s-{plugin}-request_timeout_ms"]
|
284
317
|
===== `request_timeout_ms`
|
285
318
|
|
286
|
-
* Value type is <<
|
287
|
-
*
|
319
|
+
* Value type is <<number,number>>
|
320
|
+
* Default value is `40000` milliseconds (40 seconds).
|
288
321
|
|
289
322
|
The configuration controls the maximum amount of time the client will wait
|
290
323
|
for the response of a request. If the response is not received before the timeout
|
@@ -307,11 +340,20 @@ Kafka down, etc).
|
|
307
340
|
|
308
341
|
A value less than zero is a configuration error.
|
309
342
|
|
343
|
+
Starting with version 10.5.0, this plugin will only retry exceptions that are a subclass of
|
344
|
+
https://kafka.apache.org/{kafka_client_doc}/javadoc/org/apache/kafka/common/errors/RetriableException.html[RetriableException]
|
345
|
+
and
|
346
|
+
https://kafka.apache.org/{kafka_client_doc}/javadoc/org/apache/kafka/common/errors/InterruptException.html[InterruptException].
|
347
|
+
If producing a message throws any other exception, an error is logged and the message is dropped without retrying.
|
348
|
+
This prevents the Logstash pipeline from hanging indefinitely.
|
349
|
+
|
350
|
+
In versions prior to 10.5.0, any exception is retried indefinitely unless the `retries` option is configured.
|
351
|
+
|
310
352
|
[id="plugins-{type}s-{plugin}-retry_backoff_ms"]
|
311
353
|
===== `retry_backoff_ms`
|
312
354
|
|
313
355
|
* Value type is <<number,number>>
|
314
|
-
* Default value is `100`
|
356
|
+
* Default value is `100` milliseconds.
|
315
357
|
|
316
358
|
The amount of time to wait before attempting to retry a failed produce request to a given topic partition.
|
317
359
|
|
@@ -364,7 +406,7 @@ Security protocol to use, which can be either of PLAINTEXT,SSL,SASL_PLAINTEXT,SA
|
|
364
406
|
===== `send_buffer_bytes`
|
365
407
|
|
366
408
|
* Value type is <<number,number>>
|
367
|
-
* Default value is `131072`
|
409
|
+
* Default value is `131072` (128KB).
|
368
410
|
|
369
411
|
The size of the TCP send buffer to use when sending data.
|
370
412
|
|
@@ -53,7 +53,7 @@ class LogStash::Inputs::Kafka < LogStash::Inputs::Base
|
|
53
53
|
default :codec, 'plain'
|
54
54
|
|
55
55
|
# The frequency in milliseconds that the consumer offsets are committed to Kafka.
|
56
|
-
config :auto_commit_interval_ms, :validate => :
|
56
|
+
config :auto_commit_interval_ms, :validate => :number, :default => 5000 # Kafka default
|
57
57
|
# What to do when there is no initial offset in Kafka or if an offset is out of range:
|
58
58
|
#
|
59
59
|
# * earliest: automatically reset the offset to the earliest offset
|
@@ -70,35 +70,40 @@ class LogStash::Inputs::Kafka < LogStash::Inputs::Base
|
|
70
70
|
# Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk
|
71
71
|
# corruption to the messages occurred. This check adds some overhead, so it may be
|
72
72
|
# disabled in cases seeking extreme performance.
|
73
|
-
config :check_crcs, :validate => :
|
73
|
+
config :check_crcs, :validate => :boolean, :default => true
|
74
|
+
# How DNS lookups should be done. If set to `use_all_dns_ips`, when the lookup returns multiple
|
75
|
+
# IP addresses for a hostname, they will all be attempted to connect to before failing the
|
76
|
+
# connection. If the value is `resolve_canonical_bootstrap_servers_only` each entry will be
|
77
|
+
# resolved and expanded into a list of canonical names.
|
78
|
+
config :client_dns_lookup, :validate => ["default", "use_all_dns_ips", "resolve_canonical_bootstrap_servers_only"], :default => "default"
|
74
79
|
# The id string to pass to the server when making requests. The purpose of this
|
75
80
|
# is to be able to track the source of requests beyond just ip/port by allowing
|
76
81
|
# a logical application name to be included.
|
77
82
|
config :client_id, :validate => :string, :default => "logstash"
|
78
83
|
# Close idle connections after the number of milliseconds specified by this config.
|
79
|
-
config :connections_max_idle_ms, :validate => :
|
84
|
+
config :connections_max_idle_ms, :validate => :number, :default => 540_000 # (9m) Kafka default
|
80
85
|
# Ideally you should have as many threads as the number of partitions for a perfect
|
81
86
|
# balance — more threads than partitions means that some threads will be idle
|
82
87
|
config :consumer_threads, :validate => :number, :default => 1
|
83
88
|
# If true, periodically commit to Kafka the offsets of messages already returned by the consumer.
|
84
89
|
# This committed offset will be used when the process fails as the position from
|
85
90
|
# which the consumption will begin.
|
86
|
-
config :enable_auto_commit, :validate => :
|
91
|
+
config :enable_auto_commit, :validate => :boolean, :default => true
|
87
92
|
# Whether records from internal topics (such as offsets) should be exposed to the consumer.
|
88
93
|
# If set to true the only way to receive records from an internal topic is subscribing to it.
|
89
94
|
config :exclude_internal_topics, :validate => :string
|
90
95
|
# The maximum amount of data the server should return for a fetch request. This is not an
|
91
96
|
# absolute maximum, if the first message in the first non-empty partition of the fetch is larger
|
92
97
|
# than this value, the message will still be returned to ensure that the consumer can make progress.
|
93
|
-
config :fetch_max_bytes, :validate => :
|
98
|
+
config :fetch_max_bytes, :validate => :number, :default => 52_428_800 # (50MB) Kafka default
|
94
99
|
# The maximum amount of time the server will block before answering the fetch request if
|
95
100
|
# there isn't sufficient data to immediately satisfy `fetch_min_bytes`. This
|
96
101
|
# should be less than or equal to the timeout used in `poll_timeout_ms`
|
97
|
-
config :fetch_max_wait_ms, :validate => :
|
102
|
+
config :fetch_max_wait_ms, :validate => :number, :default => 500 # Kafka default
|
98
103
|
# The minimum amount of data the server should return for a fetch request. If insufficient
|
99
104
|
# data is available the request will wait for that much data to accumulate
|
100
105
|
# before answering the request.
|
101
|
-
config :fetch_min_bytes, :validate => :
|
106
|
+
config :fetch_min_bytes, :validate => :number
|
102
107
|
# The identifier of the group this consumer belongs to. Consumer group is a single logical subscriber
|
103
108
|
# that happens to be made up of multiple processors. Messages in a topic will be distributed to all
|
104
109
|
# Logstash instances with the same `group_id`
|
@@ -108,50 +113,55 @@ class LogStash::Inputs::Kafka < LogStash::Inputs::Base
|
|
108
113
|
# consumers join or leave the group. The value must be set lower than
|
109
114
|
# `session.timeout.ms`, but typically should be set no higher than 1/3 of that value.
|
110
115
|
# It can be adjusted even lower to control the expected time for normal rebalances.
|
111
|
-
config :heartbeat_interval_ms, :validate => :
|
116
|
+
config :heartbeat_interval_ms, :validate => :number, :default => 3000 # Kafka default
|
117
|
+
# Controls how to read messages written transactionally. If set to read_committed, consumer.poll()
|
118
|
+
# will only return transactional messages which have been committed. If set to read_uncommitted'
|
119
|
+
# (the default), consumer.poll() will return all messages, even transactional messages which have
|
120
|
+
# been aborted. Non-transactional messages will be returned unconditionally in either mode.
|
121
|
+
config :isolation_level, :validate => ["read_uncommitted", "read_committed"], :default => "read_uncommitted" # Kafka default
|
112
122
|
# Java Class used to deserialize the record's key
|
113
123
|
config :key_deserializer_class, :validate => :string, :default => "org.apache.kafka.common.serialization.StringDeserializer"
|
114
124
|
# The maximum delay between invocations of poll() when using consumer group management. This places
|
115
125
|
# an upper bound on the amount of time that the consumer can be idle before fetching more records.
|
116
126
|
# If poll() is not called before expiration of this timeout, then the consumer is considered failed and
|
117
127
|
# the group will rebalance in order to reassign the partitions to another member.
|
118
|
-
|
119
|
-
config :max_poll_interval_ms, :validate => :string
|
128
|
+
config :max_poll_interval_ms, :validate => :number, :default => 300_000 # (5m) Kafka default
|
120
129
|
# The maximum amount of data per-partition the server will return. The maximum total memory used for a
|
121
130
|
# request will be <code>#partitions * max.partition.fetch.bytes</code>. This size must be at least
|
122
131
|
# as large as the maximum message size the server allows or else it is possible for the producer to
|
123
132
|
# send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying
|
124
133
|
# to fetch a large message on a certain partition.
|
125
|
-
config :max_partition_fetch_bytes, :validate => :
|
134
|
+
config :max_partition_fetch_bytes, :validate => :number, :default => 1_048_576 # (1MB) Kafka default
|
126
135
|
# The maximum number of records returned in a single call to poll().
|
127
|
-
config :max_poll_records, :validate => :
|
136
|
+
config :max_poll_records, :validate => :number, :default => 500 # Kafka default
|
128
137
|
# The period of time in milliseconds after which we force a refresh of metadata even if
|
129
138
|
# we haven't seen any partition leadership changes to proactively discover any new brokers or partitions
|
130
|
-
config :metadata_max_age_ms, :validate => :
|
139
|
+
config :metadata_max_age_ms, :validate => :number, :default => 300_000 # (5m) Kafka default
|
131
140
|
# The name of the partition assignment strategy that the client uses to distribute
|
132
141
|
# partition ownership amongst consumer instances, supported options are `range`,
|
133
142
|
# `round_robin`, `sticky` and `cooperative_sticky`
|
134
143
|
# (for backwards compatibility setting the class name directly is supported).
|
135
144
|
config :partition_assignment_strategy, :validate => :string
|
136
145
|
# The size of the TCP receive buffer (SO_RCVBUF) to use when reading data.
|
137
|
-
|
138
|
-
|
146
|
+
# If the value is `-1`, the OS default will be used.
|
147
|
+
config :receive_buffer_bytes, :validate => :number, :default => 32_768 # (32KB) Kafka default
|
148
|
+
# The base amount of time to wait before attempting to reconnect to a given host.
|
139
149
|
# This avoids repeatedly connecting to a host in a tight loop.
|
140
|
-
# This backoff applies to all
|
141
|
-
config :reconnect_backoff_ms, :validate => :
|
142
|
-
# The configuration controls the maximum amount of time the client will wait
|
143
|
-
#
|
144
|
-
#
|
145
|
-
#
|
146
|
-
config :request_timeout_ms, :validate => :string
|
150
|
+
# This backoff applies to all connection attempts by the client to a broker.
|
151
|
+
config :reconnect_backoff_ms, :validate => :number, :default => 50 # Kafka default
|
152
|
+
# The configuration controls the maximum amount of time the client will wait for the response of a request.
|
153
|
+
# If the response is not received before the timeout elapses the client will resend the request if necessary
|
154
|
+
# or fail the request if retries are exhausted.
|
155
|
+
config :request_timeout_ms, :validate => :number, :default => 40_000 # Kafka default
|
147
156
|
# The amount of time to wait before attempting to retry a failed fetch request
|
148
157
|
# to a given topic partition. This avoids repeated fetching-and-failing in a tight loop.
|
149
|
-
config :retry_backoff_ms, :validate => :
|
150
|
-
# The size of the TCP send buffer (SO_SNDBUF) to use when sending data
|
151
|
-
|
158
|
+
config :retry_backoff_ms, :validate => :number, :default => 100 # Kafka default
|
159
|
+
# The size of the TCP send buffer (SO_SNDBUF) to use when sending data.
|
160
|
+
# If the value is -1, the OS default will be used.
|
161
|
+
config :send_buffer_bytes, :validate => :number, :default => 131_072 # (128KB) Kafka default
|
152
162
|
# The timeout after which, if the `poll_timeout_ms` is not invoked, the consumer is marked dead
|
153
163
|
# and a rebalance operation is triggered for the group identified by `group_id`
|
154
|
-
config :session_timeout_ms, :validate => :
|
164
|
+
config :session_timeout_ms, :validate => :number, :default => 10_000 # (10s) Kafka default
|
155
165
|
# Java Class used to deserialize the record's value
|
156
166
|
config :value_deserializer_class, :validate => :string, :default => "org.apache.kafka.common.serialization.StringDeserializer"
|
157
167
|
# A list of topics to subscribe to, defaults to ["logstash"].
|
@@ -276,9 +286,7 @@ class LogStash::Inputs::Kafka < LogStash::Inputs::Base
|
|
276
286
|
end
|
277
287
|
end
|
278
288
|
# Manual offset commit
|
279
|
-
if @enable_auto_commit
|
280
|
-
consumer.commitSync
|
281
|
-
end
|
289
|
+
consumer.commitSync if @enable_auto_commit.eql?(false)
|
282
290
|
end
|
283
291
|
rescue org.apache.kafka.common.errors.WakeupException => e
|
284
292
|
raise e if !stop?
|
@@ -294,31 +302,33 @@ class LogStash::Inputs::Kafka < LogStash::Inputs::Base
|
|
294
302
|
props = java.util.Properties.new
|
295
303
|
kafka = org.apache.kafka.clients.consumer.ConsumerConfig
|
296
304
|
|
297
|
-
props.put(kafka::AUTO_COMMIT_INTERVAL_MS_CONFIG, auto_commit_interval_ms)
|
305
|
+
props.put(kafka::AUTO_COMMIT_INTERVAL_MS_CONFIG, auto_commit_interval_ms.to_s) unless auto_commit_interval_ms.nil?
|
298
306
|
props.put(kafka::AUTO_OFFSET_RESET_CONFIG, auto_offset_reset) unless auto_offset_reset.nil?
|
299
307
|
props.put(kafka::BOOTSTRAP_SERVERS_CONFIG, bootstrap_servers)
|
300
|
-
props.put(kafka::CHECK_CRCS_CONFIG, check_crcs) unless check_crcs.nil?
|
308
|
+
props.put(kafka::CHECK_CRCS_CONFIG, check_crcs.to_s) unless check_crcs.nil?
|
309
|
+
props.put(kafka::CLIENT_DNS_LOOKUP_CONFIG, client_dns_lookup)
|
301
310
|
props.put(kafka::CLIENT_ID_CONFIG, client_id)
|
302
|
-
props.put(kafka::CONNECTIONS_MAX_IDLE_MS_CONFIG, connections_max_idle_ms) unless connections_max_idle_ms.nil?
|
303
|
-
props.put(kafka::ENABLE_AUTO_COMMIT_CONFIG, enable_auto_commit)
|
311
|
+
props.put(kafka::CONNECTIONS_MAX_IDLE_MS_CONFIG, connections_max_idle_ms.to_s) unless connections_max_idle_ms.nil?
|
312
|
+
props.put(kafka::ENABLE_AUTO_COMMIT_CONFIG, enable_auto_commit.to_s)
|
304
313
|
props.put(kafka::EXCLUDE_INTERNAL_TOPICS_CONFIG, exclude_internal_topics) unless exclude_internal_topics.nil?
|
305
|
-
props.put(kafka::FETCH_MAX_BYTES_CONFIG, fetch_max_bytes) unless fetch_max_bytes.nil?
|
306
|
-
props.put(kafka::FETCH_MAX_WAIT_MS_CONFIG, fetch_max_wait_ms) unless fetch_max_wait_ms.nil?
|
307
|
-
props.put(kafka::FETCH_MIN_BYTES_CONFIG, fetch_min_bytes) unless fetch_min_bytes.nil?
|
314
|
+
props.put(kafka::FETCH_MAX_BYTES_CONFIG, fetch_max_bytes.to_s) unless fetch_max_bytes.nil?
|
315
|
+
props.put(kafka::FETCH_MAX_WAIT_MS_CONFIG, fetch_max_wait_ms.to_s) unless fetch_max_wait_ms.nil?
|
316
|
+
props.put(kafka::FETCH_MIN_BYTES_CONFIG, fetch_min_bytes.to_s) unless fetch_min_bytes.nil?
|
308
317
|
props.put(kafka::GROUP_ID_CONFIG, group_id)
|
309
|
-
props.put(kafka::HEARTBEAT_INTERVAL_MS_CONFIG, heartbeat_interval_ms) unless heartbeat_interval_ms.nil?
|
318
|
+
props.put(kafka::HEARTBEAT_INTERVAL_MS_CONFIG, heartbeat_interval_ms.to_s) unless heartbeat_interval_ms.nil?
|
319
|
+
props.put(kafka::ISOLATION_LEVEL_CONFIG, isolation_level)
|
310
320
|
props.put(kafka::KEY_DESERIALIZER_CLASS_CONFIG, key_deserializer_class)
|
311
|
-
props.put(kafka::MAX_PARTITION_FETCH_BYTES_CONFIG, max_partition_fetch_bytes) unless max_partition_fetch_bytes.nil?
|
312
|
-
props.put(kafka::MAX_POLL_RECORDS_CONFIG, max_poll_records) unless max_poll_records.nil?
|
313
|
-
props.put(kafka::MAX_POLL_INTERVAL_MS_CONFIG, max_poll_interval_ms) unless max_poll_interval_ms.nil?
|
314
|
-
props.put(kafka::METADATA_MAX_AGE_CONFIG, metadata_max_age_ms) unless metadata_max_age_ms.nil?
|
321
|
+
props.put(kafka::MAX_PARTITION_FETCH_BYTES_CONFIG, max_partition_fetch_bytes.to_s) unless max_partition_fetch_bytes.nil?
|
322
|
+
props.put(kafka::MAX_POLL_RECORDS_CONFIG, max_poll_records.to_s) unless max_poll_records.nil?
|
323
|
+
props.put(kafka::MAX_POLL_INTERVAL_MS_CONFIG, max_poll_interval_ms.to_s) unless max_poll_interval_ms.nil?
|
324
|
+
props.put(kafka::METADATA_MAX_AGE_CONFIG, metadata_max_age_ms.to_s) unless metadata_max_age_ms.nil?
|
315
325
|
props.put(kafka::PARTITION_ASSIGNMENT_STRATEGY_CONFIG, partition_assignment_strategy_class) unless partition_assignment_strategy.nil?
|
316
|
-
props.put(kafka::RECEIVE_BUFFER_CONFIG, receive_buffer_bytes) unless receive_buffer_bytes.nil?
|
317
|
-
props.put(kafka::RECONNECT_BACKOFF_MS_CONFIG, reconnect_backoff_ms) unless reconnect_backoff_ms.nil?
|
318
|
-
props.put(kafka::REQUEST_TIMEOUT_MS_CONFIG, request_timeout_ms) unless request_timeout_ms.nil?
|
319
|
-
props.put(kafka::RETRY_BACKOFF_MS_CONFIG, retry_backoff_ms) unless retry_backoff_ms.nil?
|
320
|
-
props.put(kafka::SEND_BUFFER_CONFIG, send_buffer_bytes) unless send_buffer_bytes.nil?
|
321
|
-
props.put(kafka::SESSION_TIMEOUT_MS_CONFIG, session_timeout_ms) unless session_timeout_ms.nil?
|
326
|
+
props.put(kafka::RECEIVE_BUFFER_CONFIG, receive_buffer_bytes.to_s) unless receive_buffer_bytes.nil?
|
327
|
+
props.put(kafka::RECONNECT_BACKOFF_MS_CONFIG, reconnect_backoff_ms.to_s) unless reconnect_backoff_ms.nil?
|
328
|
+
props.put(kafka::REQUEST_TIMEOUT_MS_CONFIG, request_timeout_ms.to_s) unless request_timeout_ms.nil?
|
329
|
+
props.put(kafka::RETRY_BACKOFF_MS_CONFIG, retry_backoff_ms.to_s) unless retry_backoff_ms.nil?
|
330
|
+
props.put(kafka::SEND_BUFFER_CONFIG, send_buffer_bytes.to_s) unless send_buffer_bytes.nil?
|
331
|
+
props.put(kafka::SESSION_TIMEOUT_MS_CONFIG, session_timeout_ms.to_s) unless session_timeout_ms.nil?
|
322
332
|
props.put(kafka::VALUE_DESERIALIZER_CLASS_CONFIG, value_deserializer_class)
|
323
333
|
props.put(kafka::CLIENT_RACK_CONFIG, client_rack) unless client_rack.nil?
|
324
334
|
|
@@ -374,15 +384,15 @@ class LogStash::Inputs::Kafka < LogStash::Inputs::Base
|
|
374
384
|
end
|
375
385
|
|
376
386
|
def set_sasl_config(props)
|
377
|
-
java.lang.System.setProperty("java.security.auth.login.config",jaas_path) unless jaas_path.nil?
|
378
|
-
java.lang.System.setProperty("java.security.krb5.conf",kerberos_config) unless kerberos_config.nil?
|
387
|
+
java.lang.System.setProperty("java.security.auth.login.config", jaas_path) unless jaas_path.nil?
|
388
|
+
java.lang.System.setProperty("java.security.krb5.conf", kerberos_config) unless kerberos_config.nil?
|
379
389
|
|
380
|
-
props.put("sasl.mechanism",sasl_mechanism)
|
390
|
+
props.put("sasl.mechanism", sasl_mechanism)
|
381
391
|
if sasl_mechanism == "GSSAPI" && sasl_kerberos_service_name.nil?
|
382
392
|
raise LogStash::ConfigurationError, "sasl_kerberos_service_name must be specified when SASL mechanism is GSSAPI"
|
383
393
|
end
|
384
394
|
|
385
|
-
props.put("sasl.kerberos.service.name",sasl_kerberos_service_name) unless sasl_kerberos_service_name.nil?
|
395
|
+
props.put("sasl.kerberos.service.name", sasl_kerberos_service_name) unless sasl_kerberos_service_name.nil?
|
386
396
|
props.put("sasl.jaas.config", sasl_jaas_config) unless sasl_jaas_config.nil?
|
387
397
|
end
|
388
398
|
end #class LogStash::Inputs::Kafka
|
@@ -67,7 +67,7 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
67
67
|
# The producer will attempt to batch records together into fewer requests whenever multiple
|
68
68
|
# records are being sent to the same partition. This helps performance on both the client
|
69
69
|
# and the server. This configuration controls the default batch size in bytes.
|
70
|
-
config :batch_size, :validate => :number, :default =>
|
70
|
+
config :batch_size, :validate => :number, :default => 16_384 # Kafka default
|
71
71
|
# This is for bootstrapping and the producer will only use it for getting metadata (topics,
|
72
72
|
# partitions and replicas). The socket connections for sending the actual data will be
|
73
73
|
# established based on the broker information returned in the metadata. The format is
|
@@ -75,10 +75,15 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
75
75
|
# subset of brokers.
|
76
76
|
config :bootstrap_servers, :validate => :string, :default => 'localhost:9092'
|
77
77
|
# The total bytes of memory the producer can use to buffer records waiting to be sent to the server.
|
78
|
-
config :buffer_memory, :validate => :number, :default =>
|
78
|
+
config :buffer_memory, :validate => :number, :default => 33_554_432 # (32M) Kafka default
|
79
79
|
# The compression type for all data generated by the producer.
|
80
80
|
# The default is none (i.e. no compression). Valid values are none, gzip, or snappy.
|
81
81
|
config :compression_type, :validate => ["none", "gzip", "snappy", "lz4"], :default => "none"
|
82
|
+
# How DNS lookups should be done. If set to `use_all_dns_ips`, when the lookup returns multiple
|
83
|
+
# IP addresses for a hostname, they will all be attempted to connect to before failing the
|
84
|
+
# connection. If the value is `resolve_canonical_bootstrap_servers_only` each entry will be
|
85
|
+
# resolved and expanded into a list of canonical names.
|
86
|
+
config :client_dns_lookup, :validate => ["default", "use_all_dns_ips", "resolve_canonical_bootstrap_servers_only"], :default => "default"
|
82
87
|
# The id string to pass to the server when making requests.
|
83
88
|
# The purpose of this is to be able to track the source of requests beyond just
|
84
89
|
# ip/port by allowing a logical application name to be included with the request
|
@@ -92,26 +97,26 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
92
97
|
# This setting accomplishes this by adding a small amount of artificial delay—that is,
|
93
98
|
# rather than immediately sending out a record the producer will wait for up to the given delay
|
94
99
|
# to allow other records to be sent so that the sends can be batched together.
|
95
|
-
config :linger_ms, :validate => :number, :default => 0
|
100
|
+
config :linger_ms, :validate => :number, :default => 0 # Kafka default
|
96
101
|
# The maximum size of a request
|
97
|
-
config :max_request_size, :validate => :number, :default =>
|
102
|
+
config :max_request_size, :validate => :number, :default => 1_048_576 # (1MB) Kafka default
|
98
103
|
# The key for the message
|
99
104
|
config :message_key, :validate => :string
|
100
105
|
# the timeout setting for initial metadata request to fetch topic metadata.
|
101
|
-
config :metadata_fetch_timeout_ms, :validate => :number, :default =>
|
106
|
+
config :metadata_fetch_timeout_ms, :validate => :number, :default => 60_000
|
102
107
|
# the max time in milliseconds before a metadata refresh is forced.
|
103
|
-
config :metadata_max_age_ms, :validate => :number, :default =>
|
108
|
+
config :metadata_max_age_ms, :validate => :number, :default => 300_000 # (5m) Kafka default
|
104
109
|
# Partitioner to use - can be `default`, `uniform_sticky`, `round_robin` or a fully qualified class name of a custom partitioner.
|
105
110
|
config :partitioner, :validate => :string
|
106
111
|
# The size of the TCP receive buffer to use when reading data
|
107
|
-
config :receive_buffer_bytes, :validate => :number, :default =>
|
112
|
+
config :receive_buffer_bytes, :validate => :number, :default => 32_768 # (32KB) Kafka default
|
108
113
|
# The amount of time to wait before attempting to reconnect to a given host when a connection fails.
|
109
|
-
config :reconnect_backoff_ms, :validate => :number, :default =>
|
114
|
+
config :reconnect_backoff_ms, :validate => :number, :default => 50 # Kafka default
|
110
115
|
# The configuration controls the maximum amount of time the client will wait
|
111
116
|
# for the response of a request. If the response is not received before the timeout
|
112
117
|
# elapses the client will resend the request if necessary or fail the request if
|
113
118
|
# retries are exhausted.
|
114
|
-
config :request_timeout_ms, :validate => :
|
119
|
+
config :request_timeout_ms, :validate => :number, :default => 40_000 # (40s) Kafka default
|
115
120
|
# The default retry behavior is to retry until successful. To prevent data loss,
|
116
121
|
# the use of this setting is discouraged.
|
117
122
|
#
|
@@ -122,9 +127,9 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
122
127
|
# A value less than zero is a configuration error.
|
123
128
|
config :retries, :validate => :number
|
124
129
|
# The amount of time to wait before attempting to retry a failed produce request to a given topic partition.
|
125
|
-
config :retry_backoff_ms, :validate => :number, :default => 100
|
130
|
+
config :retry_backoff_ms, :validate => :number, :default => 100 # Kafka default
|
126
131
|
# The size of the TCP send buffer to use when sending data.
|
127
|
-
config :send_buffer_bytes, :validate => :number, :default =>
|
132
|
+
config :send_buffer_bytes, :validate => :number, :default => 131_072 # (128KB) Kafka default
|
128
133
|
# The truststore type.
|
129
134
|
config :ssl_truststore_type, :validate => :string
|
130
135
|
# The JKS truststore path to validate the Kafka broker's certificate.
|
@@ -231,7 +236,7 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
231
236
|
remaining = @retries
|
232
237
|
|
233
238
|
while batch.any?
|
234
|
-
|
239
|
+
unless remaining.nil?
|
235
240
|
if remaining < 0
|
236
241
|
# TODO(sissel): Offer to DLQ? Then again, if it's a transient fault,
|
237
242
|
# DLQing would make things worse (you dlq data that would be successful
|
@@ -250,27 +255,39 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
250
255
|
begin
|
251
256
|
# send() can throw an exception even before the future is created.
|
252
257
|
@producer.send(record)
|
253
|
-
rescue org.apache.kafka.common.errors.
|
258
|
+
rescue org.apache.kafka.common.errors.InterruptException,
|
259
|
+
org.apache.kafka.common.errors.RetriableException => e
|
260
|
+
logger.info("producer send failed, will retry sending", :exception => e.class, :message => e.message)
|
254
261
|
failures << record
|
255
262
|
nil
|
256
|
-
rescue org.apache.kafka.common.
|
257
|
-
|
258
|
-
|
259
|
-
|
260
|
-
|
261
|
-
# TODO(sissel): Let's add DLQ here.
|
262
|
-
failures << record
|
263
|
+
rescue org.apache.kafka.common.KafkaException => e
|
264
|
+
# This error is not retriable, drop event
|
265
|
+
# TODO: add DLQ support
|
266
|
+
logger.warn("producer send failed, dropping record",:exception => e.class, :message => e.message,
|
267
|
+
:record_value => record.value)
|
263
268
|
nil
|
264
269
|
end
|
265
|
-
end
|
270
|
+
end
|
266
271
|
|
267
272
|
futures.each_with_index do |future, i|
|
268
|
-
|
269
|
-
|
270
|
-
|
271
|
-
|
272
|
-
|
273
|
-
|
273
|
+
# We cannot skip nils using `futures.compact` because then our index `i` will not align with `batch`
|
274
|
+
unless future.nil?
|
275
|
+
begin
|
276
|
+
future.get
|
277
|
+
rescue java.util.concurrent.ExecutionException => e
|
278
|
+
# TODO(sissel): Add metric to count failures, possibly by exception type.
|
279
|
+
if e.get_cause.is_a? org.apache.kafka.common.errors.RetriableException or
|
280
|
+
e.get_cause.is_a? org.apache.kafka.common.errors.InterruptException
|
281
|
+
logger.info("producer send failed, will retry sending", :exception => e.cause.class,
|
282
|
+
:message => e.cause.message)
|
283
|
+
failures << batch[i]
|
284
|
+
elsif e.get_cause.is_a? org.apache.kafka.common.KafkaException
|
285
|
+
# This error is not retriable, drop event
|
286
|
+
# TODO: add DLQ support
|
287
|
+
logger.warn("producer send failed, dropping record", :exception => e.cause.class,
|
288
|
+
:message => e.cause.message, :record_value => batch[i].value)
|
289
|
+
end
|
290
|
+
end
|
274
291
|
end
|
275
292
|
end
|
276
293
|
|
@@ -318,18 +335,19 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
318
335
|
props.put(kafka::BOOTSTRAP_SERVERS_CONFIG, bootstrap_servers)
|
319
336
|
props.put(kafka::BUFFER_MEMORY_CONFIG, buffer_memory.to_s)
|
320
337
|
props.put(kafka::COMPRESSION_TYPE_CONFIG, compression_type)
|
338
|
+
props.put(kafka::CLIENT_DNS_LOOKUP_CONFIG, client_dns_lookup)
|
321
339
|
props.put(kafka::CLIENT_ID_CONFIG, client_id) unless client_id.nil?
|
322
340
|
props.put(kafka::KEY_SERIALIZER_CLASS_CONFIG, key_serializer)
|
323
341
|
props.put(kafka::LINGER_MS_CONFIG, linger_ms.to_s)
|
324
342
|
props.put(kafka::MAX_REQUEST_SIZE_CONFIG, max_request_size.to_s)
|
325
|
-
props.put(kafka::METADATA_MAX_AGE_CONFIG, metadata_max_age_ms) unless metadata_max_age_ms.nil?
|
343
|
+
props.put(kafka::METADATA_MAX_AGE_CONFIG, metadata_max_age_ms.to_s) unless metadata_max_age_ms.nil?
|
326
344
|
unless partitioner.nil?
|
327
345
|
props.put(kafka::PARTITIONER_CLASS_CONFIG, partitioner = partitioner_class)
|
328
346
|
logger.debug('producer configured using partitioner', :partitioner_class => partitioner)
|
329
347
|
end
|
330
348
|
props.put(kafka::RECEIVE_BUFFER_CONFIG, receive_buffer_bytes.to_s) unless receive_buffer_bytes.nil?
|
331
|
-
props.put(kafka::RECONNECT_BACKOFF_MS_CONFIG, reconnect_backoff_ms) unless reconnect_backoff_ms.nil?
|
332
|
-
props.put(kafka::REQUEST_TIMEOUT_MS_CONFIG, request_timeout_ms) unless request_timeout_ms.nil?
|
349
|
+
props.put(kafka::RECONNECT_BACKOFF_MS_CONFIG, reconnect_backoff_ms.to_s) unless reconnect_backoff_ms.nil?
|
350
|
+
props.put(kafka::REQUEST_TIMEOUT_MS_CONFIG, request_timeout_ms.to_s) unless request_timeout_ms.nil?
|
333
351
|
props.put(kafka::RETRIES_CONFIG, retries.to_s) unless retries.nil?
|
334
352
|
props.put(kafka::RETRY_BACKOFF_MS_CONFIG, retry_backoff_ms.to_s)
|
335
353
|
props.put(kafka::SEND_BUFFER_CONFIG, send_buffer_bytes.to_s)
|
@@ -346,7 +364,6 @@ class LogStash::Outputs::Kafka < LogStash::Outputs::Base
|
|
346
364
|
set_sasl_config(props)
|
347
365
|
end
|
348
366
|
|
349
|
-
|
350
367
|
org.apache.kafka.clients.producer.KafkaProducer.new(props)
|
351
368
|
rescue => e
|
352
369
|
logger.error("Unable to create Kafka producer from given configuration",
|