Implementation for lazy down-conversion in a chunked manner for efficient memory usage during down-conversion. This pull request is mainly to get initial feedback on the direction of the patch. The patch includes all the main components from KIP-283.
Reviewers: Jason Gustafson <jason@confluent.io>
This commit allows secrets in Connect configs to be externalized and replaced with variable references of the form `${provider:[path:]key}`, where the "path" is optional.
There are 2 main additions to `org.apache.kafka.common.config`: a `ConfigProvider` and a `ConfigTransformer`. The `ConfigProvider` is an interface that allows key-value pairs to be provided by an external source for a given "path". An a TTL can be associated with the key-value pairs returned from the path. The `ConfigTransformer` will use instances of `ConfigProvider` to replace variable references in a set of configuration values.
In the Connect framework, `ConfigProvider` classes can be specified in the worker config, and then variable references can be used in the connector config. In addition, the herder can be configured to restart connectors (or not) based on the TTL returned from a `ConfigProvider`. The main class that performs restarts and transformations is `WorkerConfigTransformer`.
Finally, a `configs()` method has been added to both `SourceTaskContext` and `SinkTaskContext`. This allows connectors to get configs with variables replaced by the latest values from instances of `ConfigProvider`.
Most of the other changes in the Connect framework are threading various objects through classes to enable the above functionality.
Author: Robert Yokota <rayokota@gmail.com>
Author: Ewen Cheslack-Postava <me@ewencp.org>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5068 from rayokota/KAFKA-6886-connect-secrets
This implements KIP-219, where a broker returns a response with throttle time on
quota violation immediately after processing the corresponding request. After
the response is sent out, the broker will keep the channel muted until the
throttle time is over. Also, on receiving a response with throttle time, client
will block outgoing communication to the broker for the specified throttle time.
See PR 4830, 5064 and 5094 for all the review history
Author: Jon Lee <jonlee@jonlee-ld1.linkedin.biz>
Reviewers: Jun Rao <junrao@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>, Dong Lin <lindong28@gmail.com>
Closes#5064 from jonlee2/kip-219
Add the new stricter-timeout version of `poll` proposed in KIP-266.
The pre-existing variant `poll(long timeout)` would block indefinitely for metadata
updates if they were needed, then it would issue a fetch and poll for `timeout` ms
for new records. The initial indefinite metadata block caused applications to become
stuck when the brokers became unavailable. The existence of the timeout parameter
made the indefinite block especially unintuitive.
This PR adds `poll(Duration timeout)` with the semantics:
1. iff a metadata update is needed:
1. send (asynchronous) metadata requests
2. poll for metadata responses (counts against timeout)
- if no response within timeout, **return an empty collection immediately**
2. if there is fetch data available, **return it immediately**
3. if there is no fetch request in flight, send fetch requests
4. poll for fetch responses (counts against timeout)
- if no response within timeout, **return an empty collection** (leaving async fetch request for the next poll)
- if we get a response, **return the response**
The old method, `poll(long timeout)` is deprecated, but we do not change its semantics, so it remains:
1. iff a metadata update is needed:
1. send (asynchronous) metadata requests
2. poll for metadata responses *indefinitely until we get it*
2. if there is fetch data available, **return it immediately**
3. if there is no fetch request in flight, send fetch requests
4. poll for fetch responses (counts against timeout)
- if no response within timeout, **return an empty collection** (leaving async fetch request for the next poll)
- if we get a response, **return the response**
One notable usage is prohibited by the new `poll`: previously, you could call `poll(0)` to block for metadata updates, for example to initialize the client, supposedly without fetching records. Note, though, that this behavior is not according to any contract, and there is no guarantee that `poll(0)` won't return records the first time it's called. Therefore, it has always been unsafe to ignore the response.
This KIP adds the following functionality related to SASL/OAUTHBEARER:
1) Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker protocol as well as non-broker clients) to flexibly retrieve an access token from an OAuth 2 authorization server based on the declaration of a custom login CallbackHandler implementation and have that access token transparently and automatically transmitted to a broker for authentication.
2) Allow brokers to flexibly validate provided access tokens when a client establishes a connection based on the declaration of a custom SASL Server CallbackHandler implementation.
3) Provide implementations of the above retrieval and validation features based on an unsecured JSON Web Token that function out-of-the-box with minimal configuration required (i.e. implementations of the two types of callback handlers mentioned above will be used by default with no need to explicitly declare them).
4) Allow clients (both brokers when SASL/OAUTHBEARER is the inter-broker protocol as well as non-broker clients) to transparently retrieve a new access token in the background before the existing access token expires in case the client has to open new connections.
AdminClient should backoff when retrying a Call. Fixed and added a unit test
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Dong Lin <lindong28@gmail.com>
Closes#5077 from hachikuji/admin-client-retry-backoff
The Signal classes are not available in the compile classpath
if --release is used so we use reflection as a workaround.
As part of that moved the code to Java and added a simple
unit test.
Also disabled the signal handler if the IBM JDK is being used
due to KAFKA-6918.
Manually tested shutdown via ctrl+c and verified that
the message is printed.
Fix the check, add unit test to verify the change, update `DynamicBrokerReconfigurationTest` to avoid dynamic keystore update in tests which are not expected to update keystores.
Avoid dependence on the internal __consumer_offsets topic to handle `listConsumerGroups()` since it unnecessarily requires users to have Describe access on an internal topic. Instead we query each broker independently. For most clusters, this amounts to the same thing since the default number of partitions for __consumer_offsets is 50. This also provides better encapsulation since it avoids exposing the use of __consumer_offsets, which gives us more flexibility in the future.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Dong Lin <lindong28@gmail.com>
Closes#5007 from hachikuji/remove-admin-use-of-offsets-topic
If the internal metadata request fails, we must reset the state inside `AdminClientMetadataManager` or we will be stuck indefinitely in the `UPDATE_PENDING` state and have no way to fetch new metadata.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Dong Lin <lindong28@gmail.com>
Closes#5057 from hachikuji/fix-admin-client-metadata-update-failure
We no longer need them since we now require Java 8.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Andras Beni <andrasbeni@cloudera.com>, Manikumar Reddy O <manikumar.reddy@gmail.com>, Dong Lin <lindong28@gmail.com>
Closes#5049 from ijuma/remove-base64
* Set --source, --target and --release to 1.8.
* Build Scala 2.12 by default.
* Remove some conditionals in the build file now that Java 8
is the minimum version.
* Bump the version of Jetty, Jersey and Checkstyle (the newer
versions require Java 8).
* Fixed issues uncovered by the new version if Checkstyle.
* A couple of minor updates to handle an incompatible source
change in the new version of Jetty.
* Add dependency to jersey-hk2 to fix failing tests caused
by Jersey upgrade.
* Update release script to use Java 8 and to take into account
that Scala 2.12 is now built by default.
* While we're at it, bump the version of Gradle, Gradle plugins,
ScalaLogging, JMH and apache directory api.
* Minor documentation updates including the readme and upgrade
notes. A number of Streams Java 7 examples can be removed
subsequently.
Implementation of KIP-279 as described here: https://cwiki.apache.org/confluence/display/KAFKA/KIP-279%3A+Fix+log+divergence+between+leader+and+follower+after+fast+leader+fail+over
In summary:
- Added leader_epoch to OFFSET_FOR_LEADER_EPOCH_RESPONSE
- Leader replies with the pair( largest epoch less than or equal to the requested epoch, the end offset of this epoch)
- If Follower does not know about the leader epoch that leader replies with, it truncates to the end offset of largest leader epoch less than leader epoch that leader replied with, and sends another OffsetForLeaderEpoch request. That request contains the largest leader epoch less than leader epoch that leader replied with.
Reviewers: Dong Lin <lindong28@gmail.com>, Jun Rao <junrao@gmail.com>
When AdminClient gets a NOT_CONTROLLER error, it should refresh its metadata and retry the request, rather than making the end-user deal with NotControllerException.
Move AdminClient's metadata management outside of NetworkClient and into AdminMetadataManager. This will make it easier to do more sophisticated metadata management in the future, such as implementing a NodeProvider which fetches the leaders for topics.
Rather than manipulating newCalls directly, the AdminClient service thread now drains it directly into pendingCalls. This minimizes the amount of locking we have to do, since pendingCalls is only accessed from the service thread.
Grow buffers in log cleaner to hold one message set after sanity check even if message set is bigger than max.message.bytes.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
This patch removed a completedFetch from the completedFetches queue upon a failed parse if it contains no records. The following scenario explains why this is needed for an instance of this case – i.e. in TopicAuthorizationException.
0. Let's assume a scenario, in which the consumer is attempting to read from a topic without the necessary read permission.
1. In Fetcher#fetchedRecords(), after peeking the completedFetches, the Fetcher#parseCompletedFetch(CompletedFetch) throws a TopicAuthorizationException (as expected).
2. Fetcher#fetchedRecords() passes the TopicAuthorizationException up without having a chance to poll completedFetches. So, the same completedFetch remains at the completedFetches queue.
3. Upon following calls to Fetcher#fetchedRecords(), peeking the completedFetches will always return the same completedFetch independent of any updates to the ACL that the topic is trying to read from.
4. Hence, despite the creation of an ACL with correct permissions, once the consumer sees the TopicAuthorizationException, it will be unable to recover without a bounce.
Author: Adem Efe Gencer <agencer@linkedin.com>
Reviewers: Jiangjie (Becket) Qin <becket.qin@gmail.com>
Closes#4974 from efeg/fix/parseCompletedFetchRemainsInQueue
Switch from sum operations to subtraction to avoid type casting in checks and type overflow during `FlieLogInputStream` work, especially in cases where property `log.segment.bytes` was set close to the `Integer.MAX_VALUE` and used as a `position` inside `nextBatch()` function.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
For the UNKNOWN_TOPIC_OR_PARTITION error, we could change the consumer's behavior to retry after this error. While this is a rare case since the user would not commit offsets for topics unless they had been able to fetch from them, but this doesn't really handle the situation where the broker hasn't received any metadata updates.
Reviewers: Jason Gustafson <jason@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
The current Iterator-based ListConsumerGroups API is synchronous. The API should be asynchronous to fit in with the other AdminClient APIs. Also fix some error handling corner cases.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Quota tests wait for throttle metric to be updated without waiting for requests to complete to avoid waiting for potentially large throttle times. This requires the test to read metric values while a broker may be updating the value, resulting in exception in the test. Since this issue can also occur with JMX metrics reporter, change synchronization on metrics with sensors to use the sensor as lock.
Reviewers: Manikumar Reddy O <manikumar.reddy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Refactored the return types in consumer group APIs the following way:
```
Map<TopicPartition, KafkaFuture<Void>> DeleteConsumerGroupsResult#deletedGroups()
Map<TopicPartition, KafkaFuture<ConsumerGroupDescription>> DescribeConsumerGroupsResult#describedGroups()
KafkaFuture<Collection<ConsumerGroupListing>> ListConsumerGroupsResult#listings()
KafkaFuture<Map<TopicPartition, OffsetAndMetadata>> ListConsumerGroupOffsetsResult#partitionsToOffsetAndMetadata()
```
* For DeleteConsumerGroupsResult and DescribeConsumerGroupsResult, for each group id we have two round-trips to get the coordinator, and then send the delete / describe request; I leave the potential optimization of batching requests for future work.
* For ListConsumerGroupOffsetsResult, it is a simple single round-trip and hence the whole map is wrapped as a Future.
* ListConsumerGroupsResult, it is the most tricky one: we would only know how many futures we should wait for after the first listNode returns, and hence I constructed the flattened future in the middle wrapped with the underlying map of futures; also added an iterator API to compensate the "fail the whole future if any broker returns error" behavior. The iterator future will throw exception on the failing brokers, while return the consumer for other succeeded brokers.
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Jason Gustafson <jason@confluent.io>
This change makes adding a metric to a sensor idempotent.
That is, if the metric is already added to the sensor, the method
returns with success.
The current behavior is that any attempt to register a second metric
with the same name is an error.
Testing strategy: There is a new unit test covering this behavior
Reviewers: Guozhang Wang <guozhang@confluent.io>, Bill Bejeck <bill@confluent.io>
This patch fixes an edge case in producer shutdown which prevents `close()` from completing due to a pending request which will never be sent due to shutdown initiation. I have added a test case which reproduces the scenario.
Reviewers: Apurva Mehta <apurva@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Implementation of KIP-86. Client, server and login callback handlers have been made configurable for both brokers and clients.
Reviewers: Jun Rao <junrao@gmail.com>, Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
The invalid topic name is already handled locally so it is unnecessary to send the DeleteTopicsRequest. This PR adds a count to MockClient for testing.
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Jason Gustafson <jason@confluent.io>
Ignore headers when down-converting to V0/V1 since they are not supported. Added a test-case to verify down-conversion sanity in presence of headers.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
Currently the `initTransactions()` API blocks indefinitely if the broker cannot be reached. This patch changes the behavior to raise a `TimeoutException` after waiting for `max.block.ms`.
Reviewers: Apurva Mehta <apurva@confluent.io>, Jason Gustafson <jason@confluent.io>
Added configs to ProducerBenchSpec:
topicPrefix: name of topics will be of format topicPrefix + topic index. If not provided, default is "produceBenchTopic".
partitionsPerTopic: number of partitions per topic. If not provided, default is 1.
replicationFactor: replication factor per topic. If not provided, default is 3.
The behavior of producer bench is changed such that if some or all topics already exist (with topic names = topicPrefix + topic index), and they have the same number of partitions as requested, the worker uses those topics and does not fail. The producer bench fails if one or more existing topics has number of partitions that is different from expected number of partitions.
Added unit test for WorkerUtils -- for existing methods and new methods.
Fixed bug in MockAdminClient, where createTopics() would over-write existing topic's replication factor and number of partitions while correctly completing the appropriate futures exceptionally with TopicExistsException.
Reviewers: Colin P. Mccabe <cmccabe@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
Remove unnecessary null check in StringDeserializer, MockProducerInterceptor and KStreamImpl.
Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Jason Gustafson <jason@confluent.io>
Logging can get spammy during the reconnect blackout period because any requests we send to ConsumerNetworkClient will immediately be failed when poll() returns. This patch checks for connection failures prior to sending fetches and offset lookups and skips sending to any failed nodes. Test cases added for both.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
NetworkClient should use FIFO order when completing inflight requests following a disconnect.
I've added new unit tests for `InFlightRequests` and `NetworkClient` which verify completion order.
Reviewers: Jun Rao <junrao@gmail.com>
We need to reset the auto-commit deadline after sending the offset commit request so that we do not resend it while the request is still inflight.
Added unit tests ensuring this behavior and proper backoff in the case of a failure.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Contention for the lock in ConsumerNetworkClient can lead to a livelock situation in which an active commitSync is unable to make progress because its completion is blocked in the heartbeat thread. The fix is twofold:
1) We change ConsumerNetworkClient to use a fair lock to reduce the chance of each thread getting starved.
2) We eliminate the dependence on the lock in ConsumerNetworkClient for callback completion so that callbacks will not be blocked by an active poll().
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Currently we hold onto all Records references in a multi-partition fetch response until the full response has completed. This can be a problem when the records have been down-converted since they will be occupying a (potentially large) chunk of memory. This patch changes the behavior in MultiSend so that once a Send is completed, we no longer keep a reference to it, which will allow the Records objects to be freed sooner.
I have added a simple unit test to verify that sends are removed as the MultiSend progresses.
Reviewers: Ismael Juma <ismael@juma.me.uk>