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

Compare with Current View Page History

« Previous Version 9 Next »

 

Status

Current stateUnder Discussion

Discussion threadhere

JIRAhere

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Kafka Brokers today rely on Apache Zookeeper. Many folks in the community have expressed a desire to either change the zkclient and start using Apache Curator or allowing other systems like etcd, consul, Apache Cassandra and others to handle the role Zookeeper is currently playing. By allowing the brokers to have both a way to plug-in another server for storing the meta data and also for leader election then we can have the ability to-do this.

Proposed Changes

This KIP proposes approach for isolating coordination related functionality to separate modules. These modules should come with a public interface that can have pluggable implementations.

Zookeeper has advanced low-level primitives for coordinating distributed systems – ephemeral nodes, key-value storage, watchers. Such primitives may not be available in other consensus frameworks. At the same time such low-level primitives (especially ephemeral nodes) are error prone and usually a cause of subtle bugs in Kafka coordination code.

That's why instead of focusing on question “how Kafka does coordination with Zookeeper” it is proposed to concentrate on question “what general problems of distributed systems are solved in Kafka by means of Zookeeper”. Having defined interface boundaries this way, we'll be able to hide implementation details under concrete realizations developed with corresponding built-in facilities available in particular tools (e.g. ephemeral nodes vs TTLs).

It is proposed to separate such high-level concerns:

  1. Group membership protocol (Kafka brokers form a cluster; consumer connectors form a consumer group)
  2. Leader election (electing controller among brokers)
  3. Distributed key-value storage (topic config storage etc etc etc)
  4. Data-change listeners (triggering events - partition reassignment, catching up isr-s etc)

Public Interfaces

Below each module is presented by its interfaces.

 

Group Membership Protocol
/**
 * A connector for group membership protocol. Supports both parts:
 * 1) "joining" (becoming the member, leaving the group, subscribing to change notifications)
 * 2) "observing" (fetching group state, subscribing to change notifications)
 *
 * @tparam K type of the member ID - unique identifier among members of this group
 * @tparam V type of the additional data that comes with ID
 */
trait GroupMembershipClient[K, V] {
  /**
   * Each instance of this class is tightly coupled with exactly one group,
   * once set (during initialization) cannot be changed
   * @return unique group identifier among all application groups
   */
  def group: String

  /**
   * Become a member of this group. Throw an exception in case of ID conflict
   * @param id unique member identifier among members of this group
   * @param data supplemental data to be stored along with member ID
   */
  def join(id: K, data: V): Unit

  /**
   * Stop membership in this group
   * @param id unique member identifier among members of this group
   */
  def leave(id: K): Unit

  /**
   * Fetch membership of this group
   * @return IDs of the members of this group
   */
  def membershipList(): Set[K]

  /**
   * Fetch detailed membership of this group
   * @return IDs and corresponding supplemental data of the members of this group
   */
  def membership(): Map[K, V]

  /**
   * A callback fired on event
   */
  trait Listener {
    /**
     * Event fired when the group membership has changed (member(s) joined and/or left)
     * @param membership new membership of the group
     */
    def onGroupChange(membership: Set[K])
  }

  /**
   * Register permanent on group change listener.
   * There is no guarantee listener will be fired on ALL events (due to session reconnects etc)
   * @param listener see [[Listener]]
   */
  def addListener(listener: Listener)

  /**
   * Deregister on group change listener
   * @param listener see [[Listener]]
   */
  def removeListener(listener: Listener)

  /**
   * Setup everything needed for concrete implementation
   * @param context TBD. Should be abstract enough to be used by different implementations and
   *                at the same time specific because will be uniformly called from the Kafka code -
   *                regardless of the implementation
   */
  def init(context: Any): Unit

  /**
   * Release all acquired resources
   */
  def close(): Unit
}
Leader Election
/**
 * A connector for leadership election protocol. Supports both parts:
 * 1) "running for election" (joining the candidates for leadership, resigning as a leader, subscribing to change notifications)
 * 2) "observing" (getting current leader, subscribing to change notifications)
 *
 * @tparam K type of the candidate ID - unique identifier among candidates participating in leader election
 */
trait LeaderElectionClient[K]{
  /**
   * Each instance of this class is tightly coupled with leadership over exactly one service (resource),
   * once set (during initialization) cannot be changed
   *
   * @return unique group identifier among all application services (resources)
   */
  def service: String

