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

Compare with Current View Page History

« Previous Version 27 Next »

Public-Facing Changes

  • Update Java Docs

  • API signatures will remain the same

Summary

This document highlights our effort to refactor the threading model of the KafkaConsumer, and the purpose of this refactor is to address issues and shortcomings we've encountered in the past years, such as increasing code complexity, lock and race conditions, and the coupling between the client poll and the rebalance progress.

  • Code complexity: Patches and hotfixes in the past years have heavily impacted the readability of the code.  The complex code path and intertwined logic make the code difficult to modify and comprehend. For example, coordinator communication can happen in both the heartbeat thread and the polling thread, which makes it challenging to diagnose the source of the issue when there's a race condition, for example. Also, many parts of the coordinator code have been refactored to execute asynchronously; therefore, we can take the opportunity to refactor the existing loops and timers. 

  • Complex Coordinator Communication: Coordinator communication happens at both threads in the current design, and it has, caused some race conditions such as KAFKA-13563. We want to take the opportunity to move all coordinator communication to the background thread and adopt a more linear design.

  • Asynchronous Background Thread: One of our goals here is to make rebalance process happen asynchronously to the poll. Firstly, it simplifies the pd design, as it essentially only needs to handle fetch requests. Secondly, because rebalance occurs in the background thread, the polling thread won't be blocked or blocking the rebalance process.

Scope

  • Deprecate HeartbeatThread

    • Remove the heartbeat thread.
    • Move the heartbeat logic to the background thread.
  • Refactor the Coordinator classes

    • Revisit timers and loops
    • Blocking methods such as commitOffsetSync will be handled by the poling thread by waiting on the future's completion. The coordinator should not be blocking.
    • The background thread loop will poll the coordinator.
    • We will modify the existing rebalance states, specifically for the callback execution.
  • Refactor the KafkaConsumer API

    • It will send Events to the background thread if network communication is required.
    • Remove dependency on the fetcher, coordinator, and networkClient.
  • Events and communication channels.

    • We will use two channels the facilitate the two-way communication
  • Address issues in these Jira tickets

Terms

Polling Thread

The client thread, also known as the polling thread, is where the thread user invokes the poll() operation.

Background Thread

Specifically, in the new design context, we use this generic term to indicate the active process running in the background. It handles network communication, rebalancing, and heartbeat.

Heartbeat Thread

The background running thread in the current implementation (see KIP-62).

Channel

We have two channels in this design. One channel delivers messages from the polling thread to the background thread. The other channel delivers messages submitted by the background thread to the polling thread.

Event

The primary communication medium for the polling thread is to interact with the background thread.

Proposed Design

In the schematic below, the main components are:

  • Polling thread: handles all of the API requests and callback executions.

  • Background thread: handles network communication such as heartbeat, coordinator requests, and rebalance flow execution.

  • Communication channels: which are responsible for:

    • ServerEventQueue: sending events to the background thread

    • ConsumerEventQueue: sending events to the polling thread

We will discuss the strategy for handling event results in the following section. In short, we will use the CompletableFuture API to handle the asynchronous operation between the two threads.    

A note on the subscriptionState: Its reference will be shared by polling and background threads, i.e., we will refactor the current implementation to be thread-safe. The reason we are making this data structure an exception are:

  • The object is frequently accessed by both the polling thread and background thread.
  • The object often requires instaneous response; otherwise, the performance will be heavily impacted
  • Also, please review the rejected proposals section below. I talked about a few ideas that we've had.

Important Components

Background thread and its lifecycle

We use a state machine to represent the different states of the coordinator connection; these are: down, initialized, coordinator_discovery (coordinator discovery), and stable.

  • down: The coordinator is uninitialized
  • initialized: The background thread completes initialization, and the loop has started.
  • coordinator_discovery: The coordinator is requested but hasn't been connected yet. If there's no in-flight FindCoordinatorRequest, then it will send one. If one exists, check for the request results.
  • stable: The coordinator is connected.

