Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

Hence, the discovery API is part of the KafkaStreams instance. The API will provide four methods:

  • Collection<KafkaStreamsInstance> Collection<StreamsMetadata> KafkaStreams.allInstancesallMetadata()) where KafkaStreamsInstanceStreamsMetadata has fields such as list of assigned partitions, list of state store names and HostInfo that includes hostname / port, etc. The port is the listening port of a user-defined listener that users provide to listen for queries (e.g., using REST APIs). More on the user-defined agent below.
  • Collection<KafkaStreamsInstance>Collection<StreamsMetadata> KafkaStreams.allInstancesWIthStoreallMetadataForStore(String /* storeName */) would return only the KafkaStreamsInstances StreamsMetadata that include the given store name.

  • KafkaStreamsInstance StreamsMetadata KafkaStreams.instanceWithKeymetadataWithKey(String storeName, K key, Serializer<K> serializer) would return the KafkaStreamInstance StreamsMetadata that has the store that might have the key (the key might not exist in any store). The default StreamPartitioner will be used for key partitioning.

  • KafkaStreamsInstance StreamsMetadata KafkaStreams.instanceWithKeymetadataForKey(String storeName, K key, StreamPartitioner<K, ?> partitioner) same as above but will use the provided StreamPartitioner

    An additional configuration parameter, StreamsConfig.USERAPPLICATION_ENDPOINTSERVER_CONFIG, will be added. This is a host:port pair supplied by the streams developer and should map to a Server running in the same instance of the KafkaStreams application. The supplied host:port pair will form part of the KafkaStreamsInstance StreamsMetadata returned from the the above mentioned API calls.

...

We propose adding an additional method to the  KafkaStreams public API:

Code Block
public <T> T getStorestore(final String storeName, 
                      final QueryableStoreType<T> queryableStoreType)

...

Code Block
public interface QueryableStoreType<T> {
    /**
     * Called when searching for {@StateStore}s to see if they
     * match the type expected by implementors of this interface
     * @param stateStore    The stateStore
     * @return true if it is a match
     */
    boolean accepts(final StateStore stateStore);
    /**
     * Create an instance of T (usually a facade) that developers can use
     * to query the Underlying {@StateStore}s
     * @param storeProvider     provides access to all the underlying StateStore instances of type T
     * @param storeName         The name of the Store
     * @return  T usually a read-only interface over a StateStore @see {@link   QueryableStoreTypes.KeyValueStoreType}
     */
    T create(final StateStoreProvider storeProvider, final String storeName);
}
public interface StateStoreProvider {
	/**
     * Find instances of StateStore that are accepted by {@link QueryableStoreType#accepts} and
     * have the provided storeName.
     *
     * @param storeName             name of the store
     * @param queryableStoreType    filter stores based on this queryableStoreType
     * @param <T>                   The type of the Store
     * @return  List of the instances of the store in this topology. Empty List if not found
     */
    <T> List<T> getStoresstores(String storeName, QueryableStoreType<T> queryableStoreType);
}

...

Code Block
/**
 * A new config will be added to StreamsConfig
 * A user defined endpoint that can be used to connect to remote KafkaStreams instances. 
 * Should be in the format host:port
 */
public static final String USERAPPLICATION_ENDPOINTSERVER_CONFIG = "userapplication.endpointserver";


 
public class HostInfo {
   String hostname;    /* hostname for instance that contains state store */
   int port;           /* listening port for instance that contains state store */
}
public class KafkaStreamsInstanceStreamsMetadata {
    private final HostInfo hostInfo; /* hostInfo from above */
    private final Set<String> stateStores; /* state stores on this instance */
    private final Set<TopicPartition> topicPartitions; /* TopicPartitions on this instance */
}

/**
 * Find all of the instances of {@link KafkaStreamsInstanceStreamsMetadata} in a {@link KafkaStreams application}
 * Note: this is a point in time view and it may change due to partition reassignment.
 * @return collection containing all instances of {@link KafkaStreamsInstanceStreamsMetadata} in this application
 */
public Collection<KafkaStreamsInstance>Collection<StreamsMetadata> allInstancesallMetadata();

/**
 * Find the instances {@link KafkaStreamsInstanceStreamsMetadata} for a given storeName
 * Note: this is a point in time view and it may change due to partition reassignment.
 * @param storeName the storeName to find metadata for
 * @return  A collection containing instances of {@link KafkaStreamsInstanceStreamsMetadata} that have the provided storeName
 */
public Collection<KafkaStreamsInstance>Collection<StreamsMetadata> allInstancesWithStoreallMetadataForStore(final String storeName);

/**
 * Find the {@link KafkaStreamsInstanceStreamsMetadata} for a given storeName and key.
 * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore},
 * this method provides a way of finding which host it would exist on.
 * Note: this is a point in time view and it may change due to partition reassignment.
 * @param storeName         Name of the store
 * @param key               Key to use to for partition
 * @param keySerializer     Serializer for the key
 * @param <K>               key type
 * @return  The {@link KafkaStreamsInstanceStreamsMetadata} for the storeName and key
 */
