You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 37 Next »

Authors: Sriharsha Chintalapani, Suresh Srinivas, Ying Zheng, Satish Duggana


Status

Current State: Discussion

Discussion Thread: Discuss Thread

JIRA KAFKA-7739 - Getting issue details... STATUS


Motivation

Kafka is an important part of data infrastructure and is seeing significant adoption and growth. As the Kafka cluster size grows and more data is stored in Kafka for a longer duration, several issues related to scalability, efficiency, and operations become important to address.

Kafka stores the messages in append-only log segments on local disks on Kafka brokers. The retention period for the log is based on `log.retention` that can be set system-wide or per topic. Retention gives the guarantee to consumers that even if their application failed or was down for maintenance, it can come back within the retention period to read from where it left off without losing any data.

The total storage required on a cluster is proportional to the number of topics/partitions, the rate of messages, and most importantly the retention period. A Kafka broker typically has a large number of disks with the total storage capacity of 10s of TBs. The amount of data locally stored on a Kafka broker presents many operational challenges.


Kafka as a long-term storage service

Kafka has grown in adoption to become the entry point of all of the data. It allows users to not only consume data in real-time but also gives the flexibility to fetch older data based on retention policies. Given the simplicity of Kafka protocol and wide adoption of consumer API, allowing users to store and fetch data with longer retention help make Kafka one true source of data.

Currently, Kafka is configured with a shorter retention period in days (typically 3 days) and data older than the retention period is copied using data pipelines to a more scalable external storage for long-term use, such as HDFS. This results in data consumers having to build different versions of applications to consume the data from different systems depending on the age of the data.

Kafka cluster storage is typically scaled by adding more broker nodes to the cluster. But this also adds needless memory and CPUs to the cluster making overall storage cost less efficient compared to storing the older data in external storage. Larger cluster with more nodes also adds to the complexity of deployment and increases the operational costs.

Kafka local storage and operational complexity

When a broker fails, the failed node is replaced by a new node. The new node must copy all the data that was on the failed broker from other replicas. Similarly, when a new Kafka node is added to scale the cluster storage, cluster rebalancing assigns partitions to the new node which also requires copying a lot of data. The time for recovery and rebalancing is proportional to the amount of data stored locally on a Kafka broker. In setups that have many Kafka clusters running 100s of brokers, a node failure is a common occurrence, with a lot of time spent in recovery making operations difficult and time-consuming.

Reducing the amount of data stored on each broker can reduce the recovery/rebalancing time. But it would also necessitate reducing the log retention period impacting the time available for application maintenance and failure recovery.

Kafka in cloud

On-premise Kafka deployments use for Kafka broker nodes hardware SKUs with multiple high capacity disks to maximize the i/o throughput and to store the data for the retention period. Equivalent SKUs with similar local storage options are either not available or they are very expensive in the cloud. SKUs with lesser local storage capacity as Kafka broker nodes have more available options and are more suitable in the cloud.

Solution - Tiered storage for Kafka

Kafka data is mostly consumed in a streaming fashion using tail reads. Tail reads leverage OS's page cache to serve the data instead of disk reads. Older data is typically read from the disk for backfill or failure recovery purposes and is infrequent.

In the tiered storage approach, Kafka cluster is configured with two tiers of storage - local and remote. Local tier is the same as the current Kafka that uses the local disks on the Kafka brokers to store the log segments. The new remote tier uses systems, such as HDFS or S3 to store the completed log segments. Two separate retention periods are defined corresponding to each of the tiers. With remote tier enabled, the retention period for the local tier can be significantly reduced from days to few hours. The retention period for remote tier can be much longer, days or even months. When a log segment is rolled on the local tier, it is copied to the remote tier along with the corresponding offset index. Applications that are latency sensitive perform tail reads and are served from local tier leveraging the existing Kafka mechanism of efficiently using page cache to serve the data. Backfill and other applications recovering from a failure that needs data older than what is in the local tier are served from the remote tier.

This solution allows scaling storage independent of memory and CPUs in a Kafka cluster enabling Kafka to be a long-term storage solution. This also reduces the amount of data stored locally on Kafka brokers and hence the amount of data that needs to be copied during recovery and rebalancing. Log segments that are available in the remote tier need not be restored on the broker or restored lazily and are served from the remote tier. With this, increasing the retention period no longer requires scaling the Kafka cluster storage and the addition of new nodes. At the same time, the overall data retention can still be much longer eliminating the need for separate data pipelines to copy the data from Kafka to external stores, as done currently in many deployments.