The coordinator discovery process piggybacks on the background thread loop; therefore, any disconnect or connection request is handled by the state machine, and the state machine is run by the background thread loop.  If we don't need a coordinator, the background thread will stay in the initialized state.  If an event requires a coordinator, we will then move to the coordinator_discovery state and wait for the response to come back.

Background thread and its lifecycle

  1. The background thread has not been constructed, so it is in the down state.

  2. The polling thread starts the background; the background thread finishes initialization and then moves to the initialized state. Usually, this happens after new KafkaConsumer().

  3. The background thread loop performs the following tasks:

    1. Check if there is an in-flight event. If not, poll for the new events from the channel.

    2. Run the state machine, and here are the following scenario:

      • The event does not require a coordinator.  Execute the event and move on.
      • The event requires a coordinator, but it is currently disconnected.  Move the state to coordinator_discovery.
      • The background thread is currently in coorinator_discovery state; continue to loop and wait for the FindCoordinator response.
      • The background thread is in the stable state.
        • Poll the Coordinator.
        • Execute the event.
  4. Poll the networkClient.

  5. Backoff for retryBackoffMs milliseconds.

Rebalance States

We modify the existing rebalance states to aid the callback execution.

  1. We split the PREPARING_REBALANCE into PREPARE_REVOCATION, and PARTITION_REVOKED, PREPARING_REBALANCE

    1. PREPARE_REVOCATION: Pause partitions that will be reovked
    2. REVOKING_PARTITION: Send a PARTITION_REVOKE event to the polling thread.

    3. PARTITION_REVOKED: Update the subscription.  Autocommit.
  2. STABLE will become ASSIGNING_PARTITIONS and STABLE 

    1. ASSIGNING_PARTITIONS: Send an ASSIGN_PARTITIONS event

    2. STABLE: Upon receiving PARTITIONS_ASSIGNED event, transition to STABLE

After the proposed state transition is as such:

Starting from UNJOINED: UNJOINED → PREPARING_REBALANCE → COMPLETING_REBALANCE → ASSIGNING_PARTITIONS → STABLE

(EAGER) Starting from STABLE: STABLE → PREPARE_REVOCATION → REVOKING_PARTITION → PARTITION_REVOKED → PREPARING_REBALANCE → COMPLETING_REBALANCE → ASSIGNING_PARTITIONS → STABLE

(COOPERATIVE) STABLE → PREPARE_REVOCATION → REVOKING_PARTITION → PARTITION_REVOKED → COMPLETING_REBALANCE → ASSIGNING_PARTITIONS → STABLE → //rejoin

Channel and Events

We use a blocking queue to send API events from the polling thread to the background thread.

ServerEventQueue
// Channel used to send events to the background thread

private BlockingQueue<ServerEvent> queue;

abstract public class ServerEvent {
   private final ServerEventType eventType;
}

enum ServerEventType {
   FETCH,
   COMMIT,
   METADATA_UPDATE,
   CLOSE,
   ...
}

ConsumerEventQueue
// Channel used to send events to the polling thread for client side execution/notification

private BlockingQueue<ConsumerEvent> queue;

abstract public class ConsumerEvent {
   private final ConsumerEventType eventType;
}

enum ConsumerEventType {
   ERROR,
   REVOKE_PARTITIONS,
   ASSIGN_PARTITIONS,
}

Consumer.poll() 

consumer.poll() onward will be responsible for a the following tasks:

  1. Poll ConsumerEventQueue, execute the events
  2. Build FetchEvent, and send it to the background thread
  3. Return fetched data

Major Changes

Consumer Poll Changes

Currently, the consumer.poll() does two things: poll coordinator for assignment update, auto-commit and rebalances, and fetching.

Moving forward, the consumer will only be responsible for sending and returning the fetch data and polling the channel for events such as rebalancing and error:

  1. Check wakeup flag.  Throw WakeupException and interrupt the loop if a wakeup call was invoked.
  2. Poll the ConsumerQueue for events

    1. Handle exceptions

    2. Handle callback execution

  3. Send out fetch requests

  4. Returning fetch results