public <K> KafkaStreamsInstanceStreamsMetadata instanceWithKeymetadataForKey(final String storeName, 
												   final K key, 
                                                   final Serializer<K> keySerializer);

/**
 * Find the {@link KafkaStreamsInstanceStreamsMetadata} for a given storeName and key.
 * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore},
 * this method provides a way of finding which host it would exist on.
 * Note: this is a point in time view and it may change due to partition reassignment.
 * @param storeName         Name of the store
 * @param key               Key to use to for partition
 * @param partitioner       Partitioner for the store
 * @param <K>               key type
 * @return  The {@link KafkaStreamsInstanceStreamsMetadata} for the storeName and key
 */
public <K> KafkaStreamsInstanceStreamsMetadata instanceWithKeymetadataForKey(final String storeName,
                                                   final K key,
                                                   final StreamPartitioner<K, ?> partitioner);

...

Code Block
public static void main(String[] args) throws Exception {
  Properties streamsConfiguration = new Properties();
  ...
  /**
   * To use the Discovery API the developer must provide an host:port pair that
   * maps to an embedded service listening on this address. i.e.,
   * it is up to the developer to define the protocol and create the service
   */
  streamsConfiguration.put(StreamsConfig.USERAPPLICATION_ENDPOINTSERVER_CONFIG, "localhost:7070");
  ...
  KafkaStreams streams = new KafkaStreams(builder, streamsConfiguration);
  streams.start();

  /** 
   * Start your embedded service listening on the endpoint
   * provided above
   */
  new QueryableStateProxy(streams).start(7070);
}
 
/**
 * Example Rest Proxy that uses the Discovery API to locate the
 * KafkaStreamsInstances StateStores are running on. A developer would first connect 
 * to a well-known instance to find where a particular store, or store with key,
 * is located. They'd then use the returned KafkaStreamsInstances
 * to connect to the appropriate instances and perform queries, i.e, :
 * KafkaStreamsInstance instance = http.get("http://well-known-host:8080/state/instance/my-store/my-key");
 * Long result = http.get("http://" + instance.host() + ":" + instance.port() + "/state/stores/my-store/my-key");
 */
@Path("state")
public class QueryableStateProxy {
   /**
    * The KafkaStreams instance knows about all of the other instances 
    * in the application. This is maintained by the StreamPartitionAssignor
    * on partition assignments (rebalances) 
 */ 
   private final KafkaStreams streams;
   public QueryableStateProxy(final KafkaStreams streams) {
      this.streams = streams;
   }
   
   @GET()
   @Path("/instances")
   public Response streamsInstancesstreamsMetadata() {
	 // get the current collection of KafkaStreamsInstancesStreamsMetadata
     final Collection<KafkaStreamsInstance>Collection<StreamsMetadata> instancesmetadata = streams.allInstancesallMetadata();
     return respondWithInstances(streams.allInstances()respondWithMetadata(metadata);
   }
 
   @GET()
   @Path("/instances/{storeName}")
   public Response streamsInstancesForStorestreamsMetadataForStore(@PathParam("storeName") String store) {
      // find all the instancesmetadata that have the provided store
      final Collection<KafkaStreamsInstance>Collection<StreamsMetadata> instancesmetadata = streams.allInstancesWithStoreallMetadataForStore(store);
      return respondWithInstancesrespondWithMetadata(instancesmetadata);
   }
 
   @GET()
   @Path("/instance/{storeName}/{key}")
   public Response streamsInstancesForStoreAndKeystreamsMetadataForStoreAndKey(@PathParam("storeName") String store,
                                                  @PathParam("key") String key) {
 
	  // locate the instance that would have the store with the provided key (if the key exists)
      final KafkaStreamsInstanceStreamsMetadata instancemetadata = streams.instanceWithKeymetadataForKey(store, key, new StringSerializer());
      if (instance == null) {
        return Response.noContent().build();
      }
      return Response.ok(instancemetadata.toString()).build();
   }
 
   @GET()
   @Path("/stores/{storeName}/{key}")
   public Response byKey(@PathParam("storeName") String storeName, @PathParam("id") String key) {
	  // Get a handle on the Store for the provides storeName
      final ReadOnlyKeyValueStore<String, Long> store = streams.getStorestore(storeName,
                                                                         QueryableStoreTypes.keyValueStore());
      // store may not exist or might not exist yet, i.e, if partitions haven't been assigned or 
      // a rebalance is in process
      if (store == null) {
        return Response.noContent().build();
      }

	  // we have a store so we can get the result
      final Long result = store.get(key);
      if (result == null) {
        return Response.noContent().build();
      }
      return Response.ok(result).build();
    }
 
	public void start(int port) {
     // start your favourite http server
     ...
    }
}
 

...