Commit Graph

392 Commits

Author SHA1 Message Date
Ken Huang 05b1380ecb
KAFKA-16897 Move OffsetIndexTest and OffsetMapTest to storage module (#16244)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-13 06:24:23 +08:00
Kamal Chandraprakash f3dbd7ed08
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:07:12 +05:30
Kamal Chandraprakash f359908fcd
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-10 20:42:12 +05:30
Chia Chuan Yu e5b8712993
KAFKA-16885 Renamed the enableRemoteStorageSystem to isRemoteStorageSystemEnabled (#16256)
Reviewers: Kamal Chandraprakash <kchandraprakash@uber.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-06-10 02:14:15 +08:00
Kirk True d6cd83e2fb
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:53:27 +02:00
Murali Basani a41f7a4e13
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-07 00:06:25 +08:00
Kamal Chandraprakash 0ed104c3dc
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-06 23:56:08 +08:00
Okada Haruki 3835515fea
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-06 15:10:13 +09:00
Kamal Chandraprakash 02c794dfd3
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-05 14:42:23 +08:00
Kamal Chandraprakash cda2df5feb
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-05 03:11:30 +08:00
Murali Basani 64c50a274b
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-04 14:05:00 +08:00
Anatoly Popov 8a882a77a4
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:46:40 -07:00
Murali Basani 734d309a29
KAFKA-16852 Adding two thread pools kafka-16852 (#16154)
Reviewers: Christo Lolov <lolovc@amazon.com>, Chia-Ping Tasi <chia7712@gmail.com>
2024-06-03 09:52:54 +01:00
Kuan-Po (Cooper) Tseng 0c9c1d405d
MINOR: Fix missing wait topic finished in TopicBasedRemoteLogMetadataManagerRestartTest (#16171)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-03 03:59:56 +08:00
Kuan-Po (Cooper) Tseng b05f82d444
KAFKA-16785 Migrate TopicBasedRemoteLogMetadataManagerRestartTest to new test infra (#16170)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-06-02 22:18:53 +08: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
Luke Chen 834efa6606
KAFKA-16342 fix getOffsetByMaxTimestamp for compressed records (#15474)
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>
2024-03-15 06:09:45 +08:00
Johnny Hsu 3fcaa9ccc0
MINOR: remove the copy constructor of LogSegment (#15488)
In the LogSegment, the copy constructor is only used in LogLoaderTest

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-03-10 03:06:41 +08:00
John Yu 554fa57af8
KAFKA-16209 : fetchSnapshot might return null if topic is created before v2.8 (#15444)
Change the function with a better way to deal with the NULL pointer exception.

Reviewers: Luke Chen <showuon@gmail.com>
2024-03-06 09:00:58 +08:00
Nikolay eea369af94
KAFKA-14588 Log cleaner configuration move to CleanerConfig (#15387)
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>
2024-03-05 18:11:56 +08:00
John Yu 1bb9a85174
MINOR: Remove the space between two words (#15439)
Remove the space between two words

Reviewers: Luke Chen <showuon@gmail.com>
2024-02-29 08:14:35 +08:00
Satish Duggana fc8b644e56
MINOR Removed unused CommittedOffsetsFile class. (#15209)
`CommittedOffsetsFile` can be introduced when it is required for enhancing TBRLMM to consume from a specific offset when snapshots are implemented.

Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>
2024-02-12 17:35:01 +05:30
Jorge Esteban Quilcate Otoya b25c96a915
KAFKA-16229: Fix slow expired producer id deletion (#15324)
Expiration of ProducerIds is implemented with a slow removal of map keys:
        producers.keySet().removeAll(keys);
Unnecessarily going through all producer ids and then throw all expired keys to be removed.
This leads to exponential time on worst case when most/all keys need to be removed:

Benchmark                                        (numProducerIds)  Mode  Cnt           Score            Error  Units
ProducerStateManagerBench.testDeleteExpiringIds               100  avgt    3        9164.043 ±      10647.877  ns/op
ProducerStateManagerBench.testDeleteExpiringIds              1000  avgt    3      341561.093 ±      20283.211  ns/op
ProducerStateManagerBench.testDeleteExpiringIds             10000  avgt    3    44957983.550 ±    9389011.290  ns/op
ProducerStateManagerBench.testDeleteExpiringIds            100000  avgt    3  5683374164.167 ± 1446242131.466  ns/op
A simple fix is to use map#remove(key) instead, leading to a more linear growth:

Benchmark                                        (numProducerIds)  Mode  Cnt        Score         Error  Units
ProducerStateManagerBench.testDeleteExpiringIds               100  avgt    3     5779.056 ±     651.389  ns/op
ProducerStateManagerBench.testDeleteExpiringIds              1000  avgt    3    61430.530 ±   21875.644  ns/op
ProducerStateManagerBench.testDeleteExpiringIds             10000  avgt    3   643887.031 ±  600475.302  ns/op
ProducerStateManagerBench.testDeleteExpiringIds            100000  avgt    3  7741689.539 ± 3218317.079  ns/op
Flamegraph of the CPU usage at dealing with expiration when producers ids ~1Million:

Reviewers: Justine Olshan <jolshan@confluent.io>
2024-02-09 17:17:17 -08:00
Divij Vaidya 65424ab484
MINOR: New year code cleanup - include final keyword (#15072)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Sagar Rao <sagarmeansocean@gmail.com>
2024-01-11 17:53:35 +01:00
Christo Lolov d4f3bf93d3
KAFKA-16014: Implement RemoteLogSizeBytes (#15050)
This pull request aims to implement RemoteLogSizeBytes from KIP-963.

Reviewers: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>,  Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-12-22 15:00:44 +08:00
Christo Lolov 1a97de2fe6
KAFKA-16002: Implement RemoteCopyLagSegments, RemoteDeleteLagBytes and RemoteDeleteLagSegments (#15005)
This pull request aims to implement RemoteCopyLagSegments, RemoteDeleteLagBytes and RemoteDeleteLagSegments from KIP-963.

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-12-21 14:27:12 +08:00
Luke Chen 4e11de00a7
KAFKA-16014: Add RemoteLogMetadataCount metric (#15026)
Reviewers: Christo Lolov <lolovc@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>
2023-12-20 14:21:30 +05:30
Gantigmaa Selenge 7b21da9712
KAFKA-15158: Add metrics for RemoteDelete and BuildRemoteLogAuxState (#14375)
This PR implements part of KIP-963, specifically for adding new metrics.
The metrics added in this PR are:
    RemoteDeleteRequestsPerSec (emitted when expired log segments on remote storage being deleted)
    RemoteDeleteErrorsPerSec (emitted when failed to delete expired log segments on remote storage)
    BuildRemoteLogAuxStateRequestsPerSec (emitted when building remote log aux state for replica fetchers)
    BuildRemoteLogAuxStateErrorsPerSec (emitted when failed to build remote log aux state for replica fetchers)

Reviewers: Luke Chen <showuon@gmail.com>, Nikhil Ramakrishnan <ramakrishnan.nikhil@gmail.com>, Christo Lolov <lolovc@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
2023-12-19 15:02:45 +08:00
Luke Chen c240993be2
KAFKA-16014: Add RemoteLogSizeComputationTime metric (#15021)
Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>
2023-12-18 21:39:43 +05:30
Christo Lolov a87e86e015
KAFKA-15883: Implement RemoteCopyLagBytes (#14832)
This pull request implements the first in the list of metrics in KIP-963: Additional metrics in Tiered Storage.

Since each partition of a topic will be serviced by its own RLMTask we need an aggregator object for a topic. The aggregator object in this pull request is BrokerTopicAggregatedMetric. Since the RemoteCopyLagBytes is a gauge I have introduced a new GaugeWrapper. The GaugeWrapper is used by the metrics collection system to interact with the BrokerTopicAggregatedMetric. The RemoteLogManager interacts with the BrokerTopicAggregatedMetric directly.

Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>
2023-12-14 09:21:37 +08:00
Okada Haruki d71d0639d9
KAFKA-15046: Get rid of unnecessary fsyncs inside UnifiedLog.lock to stabilize performance (#14242)
While any blocking operation under holding the UnifiedLog.lock could lead to serious performance (even availability) issues, currently there are several paths that calls fsync(2) inside the lock
In the meantime the lock is held, all subsequent produces against the partition may block
This easily causes all request-handlers to be busy on bad disk performance
Even worse, when a disk experiences tens of seconds of glitch (it's not rare in spinning drives), it makes the broker to unable to process any requests with unfenced from the cluster (i.e. "zombie" like status)
This PR gets rid of 4 cases of essentially-unnecessary fsync(2) calls performed under the lock:
(1) ProducerStateManager.takeSnapshot at UnifiedLog.roll
I moved fsync(2) call to the scheduler thread as part of existing "flush-log" job (before incrementing recovery point)
Since it's still ensured that the snapshot is flushed before incrementing recovery point, this change shouldn't cause any problem
(2) ProducerStateManager.removeAndMarkSnapshotForDeletion as part of log segment deletion
This method calls Utils.atomicMoveWithFallback with needFlushParentDir = true internally, which calls fsync.
I changed it to call Utils.atomicMoveWithFallback with needFlushParentDir = false (which is consistent behavior with index files deletion. index files deletion also doesn't flush parent dir)
This change shouldn't cause problems neither.
(3) LeaderEpochFileCache.truncateFromStart when incrementing log-start-offset
This path is called from deleteRecords on request-handler threads.
Here, we don't need fsync(2) either actually.
On unclean shutdown, few leader epochs might be remained in the file but it will be handled by LogLoader on start-up so not a problem
(4) LeaderEpochFileCache.truncateFromEnd as part of log truncation
Likewise, we don't need fsync(2) here, since any epochs which are untruncated on unclean shutdown will be handled on log loading procedure

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>, Jun Rao <junrao@gmail.com>
2023-11-29 09:43:44 -08:00
Kamal Chandraprakash 20b0bf063b
MINOR: Fix the flaky TBRLMM `testInternalTopicExists` test (#14840)
The internal topic creation is asynchronous so the test gets flaky. To fix the test flakiness and in this test I want to assert that doesTopicExist should return true when a topic exists, so created a dummy internal topic.

Reviewers: Luke Chen <showuon@gmail.com>, Jun Rao <jun@confluent.io>, Satish Duggana <satishd@apache.org>
2023-11-29 10:50:22 +08:00
Kamal Chandraprakash fade3d10ea
KAFKA-15047: Roll active segment when it breaches the retention policy (#14766)
Roll the active segment and offload it to remote storage once it breaches the retention time policy.

A segment is eligible for deletion once it gets uploaded to the remote storage. We have checks to allow only the passive segments to be uploaded, so the active segment never gets removed at all even if breaches the retention time. For low-throughput/stale topics, the active segment can hold the data beyond the configured retention time by the user.

Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
2023-11-28 09:38:11 +05:30
Kamal Chandraprakash a42c846336
KAFKA-15241: Compute tiered copied offset by keeping the respective epochs in scope (#14787)
"findHighestRemoteOffset" does not take into account the leader-epoch end offset. This can cause log divergence between the local and remote log segments when there is unclean leader election.

To handle it correctly, the logic to find the highest remote offset can be updated to:
find-highest-remote-offset = min(end-offset-for-epoch-in-the-checkpoint, highest-remote-offset-for-epoch)

Discussion thread: https://github.com/apache/kafka/pull/14004#discussion_r1266864272

Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
2023-11-27 09:10:46 +05:30
Jason Gustafson e905ef1edf
MINOR: Small LogValidator clean ups (#14697)
This patch contains a few small clean-ups in LogValidator and associated classes:

1. Set shallowOffsetOfMaxTimestamp consistently as the last offset in the
   batch for v2 compressed and non-compressed data.
2. Rename `RecordConversionStats` to `RecordValidationStats` since one of its
   fields `temporaryMemoryBytes` does not depend on conversion.
3. Rename `batchIndex` in `recordIndex` in loops over the records in each batch
   inside `LogValidator`.

Reviewers: Qichao Chu <5326144+ex172000@users.noreply.github.com>, Jun Rao <junrao@gmail.com>
2023-11-20 10:40:45 -08:00
Kamal Chandraprakash feee616f73
MINOR: Query before creating the internal remote log metadata topic (#14755)
When a node starts (or) restarts, then we send a CREATE_TOPICS request to the controller to create the internal __remote_log_metadata topic.

Topic creation event is costly and handled by the controller. During re-balance, the controller can have pending requests in its queue and can lead to CREATE_TOPICS timeout. Instead of firing the CREATE_TOPICS request when a node restarts, send a METADATA request (topic describe) which is handled by the least loaded node before sending a request to create the topic.

Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
2023-11-20 14:50:11 +05:30
David Mao c6ea0a84ab
KAFKA-15780: Wait for consistent KRaft metadata when creating or deleting topics (#14695)
TestUtils.createTopicWithAdmin calls waitForAllPartitionsMetadata which waits for partition(s) to be present in each brokers' metadata cache. This is a sufficient check in ZK mode because the controller sends an LISR request before sending an UpdateMetadataRequest which means that the partition in the ReplicaManager will be updated before the metadata cache.

In KRaft mode, the metadata cache is updated first, so the check may return before partitions and other metadata listeners are fully initialized.

Testing:
Insert a Thread.sleep(100) in BrokerMetadataPublisher.onMetadataUpdate after

      // Publish the new metadata image to the metadata cache.
      metadataCache.setImage(newImage)
and run EdgeCaseRequestTest.testProduceRequestWithNullClientId and the test will fail locally nearly deterministically. After the change(s), the test no longer fails.

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-11-06 17:07:56 -08:00
gongzhongqiang d682b15eeb
KAFKA-15769: Fix logging with exception trace (#14683)
Reviewers: Divij Vaidya <diviv@amazon.com>, hudeqi <1217150961@qq.com>
2023-11-06 11:02:05 +01:00
Alok Thatikunta eca8502990
KAFKA-14484: [1/N] Move PartitionMetadataFile to storage module (#14607)
This PR moves PartitionMetadataFile to the storage module.

Existing unit tests in UnifiedLogTest like testLogFlushesPartitionMetadataOnAppend should suffice.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2023-11-01 09:40:45 -07:00
Kamal Chandraprakash 57fd8f4c36
KAFKA-15632: Drop the invalid remote log metadata events (#14576)
Reviewers: Christo Lolov <lolovc@amazon.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-10-31 15:21:33 +05:30
Calvin Liu 8f8ad6db38
KAFKA-15582: Move the clean shutdown file to the storage package (#14603)
A follow-up change to move the clean shutdown file to the storage package.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2023-10-30 16:27:40 -07:00
Jotaniya Jeel 4612fe42af
KAFKA-15481: Fix concurrency bug in RemoteIndexCache (#14483)
RemoteIndexCache has a concurrency bug which leads to IOException while fetching data from remote tier.

The bug could be reproduced as per the following order of events:-

Thread 1 (cache thread): invalidates the entry, removalListener is invoked async, so the files have not been renamed to "deleted" suffix yet.
Thread 2: (fetch thread): tries to find entry in cache, doesn't find it because it has been removed by 1, fetches the entry from S3, writes it to existing file (using replace existing)
Thread 1: async removalListener is invoked, acquires a lock on old entry (which has been removed from cache), it renames the file to "deleted" and starts deleting it
Thread 2: Tries to create in-memory/mmapped index, but doesn't find the file and hence, creates a new file of size 2GB in AbstractIndex constructor. JVM returns an error as it won't allow creation of 2GB random access file.

This commit fixes the bug by using EvictionListener instead of RemovalListener to perform the eviction atomically with the file rename. It handles the manual removal (not handled by EvictionListener) by using computeIfAbsent() and enforcing atomic cache removal & file rename.

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Arpit Goyal 
<goyal.arpit.91@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-10-23 14:50:46 +02:00
Justine Olshan e8c8969330
KAFKA-15626: Replace verification guard object with an specific type (#14568)
I've added a new class with an incrementing atomic long to represent the verification guard. Upon creation of verification guard, we will increment this value and assign it to the guard.

The expected behavior is the same as the object guard, but with better debuggability with the string value and type safety (I found a type safety issue in the current code when implementing this)

Reviewers: Ismael Juma <ismael@juma.me.uk>, Artem Livshits <alivshits@confluent.io>
2023-10-20 14:26:20 -07:00
Arpit Goyal dc6a53e196
MINOR: Rename lock variable of the entry class (#14569)
The RemoteIndexCache has a variable lock and the child class also have a variable lock in the same class file. Renaming lock of the entry(child class) to avoid confusion.

Reviewers: Luke Chen <showuon@gmail.com>, hudeqi <1217150961@qq.com>
2023-10-18 18:20:55 +08:00
Ismael Juma 1073d434ec
KAFKA-14481: Move LogSegment/LogSegments to storage module (#14529)
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>
2023-10-16 06:37:30 -07:00
hudeqi b0b8693c72
KAFKA-15536: Dynamically resize remoteIndexCache (#14511)
Dynamically resize remoteIndexCache

Reviewers: Christo Lolov <lolovc@amazon.com>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-10-16 15:24:36 +08:00
Matthias J. Sax dc0f0db864
MINOR: fix typo (#14542)
Reviewers: Bruno Cadonna <bruno@confluent.io>
2023-10-13 09:28:00 -07:00
Arpit Goyal 99ce2e081c
KAFKA-15169: Added TestCase in RemoteIndexCache (#14482)
est Cases Covered

    1. Index Files already exist on disk but not in Cache i.e. RemoteIndexCache should not call remoteStorageManager to fetch it instead cache it from the local index file present.
    2. RSM returns CorruptedIndex File i.e. RemoteIndexCache should throw CorruptedIndexException instead of successfull execution.
    3. Deleted Suffix Indexes file already present on disk i.e. If cleaner thread is slow , then there is a chance of deleted index files present on the disk while in parallel same index Entry is invalidated. To understand more refer https://issues.apache.org/jira/browse/KAFKA-15169

Reviewers: Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>
2023-10-11 10:58:17 +08:00
hudeqi 1c3eb4395a
KAFKA-14912:Add a dynamic config for remote index cache size (#14381)
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Subhrodip Mohanta <hello@subho.xyz>
2023-10-08 13:24:09 +05:30
David Mao 2c925e9f33
KAFKA-15526: Simplify the LogAppendInfo class (#14470)
The LogAppendInfo class is a bit bloated in terms of class fields. That's because it is used as an umbrella class for both leader log appends and follower log appends and needs to carry fields for both. This makes the constructor for the class a bit cludgy to use. It also ends up being a bit confusing when fields are important and when they aren't. I noticed there were a few fields that didn't seem necessary.

Below is a description of changes:

firstOffset is a LogOffsetMetadata but there are no readers of the field that use anything but the messageOffset field - simplified to a long.
LogAppendInfo.errorMessage is only set in one context - when calling LogAppendInfo.unknownLogAppendInfoWithAdditionalInfo. When we use this constructor, we pass up the original exception in LogAppendResult anyway, so the field is redundant with the LogAppendResult.exception field. This allows us to simplify the handling in KAFKA-15459: Convert coordinator retriable errors to a known producer response error #14378 since there are no custom error messages we just return whatever is in the exception message.
We only use targetCompressionType when constructing the LogValidator - just inline the call instead of including it in the LogAppendInfo.
offsetsMonotonic is only used when not assigning offsets to throw an exception - just throw the exception instead of setting a field to throw later.
shallowCount is only there to determine whether there are any messages in the append. Instead, we can just check validBytes which is incremented with a non-zero value every time we increment shallowCount.

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-10-03 17:32:44 -07:00
iit2009060 13b119aa62
KAFKA-15511: Handle CorruptIndexException in RemoteIndexCache (#14459)
A bug in the RemoteIndexCache leads to a situation where the cache does not replace the corrupted index with a new index instance fetched from remote storage. This commit fixes the bug by adding correct handling for `CorruptIndexException`.

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Alexandre Dupriez <duprie@amazon.com>
2023-09-29 12:26:46 +02:00
Kamal Chandraprakash aa399a335f
KAFKA-15499: Fix the flaky DeleteSegmentsDueToLogStartOffsetBreach test (#14439)
DeleteSegmentsDueToLogStartOffsetBreach configures the segment such that it can hold at-most 2 record-batches. And, it asserts that the local-log-start-offset based on the assumption that each segment will contain exactly two messages.

During leader switch, the segment can get rotated and may not always contain two records. Previously, we were checking whether the expected local-log-start-offset is equal to the base-offset-of-the-first-local-log-segment. With this patch, we will scan the first local-log-segment for the expected offset.

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-09-28 15:04:37 +02:00
Ismael Juma 98febb989a
KAFKA-15485: Fix "this-escape" compiler warnings introduced by JDK 21 (1/N) (#14427)
This is one of the steps required for kafka to compile with Java 21.

For each case, one of the following fixes were applied:
1. Suppress warning if fixing would potentially result in an incompatible change (for public classes)
2. Add final to one or more methods so that the escape is not possible
3. Replace method calls with direct field access.

In addition, we also fix a couple of compiler warnings related to deprecated references in the `core` module.

See the following for more details regarding the new lint warning:
https://www.oracle.com/java/technologies/javase/21-relnote-issues.html#JDK-8015831

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Chris Egerton <chrise@aiven.io>
2023-09-24 05:59:29 -07:00
Wuzhengyu97 fcd382138e
MINOR: Used Admin instead of AdminClient to create Admin (#14411)
Used Admin instead of AdminClient to create Admin

Reviewers: Ziming Deng <dengziming1993@gmail.com>
2023-09-21 11:01:08 +08:00
Luke Chen c00c5b1b66
MINOR: Add verification for (local) log start offset in txn tests (#14401)
In this PR, we noticed failed tests caused by the verification of log start offset and local log start offset. After investigation: #14347 (comment)

https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-14347/13/#showFailuresLink

After investigation, I found it's because the scala 2.12 cannot recognize the override method of maybeWaitForAtLeastOneSegmentUpload since it's using varargs in scala. I think there must be some bugs/gaps between java/scala that causes these issue. We can fix it by not using varargs, instead, using the Seq.

This PR adds back the log start offset and local log start offset verification, and make sure all tests passed.

Reviewers: Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-09-19 20:46:09 +08:00
Kamal Chandraprakash dacb3b31d9
KAFKA-15439: Transactions test with tiered storage (#14347)
This test extends the existing TransactionsTest. It configures the broker and topic with tiered storage and expects at-least one log segment to be uploaded to the remote storage.

Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>,  Divij Vaidya <diviv@amazon.com>
2023-09-14 09:52:13 +08:00
Luke Chen 8a7e5e8ea0
MINOR: Fix errors in javadoc and docs in tiered storage (#14379)
Reviewers: Satish Duggana <satishd@apache.org>
2023-09-13 12:45:36 +05:30
Luke Chen 4be174f5c1
MINOR: reduce default RLMM retry interval (#14374)
Reduce default remote.log.metadata.initialization.retry.interval.ms value to 100ms.

Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>
2023-09-12 22:06:31 +05:30
Abhijeet Kumar a63bf93dce
KAFKA-14993: Improve TransactionIndex instance handling while copying to and fetching from RSM (#14363)
- Updated the contract for RSM's fetchIndex to throw a ResourceNotFoundException instead of returning an empty InputStream when it does not have a TransactionIndex.
- Updated the LocalTieredStorage implementation to adhere to the new contract.
- Added Unit Tests for the change.

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>
2023-09-12 17:54:20 +05:30
Kamal Chandraprakash 672ea644f0
MINOR: Removed the RSM and RLMM classpath config validator (#14358)
- RSM and RLMM classpath can be empty since it's optional so removed the non-empty string validator
- Fix getting the `localTieredStorage` by brokerId after stopping a broker.

Reviewers: Christo Lolov <lolovc@amazon.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-09-09 19:02:42 +05:30
Kamal Chandraprakash 6e818c6b02
KAFKA-15410: Delete records with tiered storage integration test (4/4) (#14330)
* Added the integration test for DELETE_RECORDS API for tiered storage enabled topic
* Added validation checks before removing remote log segments for log-start-offset breach

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>
2023-09-07 21:02:39 +05:30
Luke Chen cd897e6c76
MINOR: Update the javadoc in RSM (#14352)
Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>
2023-09-07 20:55:11 +05:30
Kamal Chandraprakash 6d762480c9
KAFKA-15351: Update log-start-offset after leader election for topics enabled with remote storage (#14340)
On leadership failover, the new leader's start offset may be older than the start offset of old leader. This works fine for local storage scenario because the new leader still contains data associated with stale start offset. But in case of remote storage, although new leader has a stale offset, the data associated with it has been deleted from remote by the old leader. Hence, we end up in a situation where leader has a start offset but no data associated with it.

This commit fixes the situation by ensuring that on every leadership failover, for topics with remote storage, the leader will update it's start offset from the base of first segment in current leader chain present in the remote storage (if any).

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com>
2023-09-07 16:32:16 +02:00
Kamal Chandraprakash 80982c4ae3
KAFKA-15410: Delete topic integration test with LocalTieredStorage and TBRLMM (3/4) (#14329)
Added delete topic integration tests for tiered storage enabled topics with LocalTieredStorage and TBRLMM

Reviewers: Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>
2023-09-06 05:50:12 +05:30
Luke Chen be0de2124a
MINOR: Update comment in consumeAction (#14335)
Reviewers: Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>
2023-09-05 21:36:28 +05:30
Kamal Chandraprakash 9f2ac375c2
KAFKA-15410: Reassign replica expand, move and shrink integration tests (2/4) (#14328)
- Updated the log-start-offset to the correct value while building the replica state in ReplicaFetcherTierStateMachine#buildRemoteLogAuxState

Integration tests added:
1. ReassignReplicaExpandTest
2. ReassignReplicaMoveTest and
3. ReassignReplicaShrinkTest

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-09-05 19:28:17 +05:30
Justine Olshan b892acae5e
KAFKA-15424: Make the transaction verification a dynamic configuration (#14324)
This will allow enabling and disabling transaction verification (KIP-890 part 1) without having to roll the cluster.

Tested that restarting the cluster persists the configuration.

If a verification is disabled/enabled while we have an inflight request, depending on the step of the process, the change may or may not be seen in the inflight request (enabling will typically fail unverified requests, but we may still verify and reject when we first disable) Subsequent requests/retries will behave as expected for verification.

Sequence checks will continue to take place after disabling until the first message is written to the partition (thus clearing the verification entry with the tentative sequence) or the broker restarts/partition is reassigned which will clear the memory. On enabling, we will only track sequences that for requests received after the verification is enabled.

Reviewers: Jason Gustafson <jason@confluent.io>, Satish Duggana <satishd@apache.org>
2023-09-04 20:40:50 -07:00
Kamal Chandraprakash caaa4c55fe
KAFKA-15410: Expand partitions, segment deletion by retention and enable remote log on topic integration tests (1/4) (#14307)
Added the below integration tests with tiered storage
 - PartitionsExpandTest
 - DeleteSegmentsByRetentionSizeTest
 - DeleteSegmentsByRetentionTimeTest and
 - EnableRemoteLogOnTopicTest
 - Enabled the test for both ZK and Kraft modes.

These are enabled for both ZK and Kraft modes.

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com>
2023-09-05 05:13:16 +05:30
Luke Chen da99879df7
KAFKA-15421: fix network thread leak in testThreadPoolResize (#14320)
In SocketServerTest, we create SocketServer and enableRequestProcessing on each test class initialization. That's fine since we shutdown it in @AfterEach. The issue we have is we disabled 2 tests in this test suite. And when running these disabled tests, we will go through class initialization, but without @AfterEach. That causes 2 network thread leaked.

Compared the error message in DynamicBrokerReconfigurationTest#testThreadPoolResize test here:

org.opentest4j.AssertionFailedError: Invalid threads: expected 6, got 8: List(data-plane-kafka-socket-acceptor-ListenerName(INTERNAL)-SSL-0, data-plane-kafka-socket-acceptor-ListenerName(PLAINTEXT)-PLAINTEXT-0, data-plane-kafka-socket-acceptor-ListenerName(INTERNAL)-SSL-0, data-plane-kafka-socket-acceptor-ListenerName(EXTERNAL)-SASL_SSL-0, data-plane-kafka-socket-acceptor-ListenerName(INTERNAL)-SSL-0, data-plane-kafka-socket-acceptor-ListenerName(EXTERNAL)-SASL_SSL-0, data-plane-kafka-socket-acceptor-ListenerName(PLAINTEXT)-PLAINTEXT-0, data-plane-kafka-socket-acceptor-ListenerName(EXTERNAL)-SASL_SSL-0) ==> expected: <true> but was: <false>

The 2 unexpected network threads are leaked from SocketServerTest.

Reviewers: Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kchandraprakash@uber.com>, Chris Egerton <chrise@aiven.io>
2023-09-03 16:16:54 +08:00
Christo Lolov 134f6c07a4
KAFKA-15427: Fix resource leak in integration tests for tiered storage (#14319)
Co-authored-by: Nikhil Ramakrishnan <nikrmk@amazon.com>

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-09-01 23:12:57 +05:30
Kamal Chandraprakash d0f3cf1f9f
KAFKA-15351: Ensure log-start-offset not updated to local-log-start-offset when remote storage enabled (#14301)
When tiered storage is enabled on the topic, and the last-standing-replica is restarted, then the log-start-offset should not reset its offset to first-local-log-segment-base-offset.

Reviewers: Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
2023-09-01 06:33:33 +05:30
Kamal Chandraprakash 43fe13350f
KAFKA-15404: Disable the flaky integration tests. (#14296)
Disabled the below tests to fix the thread leak:

1. kafka.server.DynamicBrokerReconfigurationTest.testThreadPoolResize() and
2. org.apache.kafka.tiered.storage.integration.OffloadAndConsumeFromLeaderTest

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>
2023-08-31 11:39:26 -07:00
Luke Chen c2bb8eb875
MINOR: Close topic based RLMM correctly in integration tests (#14315)
Reviewers: Divij Vaidya <diviv@amazon.com>
2023-08-31 10:44:32 +02:00
Christo Lolov efec0f5756
KAFKA-15267: Do not allow Tiered Storage to be disabled while topics have remote.storage.enable property (#14161)
The purpose of this change is to not allow a broker to start up with Tiered Storage disabled (remote.log.storage.system.enable=false) while there are still topics that have 'remote.storage.enable' set.

Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-08-30 05:34:20 +05:30
Kamal Chandraprakash 68b140cb56
MINOR: Fix the TBRLMMRestart test. (#14297)
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-08-28 20:23:17 +05:30
Kamal Chandraprakash 4590d565ef
KAFKA-15399: Enable OffloadAndConsumeFromLeader test (#14285)
Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-28 12:29:50 +02:00
Gantigmaa Selenge 180dcd3969
KAFKA-15294: Publish remote storage configs (#14266)
This change does the following:

1. Make RemoteLogManagerConfigs that are implemented public

2. Add tasks to generate html docs for the configs

3. Include config docs in the main site

Reviewers: Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-28 10:35:11 +02:00
Abhijeet Kumar ff3e6842ff
KAFKA-15181: Wait for RemoteLogMetadataCache to initialize after assigning partitions (#14127)
This PR adds the following changes to the `TopicBasedRemoteLogMetadataManager`

1. Added a guard in RemoteLogMetadataCache so that the incoming request can be served from the cache iff the corresponding user-topic-partition is initalized
2. Improve error handling in ConsumerTask thread so that is not killed when there are errors in reading the internal topic
3. ConsumerTask initialization should handle the case when there are no records to read
and some other minor changes

Added Unit Tests for the changes

Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

Reviewers: Luke Chen <showuon@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-08-26 05:52:26 +05:30
Mickael Maison 30de2bb5ef
MINOR: Missing space in ProducerStateManager LogContext (#14275)
Reviewers: Divij Vaidya <diviv@amazon.com>
2023-08-25 10:34:17 +02:00
Satish Duggana d4ab3ae85a
KAFKA-14888: Added remote log segments retention mechanism based on time and size. (#13561)
This change introduces a remote log segment segment retention cleanup mechanism.

RemoteLogManager runs retention cleanup activity tasks on each leader replica. It assesses factors such as overall size and retention duration, subsequently removing qualified segments from remote storage. This process also involves adjusting the log-start-offset within the UnifiedLog accordingly. It also cleans up the segments which have epochs earlier than the earliest leader epoch in the current leader. 

Co-authored-by: Satish Duggana <satishd@apache.org>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>

Reviewers: Jun Rao <junrao@gmail.com>, Divij Vaidya <diviv@amazon.com, Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Christo Lolov <lolovc@amazon.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Nikhil Ramakrishnan <ramakrishnan.nikhil@gmail.com>
2023-08-25 05:27:59 +05:30
Kamal Chandraprakash 88d2c4460a
KAFKA-15400: Use readLock when removing an item from the RemoteIndexCache (#14283)
- Caffeine cache is thread safe, we want to hold the writeLock only during the close.
- Fix the flaky tests

Reviewers: Divij Vaidya <diviv@amazon.com>
2023-08-24 13:42:13 +02:00
Mehari Beyene 25b128de81
KAFKA-14991: KIP-937-Improve message timestamp validation (#14135)
This implementation introduces two new configurations `log.message.timestamp.before.max.ms` and `log.message.timestamp.after.max.ms` and deprecates `log.message.timestamp.difference.max.ms`.

The default value for all these three configs is maintained to be Long.MAX_VALUE for backward compatibility but with the newly added configurations we can have a finer control when validating message timestamps that are in the past and the future compared to the broker's timestamp.

To maintain backward compatibility if the default value of `log.message.timestamp.before.max.ms` is not changed, we are assuming users are still using the deprecated config `log.message.timestamp.difference.max.ms` and validation is done using its value. This ensures that existing customers who have customized the value of `log.message.timestamp.difference.max.ms` will continue to see no change in behavior.

Reviewers: Divij Vaidya <diviv@amazon.com>, Christo Lolov <lolovc@amazon.com>
2023-08-24 12:04:55 +02:00
David Mao eefa812453
MINOR: Delete unused class, LogOffsetMetadata toString formatting (#14246)
Noticed that there was a dangling unused class (LongRef, replaced by PrimitiveRef.LongRef), and the LogOffsetMetadata toString was a little oddly formatted.

Reviewers: Justine Olshan <jolshan@confluent.io>
2023-08-20 15:16:27 -07:00
Kamal Chandraprakash 6492164d9c
KAFKA-15167: Tiered Storage Test Harness Framework (#14116)
`TieredStorageTestHarness` is a base class for integration tests exercising the tiered storage functionality. This uses  `LocalTieredStorage` instance as the second-tier storage system and `TopicBasedRemoteLogMetadataManager` as the remote log metadata manager.

Co-authored-by: Alexandre Dupriez <alexandre.dupriez@gmail.com>
Co-authored-by: Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-08-20 17:15:52 +05:30
DL1231 4f88fb28f3
KAFKA-15130: Delete remote segments when deleting a topic (#13947)
* Delete remote segments when deleting a topic

Co-authored-by: Kamal Chandraprakash <kchandraprakash@uber.com>
Co-authored-by: d00791190 <dinglan6@huawei.com>
2023-08-18 18:21:09 +05:30
vamossagar12 ee27773549
KAFKA-15329: Make default remote.log.metadata.manager.class.name as topic based RLMM (#14202)
As described in the KIP here the default value of remote.log.metadata.manager.class.name should be TopicBasedRemoteLogMetadataManager

Reviewers: Luke Chen <showuon@gmail.com>, Kamal Chandraprakash <kchandraprakash@uber.com>, Divij Vaidya <diviv@amazon.com>
2023-08-16 09:46:17 +08:00
Kamal Chandraprakash 696a56dd2b
KAFKA-15295: Add config validation when remote storage is enabled on a topic (#14176)
Add config validation which verifies that system level remote storage is enabled when enabling remote storage for a topic. In case verification fails, it throws InvalidConfigurationException.

Reviewers: Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com>,  Luke Chen <showuon@gmail.com>
2023-08-15 20:43:11 +02:00
Luke Chen cdbc9a8d88
KAFKA-15083: add config with "remote.log.metadata" prefix (#14151)
When configuring RLMM, the configs passed into configure method is the RemoteLogManagerConfig. But in RemoteLogManagerConfig, there's no configs related to remote.log.metadata.*, ex: remote.log.metadata.topic.replication.factor. So, even if users have set the config in broker, it'll never be applied.

This PR fixed the issue to allow users setting RLMM prefix: remote.log.metadata.manager.impl.prefix (default is rlmm.config.), and then, appending the desired remote.log.metadata.* configs, it'll pass into RLMM, including remote.log.metadata.common.client./remote.log.metadata.producer./ remote.log.metadata.consumer. prefixes.

Ex:

# default value
# remote.log.storage.manager.impl.prefix=rsm.config.
# remote.log.metadata.manager.impl.prefix=rlmm.config.

rlmm.config.remote.log.metadata.topic.num.partitions=50
rlmm.config.remote.log.metadata.topic.replication.factor=4

rsm.config.test=value

Reviewers: Christo Lolov <christololov@gmail.com>, Kamal Chandraprakash <kchandraprakash@uber.com>, Divij Vaidya <diviv@amazon.com>
2023-08-11 10:42:14 +08:00
Luke Chen 748175ce62
KAFKA-15189: only init remote topic metrics when enabled (#14133)
Only initialize remote topic metrics when system-wise remote storage is enabled to avoid impacting performance for existing brokers. Also add tests.

Reviewers: Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-08-05 13:00:16 +08:00
Ivan Yurchenko b3db905b27
KAFKA-15107: Support custom metadata for remote log segment (#13984)
* KAFKA-15107: Support custom metadata for remote log segment

This commit does the changes discussed in the KIP-917. Mainly, changes the `RemoteStorageManager` interface in order to return `CustomMetadata` and then ensures these custom metadata are stored, propagated, (de-)serialized correctly along with the standard metadata throughout the whole lifecycle. It introduces the `remote.log.metadata.custom.metadata.max.size` to limit the custom metadata size acceptable by the broker and stop uploading in case a piece of metadata exceeds this limit.

On testing:
1. `RemoteLogManagerTest` checks the case when a piece of custom metadata is larger than the configured limit.
2. `RemoteLogSegmentMetadataTest` checks if `createWithUpdates` works correctly, including custom metadata.
3. `RemoteLogSegmentMetadataTransformTest`, `RemoteLogSegmentMetadataSnapshotTransformTest`, and `RemoteLogSegmentMetadataUpdateTransformTest` were added to test the corresponding class (de-)serialization, including custom metadata.
4. `FileBasedRemoteLogMetadataCacheTest` checks if custom metadata are being correctly saved and loaded to a file (indirectly, via `equals`).
5. `RemoteLogManagerConfigTest` checks if the configuration setting is handled correctly.

Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>
2023-08-04 18:23:25 +05:30
Divij Vaidya 7d39d7400c
MINOR: Fix debug logs to display TimeIndexOffset (#13935)
Reviewers: Luke Chen <showuon@gmail.com>
2023-08-03 11:05:01 +02:00
Kamal Chandraprakash d89b26ff44
KAFKA-12969: Add broker level config synonyms for topic level tiered storage configs (#14114)
KAFKA-12969: Add broker level config synonyms for topic level tiered storage configs.

Topic -> Broker Synonym:
local.retention.bytes -> log.local.retention.bytes
local.retention.ms -> log.local.retention.ms

We cannot add synonym for `remote.storage.enable` topic property as it depends on KIP-950

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-08-03 13:56:00 +05:30
hzh0425 660e6fe810
MINOR: Fix some typos in remote.metadata.storage (#13133)
Fix some typos in remote.metadata.storage

Reviewers: Luke Chen <showuon@gmail.com>
2023-08-01 14:53:42 +08:00
Christo Lolov 722b259961
KAFKA-14038: Optimise calculation of size for log in remote tier (#14049)
Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-07-28 11:10:37 +05:30
Justine Olshan 38781f9aea
KAFKA-14920: Address timeouts and out of order sequences (#14033)
When creating a verification state entry, we also store sequence and epoch. On subsequent requests, we will take the latest epoch seen and the earliest sequence seen. That way, if we try to append a sequence after the earliest seen sequence, we can block that and retry. This addresses potential OutOfOrderSequence loops caused by errors during verification (coordinator loading, timeouts, etc).

Reviewers:  David Jacot <david.jacot@gmail.com>,  Artem Livshits <alivshits@confluent.io>
2023-07-24 13:08:57 -07:00
Kamal Chandraprakash 84691b11f6
KAFKA-15168: Handle overlapping remote log segments in RemoteLogMetadata cache (#14004)
KAFKA-15168: Handle overlapping remote log segments in RemoteLogMetadata cache

Reviewers: Satish Duggana <satishd@apache.org>, Viktor Nikitash <nikitashvictor@pdffiller.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Abhijeet Kumar <abhijeet.cse.kgp@gmail.com>
2023-07-24 19:36:25 +05:30
Owen Leung a3204aed2e
KAFKA-15194: Prepend offset in the filenames used by LocalTieredStorage (#14057)
Reviewers: Divij Vaidya <diviv@amazon.com>
2023-07-22 13:47:26 +02:00
Jorge Esteban Quilcate Otoya cc4e699d4c
MINOR: Minor logging and doc related improvements in topic-based RLMM consumer-manager/task (#14045)
Improved logging and docs on consumer manager/task call paths.

Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>
2023-07-22 10:03:35 +05:30
Abhijeet Kumar fd3b1137d2
KAFKA-14953: Add tiered storage related metrics (#13944)
* KAFKA-14953: Adding RemoteLogManager metrics
In this PR, I have added the following metrics that are related to tiered storage mentioned in[ KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage).
|Metric|Description|
|-----------------------------------------|--------------------------------------------------------------|
| RemoteReadRequestsPerSec                    | Number of remote storage read requests per second               |
| RemoteWriteRequestsPerSec                    | Number of remote storage write requests per second              |
| RemoteBytesInPerSec                                | Number of bytes read from remote storage per second           |
| RemoteReadErrorsPerSec                          | Number of remote storage read errors per second                   |
| RemoteBytesOutPerSec                             | Number of bytes copied to remote storage per second            |
| RemoteWriteErrorsPerSec                          | Number of remote storage write errors per second                   |
| RemoteLogReaderTaskQueueSize             | Number of remote storage read tasks pending for execution.  |
| RemoteLogReaderAvgIdlePercent             | Average idle percent of the remote storage reader thread pool|
| RemoteLogManagerTasksAvgIdlePercent | Average idle percent of RemoteLogManager thread pool          |

Added unit tests for all the rate metrics.

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Staniel Yao<yaolixinylx@gmail.com>, hudeqi<1217150961@qq.com>, Satish Duggana <satishd@apache.org>
2023-07-18 20:16:19 +05:30
Kamal Chandraprakash cbd5d52ec7
KAFKA-9564: Local Tiered Storage implementation for Remote Storage Manager (#13837)
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Christo Lolov <lolovc@amazon.com>, Satish Duggana <satishd@apache.org>
2023-07-17 21:49:43 +05:30
Justine Olshan ea0bb00126
KAFKA-14884: Include check transaction is still ongoing right before append (take 2) (#13787)
Introduced extra mapping to track verification state.

When verifying, there is a race condition that the add partitions verification response returns that the partition is in the ongoing transaction, but an abort marker is written before we get to append. Therefore, we track any given transaction we are verifying with an object unique to that transaction.

We check this unique state upon the first append to the log. After that, we can rely on currentTransactionFirstOffset. We remove the verification state on appending to the log with a transactional data record or marker.

We will also clean up lingering verification state entries via the producer state entry expiration mechanism. We do not update the the timestamp on retrying a verification for a transaction, so each entry must be verified before producer.id.expiration.ms.

There were a few other fixes:
- Moved the transaction manager handling for failed batch into the future completed exceptionally block to avoid processing it twice (this caused issues in unit tests)
- handle interrupted exceptions encountered when callback thread encountered them
- change handling to throw error if we try to set verification state and leaderLogIfLocal is None.

Reviewers: David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Jason Gustafson <jason@confluent.io>
2023-07-14 15:18:11 -07:00
Satish Duggana 7e2f878713
KAFKA-14522 Rewrite/Move of RemoteIndexCache to storage module. (#13275)
KAFKA-14522 Rewrite and Move of RemoteIndexCache to storage module.
Cleanedup index file suffix usages and other minor cleanups

Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
2023-07-11 23:55:23 +05:30
Divij Vaidya 7bdcb22cf6
MINOR: Refactor & cleanup for RemoteIndexCache (#13936)
- Add new unit tests
- Change the on-disk filename from <offset>_<uuid>_.<indexSuffix> to <offset>_<uuid>.<indexSuffix> i.e. remove trailing underscore after
- Fix a small bug where we were parsing offset as Int when reading the file name from disk. Offset is long.
- Perform input validation in RemoteLogSegmentMetadata.
- Remove an extra loop in cleaner thread. Shutdownable thread already performs looping.

Reviewers: Jorge Esteban Quilcate Otoya <jorge.quilcate@aiven.io>, Satish Duggana <satishd@apache.org>
2023-07-08 12:52:22 +02:00
Jorge Esteban Quilcate Otoya 0ae1d22879
KAFKA-15135: fix(storage): pass endpoint configurations as client common to TBRLMM (#13938)
Pass endpoint properties from RLM to TBRLMM and validate those are not ignored.

Reviewers: Luke Chen <showuon@gmail.com>
2023-07-03 09:16:15 +08:00
Jorge Esteban Quilcate Otoya 43574beb97
KAFKA-15131: Improve RemoteStorageManager exception handling documentation (#13923)
Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>, Luke Chen <showuon@gmail.com>
2023-06-30 14:37:48 +02:00
Luke Chen 7a6435af49
KAFKA-15037: pass remoteLogEnabled to unifiedLog (#13779)
UnifiedLog relied on the remoteStorageSystemEnable to identify if the broker is enabling remote storage, but we never pass this value from the config into UnifiedLog. So it'll always be false.

In this PR, I did:

   - pass remoteStorageSystemEnable to UnifiedLog
   - remove remoteLogManager from the class member of UnifiedLog since only UnifiedLog#fetchOffsetByTimestamp needs remoteLogManager, and this can be passed when called from ReplicaManager.

Reviewers: Satish Duggana <satishd@apache.org>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
2023-06-05 15:00:16 +08:00
Justine Olshan 9edf2ec5cc
MINOR: Add transaction verification config to producerStateManager config (#13770)
I have moved this config into producer state manager so it can be checked easily under the log lock when we are about to append.

Only a few test files currently use the validation and those have been verified to work via running the tests.

Reviews:  David Jacot <djacot@confluent.io>
2023-05-30 13:46:17 -07:00
Satish Duggana 6f19730164
KAFKA-9579 Fetch implementation for records in the remote storage through a specific purgatory. (#13535)
This change includes
- Recognize the fetch requests with out of range local log offsets
- Add fetch implementation for the data lying in the range of [logStartOffset, localLogStartOffset]
- Add a new purgatory for async remote read requests which are served through a specific thread pool

We have an extended version of remote fetch that can fetch from multiple remote partitions in parallel, which we will raise as a followup PR.

A few tests for the newly introduced changes are added in this PR. There are some tests available for these scenarios in 2.8.x, refactoring with the trunk changes, will add them in followup PRs.

Other contributors:
Kamal Chandraprakash <kchandraprakash@uber.com> - Further improvements and adding a few tests
Luke Chen <showuon@gmail.com> - Added a few test cases for these changes.

PS: This functionality is pulled out from internal branches with other functionalities related to the feature in 2.8.x. The reason for not pulling all the changes as it makes the PR huge, and burdensome to review and it also needs other metrics, minor enhancements(including perf), and minor changes done for tests. So, we will try to have followup PRs to cover all those.

Reviewers: Jun Rao <junrao@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Divij Vaidya <diviv@amazon.com>,  Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
2023-05-18 06:37:37 +05:30
Satish Duggana e99984248d
KAFKA-9550 Copying log segments to tiered storage in RemoteLogManager (#13487)
Added functionality to copy log segments, indexes to the target remote storage for each topic partition enabled with tiered storage. This involves creating scheduled tasks for all leader partition replicas to copy their log segments in sequence to tiered storage.

Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>
2023-04-12 13:55:36 +08:00
Luke Chen 31f9a54cba
KAFKA-14850: introduce InMemoryLeaderEpochCheckpoint (#13456)
The motivation for introducing InMemoryLeaderEpochCheckpoint is to allow remote log manager to create the RemoteLogSegmentMetadata(RLSM) with the correct leader epoch info for a specific segment. To do that, we need to rely on the LeaderEpochCheckpointCache to truncate from start and end, to get the epoch info. However, we don't really want to truncate the epochs in cache (and write to checkpoint file in the end). So, we introduce this InMemoryLeaderEpochCheckpoint to feed into LeaderEpochCheckpointCache, and when we truncate the epoch for RLSM, we can do them in memory without affecting the checkpoint file, and without interacting with file system.

Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
2023-04-05 20:11:32 +08:00
Luke Chen d849d66717
Use readlock for reading epochs in LeaderEpochFIleCache (#13483)
Reviewers: Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
2023-03-31 18:03:29 +05:30
Ivan Yurchenko dc1cd001be
KAFKA-14795: Provide message formatter for RemoteLogMetadata (#13362)
* KAFKA-14795: Provide message formatter for RemoteLogMetadata

This commit introduces a formatter for `RemoteLogMetadata`.

Example usage:
```bash
bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic __remote_log_metadata --from-beginning --formatter 'org.apache.kafka.server.log.remote.metadata.storage.serialization.RemoteLogMetadataSerde$RemoteLogMetadataFormatter'

RemoteLogSegmentMetadata{remoteLogSegmentId=RemoteLogSegmentId{topicIdPartition=M1z1YtfhQ5i7oqLNve_0UQ:topic1-0, id=iWtc1Z6xQu2_DJXTklzKxQ}, startOffset=97990, endOffset=98467, brokerId=0, maxTimestampMs=1678292889855, eventTimestampMs=1678292938280, segmentLeaderEpochs={0=97990}, segmentSizeInBytes=511460, state=COPY_SEGMENT_STARTED}
```

Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
2023-03-21 14:35:49 +08:00
Calvin Liu 79b5f7f1ce
KAFKA-14617: Add ReplicaState to FetchRequest (KIP-903) (#13323)
This patch is the first part of KIP-903. It updates the FetchRequest to include the new tagged ReplicaState field which replaces the now deprecated ReplicaId field. The FetchRequest version is bumped to version 15 and the MetadataVersion to 3.5-IV1.

Reviewers: David Jacot <djacot@confluent.io>
2023-03-16 14:04:34 +01:00
Ismael Juma 77215eded7
KAFKA-14792: Race condition in LazyIndex.get() (#13359)
`LazyIndex.get()` has a race condition that can result in a ClassCastException being thrown in some cases.

This was introduced when `LazyIndex` was rewritten from Scala to Java.

I didn't include a test since it's a bit overkill to add a concurrent test for this.

Reviewers: Jun Rao <junrao@gmail.com>
2023-03-07 15:56:24 -08:00
Satish Duggana 97105a8e58
KAFKA-14726 Move/rewrite of LogReadInfo, LogOffsetSnapshot, LogStartOffsetIncrementReason to storage module (#13304)
Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2023-03-04 06:30:41 +05:30
Anastasia Vela d479d129e0
KAFKA-13999: Add ProducerCount metrics (KIP-847) (#13078)
This is the PR for the implementation of KIP-847: https://cwiki.apache.org/confluence/display/KAFKA/KIP-847%3A+Add+ProducerIdCount+metrics
Add ProducerIdCount metric at the broker level:

kafka.server:type=ReplicaManager,name=ProducerIdCount
Added unit tests below to ensure the metric reported the count correctly.

---------

Co-authored-by: Artem Livshits <84364232+artemlivshits@users.noreply.github.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Divij Vaidya <diviv@amazon.com>, Christo Lolov <christo_lolov@yahoo.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Justine Olshan <jolshan@confluent.io>
2023-03-01 14:20:15 -08:00
Satish Duggana 510e99e1a2
MINOR Moved a few log segment util methods from LocalLog to LogFileUtils (#13309)
Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>
2023-03-01 10:58:49 +05:30
Satish Duggana 069ce59e1e
KAFKA 14714: Move/Rewrite RollParams, LogAppendInfo, and LeaderHwChange to storage module. (#13255)
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2023-02-22 23:12:04 +05:30
David Jacot 4dd27a9f21
KAFKA-14673; Add high watermark listener to Partition/Log layers (#13196)
In the context of KIP-848, we implements are new group coordinator in Java. This new coordinator follows the architecture of the new quorum controller. It is basically a replicated state machine that writes to the log and commits its internal state when the writes are committed. At the moment, the only way to know when a write is committed is to use a delayed fetch. This is not ideal in our context because a delayed fetch can be completed before the write is actually committed to the log.

This patch proposes to introduce a high watermark listener to the Partition/Log layers. This will allow the new group coordinator to simply listen to changes and commit its state based on this. This mechanism is simpler and lighter as well.

Reviewers: Christo Lolov <lolovc@amazon.com>, Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
2023-02-20 08:26:17 +01:00
Christo Lolov ba0c5b0902
MINOR: Simplify JUnit assertions in tests; remove accidental unnecessary code in tests (#13219)
* assertEquals called on array
* Method is identical to its super method
* Simplifiable assertions
* Unused imports

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Divij Vaidya <diviv@amazon.com>
2023-02-16 16:13:31 +01:00
Ismael Juma 959756ae9d
MINOR: Remove unnecessary usage of `LazyIndex` (#13218)
The remote index classes use `LazyIndex`, but immediately
force materialization. This results in more verbose code
and it's misleading since the indexes are not lazily
used in practice.

Also simplify `LazyIndex.forOffset/forTime` by removing
`writable` parameter, which is always `true`.

Reviewers: Satish Duggana <satishd@apache.org>
2023-02-13 13:46:37 +05:30
Satish Duggana c576e02849
KAFKA-14480 Move/Rewrite ProducerStateManager to storage module. (#13040)
KAFKA-14480 Move/Rewrite of ProducerStateManager to the storage module.

Replaced `File.listFiles` with `Files.list` in ProducerStateManager.listSnapshotFiles
Used `Path` instead of `File` in ProducerStateManager.isSnapshotFile to check whether the given path is a file or not and has a suffix of '.snapshot'.
2023-02-12 13:00:51 +05:30
Satish Duggana 1d3fb76092
KAFKA-14688 Move package org.apache.kafka.server.log.internals to org.apache.kafka.storage.internals.log (#13213)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-02-08 09:22:42 +05:30
Christo Lolov a0a9b6ffea
MINOR: Remove unnecessary code (#13210)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Divij Vaidya <diviv@amazon.com>
2023-02-07 17:37:45 +01:00
Satish Duggana da2e8dce71
KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module. (#13046)
KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.

For broader context on this change, you may want to look at KAFKA-14470: Move log layer to the storage module

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>
2023-02-07 15:37:23 +05:30
Federico Valeri 111f02cc74
KAFKA-14568: Move FetchDataInfo and related to storage module (#13085)
Part of KAFKA-14470: Move log layer to storage module.

Reviewers: Ismael Juma <ismael@juma.me.uk>

Co-authored-by: Ismael Juma <ismael@juma.me.uk>
2023-01-12 21:32:23 -08:00
Akhilesh C db49070760
KAFKA-14493: Introduce Zk to KRaft migration state machine STUBs in KRaft controller. (#12998)
This patch introduces a preliminary state machine that can be used by KRaft
controller to drive online migration from Zk to KRaft.

MigrationState -- Defines the states we can have while migration from Zk to
KRaft.

KRaftMigrationDriver -- Defines the state transitions, and events to handle
actions like controller change, metadata change, broker change and have
interfaces through which it claims Zk controllership, performs zk writes and
sends RPCs to ZkBrokers.

MigrationClient -- Interface that defines the functions used to claim and
relinquish Zk controllership, read to and write from Zk.

Co-authored-by: David Arthur <mumrah@gmail.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2023-01-09 10:44:11 -08:00
Ismael Juma 55212ec2af
MINOR: Remove public mutable fields from ProducerAppendInfo (#13091)
Replace them with accessors and mutators.

Reviewers: Satish Duggana <satishd@apache.org>
2023-01-08 20:46:14 +05:30
Satish Duggana 2dec39d6e4
KAFKA-14558: Move/Rewrite LastRecord, TxnMetadata, BatchMetadata, ProducerStateEntry, and ProducerAppendInfo to the storage module (#13043)
For broader context on this change, see:
* KAFKA-14470: Move log layer to storage module.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-01-07 20:13:38 -08:00
Ivan Yurchenko 5126fe251b
KAFKA-14482: Move LoadedLogOffsets to storage module (#13065)
This is a relatively independent change in the context of KAFKA-14482.

Reviewers: Ismael Juma <ismael@juma.me.uk>, Christo Lolov <lolovc@amazon.com>, Divij Vaidya <diviv@amazon.com>, Satish Duggana <satishd@apache.org>
2023-01-05 08:57:55 +05:30
Ismael Juma 96d9710c17
KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module (#13049)
Additional notable changes to fix multiple dependency ordering issues:

* Moved `ConfigSynonym` to `server-common`
* Moved synonyms from `LogConfig` to `ServerTopicConfigSynonyms `
* Removed `LogConfigDef` `define` overrides and rely on
   `ServerTopicConfigSynonyms` instead.
* Moved `LogConfig.extractLogConfigMap` to `KafkaConfig`
* Consolidated relevant defaults from `KafkaConfig`/`LogConfig` in the latter
* Consolidate relevant config name definitions in `TopicConfig`
* Move `ThrottledReplicaListValidator` to `storage`

Reviewers: Satish Duggana <satishd@apache.org>, Mickael Maison <mickael.maison@gmail.com>
2023-01-04 02:42:52 -08:00
Greg Harris 1477c31198
KAFKA-13881: Add Storage package info (#13053)
This module exposes a single package as public API that needs a package description.

Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2023-01-02 18:38:52 +01:00
Satish Duggana 026105d05f
KAFKA-14550: Move SnapshotFile and CorruptSnapshotException to storage module (#13039)
For broader context on this change, see:

* KAFKA-14470: Move log layer to storage module

Reviewers: Ismael Juma <ismael@juma.me.uk>
2023-01-02 07:31:40 -08:00
Federico Valeri 9f026dee6b
KAFKA-14554: Move ClassLoaderAwareRemoteStorageManagerTest to storage module (#13048)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-12-28 06:42:26 -08:00
Mickael Maison 8184ada6a5
KAFKA-14543: Move LogOffsetMetadata to storage module (#13038)
Reviewers: Ismael Juma <ismael@juma.me.uk>, dengziming <dengziming1993@gmail.com>, Satish Duggana <satishd@apache.org>, Federico Valeri <fedevaleri@gmail.com>
2022-12-27 17:12:02 -08:00
Ismael Juma 871289c5c4
KAFKA-14476: Move OffsetMap and related to storage module (#13042)
For broader context on this change, please check:

* KAFKA-14470: Move log layer to storage module

Reviewers: dengziming <dengziming1993@gmail.com>, Satish Duggana <satishd@apache.org>, Federico Valeri <fedevaleri@gmail.com>
2022-12-23 08:19:00 -08:00
Federico Valeri 06af8fc630
KAFKA-14549: Move LogDirFailureChannel to storage module (#13041)
For broader context on this change, please check:

* KAFKA-14470: Move log layer to storage module

Reviewers: dengziming <dengziming1993@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Satish Duggana <satishd@apache.org>, Ismael Juma <ismael@juma.me.uk>
2022-12-23 07:13:43 -08:00
Ismael Juma e8232edd24
KAFKA-14477: Move LogValidator and related to storage module (#13012)
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>
2022-12-21 16:57:02 -08:00
Ismael Juma aad5b0a463
MINOR: Avoid unnecessary allocations in index binary search (#13024)
* MINOR: Avoid unnecessary allocations in index binary search

* Fix bug due to inverse usage of SearchType.

Also improve name clarity.
2022-12-21 05:10:44 -08:00
Ismael Juma c4f10364cd
KAFKA-14475: Move TimeIndex/LazyIndex to storage module (#13010)
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>
2022-12-20 19:08:40 -08:00
Ismael Juma 82c9216c77
KAFKA-14474: Move OffsetIndex to storage module (#13009)
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>
2022-12-20 11:45:47 -08:00
Ismael Juma d521f8110e
KAFKA-14473: Move AbstractIndex to storage module (#13007)
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>
2022-12-19 19:33:24 -08:00
Ismael Juma 26fcf73feb
MINOR: Use `withClassLoader` in a couple more places (#13018)
Instead of try/finally.

Reviewers: Ron Dagostino <rndgstn@gmail.com>
2022-12-19 16:30:54 -05:00
Ismael Juma e2678d57d0
KAFKA-14472: Move TransactionIndex and related to storage module (#12996)
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>
2022-12-19 11:31:37 -08:00
Satish Duggana e3cb2deff6
KAFKA-14466: Move ClassloaderAwareRemoteStorageManager to storage module (#13013)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-12-19 06:37:33 -08:00
Ismael Juma 95dc9d9eed
Move IndexEntry and related to storage module (#12993)
For broader context on this change, please check:

* KAFKA-14470: Move log layer to storage module

Reviewers: dengziming <dengziming1993@gmail.com>
2022-12-17 10:07:11 -08:00
Satish Duggana 7146ac57ba
[KAFKA-13369] Follower fetch protocol changes for tiered storage. (#11390)
This PR implements the follower fetch protocol as mentioned in KIP-405.

Added a new version for ListOffsets protocol to receive local log start offset on the leader replica. This is used by follower replicas to find the local log star offset on the leader.

Added a new version for FetchRequest protocol to receive OffsetMovedToTieredStorageException error. This is part of the enhanced fetch protocol as described in KIP-405.

We introduced a new field locaLogStartOffset to maintain the log start offset in the local logs. Existing logStartOffset will continue to be the log start offset of the effective log that includes the segments in remote storage.

When a follower receives OffsetMovedToTieredStorage, then it tries to build the required state from the leader and remote storage so that it can be ready to move to fetch state.

Introduced RemoteLogManager which is responsible for

initializing RemoteStorageManager and RemoteLogMetadataManager instances.
receives any leader and follower replica events and partition stop events and act on them
also provides APIs to fetch indexes, metadata about remote log segments.
Followup PRs will add more functionality like copying segments to tiered storage, retention checks to clean local and remote log segments. This will change the local log start offset and make sure the follower fetch protocol works fine for several cases.

You can look at the detailed protocol changes in KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage#KIP405:KafkaTieredStorage-FollowerReplication

Co-authors: satishd@apache.org, kamal.chandraprakash@gmail.com, yingz@uber.com

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Tirtha Chatterjee <tirtha.p.chatterjee@gmail.com>, Yaodong Yang <yangyaodong88@gmail.com>, Divij Vaidya <diviv@amazon.com>, Luke Chen <showuon@gmail.com>, Jun Rao <junrao@gmail.com>
2022-12-17 09:36:44 -08:00
Divij Vaidya 0a50005408
KAFKA-13929: Replace legacy File.createNewFile() with NIO.2 Files.createFile() (#12197)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-06-10 13:28:55 +02:00
Jason Gustafson 711b603ddc
MINOR: Cleanup admin creation logic in integration tests (#11790)
There seemed to be a little sloppiness in the integration tests in regard to admin client creation. Not only was there duplicated logic, but it wasn't always clear which listener the admin client was targeting. This made it difficult to tell in the context of authorization tests whether we were indeed testing with the right principal. As an example, we had a method in TestUtils which was using the inter-broker listener implicitly. This meant that the test was using the broker principal which had super user privilege. This was intentional, but I think it would be clearer to make the dependence on this listener explicit. This patch attempts to clean this up a bit by consolidating some of the admin creation logic and making the reliance on the listener clearer.

Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
2022-02-24 07:37:28 -08:00
Colin Patrick McCabe af8100b94f
KAFKA-13340: Change ZooKeeperTestHarness to QuorumTestHarness (#11417)
Change ZooKeeperTestHarness to QuorumTestHarness so that integration tests which inherit from
this class can test Kafka in both ZK and KRaft mode. Test cases which do this can specify the
modes they support by including a ParameterizedTest annotation before each test case, like the
following:

@ParameterizedTest
@valuesource(strings = Array("zk", "kraft"))
def testValidCreateTopicsRequests(quorum: String): Unit = { ... }

For each value that is specified here (zk, kraft), the test case will be run once in the
appropriate mode. So the test shown above is run twice. This allows integration tests to be
incrementally converted over to support KRaft mode, rather than rewritten to support it. For
now, test cases which do not specify a quorum argument will continue to run only in ZK mode.

JUnit5 makes the quorum annotation visible in the TestInfo object which each @BeforEeach
function in a test can optionally take. Therefore, this PR converts over the setUp function of
the quorum base class, plus every derived class, to take a TestInfo argument. The TestInfo
object gets "passed up the chain" to the base class, where it determines which quorum type we
create (ZK or KRaft). In a few cases, I discovered test cases inheriting from the test harness
that had more than one @BeforeEach function. Because the JUnit5 framework does not define the
order in which @BeforeEach hooks are run, I changed these to overload setUp() instead, to avoid
undefined behavior.

The general approach taken here is to make as much as possible work with KRaft, but to leave some
things as ZK-only when appropriate. For example, a test that explicitly requests an AdminZkClient
object will get an exception if it is running in KRaft mode. Similarly, tests which explicitly
request KafkaServer rather than KafkaBroker will get an exception when running in KRaft mode.
As a proof of concept, this PR converts over kafka.api.MetricsTest to support KRaft.

This PR also renames the quorum controller event handler thread to include the text
"QuorumControllerEventHandler". This allows QuorumTestHarness to check for hanging quorum
controller threads, as it does for hanging ZK-based controller threads.

Finally, ConsumerBounceTest#testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup
caused many failing test runs. Therefore, I disabled it here and filed KAFKA-13421 to fix the
test logic to be more reliable.

Reviewers: Jason Gustafson <jason@confluent.io>, Igor Soarez <soarez@apple.com>
2021-10-30 08:00:34 -07:00
Satish Duggana 6288b5370d
MINOR: Renamed a few record definition files with the existing convention. (#11414)
Reviewers: Jun Rao <junrao@gmail.com>
2021-10-21 13:44:08 -07:00
Satish Duggana 34d56dc8d0
KAFKA-12802 Added a file based cache for consumed remote log metadata for each partition to avoid consuming again incase of broker restarts. (#11058)
Added snapshots for consumed remote log metadata for each partition to avoid consuming again in case of broker restarts. These snapshots are stored in the respective topic partition log directories.

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Jun Rao <junrao@gmail.com>
2021-10-11 10:24:55 -07:00
Ismael Juma 1a3e23a579
MINOR: TopicIdPartition improvements (#11374)
1. It should not require a TopicPartition during construction and normal
usage.
2. Simplify `equals` since `topicId` and `topicPartition` are never
null.
3. Inline `Objects.hash` to avoid array allocation.
4. Make `toString` more concise using a similar approach as
`TopicPartition` since this `TopicIdPartition` will replace
`TopicPartition` in many places in the future.
5. Add unit tests for `TopicIdPartition`, it seems like we had none.
6. Minor clean-up in calling/called classes.

Reviewers: David Jacot <djacot@confluent.io>, Satish Duggana <satishd@apache.org>
2021-10-05 11:37:38 -07:00
Satish Duggana 1e19de3199
KAFKA-12988 Asynchronous API support for RemoteLogMetadataManager add/update methods. (#11033)
Added asynchronous API support for RemoeLogMetadataManager add/update/put methods.

Implemented the changes on default topic based RemoteLogMetadataManager.

Refactored the respective tests to cover the introduced asynchronous APIs.

Reviewers: Cong Ding <cong@ccding.com>, Jun Rao <junrao@gmail.com>
2021-09-09 10:06:25 -07:00
dengziming 1d22b0d706
KAFKA-10774; Admin API for Describe topic using topic IDs (#9769)
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Satish Duggana <satishd@apache.org>, Rajini Sivaram <rajinisivaram@googlemail.com>
2021-08-28 09:00:36 +01:00
Cong Ding 555f709175
MINOR: move tiered storage related configs to a separate class within LogConfig (#11110)
The original code uses a RemoteLogManagerConfig class to store KIP-405 configs and adds three configs to LogConfig. This makes the code complicated and developers may be confused.

This PR allows us to access RemoteLogManagerConfig from KafkaConfig and do the same for LogConfig. Kafka developers will see the same interface for the KIP-405 configs. After this change, if we want to read remoteStorageEnable we should use LogConfig.tieredLogConfig.remoteStorageEnable instead of LogConfig.remoteStorageEnable. The same for localRetentionMs and localRetentionBytes. If we want to read configs in RemoteLogManagerConfig, we should use KafkaConfig.tieredKafkaConfig.xxx.

Reviewers: Satish Duggana <satishd@apache.org>, Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-08-27 11:10:58 -07:00
Satish Duggana e8ce93bd53
KAFKA-9555 Added default RLMM implementation based on internal topic storage. (#10579)
KAFKA-9555 Added default RLMM implementation based on internal topic storage.

This is the initial version of the default RLMM implementation.
This includes changes containing default RLMM configs, RLMM implementation, producer/consumer managers.
Introduced TopicBasedRemoteLogMetadataManagerHarness which takes care of bringing up a Kafka cluster and create remote log metadata topic and initializes TopicBasedRemoteLogMetadataManager.
Refactored existing RemoteLogMetadataCacheTest to RemoteLogSegmentLifecycleTest to have parameterized tests to run both RemoteLogMetadataCache and also TopicBasedRemoteLogMetadataManager.
Refactored existing InmemoryRemoteLogMetadataManagerTest, RemoteLogMetadataManagerTest to have parameterized tests to run both InmemoryRemoteLogMetadataManager and also TopicBasedRemoteLogMetadataManager.

This is part of tiered storage KIP-405 efforts.

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Jun Rao <junrao@gmail.com>
2021-07-19 09:05:46 -07:00
Satish Duggana 56250f446a
KAFKA-12816 Added tiered storage related configs including remote log manager configs. (#10733)
Added tiered storage related configs including remote log manager configs.
Added local log retention configs to LogConfig.
Added tests for the added configs.

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-06-18 09:38:42 -07:00
Satish Duggana 7c7cecded6
MINOR Updated transaction index as optional in LogSegmentData. (#10848)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2021-06-15 13:56:22 -07:00
Satish Duggana 7ef3879429
KAFKA-12758 Added `server-common` module to have server side common classes. (#10638)
Added server-common module to have server side common classes. Moved ApiMessageAndVersion, RecordSerde, AbstractApiMessageSerde, and BytesApiMessageSerde to server-common module.

Reivewers:  Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-05-11 09:58:28 -07:00
Satish Duggana a1367f57f5
KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage. (#10271)
KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage. This topic will receive events of RemoteLogSegmentMetadata, RemoteLogSegmentUpdate, and RemotePartitionDeleteMetadata. These events are serialized into Kafka protocol message format.
Added tests for all the event types for that topic.

This is part of the tiered storaqe implementation KIP-405.

Reivewers:  Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-05-05 07:48:52 -07:00
Satish Duggana 327809024f
KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager. (#10218)
KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.

Added inmemory implementation for RemoteStorageManager and RemoteLogMetadataManager. A major part of inmemory RLMM will be used in the default RLMM implementation which will be based on topic storage. These will be used in unit tests for tiered storage.
Added tests for both the implementations and their supported classes.
This is part of tiered storage implementation, KIP-405.

Reivewers:  Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-04-13 10:14:03 -07:00
Satish Duggana 6e1723b483
MINOR Moved tiered storage API classes from clients module to a new storage-api wmodule. (#10489)
Moved tiered storage API classes from clients module to a new storage-api module.
Created storage and storage-api modules. All the remote storage API classes are moved to storage-api module. All the remote storage implementation classes will be added to storage module.

Reviewers: Jun Rao <junrao@gmail.com>
2021-04-07 13:52:50 -07:00