  /**
   * Fetch leader of this service until one is elected
   * @return future result of the leader ID
   */
  def getLeader: Future[K]

  /**
   * Make this candidate eligible for leader election and try to obtain leadership for this service if it's vacant
   *
   * @param candidate ID of the candidate which is eligible for
   * @return future result of the leader election
   */
  def nominate(candidate: K): Future[K]

  /**
   * Voluntarily resign as a leader and initiate new leader election.
   * It's a client responsibility to stop all leader duties before calling this method to avoid more-than-one-leader cases
   *
   * @param leader current leader ID (will be ignored if not a leader)
   * @return future result of the leader election
   */
  def resign(leader: K): Future[K]

  /**
   * A callback fired on event
   */
  trait Listener {
    /**
     * Event fired when the leader has changed and the new one has been elected
     * @param leader new leader for the given service
     */
    def onLeaderChange(leader: K)
  }

  /**
   * Register permanent on leader change listener
   * There is no guarantee listener will be fired on ALL events (due to session reconnects etc)
   * @param listener see [[Listener]]
   */
  def addListener(listener: Listener)
  /**
   * Deregister on leader change listener
   * @param listener see [[Listener]]
   */
  def removeListener(listener: Listener)

  /**
   * Setup everything needed for concrete implementation
   * @param context TBD. Should be abstract enough to be used by different implementations and
   *                at the same time specific because will be uniformly called from the Kafka code -
   *                regardless of the implementation
   */
  def init(context: Any): Unit

  /**
   * Release all acquired resources
   */
  def close(): Unit
}
Storage
/**
 * Interface to a (persistent) key value storage
 * @tparam K type of data key
 * @tparam V type of the fetched data
 */
trait Storage[K, V] {
  /**
   * Get data by its key
   * @param key data ID in this storage
   * @return future result of the value (if exists) associated with the key
   */
  def fetch(key: K): Future[Option[V]]

  /**
   * Persist value with its associated key. The contract is to throw an exception
   * if such key already exists
   *
   * @param key data ID in this storage
   * @param data value associated with the key
   */
  def put(key: K, data: V)

  /**
   * Update value by its associated key. The contract is to throw an exception
   * if such key doesn't exist
   *
   * @param key data ID in this storage
   * @param data value associated with the key
   */
  def update(key: K, data: V)

  /**
   * Setup everything needed for concrete implementation
   * @param context TBD. Should be abstract enough to be used by different implementations and
   *                at the same time specific because will be uniformly called from the Kafka code -
   *                regardless of the implementation
   */
  def init(context: Any): Unit

  /**
   * Release all acquired resources
   */
  def close(): Unit
}
Listener Registry
/**
 * A registry for async data change notifications
 */
trait ListenerRegistry {
  /**
   * Register permanent callback for data change event
   * @param key the listenable data identifier
   * @param eventListener see [[EventListener]]
   * @tparam T type of the data ID
   */
  def addListener[T](key: T, eventListener: EventListener): Unit

  /**
   * Deregister permanent callback for data change event
   * @param key the listenable data identifier
   * @param eventListener see [[EventListener]]
   * @tparam T type of the data ID
   */
  def removeLister[T](key: T, eventListener: EventListener): Unit

  /**
   * Setup everything needed for concrete implementation
   * @param context TBD. Should be abstract enough to be used by different implementations and
   *                at the same time specific because will be uniformly called from the Kafka code,
   *                regardless of the implementation
   */
  def init(context: Any): Unit

  /**
   * Release all acquired resources
   */
  def close(): Unit
}

/**
 * Base class for change events
 */
sealed trait Event

/**
 * Event fired if the listenable data state has changed. In terms of k-v storage - if the
 * value associated with the listenable key has changed
 * @tparam T type
 */
trait DataChangeEvent[T] extends Event {
  /**
   * @return new data state
   */
  def data: T
}

/**
 * Event fired if the listenable collection of data has changed. In terms of k-v storage - if the
 * key set in the particular namespace has changed
 * @tparam T collection data type
 */
trait CollectionChangeEvent[T] extends Event {
  /**
   * @return new collection states
   */
  def collection: Set[T]
}

/**
 * A callback fired on event
 */
trait EventListener {
  def onEvent(event: Event)
}

 

Compatibility, Deprecation, and Migration Plan

The use of zookeeper.connect in server.properties will still be honored. In the code we should have the default value of the new configuration.

Rejected Alternatives

If there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.

 

  • No labels