Goal

Extend Kafka's storage beyond the local storage available on the Kafka cluster by retaining the older data in an external store, such as HDFS or S3 with minimal impact on the internals of Kafka. Kafka behavior and operational complexity must not change for existing users that do not have tiered storage feature configured.


Non-Goals

Tiered storage does not replace ETL pipelines and jobs. Existing ETL pipelines continue to consume data from Kafka as is, albeit with data in Kafka having much longer retention period.

High-level design



Remote Log Manager (RLM) is a new component that copies the completed LogSegments and corresponding OffsetIndex to remote tier.

  • RLM component will keep tracks of topic-partition and its segments. It will delegate the copy and read of these segments to pluggable storage manager implementation.
  • RLM has two modes:
    • RLM Leader - In this mode, RLM that is the leader for topic-partition, checks for rolled over LogSegments and copies it along with OffsetIndex to the remote tier. RLM creates an index file, called RemoteLogSegmentIndex, per topic-partition to track remote LogSegments. Additionally, RLM leader also serves the read requests for older data from the remote tier.
    • RLM Follower - In this mode, RLM keeps track of the segments and index files on remote tier and updates its RemoteLogSegmentIndex file per topic-partition. RLM follower does not serve reading old data from the remote tier.

Core Kafka changes

To satisfy the goal of keeping Kafka changes minimal when RLM is not configured, Kafka behavior remains unchanged for existing users.

  • Core Kafka starts RLM service if tiered storage is configured
  • When an offset index is not found, if RLM is configured, the read request is delegated to RLM to serve the data from the remote tier.

Serving Data from Remote Storage

For each topic partition that has RLM configured, RLM will ship the log segment files that are older than a configurable time to remote storage. The active segment file (the last segment file of each partition, to which the new records are appending) is never shipped to remote storage.

After successfully copied a segment file to remote storage, RLM will append a set of index entries to 3 local index files: remotelogindex, remoteoffsetindex, remotetimeindex. These index files are rotated by RLM at a configurable time interval (or a configurable size).

(active segment)

{log.dirs}/{topic-partition}/0000002400013.index

{log.dirs}/{topic-partition}/0000002400013.timeindex

{log.dirs}/{topic-partition}/0000002400013.log


(inactive segments)

{log.dirs}/{topic-partition}/0000002000238.index

{log.dirs}/{topic-partition}/0000002000238.timeindex

{log.dirs}/{topic-partition}/0000002000238.log

{log.dirs}/{topic-partition}/0000001600100.index

{log.dirs}/{topic-partition}/0000001600100.timeindex

{log.dirs}/{topic-partition}/0000001600100.log


(active remote segment)

{log.dirs}/{topic-partition}/0000001000121.remoteoffsetindex

{log.dirs}/{topic-partition}/0000001000121.remotetimeindex

{log.dirs}/{topic-partition}/0000001000121.remotelogindex


(inactive remote segments)

{log.dirs}/{topic-partition}/0000000512002.remoteoffsetindex

{log.dirs}/{topic-partition}/0000000512002.remotetimeindex

{log.dirs}/{topic-partition}/0000000512002.remotelogindex



Each index entry of the remotelogindex file contains the information of a sequence of records in the remote log segment file. The format of a remotelogindex entry:

magic: int16 (current magic value is 0)

length: int16 (length of this entry)

crc: int32 (checksum from firstOffset to the end of this entry)

firstOffset: int64 (the Kafka offset of the 1st record)

lastOffset: int64 (the Kafka offset of the last record)

firstTimestamp: int64

lastTimestamp: int64

dataLength: int32 (length of the remote data)

rdiLength: int16

rdi: byte[] (Remote data identifier)


RDI (Remote data identifier) is the "pointer" or "URI" of the remote data. The format of RDI depends on the implementation. For example, RDI can be HDFS file path and offset, or S3 key and offset. When reading the remote records, RLM will use RDI to retrieve the remote data.

Depends on the implementation, RLM may append 1 or more entries to the remotelogindex file for each remote segment file. More entries will provide fine grained indexing of the remote data with the cost of local disk space.

The RemoteLogIndex entries are shipped to remote storage along with the segment data. The followers will retrieve those index entries from remote storage to build their own indices.

Remoteoffsetindex file and remotetimestampindex file are similar with the existing .index file (offset index) and .timeindex file (timestamp index). The only difference is that they point to the index in the corresponding remotelogindex file instead of a log segment file.


