Commit Graph

197 Commits

Author SHA1 Message Date
Mickael Maison 232285e534
KAFKA-17749: Fix Throttler metrics name (#17430)
Reviewers: Josep Prat <josep.prat@aiven.io>
2024-10-10 17:38:26 +02:00
Kuan-Po Tseng d9a26a95a7
KAFKA-17360 local log retention ms/bytes "-2" is not treated correctly (#16995)
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: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-26 06:22:44 +08:00
Mickael Maison a75d4d4e2b
KAFKA-17227: Refactor compression code to only load codecs when used (#16782) (#16811)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-08-06 20:56:46 +02:00
PoAn Yang 7495337c76 KAFKA-17166 Use NoOpScheduler to rewrite LogManagerTest#testLogRecoveryMetrics (#16641)
Reviewers: Okada Haruki <ocadaruma@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-07-23 01:22:15 +08:00
PoAn Yang 29e7796747 KAFKA-17142 Fix deadlock caused by LogManagerTest#testLogRecoveryMetrics (#16614)
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-07-23 01:12:59 +08:00
Kamal Chandraprakash 4ff8e16c91 KAFKA-15265: Reapply dynamic remote configs after broker restart (#16353)
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>
2024-06-18 10:41:49 +05:30
Okada Haruki 7c30eed66c KAFKA-16541 Fix potential leader-epoch checkpoint file corruption (#15993)
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>
2024-06-12 06:33:09 +05:30
Kamal Chandraprakash bcd95f6485 KAFKA-16904: Metric to measure the latency of remote read requests (#16209)
Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>, Luke Chen <showuon@gmail.com>
2024-06-11 21:08:39 +05:30
Kamal Chandraprakash d94a28b4a4 KAFKA-15776: Support added to update remote.fetch.max.wait.ms dynamically (#16203)
Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2024-06-11 12:35:26 +05:30
Chia Chuan Yu 781b93b00d KAFKA-16885 Renamed the enableRemoteStorageSystem to isRemoteStorageSystemEnabled (#16256)
Reviewers: Kamal Chandraprakash <kchandraprakash@uber.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-11 12:35:25 +05:30
Murali Basani 9460e6b266 KAFKA-16884 Refactor RemoteLogManagerConfig with AbstractConfig (#16199)
Reviewers: Greg Harris <gharris1727@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-11 12:35:25 +05:30
Kamal Chandraprakash 025e791d0c MINOR: Cleanup the storage module unit tests (#16202)
- 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>
2024-06-11 12:35:25 +05:30
Kamal Chandraprakash b6848d699d KAFKA-15776: Introduce remote.fetch.max.timeout.ms to configure DelayedRemoteFetch timeout (#14778)
KIP-1018, part1, Introduce remote.fetch.max.timeout.ms to configure DelayedRemoteFetch timeout

Reviewers: Luke Chen <showuon@gmail.com>
2024-06-11 12:35:25 +05:30
Kamal Chandraprakash 69158f67f8 KAFKA-16882 Migrate RemoteLogSegmentLifecycleTest to ClusterInstance infra (#16180)
- Removed the RemoteLogSegmentLifecycleManager
- Removed the TopicBasedRemoteLogMetadataManagerWrapper, RemoteLogMetadataCacheWrapper, TopicBasedRemoteLogMetadataManagerHarness and TopicBasedRemoteLogMetadataManagerWrapperWithHarness

Reviewers: Kuan-Po (Cooper) Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-11 12:35:25 +05:30
Murali Basani 944f4699a7 KAFKA-16880 Update equals and hashcode methods for two attributes (#16173)
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-11 12:35:24 +05:30
Murali Basani e4a3da6b09 KAFKA-16852 Adding two thread pools kafka-16852 (#16154)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tasi <chia7712@gmail.com>
2024-06-11 12:35:24 +05:30
Kuan-Po (Cooper) Tseng 2273e06138 MINOR: Fix missing wait topic finished in TopicBasedRemoteLogMetadataManagerRestartTest (#16171)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-11 12:35:24 +05:30
Kuan-Po (Cooper) Tseng c6f0db3c60 KAFKA-16785 Migrate TopicBasedRemoteLogMetadataManagerRestartTest to new test infra (#16170)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-11 12:35:24 +05:30
Kirk True fc267f4eb8 KAFKA-16200: Enforce that RequestManager implementations respect user-provided timeout (#16031)
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>
2024-06-07 09:55:45 +02:00
Anatoly Popov cd52f33746 KAFKA-16105: Reset read offsets when seeking to beginning in TBRLMM (#15165)
Reviewers: Greg Harris <greg.harris@aiven.io>, Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2024-06-03 13:48:50 -07:00
Kuan-Po (Cooper) Tseng 3d125a2322
MINOR: Add more unit tests to LogSegments (#16085)
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>
2024-05-31 16:07:38 +08:00
Abhijeet Kumar bb7db87f98
KAFKA-15265: Add Remote Log Manager quota manager (#15625)
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>
2024-05-30 09:06:49 -07:00
Calvin Liu c8af740bd4
Improve producer ID expiration performance (#16075)
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>
2024-05-29 16:49:55 -07:00
Luke Chen 897cab2a61
KAFKA-16399: Add JBOD support in tiered storage (#15690)
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>
2024-05-29 15:30:18 +08:00
Kamal Chandraprakash 524ad1e14b
KAFKA-16452: Don't throw OOORE when converting the offset to metadata (#15825)
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>
2024-05-27 17:44:23 +08:00
Mickael Maison e4e1116156
MINOR: Move Throttler to storage module (#16023)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-22 18:47:31 +02:00
PoAn Yang e93aae0664
KAFKA-16783: Migrate RemoteLogMetadataManagerTest to new test infra (#15983)
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>
2024-05-22 11:28:01 +08:00
Mickael Maison affe8da54c
KAFKA-7632: Support Compression Levels (KIP-390) (#15516)
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
2024-05-21 17:58:49 +02:00
PoAn Yang 9fe3932e5c
KAFKA-16784 Migrate TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest to use ClusterTestExtensions (#15992)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-21 22:42:43 +08:00
Chia-Ping Tsai 2c51594607
MINOR: rewrite TopicBasedRemoteLogMetadataManagerTest by ClusterTestExtensions (#15917)
Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Luke Chen <showuon@gmail.com>
2024-05-16 21:26:08 +08:00
Gaurav Narula a1c2c68db1
KAFKA-16712 Fix race in TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest (#15962)
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>
2024-05-16 14:56:06 +08:00
Gaurav Narula eb5559a40e
KAFKA-16686 Wait for given offset in TopicBasedRemoteLogMetadataManagerTest (#15885)
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>
2024-05-15 12:59:38 +08:00
Kamal Chandraprakash 576facfdf2
KAFKA-16696 Removed the in-memory implementation of RSM and RLMM (#15911)
Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-05-13 19:26:49 +08:00
Linu Shibu aeca384641
KAFKA-16356: Remove class-name dispatch in RemoteLogMetadataSerde (#15620)
Reviewers: Greg Harris <greg.harris@aiven.io>, Luke Chen <showuon@gmail.com>, Igor Soarez <soarez@apple.com>, The-Gamer-01 <19974361760@163.com>
2024-05-06 16:49:35 -07:00
Chia Chuan Yu 55a00be4e9
MINOR: Replaced Utils.join() with JDK API. (#15823)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-05-06 15:13:01 +08:00
Gaurav Narula 025f9816f1
MINOR: fix javadoc warnings (#15527)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-26 08:31:52 +08:00
PoAn Yang a38185280c
KAFKA-16424: remove truncated logs after alter dir (#15616)
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>
2024-04-24 17:51:29 +08:00
Omnia Ibrahim cfe5ab5cf2
KAFKA-15853 Move quota configs into server-common package (#15774)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 13:05:18 +08:00
Cheng-Kai, Zhang b6e70e9a54
MINOR: Add test for PartitionMetadataFile (#15714)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-24 13:01:35 +08:00
Kamal Chandraprakash 18572f5f8f
MINOR: Reduce the time taken to execute the TieredStorage tests. (#15780)
Reduce the time taken to execute the TieredStorage tests

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-23 10:21:46 +08:00
Omnia Ibrahim ecb2dd4cdc
KAFKA-15853 Move KafkaConfig log properties and docs out of core (#15569)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Nikolay <nizhikov@apache.org>, Federico Valeri <fvaleri@redhat.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-20 04:14:23 +08:00
Josep Prat 8f2fca7bd8
MINOR: Use Parametrized types correctly in RemoteLogMetadataSerde (#13824)
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>
2024-04-19 09:04:27 +02:00
Omnia Ibrahim 8c0458861c
KAFKA-15853 Move KafkaConfig Replication properties and docs out of … (#15575)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-16 15:28:35 +08:00
Mickael Maison 3617dda9a5
MINOR: Various cleanups in storage (#15711)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-15 13:30:41 +02:00
Alok Thatikunta c034cf2953
MINOR: Fix incorrect Java equals comparison of Uuid by reference (#15707)
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2024-04-13 20:55:48 +08:00
Omnia Ibrahim 61baa7ac6b
KAFKA-15853 Move transactions configs out of core (#15670)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-04-13 00:29:51 +08:00
Chia-Ping Tsai 9a6760f130
KAFKA-16310 ListOffsets doesn't report the offset with maxTimestamp a… (#15621)
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>
2024-04-10 11:36:07 +08:00
Erik van Oosten 8e61f04228
MINOR: Fix usage of none in javadoc (#15674)
- 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>
2024-04-08 08:43:05 +08:00
Kamal Chandraprakash 2f733ac583
KAFKA-16161: Avoid empty remote metadata snapshot file in partition dir (#15636)
Avoid empty remote metadata snapshot file in partition dir

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Satish Duggana <satishd@apache.org>
2024-04-02 10:07:54 +08:00
Johnny Hsu bf3f088c94
KAFKA-16341 fix the LogValidator for non-compressed type (#15476)
- 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>
2024-03-19 23:00:30 +08:00