The PR implements the BrokerTopicMetrics defined in KIP-1103.
The PR also corrected the share-acknowledgement-rate and share-acknowledgement-count metrics defined in KIP-932 as they are moved to BrokerTopicMetrics, necessary changes to KIP-932 broker metrics will be done once we complete KIP-1103.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
This patch reorganizes our test infrastructure into three Gradle modules:
":test-common:test-common-internal-api" is now a minimal dependency which exposes interfaces and annotations only. It has one project dependency on server-common to expose commonly used data classes (MetadataVersion, Feature, etc). Since this pulls in server-common, this module is Java 17+. It cannot be used by ":clients" or other Java 11 modules.
":test-common:test-common-util" includes the auto-quarantined JUnit extension. The @Flaky annotation has been moved here. Since this module has no project dependencies, we can add it to the Java 11 list so that ":clients" and others can utilize the @Flaky annotation
":test-common:test-common-runtime" now includes all of the test infrastructure code (TestKitNodes, etc). This module carries heavy dependencies (core, etc) and so it should not normally be included as a compile-time dependency.
In addition to this reorganization, this patch leverages JUnit SPI service discovery so that modules can utilize the integration test framework without depending on ":core". This will allow us to start moving integration tests out of core and into the appropriate sub-module. This is done by adding ":test-common:test-common-runtime" as a testRuntimeOnly dependency rather than as a testImplementation dependency. A trivial example was added to QuorumControllerTest to illustrate this.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>
Currently, each log.append() will add at most 1 index entry, even when the appended data is larger than log.index.interval.bytes. One potential issue is that if a follower restarts after being down for a long time, it may fetch data much bigger than log.index.interval.bytes at a time. This means that fewer index entries are created, which can increase the fetch time from the consumers.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
Apache Kafka 4.0 will only support KRaft and 3.0-IV1 is the minimum version supported by KRaft. So, we can assume that Apache Kafka 4.0 will only communicate with brokers that are 3.0-IV1 or newer.
Note that KRaft was only marked as production-ready in 3.3, so we could go further and set the baseline to 3.3. I think we should have that discussion, but it made sense to start with the non controversial parts.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <david.jacot@gmail.com>
Convert v0/v1 record batches to v2 during compaction even if said record batches would be
written with no change otherwise. A few important details:
1. V0 compressed record batch with multiple records is converted into single V2 record batch
2. V0 uncompressed records are converted into single record V2 record batches
3. V0 records are converted to V2 records with timestampType set to `CreateTime` and the
timestamp is `-1`.
4. The `KAFKA-4298` workaround is no longer needed since the conversion to V2 fixes
the issue too.
5. Removed a log warning applicable to consumers older than 0.10.1 - they are no longer
supported.
6. Added back the ability to append records with v0/v1 (for testing only).
7. The creation of the leader epoch cache is no longer optional since the record version
config is effectively always V2.
Add integration tests, these tests existed before #18267 - restored, modified and
extended them.
Reviewers: Jun Rao <jun@confluent.io>
Relevant methods:
1. `List.of`, `Set.of`, `Map.of` and similar (introduced in Java 9)
2. Optional: `isEmpty` (introduced in Java 11), `stream` (introduced in Java 9).
Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
Included in this change:
1. Remove deprecated protocol api versions from json files.
3. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
4. Remove record down-conversion support from KafkaApis.
5. No longer return `Errors.UNSUPPORTED_COMPRESSION_TYPE` on the fetch path[1].
6. Deprecate `TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG` and made the relevant
configs (`message.downconversion.enable` and `log.message.downcoversion.enable`) no-ops since
down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
7. Fix `shouldRetainsBufferReference` to handle null request schemas for a given version.
8. Simplify producer logic since it only supports the v2 record format now.
9. Fix tests so they don't exercise protocol api versions that have been removed.
10. Add upgrade note.
Testing:
1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
3. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
4. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
5. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
6. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
7. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
8. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
9. kcat 1.6.0-1.7.0 produce works fine.
10. kcat 1.7.1 (based on librdkafka 1.8.2) works fine for consumer and produce.
11. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
12. I will test more clients, but I don't think we need to block the PR on that.
Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.
Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.
Note that I did not touch `ignorable` fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields _if_ we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).
[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] ```org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1```
[3]```METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[4]```METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[5] `ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker`
Reviewers: David Arthur <mumrah@gmail.com>
- Disallow configuring -1 for copier and expiration thread pools dynamically
Co-authored-by: Peter Lee <peterxcli@gmail.com>
Reviewers: Peter Lee <peterxcli@gmail.com>, Satish Duggana <satishd@apache.org>
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:
1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations
Co-authored-by: Lee Dongjin <dongjin@apache.org>
Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
- Deprecates OffsetResetStrategy enum
- Adds new internal class AutoOffsetResetStrategy
- Replaces all OffsetResetStrategy enum usages with AutoOffsetResetStrategy
- Deprecate old/Add new constructors to MockConsumer
Reviewers: Andrew Schofield <aschofield@confluent.io>, Matthias J. Sax <matthias@confluent.io>
Remove or update custom display names to make sure we actually include the test method as the first part of the display name.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Bill Bejeck <bill@confluent.io>
- The isReady API in RemoteLogMetadataManager (RLMM) is used to denote whether the partition metadata is ready for remote storage operations. The plugin implementors can use this API to denote the partition status while bootstrapping the RLMM.
- Using this API, we are gracefully starting the remote log components. The segment copy, delete, and other operations that hits remote storage will be invoked once the metadata is ready for a given partition.
- See KIP-1105 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-1105%3A+Make+remote+log+manager+thread-pool+configs+dynamic) for more details.
Reviewers: Federico Valeri <fvaleri@redhat.com>, Satish Duggana <satishd@apache.org>
This patch fixes the valid values generated doc of remote.log.manager.copier.thread.pool.size and remote.log.manager.expiration.thread.pool.size.
Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
According to KIP-950, remote.log.manager.thread.pool.size should be marked as deprecated and replaced by two new configurations: remote.log.manager.copier.thread.pool.size and remote.log.manager.expiration.thread.pool.size. Fix default handling so that -1 works as expected.
Reviewers: Luke Chen <showuon@gmail.com>, Gaurav Narula <gaurav_narula2@apple.com>, Satish Duggana <satishd@apache.org>, Colin P. McCabe <cmccabe@apache.org>
KAFKA-17766: Issue Details: Inside TopicBasedRemoteLogMetadataManager::close, one thread(t1) is calling join on initializationThread thread after taking writeLock on "lock" object => t1 will wait for initializationThread to complete. Internally initializationThread is also using writeLock on "lock" object. This can cause deadlock in below situation
initializationThread is started
close has been invoked as part of a separate thread. But this thread is not yet scheduled by OS.
At line 430, initializationThread is preempted and OS has started running close thread. close takes writeLock and invoked join on initializationThread.
Now OS schedules initializationThread again and at line 433 this thread also tries to take writeLock. But since writeLock is already held by close thread => both are waiting on each other to complete. initializationThread will wait on close to release the writeLock, while close thread will wait for completion of initializationThread
Fix Details: We can avoid taking lock inside close() method as there no operations with any side effects. closing instance variable is of type AtomicBoolean => no race condition when updating it to true.
Co-authored-by: Anshul Goyal <anshul.goyal@broadcom.com>
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
This patch completely removes the compile-time dependency on core for both test and main sources by introducing two new modules.
1) `test-common` include all the common test implementation code (including dependency on :core for BrokerServer, ControllerServer, etc)
2) `test-common:api` new sub-module that just includes interfaces including our junit extension
Reviewers: David Arthur <mumrah@gmail.com>
This is the part-2 of the KIP-1075
To find the offset for a given timestamp, ListOffsets API is used by the client. When the topic is enabled with remote storage, then we have to fetch the remote indexes such as offset-index and time-index to serve the query. Also, the ListOffsets request can contain the query for multiple topics/partitions.
The time taken to read the indexes from remote storage is non-deterministic and the query is handled by the request-handler threads. If there are multiple LIST_OFFSETS queries and most of the request-handler threads are busy in reading the data from remote storage, then the other high-priority requests such as FETCH and PRODUCE might starve and be queued. This can lead to higher latency in producing/consuming messages.
In this patch, we have introduced a delayed operation for remote list-offsets call. If the timestamp need to be searched in the remote-storage, then the request-handler threads will pass-on the request to the remote-log-reader threads. And, the request gets handled in asynchronous fashion.
Covered the patch with unit and integration tests.
Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
In KAFKA-16424, we added a fallback logic to delete the logs, but the file has no parent. It'd be better we have some guard from it.
Signed-off-by: PoAn Yang <payang@apache.org>
Reviewers: Luke Chen <showuon@gmail.com>
Previously in KRaft mode, we could request an unclean leader election for a specific topic using
the electLeaders API. This PR adds an additional way to trigger unclean leader election when in
KRaft mode via the static controller configuration and various dynamic configurations.
In order to support all possible configuration methods, we have to do a multi-step configuration
lookup process:
1. check the dynamic topic configuration for the topic.
2. check the dynamic node configuration.
3. check the dynamic cluster configuration.
4. check the controller's static configuration.
Fortunately, we already have the logic to do this multi-step lookup in KafkaConfigSchema.java.
This PR reuses that logic. It also makes setting a configuration schema in
ConfigurationControlManager mandatory. Previously, it was optional for unit tests.
Of course, the dynamic configuration can change over time, or the active controller can change
to a different one with a different configuration. These changes can make unclean leader
elections possible for partitions that they were not previously possible for. In order to address
this, I added a periodic background task which scans leaderless partitions to check if they are
eligible for an unclean leader election.
Finally, this PR adds the UncleanLeaderElectionsPerSec metric.
Co-authored-by: Luke Chen showuon@gmail.com
Reviewers: Igor Soarez <soarez@apple.com>, Luke Chen <showuon@gmail.com>
1) When the local.retention.ms/bytes is set to -2, we didn't replace it with the server-side retention.ms/bytes config, so the -2 local retention won't take effect.
2) When setting retention.ms/bytes to -2, we can notice this log message:
```
Deleting segment LogSegment(baseOffset=10045, size=1037087, lastModifiedTime=1724040653922, largestRecordTimestamp=1724040653835) due to local log retention size -2 breach. Local log size after deletion will be 13435280. (kafka.log.UnifiedLog) [kafka-scheduler-6]
```
This is not helpful for users. We should replace -2 with real retention value when logging.
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Handle local log deletion when remote.log.copy.disabled=true based on the KIP-950.
When tiered storage is disabled or becomes read-only on a topic, the local retention configuration becomes irrelevant, and all data expiration follows the topic-wide retention configuration exclusively.
- added remoteLogEnabledAndRemoteCopyEnabled method to check if this topic enables tiered storage and remote log copy is enabled. We should adopt local.retention.ms/bytes when remote.storage.enable=true,remote.log.copy.disable=false.
- Changed to use retention.bytes/retention.ms when remote copy disabled.
- Added validation to ask users to set local.retention.ms == retention.ms and local.retention.bytes == retention.bytes
- Added tests
Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
The method overrides of metricName in KafkaMetricsGroup are no longer required since there's a new constructor that implement this with the same behavior.
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
Add a remote.log.disable.policy on a topic-level only as part of KIP-950
Reviewers: Kamal Chandraprakash <kchandraprakash@uber.com>, Luke Chen <showuon@gmail.com>, Murali Basani <muralidhar.basani@aiven.io>
Create 3 new metadata versions:
- 3.8-IV0, for the upcoming 3.8 release.
- 3.9-IV0, to add support for KIP-1005.
- 3.9-IV1, as the new release vehicle for KIP-966.
Create ListOffsetRequest v9, which will be used in 3.9-IV0 to support KIP-1005. v9 is currently an unstable API version.
Reviewers: Jun Rao <junrao@gmail.com>, Justine Olshan <jolshan@confluent.io>
Add field tieredEpoch to RemoteLogSegmentMetadata
Update relevant tests
Add two fields tieredEpoch and tieredState to TopicRecord.json
Reviewers: Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>
The below remote log configs can be configured dynamically:
1. remote.log.manager.copy.max.bytes.per.second
2. remote.log.manager.fetch.max.bytes.per.second and
3. remote.log.index.file.cache.total.size.bytes
If those values are configured dynamically, then during the broker restart, it ensures the dynamic values are loaded instead of the static values from the config.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
Improve consistency and correctness for user-provided timeouts at the Consumer network request layer, per the Java client Consumer timeouts design (https://cwiki.apache.org/confluence/display/KAFKA/Java+client+Consumer+timeouts). While the changes introduced in KAFKA-15974 enforce timeouts at the Consumer's event layer, this change enforces timeouts at the network request layer.
The changes mostly fit into the following areas:
1. Create shared code and idioms so timeout handling logic is consistent across current and future RequestManager implementations
2. Use deadlineMs instead of expirationMs, expirationTimeoutMs, retryExpirationTimeMs, timeoutMs, etc.
3. Update "preemptive pruning" to remove expired requests that have had at least one attempt
Reviewers: Lianet Magrans <lianetmr@gmail.com>, Bruno Cadonna <cadonna@apache.org>
- Use SystemTime instead of MockTime when time is not mocked
- Use static assertions to reduce the line length
- Fold the lines if it exceeds the limit
- rename tp0 to tpId0 when it refers to TopicIdPartition
Reviewers: Kuan-Po (Cooper) Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
A patch for KAFKA-15046 got rid of fsync on LeaderEpochFileCache#truncateFromStart/End for performance reason, but it turned out this could cause corrupted leader-epoch checkpoint file on ungraceful OS shutdown, i.e. OS shuts down in the middle when kernel is writing dirty pages back to the device.
To address this problem, this PR makes below changes: (1) Revert LeaderEpochCheckpoint#write to always fsync
(2) truncateFromStart/End now call LeaderEpochCheckpoint#write asynchronously on scheduler thread
(3) UnifiedLog#maybeCreateLeaderEpochCache now loads epoch entries from checkpoint file only when current cache is absent
Reviewers: Jun Rao <junrao@gmail.com>
add more unit tests to LogSegments and do some small refactor in LogSegments.java
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Added the implementation of the quota manager that will be used to throttle copy and fetch requests from the remote storage. Reference KIP-956
Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kchandraprakash@uber.com>, Jun Rao <junrao@gmail.com>
Skip using stream when expiring the producer ID. This can improve the performance significantly when the count is high.
Before
Benchmark (numProducerIds) Mode Cnt Score Error Units
ProducerStateManagerBench.testDeleteExpiringIds 10000 avgt 3 101.253 ± 28.031 us/op
ProducerStateManagerBench.testDeleteExpiringIds 100000 avgt 3 2297.219 ± 1690.486 us/op
ProducerStateManagerBench.testDeleteExpiringIds 1000000 avgt 3 30688.865 ± 16348.768 us/op
After
Benchmark (numProducerIds) Mode Cnt Score Error Units
ProducerStateManagerBench.testDeleteExpiringIds 10000 avgt 3 39.122 ± 1.151 us/op
ProducerStateManagerBench.testDeleteExpiringIds 100000 avgt 3 464.363 ± 98.857 us/op
ProducerStateManagerBench.testDeleteExpiringIds 1000000 avgt 3 5731.169 ± 674.380 us/op
Also, made a change to the JMH testing which excludes the producer ID populating from the testing.
Reviewers: Artem Livshits <alivshits@confluent.io>, Justine Olshan <jolshan@confluent.io>
After JBOD is supported in KRaft, we should also enable JBOD support in tiered storage. Unit tests and Integration tests are also added.
Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Igor Soarez <soarez@apple.com>, Mickael Maison <mickael.maison@gmail.com>
Don't throw OFFSET_OUT_OF_RANGE error when converting the offset to metadata, and next time the leader should increment the high watermark by itself after receiving fetch requests from followers. This can happen when checkpoint files are missing and being elected as a leader.
Reviewers: Luke Chen <showuon@gmail.com>, Jun Rao <junrao@apache.org>
1. Replace TopicBasedRemoteLogMetadataManagerWrapperWithHarness with RemoteLogMetadataManagerTestUtils#builder in RemoteLogMetadataManagerTest.
2. Use ClusterTestExtention for RemoteLogMetadataManagerTest.
Signed-off-by: PoAn Yang <payang@apache.org>
Reviewers: Luke Chen <showuon@gmail.com>
TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest has a race when it sets RemoteLogMetadataTopicPartitioner using the setter.
This change fixes the race condition by passing the RemoteLogMetadataTopicPartitioner instance in a Function<Integer, RemoteLogMetaedataTopicPartitioner> which is used in configure() in TopicBasedRemoteLogMetadataManager.
It also improves the waitingFor condition by spying on RemotePartitionMetadataStore and awaiting on Phasers to ensure ConsumerManager makes progress before performing assertions.
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Some tests in TopicBasedRemoteLogMetadataManagerTest flake because waitUntilConsumerCatchesUp may break early before consumer manager has caught up with all the events.
This PR adds an expected offsets for leader/follower metadataOffset partitions and ensures we wait for the offset to be at least equal to the argument to avoid flakyness.
Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
If there are some logs to be deleted during the log dir movement, we'll send for a scheduler to do the deletion later.
However, when the log dir movement completed, the future log is renamed, the async log deletion will fail with no file existed error.
Signed-off-by: PoAn Yang <payang@apache.org>
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, SoontaekLim <soontaek.lim@neya.kr>, Johnny Hsu <johnnyhsu@fb.com>
RemoteLogMetadataSerde references RemoteLogMetadataTransform in a Raw
form. Given that the class is parametrized we should make use of it.
Signed-off-by: Josep Prat <josep.prat@aiven.io>
Reviewers: Matthew de Detrich <matthew.dedetrich@aiven.io>, Mickael Maison <mickael.maison@gmail.com>
We do iterate the records to find the offsetOfMaxTimestamp instead of returning the cached one when handling ListOffsetsRequest.MAX_TIMESTAMP, since it is hard to align all paths to get correct offsetOfMaxTimestamp. The known paths are shown below.
1. convertAndAssignOffsetsNonCompressed -> we CAN get correct offsetOfMaxTimestamp when validating all records
2. assignOffsetsNonCompressed -> ditto
3. validateMessagesAndAssignOffsetsCompressed -> ditto
4. validateMessagesAndAssignOffsetsCompressed#buildRecordsAndAssignOffsets -> ditto
5. appendAsFollow#append#analyzeAndValidateRecords -> we CAN'T get correct offsetOfMaxTimestamp as iterating all records is expensive when fetching records from leader
6. LogSegment#recover -> ditto
Reviewers: Jun Rao <junrao@gmail.com>
- Use `Empty` instead of 'none' when referring to `Optional` values.
- `Headers.lastHeader` returns `null` when no header is found.
- Fix minor spelling mistakes.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
- Fix the verifying logic. If it's LOG_APPEND_TIME, we choose the offset of the first record. Else, we choose the record with the maxTimeStamp.
- rename the shallowOffsetOfMaxTimestamp to offsetOfMaxTimestamp
Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Fix getOffsetByMaxTimestamp for compressed records.
This PR adds:
1) For inPlaceAssignment case, compute the correct offset for maxTimestamp when traversing the batch records, and set to ValidationResult in the end, instead of setting to last offset always.
2) For not inPlaceAssignment, set the offsetOfMaxTimestamp for the log create time, like non-compressed, and inPlaceAssignment cases, instead of setting to last offset always.
3) Add tests to verify the fix.
Reviewers: Jun Rao <junrao@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
In order to move ConfigCommand to tools we must move all it's dependencies which includes KafkaConfig and other core classes to java. This PR moves log cleaner configuration to CleanerConfig class of storage module.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>