Manage Remote Log Segments Info in Followers

The RLM of a follower broker will retrieve remote segments information from the remote storage. It periodically checks the remote storage to find out the new segments shipped by the leader.

When the follower discovers a new segment in remote storage, it will retrieve the index entries from remote storage and append them into its local remote log index file. The RemoteOffsetIndex file and RemoteTimestampIndex file are updated accordingly.

The leader may fail to ship segment data to remote storage on time. In such a situation, the follower has to keep its local segment files, even if the configured retention time is reached. The local segment files (and the corresponding index files) can only be deleted in the following 2 cases:

  1. the follower received the corresponding segment data info from remote storage and updated its index files
  2. the local files are already older than the configured remote retention time

After restarted or after assigned new topic-partitions, the RLM of a follower will retrieve remote log segment info from remote storage, and the fetcher threads will fetch local log segment data from the leader.

When a remote segment is deleted in the remote storage, the follower deletes the corresponding index files from its local disk. A follower does not delete remote index files based on remote retention time. It always waits for the leader to delete the remote segments in remote storage first.

Public Interfaces

Compacted topics will not have remote storage support. 

Configs

System-Wide
  • remote.log.storage.enable = false (to support backward compatibility)
  • remote.storage.manager.class =  org.apache.kafka.log.HdfsRemoteStorageManager
RemoteStorageManager

(These configs are dependent on remote storage manager implementation)

  • remote.storage.manager.config
Per Topic Configuration
  • remote.retention.period

  • remote.retention.bytes


RemoteLogManager (RLM)

RemoteLogManager is a new class added to the broker. It is responsible to copy the completed log segments to the remote storage and update RemoteOffsetIndex file. 

Only the broker that is the Leader for topic-partitions is allowed to copy to the remote storage.

Note: Early proposal. To be finalized during implementation.

class RemoteLogManager extends Configurable {
     val RemoteStorageManager
     // configure
     def configure(Map<String, ?> configs)

     // Ask RLM to monitor the given TopicPartitions, and copy inactive Log
     // Segments to remote storage. RLM updates local index files once a
     // Log Segment in a TopicPartition is copied to Remote Storage
     def addPartitions(topicPartitions: Set[TopicPartition]): boolean

     // Stops copy of LogSegment if TopicPartition ownership is moved from a broker.
     def removePartitions(topicPartitions: Set[TopicPartition]): boolean
   
     
     // Read topic partition data from remote
     def read(fetchMaxByes: Int, hardMaxBytesLimit:Boolean, readPartitionInfo: Seq[(TopicPartition, PartitionData)]): LogReadResult

     // Stops all the threads and closes the instance.
     def shutdown(): Unit   
}


Remote Storage Manager:

        RemoteStorageManager is an interface that allows to plugin different remote storage implementations to copy the log segments. The default implementation of the interface supports HDFS as remote storage. Additional remote storage support, such as S3 can be added later by providing other implementations using the configuration remote.storage.manager.class.

trait RemoteStorageManager extends Configurable with AutoCloseable {

  /**
   * Copies LogSegment provided by [[RemoteLogManager]]
   * Returns the RDIs of the remote data
   * This method is used by the leader
   */
  @throws(classOf[IOException])
  def copyLogSegment(topicPartition: TopicPartition, logSegment: LogSegment): util.List[RemoteLogIndexEntry]

  /**
   * Cancels the unfinished LogSegment copying of this given topic-partition
   */
  def cancelCopyingLogSegment(topicPartition: TopicPartition): Unit

  /**
   * List the remote log segment files of the specified topicPartition
   * The RLM of a follower uses this method to find out the remote data
   *
   * @return List of remote segments, sorted by baseOffset in ascending order.
   */
  @throws(classOf[IOException])
  def listRemoteSegments(topicPartition: TopicPartition): util.List[RemoteLogSegmentInfo] = {
    listRemoteSegments(topicPartition, 0)
  }

  /**
   * List the remote log segment files of the specified topicPartition starting from the base offset minBaseOffset.
   * The RLM of a follower uses this method to find out the remote data
   *
   * @param minBaseOffset The minimum base offset for a segment to be returned.
   * @return List of remote segments starting from the base offset minBaseOffset, sorted by baseOffset in ascending order.
   */
  @throws(classOf[IOException])
  def listRemoteSegments(topicPartition: TopicPartition, minBaseOffset: Long): util.List[RemoteLogSegmentInfo]