ConsumerCoordinator and AbstractCoordinator

  • New states will be introduced (see Rebalance States section above).  The main purpose is to make the background thread drive the poll, and letting the polling thread to invoke the callbacks.
  • Remove HeartbeatThread. Therefore, we won't be starting the heartbeat thread.

    • It will still require a fetch event to poll heartbeat.  As only polling thread issues fetch events, and we want to respect the existing implementation.
  • Timeouts and timers will be reevaluated and possibly removed.
  • while loops will be reevaluated and possibly thrown out.  In the new implementation the coordinator will be non-blocking, and its states are managed by the background thread loop.

Wakeup() and WakeupException

We will have to reimplement wakeup() and WakeupException.

wakeup()

In the new design, wakeup will interrupt the blocking polling loop in the polling thread.

WakeupException

The exception will be thrown by the polling thread when the loop is interrupted.

Timeout Policy

Consumer.poll() - user provide timeout

Coordinator rediscovery backoff: retry.backoff.ms

Coordinator discovery timeout: Currently uses the user-provided timeout in the consumer.poll(). Maybe we should use request.timeout.ms. And re-attempt in the next loop if failed

CommitOffsetSync: user provided

Is there a better way to configure session interval and heartbeat interval?

Consumer API Internal Changes 

Assign

  1. The polling thread updates the subscriptionState.
  2. Create an ASSIGN event, and queue up the event to the background thread.
  3. Background thread receives the event and executes the event.
    1. send a commitOffset
    2. Update metadata

Unsubscribe

  1. The polling thread removes the subscriptions from the subscriptionState
  2. executes the onPartitionRevoke callback
  3. Send background thread an LEAVE_GROUP event
  4. Returns
  5. The background thread:
    1. Initiate LeaveGroup

Subscribe

  1. The polling thread updates the subscriptionState.
  2. Create a SUBSCRIBE event, and send it to the background thread.
  3. Background thread executes the event:
    1. Update metadata

Poll

  1. Poll the ConsumerEventQueue, and execute the events accordingly
  2. Prepare a FETCH event, and send it to the background thread.
  3. Wait for the CompleteableFuture to finish
  4. Execute the interceptor.onConsumer and return the data.
  5. Background thread execute the FETCH event:
    1. Autocommits // I think we can move the autocommit to the background thread
    2. issue fetch requests


Event Data Models

ConsumerEvent
NameTypeDescription
typeConsumerEventTypeType of event
requiredCoordinatorboolIndicate if the event uses coordinator
ServerEvent
typeServerEventTypeType of event

Test Plan

We will first write and use the existing unit tests and integration tests.

We need to ensure the timing of the critical events are happening in the correct sequence. For example: We need to first discover the coordinator, second, commit the offset while pausing the partition for being fetched, revoke the partition, and then continue onto rest of the rebalance process.

We will also ensure the heartbeat interval, and poll interval are respected.

We also need to ensure there's no event lost, and the event should happen in the correct sequence.

Please review https://en.wikipedia.org/wiki/Failure_mode_and_effects_analysis and try to assess potential failures.

Compatibility

The new consumer should be backward compatible.

Alternative Proposals

Fetcher

  • Should some of the fetcher configuration be dynamic

  • Configurable prefetch buffer

SubscriptionState

  • Non-shared: We can make a copy of the subscriptionState in the background thread, and use event to drive the synchronization.

    • There could be out of sync issues, which can subsequently causes in correct fetching, etc..

API Changes

  • Poll returns CompletableFuture<ConsumerRecord<K,V>>

User Adoption

The refactor should have (almost) no regression or breaking changes upon merge. So user should be able to continue using the new client.

Release Plan

  • Support code will exist in parallel from the current code.  The support code are:
    • Background thread
    • A new coordinator implementation, AsyncConsumerCoordinator, for example.
    • Events and event executors
  • We will create a new KafkaConsumer class first, then have it override the existing one once reach stability
  • No labels