The client caches metadata fetched from Metadata requests. Previously, each metadata response overwrote all of the metadata from the previous one, so we could rely on the expectation that the broker only returned the leaderId for a partition if it had connection information available. This behavior changed with KIP-320 since having the leader epoch allows the client to filter out partition metadata which is known to be stale. However, because of this, we can no longer rely on the request-level guarantee of leader availability. There is no mechanism similar to the leader epoch to track the staleness of broker metadata, so we still overwrite all of the broker metadata from each response, which means that the partition metadata can get out of sync with the broker metadata in the client's cache. Hence it is no longer safe to validate inside the `Cluster` constructor that each leader has an associated `Node`
Fixing this issue was unfortunately not straightforward because the cache was built to maintain references to broker metadata through the `Node` object at the partition level. In order to keep the state consistent, each `Node` reference would need to be updated based on the new broker metadata. Instead of doing that, this patch changes the cache so that it is structured more closely with the Metadata response schema. Broker node information is maintained at the top level in a single collection and cached partition metadata only references the id of the broker. To accommodate this, we have removed `PartitionInfoAndEpoch` and we have altered `MetadataResponse.PartitionMetadata` to eliminate its `Node` references.
Note that one of the side benefits of the refactor here is that we virtually eliminate one of the hotspots in Metadata request handling in `MetadataCache.getEndpoints` (which was renamed to `maybeFilterAliveReplicas`). The only reason this was expensive was because we had to build a new collection for the `Node` representations of each of the replica lists. This information was doomed to just get discarded on serialization, so the whole effort was wasteful. Now, we work with the lower level id lists and no copy of the replicas is needed (at least for all versions other than 0).
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>
When a follower's fetch offset is behind the leader's log start offset, the
follower will do a full log truncation. When it does so, it must update both
its log start offset and high watermark. The previous code did the former,
but not the latter. Failure to update the high watermark in this case can lead
to out of range errors if the follower becomes leader before getting the latest
high watermark from the previous leader. The out of range errors occur when
we attempt to resolve the log position of the high watermark in DelayedFetch
in order to determine if a fetch is satisfied.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Also add support for flexible versions to both protocol types.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Colin Patrick McCabe <cmccabe@apache.org>
Co-authored-by: Rajini Sivaram <rajinisivaram@googlemail.com>
Co-authored-by: Jason Gustafson <jason@confluent.io>
Refactors CheckpointFile such that buffers can be read in lieu of files. This
is a relatively simple refactoring as we already create a buffered reader over
the checkpoint file.
#6742, which improves the performance of the checkpointing code, requires
a similar refactoring of code, although it does go further than this.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
This PR implements the KIP-559: https://cwiki.apache.org/confluence/display/KAFKA/KIP-559%3A+Make+the+Kafka+Protocol+Friendlier+with+L7+Proxies
- it adds the Protocol Type and the Protocol Name fields in JoinGroup and SyncGroup API;
- it validates that the fields are provided by the client when the new version of the API is used and ensure that they are consistent. it errors out otherwise;
- it validates that the fields are consistent in the client and errors out otherwise;
- it adds many tests related to the API changes but also extends the testing coverage of the requests/responses themselves.
- it standardises the naming in the coordinator. now, `ProtocolType` and `ProtocolName` are used across the board in the coordinator instead of having a mix of protocol type, protocol name, subprotocol, protocol, etc.
Reviewers: Jason Gustafson <jason@confluent.io>
As the feature freeze approaches, we should support `2.5` as the inter.broker.protocol.version value. There are no new APIs so far, so `2.5` is effectively equivalent to `2.4`.
This PR implements `default.api.timeout.ms` as documented by KIP-533. This is a rebased version of #6913 with some additional test cases and small cleanups.
Reviewers: David Arthur <mumrah@gmail.com>
Co-authored-by: huxi <huxi_2b@hotmail.com>
Also: Improve error message, Add test, Minor code quality fixes
Verified that the test fails if the broker default for max message bytes is lower or higher than the currently set value.
Reviewers: Andrew Choi <andchoi@linkedin.com>, Viktor Somogyi <viktorsomogyi@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Currently, when a dynamic change is made to the broker-level default log configuration, existing log configs will be recreated with an empty overridden configs. In such case, when updating dynamic broker configs a second round, the topic-level configs are lost. This can cause unexpected data loss, for example, if the cleanup policy changes from "compact" to "delete."
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Jason Gustafson <jason@confluent.io>
This patch adds a new API to the producer to implement transactional offset commit fencing through the group coordinator as proposed in KIP-447. This PR mainly changes on the Producer end for compatible paths to old `sendOffsetsToTxn(offsets, groupId)` vs new `sendOffsetsToTxn(offsets, groupMetadata)`.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Since the leader epoch was not maintained in the fetch session cache, no validation would be done except for the initial (full) fetch request. This patch adds the leader epoch to the session cache and addresses the testing gaps.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Colin Patrick McCabe <cmccabe@apache.org>
The `KafkaController::replicasAreValid` method currently returns a
boolean indicating if replicas are valid or not. But the failure
condition loses any context on why replicas are not valid. This change
updates the metod to return the error conition if validation fails. This
allows caller to report the error to the client.
The change also renames the `replicasAreValid` method to
`validateReplicas` to reflect updated semantics.
Reviewers: Sean Li <seanli-rallyhealth@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
Fixed an intermittent failure in the test that was caused by a mocked instance not handling expandIsr. Also updated the test to use the offset from the batch to trigger expandIsr more often. With this change, the test may try to acquire write lock while appends are blocked on read lock, so separated out the test using write lock to make it safer.
Reviewers: Jason Gustafson <jason@confluent.io>
Removes references to the old scala Acl classes from kafka.security.auth (Acl, Operation, ResourceType, Resource etc.) and replaces these with the Java API. Only the old SimpleAclAuthorizer, AuthorizerWrapper used to wrap legacy authorizer instances and tests using SimpleAclAuthorizer continue to use the old API. Deprecates the old scala API.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
During a reassignment, it can happen that the current leader of a partition is demoted and removed from the replica set at the same time. In this case, we rely on the StopReplica request in order to stop replica fetchers and to clear the group coordinator cache. This patch adds similar logic to ensure that the transaction coordinator state cache also gets cleared.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
To be able to correctly fence zombie producer txn commit, we propose to add (member.id, group.instance.id, generation) into the transaction commit protocol to raise the same level of correctness guarantee as consumer commit.
Major changes involve:
1. Upgrade transaction commit protocol with (member.id, group.instance.id, generation). The client will fail if the broker is not supporting the new protocol.
2. Refactor group coordinator logic to handle new txn commit errors such as FENCED_INSTANCE_ID, UNKNOWN_MEMBER_ID and ILLEGAL_GENERATION. We loose the check on transaction commit when the member.id is set to empty. This is because the member.id check is an add-on safety for producer commit, and we also need to consider backward compatibility for old producer clients without member.id information. And if producer equips with group.instance.id, then it must provide a valid member.id (not empty definitely), the same as a consumer commit.
Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This fix makes the LogCleaner tolerant of gaps in the offset sequence. Previously, this could lead to endless loops of cleaning which required manual intervention.
Reviewers: Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>
Check for ISR updates using ISR read lock and acquire ISR write lock only if ISR needs to be updated. This avoids lock contention between request handler threads processing log appends on the leader holding the ISR read lock and request handler threads processing replica fetch requests that check/update ISR.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Let consumer back-off and retry offset fetch when the specific offset topic has pending commits.
The major change lies in the broker side offset fetch logic, where a request configured with flag WaitTransaction to true will be required to back-off when some pending transactional commit is ongoing. This prevents any ongoing transaction being modified by third party, thus guaranteeing the correctness with input partition writer shuffling.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Prior to this commit, broker's Selector used to read all requests available on the socket when the socket is ready for read. These are queued up as staged receives. This can result in excessive memory usage when socket read buffer size is large. We mute the channel and stop reading any more data until all the staged requests are processed. This behaviour is slightly inconsistent since for the initial read we drain the socket buffer, allowing it to get filled up again, but if data arrives slighly after the initial read, then we dont read from the socket buffer until pending requests are processed.
To avoid holding onto requests for longer than required, this commit removes staged receives and reads one request at a time even if more data is available in the socket buffer. This is especially useful for produce requests which may be large and may take long to process. Additional data read from the socket for SSL is limited to the pre-allocated intermediate SSL buffers.
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
When `kafka-topics.sh` is used with the --zookeeper option, it prints a confirmation message. This patch adds the same message when --bootstrap-server is used.
Reviewers: Jason Gustafson <jason@confluent.io>
Changed `CreatePartitionMetadata` to not include partition assignment
information since this is not needed to generate a
`CreatePartitionResponse` message.
Reviewers: Vikas Singh <vikas@confluent.io>, Jason Gustafson <jason@confluent.io>
The method updatePartitionReplicaAssignment allows the user to directly
modify the replicas assigned to a partition without also modifying the
addingReplicas and removingReplicas fields. This is not a safe operation
for all inputs. Clients should instead use updatePartitionFullReplicaAssignment.
Reviewers: Vikas Singh <vikas@confluent.io>, Jason Gustafson <jason@confluent.io>
The test blocks requests threads while sending the ACL update requests and occasionally hits request timeout. Updated the test to tolerate timeouts and retry the request for that case. Added an additional check to verify that the requests threads are unblocked when the semaphore is released, ensuring that the timeout is not due to blocked threads.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Not wait until updateAssignmentMetadataIfNeeded returns true, but only call it once with 0 timeout. Also do not return empty if in rebalance.
Trim the pre-fetched records after long polling since assignment may have been changed.
Also need to update SubscriptionState to retain the state in assignFromSubscribed if it already exists (similar to assignFromUser), so that we do not need the transition of INITIALIZING to FETCHING.
Unit test: this actually took me the most time :)
Reviewers: John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Sophie Blee-Goldman <sophie@confluent.io>, Jason Gustafson <jason@confluent.io>, Richard Yu <yohan.richard.yu@gmail.com>, dengziming <dengziming1993@gmail.com>
Existing producer state expiration uses timestamps from data records only and not from transaction markers. This can cause premature producer expiration when the coordinator times out a transaction because we drop the state from existing batches. This in turn can allow the coordinator epoch to revert to a previous value, which can lead to validation failures during log recovery. This patch fixes the problem by also leveraging the timestamp from transaction markers.
We also change the validation logic so that coordinator epoch is verified only for new marker appends. When replicating from the leader and when recovering the log, we only log a warning if we notice that the coordinator epoch has gone backwards. This allows recovery from previous occurrences of this bug.
Finally, this patch fixes one minor issue when loading producer state from the snapshot file. When the only record for a given producer is a control record, the "last offset" field will be set to -1 in the snapshot. We should check for this case when loading to be sure we recover the state consistently.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
With the old SimpleAclAuthorizer, we were handling delete filters that matched a single resource by looking up that resource directly, even if it wasn't in the cache. AclAuthorizerTest.testHighConcurrencyDeletionOfResourceAcls relies on this behaviour and fails intermittently when the cache is not up-to-date. This PR includes the resource from non-matching filters even if it is not in the cache to retain the old behaviour.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Also document the parameters of `ReplicaAssignment` in `ControllerContext`.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
This change updates the CreatePartitions request and response api objects
to use the generated protocol classes.
Reviewers: Jason Gustafson <jason@confluent.io>
Since `retry` expects a `by name` parameter, passing a nullary function
causes assertion failures to be ignored. I verified this by introducing
a failing assertion before/after the fix.
I checked other `retry` usages and they looked correct.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
The current coordinator loading logic causes an infinite loop when there is a gap between the last record in the log and the log end offset. This is possible because of compaction if the active segment is empty. The patch fixes the problem by breaking from the loading loop when a read from the log returns no additional data.
Reviewers: Jason Gustafson <jason@confluent.io>
The log context is useful when debugging applications which have multiple clients. This patch propagates the context to the channel builders and the SASL authenticator.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
* Adjust build and documentation.
* Use lambda syntax for SAM types in `core`, `streams-scala` and
`connect-runtime` modules.
* Remove `runnable` and `newThread` from `CoreUtils` as lambda
syntax for SAM types make them unnecessary.
* Remove stale comment in `FunctionsCompatConversions`,
`KGroupedStream`, `KGroupedTable' and `KStream` about Scala 2.11,
the conversions are needed for Scala 2.12 too.
* Deprecate `org.apache.kafka.streams.scala.kstream.Suppressed`
and use `org.apache.kafka.streams.kstream.Suppressed` instead.
* Use `Admin.create` instead of `AdminClient.create`. Static methods
in Java interfaces can be invoked since Scala 2.12. I noticed that
MirrorMaker 2 uses `AdminClient.create`, but I did not change them
as Connectors have restrictions on newer client APIs.
* Improve efficiency in a few `Gauge` implementations by avoiding
unnecessary intermediate collections.
* Remove pointless `Option.apply` in `ZookeeperClient`
`SessionState` metric.
* Fix unused import/variable and other compiler warnings.
* Reduce visibility of some vals/defs.
Reviewers: Manikumar Reddy <manikumar@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Gwen Shapira <gwen@confluent.io>
This patch fixes a brittle expectation on the `toString` implementation coming from `Set`. This was failing on jenkins with the following error:
```
java.lang.AssertionError: expected:<Some(producerId:1334,producerEpoch:0,state=Ongoing,partitions=Set(topic-0),txnLastUpdateTimestamp=0,txnTimeoutMs=1000)> but was:<Some(producerId:1334,producerEpoch:0,state=Ongoing,partitions=HashSet(topic-0),txnLastUpdateTimestamp=0,txnTimeoutMs=1000)>
```
Instead we convert the collection to a string directly.
Reviewers: Boyang Chen <boyang@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
* Warnings
1. `kafka/core/src/test/scala/integration/kafka/server/DelayedFetchTest.scala:110: local val partition in method testReplicaNotAvailable is never used`
2. `kafka/core/src/test/scala/unit/kafka/admin/ConfigCommandTest.scala:527: local val alterResourceName in method verifyAlterBrokerConfig is never used`
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Fixes an NPE when UserData in a member's subscription is null and adds similar checks for transaction log parser. Also modifies the output logic so that we show the keys of tombstones for both group and transaction metadata.
Reviewers: David Arthur <mumrah@gmail.com>
This patch removes a spammy log message in the controller which is printed every time the leader imbalance ratio is checked. It is unhelpful because preferred leaders are generally deterministic and is spammy because it includes _every_ partition in the cluster.
Reviewers: Jonathan Santilli <jonathansantilli@users.noreply.github.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
The Deserializer interface has two methods, one that gives access to the headers and one that does not. ConsoleConsumer.scala only calls the latter method. It would be nice if it were to call the default method that provides header access, so that custom serde that depends on headers becomes possible.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
The v3 JoinGroup logic does not properly complete the initial heartbeat for new members, which then expires after the static 5 minute timeout if the member does not rejoin. The core issue is in the `shouldKeepAlive` method, which returns false when it should return true because of an inconsistent timeout check.
Reviewers: Jason Gustafson <jason@confluent.io>