  /**
   * Called by the RLM to retrieve the RemoteLogIndex entries of the specified remoteLogSegment.
   */
  @throws(classOf[IOException])
  def getRemoteLogIndexEntries(remoteLogSegment: RemoteLogSegmentInfo): util.List[RemoteLogIndexEntry]

  /**
   * Deletes remote LogSegment file provided by the RLM
   */
  @throws(classOf[IOException])
  def deleteLogSegment(remoteLogSegment: RemoteLogSegmentInfo): Boolean

  /**
   * Delete all the log segments for the given topic partition. This can be done by rename the existing locations
   * and delete them later in asynchronous manner.
   */
  @throws(classOf[IOException])
  def deleteTopicPartition(topicPartition: TopicPartition): Boolean

  /**
   * Remove the log segments which are older than the given cleanUpTillMs
   */
  @throws(classOf[IOException])
  def cleanupLogUntil(topicPartition: TopicPartition, cleanUpTillMs: Long): Long

  /**
   * Read up to maxBytes data from remote storage, starting from the 1st batch that
   * is greater than or equals to the startOffset.
   *
   * Will read at least one batch, if the 1st batch size is larger than maxBytes.
   *
   * @param remoteLogIndexEntry The first remoteLogIndexEntry that remoteLogIndexEntry.lastOffset >= startOffset
   * @param maxBytes            maximum bytes to fetch for the given entry
   * @param startOffset         initial offset to be read from the given rdi in remoteLogIndexEntry
   * @param minOneMessage       if true read at least one record even if the size is more than maxBytes
   * @return
   */
  @throws(classOf[IOException])
  def read(remoteLogIndexEntry: RemoteLogIndexEntry, maxBytes: Int, startOffset: Long, minOneMessage: Boolean): Records

  /**
   * stops all the threads and closes the instance.
   */
  def close(): Unit
}


Replica Manager

If RLM is configured, ReplicaManager will call RLM to assign topic-partitions or remove topic-partitions similar to how the replicaFetcherManager works today.

If the broker changes its state from Leader to Follower for a topic-partition and RLM is in the process of copying the segment, it will finish the copy before it relinquishes the copy for topic-partition. This might leave duplicated messages

ReplicaManager.readLocalLog works as it does today. But only in case of OffsetOutOfRange of exception and RLM is configured we will delegate the read request to RLM which returns LogReadResult

def readFromLocaLog(): Seq[(TopicPartition, LogReadResult)] = {
catch {
case e@ (_: OffsetOutOfRangeException) =>
    RemoteLogManager.read(fetchMaxBytes: Int,
                          hardMaxBytesLimit: Boolean, 
                          tp: TopicPartition, 
                          fetchInfo: PartitionData 
                          quota: ReplicaQuota)
}


Proposed Changes

When an RLM class is configured and all the required configs are present, RLM will send a list of topic-partitions and invoke the RemoteLogManager.addTopicPartitions(). This function’s responsibility is to delegate these topic-partitions to RLM. RLM will monitor the log.dirs for these topic-partitions and copy the rolled over LogSegment to the configured remote storage. Once a LogSegment is copied over it should mark it as done.

Log Retention

Log retention will continue to work as it is today except for one case, where If a LogSegment is in the process of getting copied over and it doesn't have associated "copy-done" file, LogCleaner will skips these LogSegments until it has the marker to denote its copied over to remote and its safe to delete.

Consumer Fetch Requests

For any fetch requests, ReplicaManager will proceed with making a call to readFromLocalLog, if this method returns OffsetOutOfRange exception it will delegate the read call to RemoteLogManager.readFromRemoteLog and returns the LogReadResult. If the fetch request is from a consumer, RLM will read the data from remote storage, and return the result to the consumer.

Follower Requests/Replication

For follower fetch, the leader only returns the data that is still in the leader's local storage. If a LogSegment copied into Remote Storage by a Leader Broker, it's not necessary for Follower to copy this segment which is already present in Remote Storage. Instead, a follower will retrieve the information of the segment from remote storage. If a Replica becomes a Leader, It can still locate and serve data from Remote storage.

Thread Pools

Remote storage (e.g. S3 / HDFS) is likely have higher I/O latency and lower availability than local storage.

When the remote storage becoming temporarily unavailable (up to several hours) or having high latency (up to minutes) , Kafka should still be able to operate normally. All the Kafka operations (produce, consume local data, create / expand topics etc.) that do not rely on remote storage should not be impacted. The consumers that try to consume the remote data should get reasonable errors, when remote storage is unavailable or the remote storage requests timeout.

To achieve this, we have to handle remote storage operations in dedicated threads pools, instead of Kafka I/O threads and fetcher threads.


1. Remote Storage Manager (RLM) Thread Pool


RLM maintains a list of the topic-partitions it manages. The list is updated in Kafka I/O threads, when topic-partitions are added to / removed from RLM. Each topic-partition in the list is assigned a scheduled processing time. The RLM thread pool processes the topic-partitions that the "scheduled processing time" is less than or equal to the current time.

When a new topic-partition is assigned to the broker, the topic-partition is added to the list, with scheduled processing time = 0, which means the topic-partition has to be processed immediately, to retrieve information from remote storage.

After a topic-partition is successfully processed by the thread pool, it's scheduled processing time is set to ( now() + rlm_process_interval_ms ). rlm_process_interval_ms can be configured in broker config file.

If the process of a topic-partition is failed due to remote storage error, its scheduled processing time is set to ( now() + rlm_retry_interval_ms ). rlm_retry_interval_ms can be configured in broker config file.

When a topic-partition is unassigned from the broker, the topic-partition is not currently processed by the thread pool, the topic-partition is directly removed from the list; otherwise, the topic-partition is marked as "deleted", and will be removed after the current process is done.

Each thread in the thread pool processes one topic-partition at a time in the following steps:

Step 1) Delete old remote segments (leader)

