Fixes the LICENSE files that we ship with our releases:
* the source-distribution license included wrong and unnecessary dependencies
* the binary-distribution license was missing most of our actual dependencies
Reviewers: A. Sophie Blee-Goldman <ableegoldman@apache.org>, Ewen Cheslack-Postava <ewencp@apache.org>, Justin Mclean <jmclean@apache.org>
Implement controller-side snapshot generation.Implement QuorumController snapshot
generation. Note that this PR does not handle KRaft integration, just the internal
snapshot record generation and consumption logic.
Reading a snapshot is relatively straightforward. When the QuorumController
starts up, it loads the most recent snapshot. This is just a series of records
that we replay, plus a log offset ("snapshot epoch") that we advance to.
Writing a snapshot is more complex. There are several components:
the SnapshotWriter which persists the snapshot, the SnapshotGenerator
which manages writing each batch of records, and the SnapshotGeneratorManager
which interfaces the preceding two classes with the event queue.
Controller snapshots are done incrementally. In order to avoid blocking the
controller thread for a long time, we pull a few record batches at a time from
our record batch iterators. These iterators are implemented by controller
manager classes such as ReplicationControlManager, ClusterControlManager, etc.
Finally, this PR adds ControllerTestUtils#deepSortRecords and
ControllerTestUtils#assertBatchIteratorContains, which make it easier to write
unit tests. Since records are often constructed from unsorted data structures,
it is often useful to sort them before comparing them.
Reviewers: David Arthur <mumrah@gmail.com>
It is useful to allow ExponentialBackoff to be configured to work
without jitter, in order to make unit tests more repeatable.
Reviewers: David Arthur <mumrah@gmail.com>
* Remove `ExtendedSerializer` and `ExtendedDeserializer`, deprecated since 2.1.
The extra functionality was also made available in `Serializer` and `Deserializer`.
* Remove `close(long, TimeUnit)` from the producer, consumer and admin client,
deprecated since 2.0 for the consumer and 2.2 for the rest. The replacement is `close(Duration)`.
* Remove `ConsumerConfig.addDeserializerToConfig` and `ProducerConfig.addSerializerToConfig`,
deprecated since 2.7 with no replacement. These methods were not intended to be public API
and are likely not used much (if at all).
* Remove `NoOffsetForPartitionException.partition()`, deprecated since 0.11. `partitions()`
should be used instead.
* Remove `MessageFormatter.init(Properties)`, deprecated since 2.7. The `configure(Map)`
method should be used instead.
* Remove `kafka.common.MessageFormatter`, deprecated since 2.7.
`org.apache.kafka.common.MessageFormatter` should be used instead.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
The `node` block achieves that.
Tested that an email was sent to the mailing list for:
592a0c31d5
Added back the condition not to send emails for PR builds after
such test.
Reviewers: Gwen Shapira <cshapi@gmail.com>
When auto-creating topics in KIP-500, the broker will send a `CreateTopics` request to the controller. It is useful in this case to preserve the original principal from the corresponding `Metadata` request by wrapping the `CreateTopics` request in an envelope so that the controller may repeat the authorization and to improve auditability. This follows a similar pattern to how standard `CreateTopics` requests are forwarded to the controller.
Reviewers: Jason Gustafson <jason@confluent.io>
KIP-467 added a field in the produce response to allow the broker to indicate which specific records failed validation. This patch adds the logic to propagate this message up to the application.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
1. Add `canGrantVote` to `EpochState`
2. Move the if-else in `KafkaRaftCllient.handleVoteRequest` to `EpochState`
3. Add unit tests for `canGrantVote`
Reviewers: Jason Gustafson <jason@confluent.io>
* Run all JDK/Scala version combinations for trunk/release branch builds.
* Only retry failures in PR builds for now (we can remove this distinction if/when
we report flaky failures as described in KAFKA-12216).
* Disable concurrent builds
* Send email to dev list on build failure
* Use triple double quotes in `doValidation` since we use string interpolation
for `SCALA_VERSION`.
* Update release.py to output new `Unit/integration tests` Jenkins link
Reviewers: Gwen Shapira <cshapi@gmail.com>, David Arthur <mumrah@gmail.com>
The second `clearCompletedSends()` reference should be `clearCompletedReceives()`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Zhao Haiyuan <zhaohaiyuan@mobike.com>
These were deprecated in Apache Kafka 2.4 (released in December 2019) to be replaced
by `org.apache.kafka.server.authorizer.Authorizer` and `AclAuthorizer`.
As part of KIP-500, we will implement a new `Authorizer` implementation that relies
on a topic (potentially a KRaft topic) instead of `ZooKeeper`, so we should take the chance
to remove related tech debt in 3.0.
Details on the issues affecting the old Authorizer interface can be found in the KIP:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
Kafka does not call fsync() on the directory when a new log segment is created and flushed to disk.
The problem is that following sequence of calls doesn't guarantee file durability:
fd = open("log", O_RDWR | O_CREATE); // suppose open creates "log"
write(fd);
fsync(fd);
If the system crashes after fsync() but before the parent directory has been flushed to disk, the log file can disappear.
This PR is to flush the directory when flush() is called for the first time.
Reviewers: Jun Rao <junrao@gmail.com>
Support sorting the elements in ImplicitLinkedHashCollection.
This is useful sometimes in unit tests for comparing collections.
Reviewers: Ismael Juma <ismael@juma.me.uk>
KIP-516 introduced partition.metadata file to persist the topic ID on the broker. It is created through handling the LeaderAndIsrRequest in ReplicaManager. (See https://github.com/apache/kafka/pull/10143 for the code path.) RaftReplicaManager was missing the analogue code path for Kip-500 code. Like in ReplicaManager, RaftReplicaManager will now check the partition.metadata file when handling metadata records.
However, since we know that all raft topics will have topic IDs, we can simply set the ID in the log upon the log's creation.
Updated the ReplicaManager path to do the same on newly created topics.
There are also some tweaks to the checking logic to better handle the scenario when the log exists but is not yet associated to Partition (for example, upon startup after a shutdown).
Tests added to ensure the file is created and that the correct error is thrown when the id is inconsistent.
Added tests for creating the log with the new topic ID parameter.
Also adds a few methods to get topic ID from MetadataImageBuilder as this is the most convenient way to get topic ID from RaftReplicaManager.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Jason Gustafson <jason@confluent.io>
When a `@Property` tests fail, jqwik helpfully reports the initial seed that resulted in the failure. For example, if we are executing a test scenario 100 times and it fails on the 51st run, then we will get the initial seed that generated . But if you specify the seed in the `@Property` annotation as the previous comment suggested, then the test still needs to run 50 times before we get to the 51st case, which makes debugging very difficult given the complex nature of the simulation tests. Jqwik also gives us the specific argument list that failed, but that is not very helpful at the moment since `Random` does not have a useful `toString` which indicates the initial seed.
To address these problems, I've changed the `@Property` methods to take the random seed as an argument directly so that it is displayed clearly in the output of a failure. I've also updated the documentation to clarify how to reproduce failures.
Reviewers: David Jacot <djacot@confluent.io>
If a distributed worker fails to write (or read back) a new session key to/from the config topic, it dies. This fix softens the blow a bit by instead restarting the herder tick loop anew and forcing a read to the end of the config topic until the worker is able to successfully read to the end.
At this point, if the worker was able to successfully write a new session key in its first attempt, it will have read that key back from the config topic and will not write a new key during the next tick iteration. If it was not able to write that key at all, it will try again to write a new key (if it is still the leader).
Verified with new unit tests for both cases (failure to write, failure to read back after write).
Author: Chris Egerton <chrise@confluent.io>
Reviewers: Greg Harris <gregh@confluent.io>, Randall Hauch <rhauch@gmail.com>
* Standardize license headers in scala, python, and gradle files.
* Relocate copyright attribution to the NOTICE.
* Add a license header check to `spotless` for scala files.
Reviewers: Ewen Cheslack-Postava <ewencp@apache.org>, Matthias J. Sax <mjsax@apache.org>, A. Sophie Blee-Goldman <ableegoldman@apache.org
The config has been deprecated since Kafka 2.6 (released ~1 year before
3.0), but it was the default before it got deprecated. As such, it's
reasonably unlikely that people would have set it explicitly.
Given the confusing `default` name even though it's _not_ the default, I
think we should remove it in 3.0.
Also remove `ClientDnsLookup.DEFAULT` (not public API), which unlocks
a number of code simplications.
Reviewers: David Jacot <djacot@confluent.io>
Deprecated since 1.0.0 for misleading name. Not a public API technically,
but we were conservative originally.
Reviewers: David Jacot <djacot@confluent.io>
This PR is a precursor to the recovery logic refactor work (KAFKA-12553).
I've renamed the file: core/src/test/scala/unit/kafka/log/LogUtils.scala to core/src/test/scala/unit/kafka/log/LogTestUtils.scala. Also I've renamed the underlying lass from LogUtils to LogTestUtils. This is going to help avoid a naming conflict with a new file called LogUtils.scala that I plan to introduce in core/src/main/scala/kafka/log/ as part of the recovery logic refactor. The new file will also contain a bunch of static functions.
Tests:
Relying on existing tests to catch regressions (if any) since this is a simple change.
Reviewers: Satish Duggana <satishd@apache.org>, Dhruvil Shah <dhruvil@confluent.io>, Jun Rao <junrao@gmail.com>
This PR is a precursor to the recovery logic refactor work (KAFKA-12553).
I have made a change to eliminate Log.isLogDirOffline attribute. This boolean also comes in the way of refactoring the recovery logic. This attribute was added in #9676. But it is redundant and can be eliminated in favor of looking up LogDirFailureChannel to check if the logDir is offline. The performance/latency implication of such a ConcurrentHashMap lookup inside LogDirFailureChannel should be very low given that ConcurrentHashMap reads are usually lock free.
Tests:
Relying on existing unit/integration tests.
Reviewers: Dhruvil Shah <dhruvil@confluent.io>, Jun Rao <junrao@gmail.com>
The `kafka-preferred-replica-election` command was deprecated in 2.4. This path removes it for 3.0. `kafka-leader-election` can be used instead.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
Replaced File with Path in LogSegment Data.
This is a followup of #10173
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
ConfigEntry's public constructor was deprecated in 1.1.0. This patch removes it in AK 3.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
`Admin.electLeaders` is the replacement since the deprecation in Apache Kafka 2.4.0.
The methods were originally introduced in Apache Kafka 2.2.0, so they were only
non deprecated for two releases.
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
More specifically, remove deprecated:
- Constants in SslConfigs
- Constants in SaslConfigs
- AclBinding constructor
- AclBindingFilter constructor
- PrincipalBuilder and DefaultPrincipalBuilder classes
- ResourceFilter
Also simplify tests and code that no longer have to handle the removed `PrincipalBuilder`.
These removals seem non controversial. There is a straightforward alternative. The
deprecations happened in 1.0.0 and 2.0.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Minor followup to #10407 -- we need to extract the rebalanceInProgress check down into the commitAndFillInConsumedOffsetsAndMetadataPerTaskMap method which is invoked during handleCorrupted, otherwise we may attempt to commit during a a rebalance which will fail
Reviewers: Matthias J. Sax <mjsax@confluent.io>
This PR is a precursor to the recovery logic refactor work (KAFKA-12553).
Problems:
For refactoring the recovery logic (KAFKA-12553), we would like to move the logic to initialize LeaderEpochFileCache out of the Log class and into a separate static function. In the future, once we successfully initialize LeaderEpochFileCache outside Log, we will be able pass it as a dependency into both the Log recovery module and Log class constructor. However, currently the LeaderEpochFileCache constructor takes a dependency on logEndOffset (via a callback), which poses the following problems:
Blocks the instantiation of LeaderEpochFileCache outside Log class. Because, outside Log the logEndOffset is unavailable to be passed into LeaderEpochFileCache constructor. As a result, this situation blocks the recovery logic (KAFKA-12553) refactor work.
It turns out the logEndOffset dependency is used only in 1 of the LeaderEpochFileCache methods: LeaderEpochFileCache.endOffsetFor, and just for 1 particular case. Therefore, it is overkill to pass it in the constructor as a dependency. Also a callback is generally not a neat way to access dependencies and it poses code readability problems too.
Solution:
This PR modifies the code such that we only pass the logEndOffset as a parameter into LeaderEpochFileCache.endOffsetFor whenever the method is called, thus eliminating the constructor dependency. This will also unblock the recovery logic refactor work (KAFKA-12553).
Tests:
I have modified the existing tests to suit the above refactor.
Reviewers: Dhruvil Shah <dhruvil@confluent.io>, Jun Rao <junrao@gmail.com>
The filesystem locks don't protect access between StreamThreads, only across different instances of the same Streams application. Running multiple processes in the same physical state directory is not supported, and as of PR #9978 it's explicitly guarded against), so there's no reason to continue locking the task directories with anything heavier than an in-memory map.
Reviewers: Rohan Desai <rodesai@confluent.io>, Walker Carlson <wcarlson@confluent.io>, Guozhang Wang <guozhang@confluent.io>
This patch fixes a race condition between the background thread calling `ready` and the call to `MockTime.sleep` in the test. If the call to `sleep` happens first, then the test hangs. I fixed it by giving `MockClient` a way to listen to `ready` calls. This combined with a latch fixes the race.
This patch also fixes a similar race condition in `testClientSideTimeoutAfterFailureToReceiveResponse`. After the disconnect, there is a race between the background thread sending the retry request and the foreground sleeping for the needed backoff delay.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, David Arthur <mumrah@gmail.com>
This PR is a precursor to the recovery logic refactor work (KAFKA-12553).
In this PR, I've extracted the behavior surrounding segments map access within kafka.log.Log class into a new class: kafka.log.LogSegments. This class encapsulates a thread-safe navigable map of kafka.log.LogSegment instances and provides the required read and write behavior on the map. The Log class now encapsulates an instance of the LogSegments class.
Couple advantages of this PR:
Makes the Log class a bit more modular as it moves out certain private behavior thats otherwise within the Log class.
This is a precursor to refactoring the recovery logic (KAFKA-12553). In the future, the logic for recovery and loading of segments from disk (during Log) init will reside outside the Log class. Such logic would need to instantiate and access an instance of the newly added LogSegments class.
Tests:
Added a new test suite: kafka.log.LogSegmentsTest covering the APIs of the newly introduced class.
Reviewers: Satish Duggana <satishd@apache.org>, Dhruvil Shah <dhruvil@confluent.io>, Jun Rao <junrao@gmail.com>
Fix a hanging test in KafkaAdminClientTest by forcing the admin client to shut down
whether or not there are pending requests once the test harness enters shutdown.
Reviewers: Ismael Juma <ijuma@apache.org>, Guozhang Wang <guozhang@apache.org>
`Self-managed` is also used in the context of Cloud vs on-prem and it can
be confusing.
`KRaft` is a cute combination of `Kafka Raft` and it's pronounced like `craft`
(as in `craftsmanship`).
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jose Sancio <jsancio@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
Remove the default close implementation for RocksDBConfigSetter to avoid accidental memory leaks via C++ backed objects which are constructed but not closed by the user
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Need to handle TaskCorruptedException and TimeoutException that can be thrown from offset commit during handleRevocation or handleCorruption
Reviewers: Matthias J. Sax <mjsax@confluent.org>, Guozhang Wang <guozhang@confluent.io>