Commit Graph

76 Commits

Author SHA1 Message Date
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