Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-14470

Move log layer to storage module

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Open
    • Major
    • Resolution: Unresolved
    • None
    • None
    • None
    • None

    Description

      We introduced the `storage` module as part of KIP-405, but the existing log layer remains in the `core` module. Moving the log layer to the `storage` module would be another step towards improved modularity and build times (similar to `metadata`, `raft` and `group-coordinator`).

      We should do this in an incremental manner to make the code review process easier. I will create separate tasks, each one mapping to one pull request. In order to understand the feasibility, I tackled a few of the tasks myself.

      Help from the community is appreciated for the unassigned tasks, but it probably makes sense to do that after the initial PRs have been submitted.

      Attachments

        Issue Links

          1.
          Move IndexEntry and related to storage module Sub-task Resolved Ismael Juma
          2.
          Move TransactionIndex and related to storage module Sub-task Resolved Ismael Juma
          3.
          Move AbstractIndex to storage module Sub-task Resolved Ismael Juma
          4.
          Move OffsetIndex to storage module Sub-task Resolved Ismael Juma
          5.
          Move TimeIndex/LazyIndex to storage module Sub-task Resolved Ismael Juma
          6.
          Move OffsetMap to storage module Sub-task Resolved Ismael Juma
          7.
          Move LogValidator and related to storage module Sub-task Resolved Ismael Juma
          8.
          Move LogConfig/CleanerConfig to storage module Sub-task Resolved Ismael Juma
          9.
          Move CleanerConfig to storage module Sub-task Resolved Ismael Juma
          10.
          Move ProducerStateManager to storage module Sub-task Resolved Satish Duggana
          11.
          Move LogSegment/LogSegments to storage module Sub-task Resolved Ismael Juma
          12.
          Move LogLoader to storage module Sub-task Resolved Mickael Maison
          13.
          Move LocalLog to storage module Sub-task Resolved Mickael Maison
          14.
          Move UnifiedLog to storage module Sub-task In Progress Satish Duggana
          15.
          Move LogCleaner to storage module Sub-task Open Dmitry Werner
          16.
          Move LogCleanerManager to storage module Sub-task Open Dmitry Werner
          17.
          Move LogManager to storage module Sub-task Open Unassigned
          18.
          Move log layer tests to storage module Sub-task Open Unassigned
          19.
          Adjust visibility of classes moved to storage module Sub-task Open Unassigned
          20.
          Consider using UncheckedIOException instead of IOException in the log layer Sub-task Open Unassigned
          21.
          Replace BrokerCompressionCodec with BrokerCompressionType Sub-task Resolved Ismael Juma
          22.
          Move RemoteIndexCache to the storage module Sub-task Resolved Satish Duggana
          23.
          Move RemoteLogManager to the storage module Sub-task In Progress Satish Duggana
          24.
          Move LogDirFailureChannel to storage module Sub-task Resolved Federico Valeri
          25.
          Move LogOffsetMetadata to storage module Sub-task Resolved Mickael Maison
          26.
          MoveSnapshotFile and CorruptSnapshotException to storage module Sub-task Resolved Satish Duggana
          27.
          Move LeaderEpochFileCache to storage module Sub-task Resolved Satish Duggana
          28.
          Move ClassLoaderAwareRemoteStorageManagerTest to storage module Sub-task Resolved Federico Valeri
          29.
          Move LastRecord, TxnMetadata, BatchMetadata, ProducerStateEntry, and ProducerAppendInfo to the storage module. Sub-task Resolved Satish Duggana
          30.
          Move FetchDataInfo and related to storage module Sub-task Resolved Federico Valeri
          31.
          Move KafkaMetricsGroup to server-common module. Sub-task Resolved Ivan Yurchenko
          32.
          Move Scheduler/KafkaScheduler to server-common Sub-task Resolved Ismael Juma
          33.
          Move BrokerReconfigurable/KafkaConfig to server-common module. Sub-task Resolved Unassigned
          34.
          Move org.apache.kafka.server.log.internals to org.apache.kafka.storage.internals.log Sub-task Resolved Satish Duggana
          35.
          Move ShutdownableThread to server-commons module. Sub-task Resolved Satish Duggana
          36.
          Move/Rewrite RollParams, LogAppendInfo, and LeaderHwChange to storage module. Sub-task Resolved Satish Duggana
          37.
          Move/rewrite LogReadInfo, LogOffsetSnapshot, and LogStartOffsetIncrementReason to storage module. Sub-task Resolved Satish Duggana
          38.
          Move classes in kafka.raft from core module to raft module Sub-task Open Ismael Juma
          39.
          Move `BrokerTopicStats` and `BrokerTopicMetrics` to `org.apache.kafka.storage.log.metrics` (storage module) Sub-task Resolved PoAn Yang
          40.
          Move kafka.log.remote.quota to storage module Sub-task Resolved Mickael Maison
          41.
          Move LogSegmentTest to storage module Sub-task Resolved kangning.li
          42.
          Move LogSegmentsTest to storage module Sub-task Resolved kangning.li

          Activity

            People

              ijuma Ismael Juma
              ijuma Ismael Juma
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated: