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

Compare with Current View Page History

« Previous Version 76 Next »

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

Status

Current State: Discussion

Discussion Thread: Discuss Thread

JIRA Unable to render Jira issues macro, execution error.

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 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 unavailable or they are very expensive in the cloud. There are more available options for SKUs with lesser local storage capacity as Kafka broker nodes and they 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. Latency sensitive applications 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 a much longer retention period.

It does not support compact topics.   

Public Interfaces

Compacted topics will not have remote storage support. 

Configs

System-Wide

remote.log.storage.manager.class.name =  org.apache.kafka.rsm.hdfs.HDFSRemoteStorageManager

Not configuring the above property gives backward comaptibility. 

RemoteStorageManager

(These configs are dependent on remote storage manager implementation)

remote.log.storage.*

Thread pools

remote.log.manager.thread.pool.size
Remote log thread pool size, which is used in scheduling tasks to copy segments, fetch remote log indexes and clean up remote log segments.

remote.log.manager.task.interval.ms
Interval at which remote log manager runs the scheduled tasks like copy segments, fetch remote log indexes and clean up remote log segments.

remote.log.reader.threads
Remote log reader thread pool size

remote.log.reader.max.pending.tasks
Maximum remote log reader thread pool task queue size. If the task queue is full, broker will stop reading remote log segments.

Per Topic Configuration

remote.log.retention.minutes

remote.log.retention.bytes

Remote Storage Manager:

         `RemoteStorageManager` is an interface to provide the lifecycle of remote log segments. We will provide a simple implementation of RSM to get a better understanding of the APIs. HDFS and S3 implementation are planned to be hosted in external repos and these will not be part of Apache Kafka repo. This is inline with the approach taken for Kafka connectors.


/**
 * RemoteStorageManager is an interface that allows to plugin different remote storage implementations to provide
 * the lifecycle of remote log segments.
 *
 * All these APIs are still experimental.
 */
@InterfaceStability.Unstable
trait RemoteStorageManager extends Configurable with AutoCloseable {

  /**
   * Earliest log offset if exists for the given topic partition in the remote storage. Return -1 if there are no
   * segments in the remote storage.
   *
   * @param tp
   * @return
   */
  @throws(classOf[IOException])
  def earliestLogOffset(tp: TopicPartition): Long

  /**
   * Copies LogSegment provided by [[RemoteLogManager]] for the given topic partition with the given leader epoch.
   * Returns the RDIs of the remote data. This method is invoked by the leader of topic partition.
   *
   * //todo LogSegment is not public, this will be changed with an interface which provides base and end offset of the
   * segment, log and offset/time indexes.
   *
   * @param topicPartition
   * @param logSegment
   * @return
   */
  @throws(classOf[IOException])
  def copyLogSegment(topicPartition: TopicPartition, logSegment: LogSegment, leaderEpoch: Int): util.List[RemoteLogIndexEntry]

  /**
   * List the remote log segment files of the given topicPartition.
   * The RemoteLogManager of a follower uses this method to find out the remote data for the given topic partition.
   *
   * @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]

  /**
   * Returns a List of RemoteLogIndexEntry for the given RemoteLogSegmentInfo. This is used by follower to store remote
   * log indexes locally.
   *
   * @param remoteLogSegment
   * @return
   */
  @throws(classOf[IOException])
  def getRemoteLogIndexEntries(remoteLogSegment: RemoteLogSegmentInfo): util.List[RemoteLogIndexEntry]

  /**
   * Deletes remote LogSegment file and indexes for the given remoteLogSegmentInfo
   *
   * @param remoteLogSegmentInfo
   * @return
   */
  @throws(classOf[IOException])
  def deleteLogSegment(remoteLogSegmentInfo: 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.
   *
   * @param topicPartition
   * @return
   */
  @throws(classOf[IOException])
  def deleteTopicPartition(topicPartition: TopicPartition): Boolean

  /**
   * Remove the log segments which are older than the given cleanUpTillMs. Return the log start offset of the
   * earliest remote log segment if exists or -1 if there are no log segments in the remote storage.
   *
   * @param topicPartition
   * @param cleanUpTillMs
   * @return
   */
  @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. It 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

  /**
   * Release any system resources used by this instance.
   */
  def close(): Unit
}



Proposed Changes

High-level design


RemoteLogManager (RLM) is a new component which

  • receives callback events for leadership changes and stop/delete events of topic partitions on a broker.
  • delegates copy and read of these segments and deleting topic partitions to a pluggable storage manager(viz RemoteStorageManager) implementation.

RLM creates tasks for each leader or follower topic partition:

  • RLM Leader Task - It checks for rolled over LogSegments (which have last message offset less than last stable offset of that topic partition) and copies them along with their remote offset/time indexes to the remote tier. RLM creates an index file, called RemoteLogSegmentIndex, per topic-partition to track remote LogSegments. These indexes are described in detail here. It also serves the fetch requests for older data from the remote tier. Local logs are not cleanedup till those segments are copied successfully to remote even though their retention time/size is reached.
  • RLM Follower Task - it keeps track of the segments and index files on remote tier and updates its RemoteLogSegmentIndex file per topic-partition. Local logs are not cleanedup till their remote log indexes are copied locally from remote storage even though their retention time/size is reached. RLM follower can also 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.

Local and Remote log offset constraints

Below are the leader topic partition's log offsets

Lx  = Local log start offset           Lz  = Local log end offset            Ly  = Last stable offset(LSO)

Ry  = Remote log end offset       Rx  = Remote log start offset


Lz >= Ly and Ly >= Lx and Ly >= Rand Ry >= Rx

Remote Log Indexes

For each topic partition that has RLM configured, RLM leader for a topic partition copies log segments which have last message offset less than last stable offset of that topic partition 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 a segment file is copied 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


( remote segment)

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

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

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


( 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

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 a remote storage and updated its index files and
  2. the local files are already older than the configured remote retention time

This is explained in detail here.

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)
}

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. More details are explained in the RLM/RSM tasks section.

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, the follower doesn't need 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.

RLM/RSM tasks and thread pools

Remote storage (e.g. S3 / HDFS) is likely to 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 Log 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:

Copy log segments to remote storage (leader)

Copy the log segment files that are

       - inactive and

       - the offset range is not covered by the segments on the remote storage and

      - those segments have the last offset < last-stable-offset of the partition.

If multiple log segment files are ready, they are copied to remote storage one by one, from the earliest to the latest. It uses the below copy API from RSM

copyLogSegment(topicPartition: TopicPartition, logSegment: LogSegment, leaderEpoch: Int): util.List[RemoteLogIndexEntry]

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.

Handle expired remote segments (leader and follower)

For leader, it invokes RSM.cleanupLogUntil(topicPartition: TopicPartition, cleanUpTillMs: Long) to delete remote log segments and return the start offset of the earliest remote log segment.

For follower, it fetches the earliest offset by calling RSM.earliestLogOffset(tp: TopicPartition).

Both leader and follower cleansup the existing indexes till that offset and updates start offset with the received value.


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 to 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 the 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