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>
Similar to KAFKA-5258 which move all partition and replica state transition rules into their states, we move the group state transition rules into their states.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Allow transaction metadata to be reloaded, even if it already exists as of a previous epoch. This helps with cases where a previous become-follower transition failed to unload corresponding metadata.
Reviewers: Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>
Allow ConfigCommand to handle more operations without using direct ZooKeeper access, as described by KIP-543. Also allow specifying entity type and name via a single flag-- again, as the KIP describes.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Adds support for TLSv1.3 in SslTransportLayer. Note that TLSv1.3 is only enabled from Java 11 onwards, so we test the code only when running with Java11 and above.
Tests run on this PR:
- SslTransportLayerTest: This covers testing of our SslTransportLayer and all tests are run with TLSv1.3 when running with Java 11. These tests are also run with TLSv1.2 for all Java versions.
- SslFactoryTest: Also run with TLSv1.3 on Java 11 onwards in addition to TLSv1.2 for all Java versions.
- SslEndToEndAuthorizationTest - Run only with TLSv1.3 on Java 11 onwards and only with TLSv1.2 on earlier Java versions. We have other versions of this test which use SSL that continue to be with TLSv1.2 on Java 11 to avoid reducing test coverage for TLSv1.2
Additional testing for done for TLSv1.3:
- Most tests that use SSL use TestSslUtils.DEFAULT_TLS_PROTOCOL_FOR_TESTS which is set to TLSv1.2. I have run all clients and core tests with DEFAULT_TLS_PROTOCOL_FOR_TESTS=TLSv1.3 with Java 11.
- Ran a few system tests locally with TKSv1.3
Reviewers: Ismael Juma <ismael@juma.me.uk>, Manikumar Reddy <manikumar.reddy@gmail.com>
The replication throttle in `testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress` was not setting the quota on the partition correctly, so the test case was not working as expected. This patch fixes the problem and also fixes a bug in `waitForTopicCreated` which caused the function to always wait for the full timeout.
Reviewers: Ismael Juma <ismael@juma.me.uk>
The create topic api do not work with older version of the api. It can be reproduced by trying to create a topic with kafka-topics.sh from 2.3. It timeouts.
b94c7f4 has added a check which raises an exception if a field has been set to a non-default value unless the field is marked as "ignorable".
The fields added in the version 5 of the response are always set regardless of the version used by the client. If an older version is used, an exception is thrown during the serialization because the fields have non-default values. We should either not set the fields for older versions in the api layer or mark them as ignorable. I have chosen the later in this case because it looks cleaner.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>, Mickael Maison <mickael.maison@gmail.com>
Collect and expose the KIP-511 client name and version information the clients now provide to the server as part of ApiVersionsRequests. Also refactor how we handle selector metrics by creating a ChannelMetadataRegistry class. This will make it easier for various parts of the networking code to modify channel metrics.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
https://issues.apache.org/jira/browse/KAFKA-9025
If a chroot is configured, ZkSecurityMigrator should prompt a confirm to user to ensure whether chroot is specified correctly.
Author: huxihx <huxi_2b@hotmail.com>
Author: huxi <huxi_2b@hotmail.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#7618 from huxihx/KAFKA-9025
Brokers are supposed to force SASL clients to re-authenticate (and kill such connections in the absence of a timely and successful re-authentication) when KIP-368 SASL Re-Authentication is enabled via a positive connections.max.reauth.ms configuration value. There was a flaw in the logic that caused connections to not be killed in the absence of a timely and successful re-authentication if the client did not leverage the SaslAuthenticateRequest API (which was defined in KIP-152).
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
KIP-320 improved fetch semantics by adding leader epoch validation. This relies on
reliable propagation of leader epoch information from the controller. Unfortunately, we
have encountered a bug during partition reassignment in which the leader epoch in the
controller context does not get properly updated. This causes UpdateMetadata requests
to be sent with stale epoch information which results in the metadata caches on the
brokers falling out of sync.
This bug has existed for a long time, but it is only a problem due to the new epoch
validation done by the client. Because the client includes the stale leader epoch in its
requests, the leader rejects them, yet the stale metadata cache on the brokers prevents
the consumer from getting the latest epoch. Hence the consumer cannot make progress
while a reassignment is ongoing.
Although it is straightforward to fix this problem in the controller for the new releases
(which this patch does), it is not so easy to fix older brokers which means new clients
could still encounter brokers with this bug. To address this problem, this patch also
modifies the client to treat the leader epoch returned from the Metadata response as
"unreliable" if it comes from an older version of the protocol. The client in this case will
discard the returned epoch and it won't be included in any requests.
Also, note that the correct epoch is still forwarded to replicas correctly in the
LeaderAndIsr request, so this bug does not affect replication.
Reviewers: Jun Rao <junrao@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>
[KAFKA-9267](https://issues.apache.org/jira/browse/KAFKA-9267)
ZkSecurityMigrator might create a PERSISTENT /controller node with null data, it will lead to controller can't elect.
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*
Author: NanerLee <nanerlee@qq.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#7778 from NanerLee/fix-ZkSecurityMigrator
This patch fixes a race condition on reassignment completion. The previous code fetched metadata first and then fetched the reassignment state. It is possible in between those times for the reassignment to complete, which leads to spurious URPs being reported. The fix here is to change the order of these checks and to explicitly check for reassignment completion.
Note this patch fixes the flaky test `TopicCommandWithAdminClientTest.testDescribeUnderReplicatedPartitionsWhenReassignmentIsInProgress`.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
This patch eliminates some redundancy and general messiness around the usage of `BaseRequestTest` and specifically response deserialization.
Reviewers: Ismael Juma <ismael@juma.me.uk>
This patch fixes an NPE in `DefaultMetadataUpdater` due to an inconsistency in event expectations. Whenever there is an authentication failure, we were treating it as a failed update even if was from a separate connection from an inflight metadata request. This patch fixes the problem by making the `MetadataUpdater` api clearer in terms of the events that are handled.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
This patch fixes a bug in `SocketServer` in the expiration of connections which have not re-authenticated quickly enough. Previously these connections were left hanging, but now they are properly closed and cleaned up. This was one cause of the flaky test failures in `EndToEndAuthorizationTest.testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl`.
Reviewers: Jason Gustafson<jason@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
KAFKA-8448 fixes problem with similar leak. The Log objects are being
held in ScheduledExecutor PeriodicProducerExpirationCheck callback. The
fix in KAFKA-8448 was to change the policy of ScheduledExecutor to
remove the scheduled task when it gets canceled (by calling
setRemoveOnCancelPolicy(true)).
This works when a log is closed using close() method. But when a log is
deleted either when the topic gets deleted or when the rebalancing
operation moves the replica away from broker, the delete() operation is
invoked. Log.delete() doesn't close the pending scheduled task and that
leaks Log instance.
Fix is to close the scheduled task in the Log.delete() method too.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Race condition in concurrent `get` method invocation of lazy indexes might lead
to multiple `OffsetIndex`/`TimeIndex` objects being concurrently created. When
that happens position of `MappedByteBuffer` in `AbstractIndex` is advanced to
the last entry which in turn leads to a critical `BufferOverflowException` being
thrown whenever leader or replica tries to append a record to the segment.
Moreover, `file_=` setter is seemingly also vulnerable to the race, since multiple
threads can attempt to set a new file reference as well as create new
Time/OffsetIndex objects at the same time. This might lead to the discrepant
File references being held by e.g. LazyTimeIndex and its TimeIndex counterpart.
This patch attempts to fix the issue by making sure that index objects are
atomically constructed when loaded lazily via `get` method. Additionally, `file`
reference modifications are also made atomic and thread safe.
Note that the `Lazy*Index` mutation operations are executed with a lock held by
the callers, but `get` can be called without a lock (e.g. from `Log.read`).
Reviewers: Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>, Shilin Lu, Ismael Juma <ismael@juma.me.uk>
https://issues.apache.org/jira/browse/KAFKA-9069
Make `getTopicMetadata` in AdminClientIntegrationTest always read metadata from controller to get a consistent view.
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*
Author: huxihx <huxi_2b@hotmail.com>
Reviewers: Guozhang Wang <wangguoz@gmail.com>, José Armando García Sancio <jsancio@gmail.com>
Closes#7619 from huxihx/KAFKA-9069
If a field is not marked as ignorable, we should raise an exception if it has been set to a non-default value. This check already exists in `Message.write`, so this patch adds it to `Message.toStruct`. Additionally, we fix several fields which should have been marked ignorable and we fix some related test assertions.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Manikumar Reddy <manikumar.reddy@gmail.com>, Colin Patrick McCabe <cmccabe@apache.org>
This test was taking more than 5 minutes because the producer writes were timing out. The problem was that broker 100 was being shutdown before broker 101, which meant that the partition was still offline after broker 100 was restarted. The producer timeouts were not detected because the produce future was not checked. After the fix, test time drops to about 15s.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Fixes#7717, which did not actually achieve its intended effect. The event manager failed to process the new event because we disabled the rate metric, which it expected to be present.
Reviewers: Ismael Juma <ismael@juma.me.uk
Create a controller event for handling UpdateMetadata responses and log a message when a response contains an error.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>
The latter was previously hardcoding logDirCount instead of using the
method defined in the superclass since it was unnecessarily duplicating
logic.
Also tweak IntegrationTestHarness and remove unnecessary method
override from SaslPlainPlaintextConsumerTest.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Force completion of delayed operations when receiving a StopReplica request. In the case of a partition reassignment, we cannot rely on receiving a LeaderAndIsr request in order to complete these operations because the leader may no longer be a replica. Previously when this happened, the delayed operations were left to eventually timeout.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>
Co-Authored-By: Kun Du <kidkun@users.noreply.github.com>
While investigating KAFKA-9180, I noticed that we had no
unit test coverage. It turns out that the behavior was
correct, so we just fix the test coverage issue.
Also updated .gitignore with jmh-benchmarks/generated.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
When we roll a new segment, the log offset metadata tied to the high watermark may
need to be updated. This is needed when the high watermark is equal to the log end
offset at the time of the roll. Otherwise, we risk exposing uncommitted data early.
Reviewers: Dhruvil Shah <dhruvil@confluent.io>, Ismael Juma <ismael@juma.me.uk>
This patch creates a `BaseAdminIntegrationTest` to be the root for all integration test extensions. Most of the existing tests will only be tested in `PlaintextAdminIntegrationTest`, which extends from `BaseAdminIntegrationTest`. This should cut off about 30 minutes from the overall build time.
Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Fixes `java.util.concurrent.ExecutionException: java.lang.AssertionError: Close finished too quickly 5999`.
The close test sets a close duration in milliseconds, but measures the time taken in nanoseconds. This leads to small error due to the resolution in each, where the close is deemed to have taken too little time.
When I measured the start and end with nanoTime, I found the time taken to close was `5999641566 ns (5999.6ms)` which seems close enough to be a resolution error. I've run the test 50 times and have not hit the "Close finished too quickly" issue again, whereas previously I hit a failure pretty quickly.
Author: Lucas Bradstreet <lucas@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#7683 from lbradstreet/flaky-consumer-bounce-test
This patch implements the broker-side changes for KIP-360. It adds two new fields to InitProducerId: lastEpoch and producerId. Passing these values allows the TransactionCoordinator to safely bump a producer's epoch after some failures (such as UNKNOWN_PRODUCER_ID and INVALID_PRODUCER_ID_MAPPING). When a producer calls InitProducerId after a failure, the coordinator first checks the producer ID from the request to make sure no other producer has been started using the same transactional ID. If it is safe to continue, the coordinator checks the epoch from the request; if it matches the existing epoch, the epoch is bumped and the producer can safely continue. If it matches the previous epoch, the the current epoch is returned without bumping. Otherwise, the producer is fenced.
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
This change fixes a performance regression due to follower last seen highwatermark
handling introduced in 23beeea. maybeUpdateHwAndSendResponse is expensive for
brokers with high partition counts, as it requires a partition and a replica lookup for every
partition being fetched. This refactor moves the last seen watermark update into the follower
fetch state update where we have already looked up the partition and replica.
I've seen cases where maybeUpdateHwAndSendResponse is responsible 8% of CPU usage, not including the responseCallback call that is part of it.
I have benchmarked this change with `UpdateFollowerFetchStateBenchmark` and it adds 5ns
of overhead to Partition.updateFollowerFetchState, which is a rounding error compared to the
current overhead of maybeUpdateHwAndSendResponse.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>