The PR handles slicing of fetched records based on acquire response for
share fetch. There could be additional bytes fetched from log but
acquired offsets can be a subset, typically with `max fetch records`
configuration. Rather sending additional bytes of fetched data to client
we should slice the file and wire only needed batches.
Note: If the acquired offsets are within a batch then we need to send
the entire batch within the file record. Hence rather checking for
individual batches, PR finds the first and last acquired offset, and
trims the file for all batches between (inclusive) these two offsets.
Reviewers: Christo Lolov <lolovc@amazon.com>, Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>
This is no longer required since we dropped support for Java 8. Also update `NOTICE*` and
`spotbugs-exclude.xml` files.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
Part of KIP-1106.
Adds support for "by_duration" and "none" reset strategy
to the Kafka Streams runtime.
Reviewers: Bill Bejeck <bill@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This commit implements the changes for KIP-1032. This updates Kafka to Jakarta specs, JavaEE 10 and Jetty 12. The changes here primarily effect Kafka Connect and MM2.
Todo/Notes:
1) I bumped the connect modules to JDK 17 but I also had to bump a couple other things that had a dependency on conect. The tools project depends on connect so that had to be bumped, and streams depends on tools so that needed to be bumped. This means we may need to separate some things if we don't want to enforce JDK 17 on streams.
2) There is an issue with a test in DedicatedMirrorIntegrationTest that I had to change for now that involves escaping characters and not quite sure what to do about it yet. The cause is the Servlet 6 spec changing what is allowed in the path. See: Jetty 12: 400: Ambiguous URI path encoding for path <%=FOO%>~1 (encoded: %3C%25%3DFOO%25%3E%7E1) jetty/jetty.project#11890
3) I had to configure the idle timeout in Jetty requests to match our request timeout so tests didn't fail. This was needed to fix the ConnectWorkerIntegrationTest#testPollTimeoutExpiry() test
Testing is being done by just using the existing tests for Connect and MM2 which should be sufficient.
Reviewers: Greg Harris <greg.harris@aiven.io>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This is the initial version of the share group consumer client code. It implements the complete ShareConsumer interface.
There are unit tests, but not integration tests yet since those would depend upon complete broker code, which is not available at this point.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Lianet Magrans <lianetmr@gmail.com>
Introduce a dummy node connected to every other node and run Bellman-ford from the dummy node once instead of from every node in the graph.
Reviewers: Qichao Chu (@ex172000), Matthias J. Sax <matthias@confluent.io>
The consumer refactoring project introduced another `Consumer` implementation, creating two different, coexisting implementations of the `Consumer` interface:
* `KafkaConsumer` (AKA "existing", "legacy" consumer)
* `PrototypeAsyncConsumer` (AKA "new", "refactored" consumer)
The goal of this task is to refactor the code via the delegation pattern so that we can keep a top-level `KafkaConsumer` but then delegate to another implementation under the covers. There will be two delegates at first:
* `LegacyKafkaConsumer`
* `AsyncKafkaConsumer`
`LegacyKafkaConsumer` is essentially a renamed `KafkaConsumer`. That implementation handles the existing group protocol. `AsyncKafkaConsumer` is renamed from `PrototypeAsyncConsumer` and will implement the new consumer group protocol from KIP-848. Both of those implementations will live in the `internals` sub-package to discourage their use.
This task is part of the work to implement support for the new KIP-848 consumer group protocol.
Reviewers: Philip Nee <pnee@confluent.io>, Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>
A few notes:
* Delete a few methods from `UnifiedLog` that were simply invoking the related method in `LogFileUtils`
* Fix `CoreUtils.swallow` to use the passed in `logging`
* Fix `LogCleanerParameterizedIntegrationTest` to close `log` before reopening
* Minor tweaks in `LogSegment` for readability
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
Spotbugs was temporarily disabled as part of KAFKA-15485 to support Kafka build with JDK 21. This PR upgrades the spotbugs version to 4.8.0 which adds support for JDK 21 and enables it's usage on build again.
Reviewers: Divij Vaidya <diviv@amazon.com>
Avoid busy waiting for processable tasks. We need to be a bit careful here to not have the task executors to sleep when work is available. We have to make sure to signal on the condition variable any time a task becomes "processable". Here are some situations where a task becomes processable:
- Task is unassigned from another TaskExecutor.
- Task state is changed (should only happen inside when a task is locked inside the polling phase).
- When tasks are unlocked.
- When tasks are added.
- New records available.
- A task is resumed.
So in summary, we
- We should probably lock tasks when they are paused and unlock them when they are resumed. We should also wake the task executors after every polling phase. This belongs to the StreamThread integration work (separate PR). We add DefaultTaskManager.signalProcessableTasks for this.
- We need to awake the task executors in DefaultTaskManager.unassignTask, DefaultTaskManager.unlockTasks and DefaultTaskManager.add.
Reviewers: Walker Carlson <wcarlson@confluent.io>, Bruno Cadonna <cadonna@apache.org>
Motivation
Reading/writing the protocol buffer varInt32 and varInt64 (also called varLong in our code base) is in the hot path of data plane code in Apache Kafka. We read multiple varInt in a record and in long. Hence, even a minor change in performance could extrapolate to larger performance benefit.
In this PR, we only update varInt32 encoding/decoding.
Changes
This change uses loop unrolling and reduces the amount of repetition of calculations. Based on the empirical results from the benchmark, the code has been modified to pick up the best implementation.
Results
Performance has been evaluated using JMH benchmarks on JDK 17.0.6. Various implementations have been added in the benchmark and benchmarking has been done for different sizes of varints and varlongs. The benchmark for various implementations have been added at ByteUtilsBenchmark.java
Reviewers: Ismael Juma <mlists@juma.me.uk>, Luke Chen <showuon@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>
Introduces a new Serde, that serializes a value and timestamp as a single byte array, where the value may be null (in order to represent putting a tombstone with timestamp into the versioned store).
Part of KIP-889.
Reviewers: Matthias J. Sax <matthias@confluent.io>
There were some concurrency inconsistencies in `KafkaScheduler` flagged by spotBugs
that had to be fixed, summary of changes below:
* Executor is `volatile`
* We always synchronize and check `isStarted` as the first thing within the critical
section when a mutating operation is performed.
* We don't synchronize (but ensure the executor is not null in a safe way) in read-only
operations that operate on the executor.
With regards to `MockScheduler/MockTask`:
* Set the type of `nextExecution` to `AtomicLong` and replaced inconsistent synchronization
* Extracted logic into `MockTask.rescheduleIfPeriodic`
Tweaked the `Scheduler` interface a bit:
* Removed `unit` parameter since we always used `ms` except one invocation
* Introduced a couple of `scheduleOnce` overloads to replace the usage of default
arguments in Scala
* Pulled up `resizeThreadPool` to the interface and removed `isStarted` from the
interface.
Other cleanups:
* Removed spotBugs exclusion affecting `kafka.log.LogConfig`, which no longer exists.
For broader context, see:
* KAFKA-14470: Move log layer to storage module
Reviewers: Jun Rao <junrao@gmail.com>
Also improved `LogValidatorTest` to cover a bug that was originally
only caught by `LogAppendTimeTest`.
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: Jun Rao <junrao@gmail.com>
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: Jun Rao <junrao@gmail.com>, Satish Duggana <satishd@apache.org>
* KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package
Move classes into a sub-package of "internal" named "secured" that
matches the layout more closely of the "unsecured" package.
Replaces the concrete implementations in the former packages with
sub-classes of the new package layout and marks them as deprecated. If
anyone is already using the newer OAuth code, this should still work.
* Fix checkstyle and spotbugs violations
Co-authored-by: Kirk True <kirk@mustardgrain.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Implements the major part of the IQv2 framework as proposed in KIP-796.
Reviewers: Patrick Stuedi <pstuedi@apache.org>, Vicky Papavasileiou <vpapavasileiou@confluent.io>, Bruno Cadonnna <cadonna@apache.org>
* Improve the test prior to reimplementing KafkaFutureImpl using CompletableFuture.
* KAFKA-9687: Reimplement KafkaFutureImpl using a CompleteableFuture
* KIP-707: Add KafkaFuture.toCompletionStage
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>, Konstantine Karantasis <k.karantasis@gmail.com>
It's quite verbose to include all configs for every partition loaded/created.
Also make sure to redact sensitive and unknown config values.
Unit test included.
Reviewers: David Jacot <djacot@confluent.io>, Kowshik Prakasam <kprakasam@confluent.io>, Luke Chen <showuon@gmail.com>
We don't really need it and it causes problems in older Android versions
and GraalVM native image usage (there are workarounds for the latter).
Move the logic to separate classes that are only invoked when the
relevant compression library is actually used. Place such classes
in their own package and enforce via checkstyle that only these
classes refer to compression library packages.
To avoid cyclic dependencies, moved `BufferSupplier` to the `utils`
package.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Add KafkaEventQueue, which is used by the KIP-500 controller to manage its event queue.
Compared to using an Executor, KafkaEventQueue has the following advantages:
* Events can be given "deadlines." If an event lingers in the queue beyond the deadline, it
will be completed with a timeout exception. This is useful for implementing timeouts for
controller RPCs.
* Events can be prepended to the queue as well as appended.
* Events can be given tags to make them easier to manage. This is especially useful for
rescheduling or cancelling events which were previously scheduled to execute in the future.
Reviewers: Jun Rao <junrao@gmail.com>, José Armando García Sancio <jsancio@gmail.com>
Also updated the jmh readme to make it easier for new people to know
what's possible and best practices.
There were some changes in the generated benchmarking code that
required adjusting `spotbugs-exclude.xml` and for a `javac` warning
to be suppressed for the benchmarking module. I took the chance
to make the spotbugs exclusion mode maintainable via a regex
pattern.
Tested the commands on Linux and macOS with zsh.
JMH highlights:
* async-profiler integration. Can be used with -prof async,
pass -prof async:help to look for the accepted options.
* perf c2c [2] integration. Can be used with -prof perfc2c,
if available.
* JFR profiler integration. Can be used with -prof jfr, pass
-prof jfr:help to look for the accepted options.
Full details:
* 1.24: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-August/002982.html
* 1.25: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-August/002987.html
* 1.26: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-October/003024.html
* 1.27: https://mail.openjdk.java.net/pipermail/jmh-dev/2020-December/003096.html
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Lucas Bradstreet <lucasbradstreet@gmail.com>
Scala 2.13.4 restores default global `ExecutionContext` to 2.12 behavior
(to fix a perf regression in some use cases) and improves pattern matching
(especially exhaustiveness checking). Most of the changes are related
to the latter as I have enabled the newly introduced `-Xlint:strict-unsealed-patmat`.
More details on the code changes:
* Don't swallow exception in `ReassignPartitionsCommand.topicDescriptionFutureToState`.
* `RequestChannel.Response` should be `sealed`.
* Introduce sealed ClientQuotaManager.BaseUserEntity to avoid false positive
exhaustiveness warning.
* Handle a number of cases where pattern matches were not exhaustive:
either by marking them with @unchecked or by adding a catch-all clause.
* Workaround scalac bug related to exhaustiveness warnings in ZooKeeperClient
* Remove warning suppression annotations related to the optimizer that are no
longer needed in ConsumerGroupCommand and AclAuthorizer.
* Use `forKeyValue` in `AclAuthorizer.acls` as the scala bug preventing us from
using it seems to be fixed.
* Also update scalaCollectionCompat to 2.3.0, which includes minor improvements.
Full release notes:
* https://github.com/scala/scala/releases/tag/v2.13.4
* https://github.com/scala/scala-collection-compat/releases/tag/v2.3.0
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
`forKeyValue` invokes `foreachEntry` in Scala 2.13 and falls back to
`foreach` in Scala 2.12.
This change requires a newer version of scala-collection-compat, so
update it to the latest version (2.2.0).
Finally, included a minor clean-up in `GetOffsetShell` to use `toArray`
before `sortBy` since it's more efficient.
Reviewers: Jason Gustafson <jason@confluent.io>, David Jacot <djacot@confluent.io>, José Armando García Sancio <jsancio@users.noreply.github.com>, Chia-Ping Tsai <chia7712@gmail.com>
Only check if positions need validation if there is new metadata.
Also fix some inefficient java.util.stream code in the hot path of SubscriptionState.
This PR includes 3 MessageFormatters for MirrorMaker2 internal topics:
- HeartbeatFormatter
- CheckpointFormatter
- OffsetSyncFormatter
This also introduces a new public interface org.apache.kafka.common.MessageFormatter that users can implement to build custom formatters.
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, David Jacot <djacot@confluent.io>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Fix findbugs multithreaded correctness warnings for streams, updated variables to be threadsafe
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Boyang Chen <boyang@confluent.io>, John Roesler <vvcephei@apache.org>
In the case described in the JIRA, there was a 50%+ increase in the total fetch request rate in
2.4.0 due to this change.
I included a few additional clean-ups:
* Simplify `findPreferredReadReplica` and avoid unnecessary collection copies.
* Use `LongSupplier` instead of `Supplier<Long>` in `SubscriptionState` to avoid unnecessary boxing.
Added a unit test to ReplicaManagerTest and cleaned up the test class a bit including
consistent usage of Time in MockTimer and other components.
Reviewers: Gwen Shapira <gwen@confluent.io>, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>