/** * Find all currently running {@code KafkaStreams} instances (potentially remotely) that use the same * {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all instances that belong to * the same Kafka Streams application) and return {@link StreamsMetadata} for each discovered instance. * <p> * Note: this is a point in time view and it may change due to partition reassignment. * * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances of this application */ public Collection<StreamsMetadata> allMetadata() { validateIsRunning(); return streamsMetadataState.getAllMetadata(); }
/** * Returns runtime information about the local threads of this {@link KafkaStreams} instance. * * @return the set of {@link ThreadMetadata}. */ public Set<ThreadMetadata> localThreadsMetadata() { validateIsRunning(); final Set<ThreadMetadata> threadMetadata = new HashSet<>(); for (final StreamThread thread : threads) { threadMetadata.add(thread.threadMetadata()); } return threadMetadata; } }
/** * Get a facade wrapping the local {@link StateStore} instances with the provided {@code storeName} if the Store's * type is accepted by the provided {@link QueryableStoreType#accepts(StateStore) queryableStoreType}. * The returned object can be used to query the {@link StateStore} instances. * * @param storeName name of the store to find * @param queryableStoreType accept only stores that are accepted by {@link QueryableStoreType#accepts(StateStore)} * @param <T> return type * @return A facade wrapping the local {@link StateStore} instances * @throws InvalidStateStoreException if Kafka Streams is (re-)initializing or a store with {@code storeName} and * {@code queryableStoreType} doesn't exist */ public <T> T store(final String storeName, final QueryableStoreType<T> queryableStoreType) { validateIsRunning(); return queryableStoreProvider.getStore(storeName, queryableStoreType); }
/** * Find all currently running {@code KafkaStreams} instances (potentially remotely) that * <ul> * <li>use the same {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all * instances that belong to the same Kafka Streams application)</li> * <li>and that contain a {@link StateStore} with the given {@code storeName}</li> * </ul> * and return {@link StreamsMetadata} for each discovered instance. * <p> * Note: this is a point in time view and it may change due to partition reassignment. * * @param storeName the {@code storeName} to find metadata for * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances with the provide {@code storeName} of * this application */ public Collection<StreamsMetadata> allMetadataForStore(final String storeName) { validateIsRunning(); return streamsMetadataState.getAllMetadataForStore(storeName); }
/** * Find the currently running {@code KafkaStreams} instance (potentially remotely) that * <ul> * <li>use the same {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all * instances that belong to the same Kafka Streams application)</li> * <li>and that contain a {@link StateStore} with the given {@code storeName}</li> * <li>and the {@link StateStore} contains the given {@code key}</li> * </ul> * and return {@link StreamsMetadata} for it. * <p> * Note: * <ul> * <li>this is a point in time view and it may change due to partition reassignment</li> * <li>the key may not exist in the {@link StateStore}; this method provides a way of finding which host it * <em>would</em> exist on</li> * </ul> * * @param storeName the {@code storeName} to find metadata for * @param key the key to find metadata for * @param partitioner the partitioner to be use to locate the host for the key * @param <K> key type * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provide {@code storeName} and * {@code key} of this application or {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing */ public <K> StreamsMetadata metadataForKey(final String storeName, final K key, final StreamPartitioner<? super K, ?> partitioner) { validateIsRunning(); return streamsMetadataState.getMetadataWithKey(storeName, key, partitioner); }
final K key, final Serializer<K> keySerializer) { validateIsRunning(); return streamsMetadataState.getMetadataWithKey(storeName, key, keySerializer);