Call RSM.listRemoteSegments() to get the list of remote segments.

If any of the remote segments are older than the configured remote log retention time, delete the segment from remote storage.

Ask RSM to delete the temporary files in remote storage that are older than the configured remote retention time. Failed RSM operations may leave some temporary files in remote storage. Instead of introducing a "garbage collection time" configuration, we will just delete those temporary files after the remote log retention time.

Step 2) Copy log segments to remote storage (leader)

Copy the log segment files that are 1) inactive and 2) the offset range is not covered by the segments on the remote storage.

If multiple log segment files are ready, they are copied to remote storage one by one, from the earliest to the latest.

Step 3) Retrieve the latest remote storage information (both leader and follower)

Call RSM.listRemoteSegments() to get the list of remote segments

If any of the remote segments are not added to the local index yet, call RSM.getRemoteLogIndexEntries to retrieve the remote log index data, append to the local remoteLogIndex file, and build the local index files accordingly. Roll the local remoteLogIndex file, when needed. Update the "latest remote offset" of the topic-partition.

Update the "earliest remote offset" of the topic-partition to the base offset of the earliest remote segment, when old remote segments are removed from the remote storage. Delete the old local remoteLogIndex file, if the "earliest remote offset" is larger than the last offset of the file.


2. Remote Storage Fetcher Thread Pool


When handling consumer fetch request, if the required offset is in remote storage, the request is added into "RemoteFetchPurgatory", to handle timeout. RemoteFetchPurgatory is an instance of kafka.server.DelayedOperationPurgatory, and is similar with the existing produce / fetch purgatories. At the same time, the request is put into the task queue of "remote storage fetcher thread pool".

Each thread in the thread pool processes one remote fetch request at a time. The remote storage fetch thread will

1) find out the corresponding RDI from the remote log index

2) try to retrieve the data from remote storage

2.1) if success, RemoteFetchPurgatory will be notified to return the data to the client

2.2) if the remote segment file is already deleted, RemoteFetchPurgatory will be notified to return an error to the client.

2.3) if the remote storage operation failed (remote storage is temporarily unavailable), the operation will be retried with Exponential Back-Off, until the original consumer fetch request timeout.

Alternatives considered

Following alternatives were considered:

  1. Replace all local storage with remote storage - Instead of using local storage on Kafka brokers, only remote storage is used for storing log segments and offset index files. While this has the benefits related to reducing the local storage, it has the problem of not leveraging the local disk for efficient latest reads as done in Kafka today.

  2. Implement Kafka API on another store - This is an approach that is taken by some vendors where Kafka API is implemented on a different distributed, scalable storage (example HDFS). Such an option does not leverage Kafka other than API compliance and requires the much riskier option of replacing entire Kafka cluster with another system.

  3. Client directly reads remote log segments from the remote storage - The log segments on the remote storage can be directly read by the client instead of serving it from Kafka broker. This reduces Kafka broker changes and has benefits of removing an extra hop. However, this bypasses Kafka security completely, increases Kafka client library complexity and footprint and hence is not considered.
  • No labels