Versions Compared

Key

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

...

Code Block
languagejava
titleStoreQueryParams.java
collapsetrue
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements. See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License. You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
package org.apache.kafka.streams;

import org.apache.kafka.streams.state.QueryableStoreType;

import java.util.Objects;

/**
 * Represents all the query options that a user can provide to state what kind of stores it is expecting. The options would be whether a user would want to enable/disable stale stores* or whether it knows the list of partitions that it specifically wants to fetch. If this information is not provided the default behavior is to fetch the stores for all the partitions available on that instance* for that particular store name.
 * It contains a partition, which for a point queries can be populated from the  KeyQueryMetadata.
 */
public class StoreQueryParams<T> {

    private Integer partition;
    private boolean includeStaleStores;
    private final String storeName;
    private final QueryableStoreType<T> queryableStoreType;

    public StoreQueryParams(final String storeName, final QueryableStoreType<T>  queryableStoreType) {
        this.storeName = storeName;
        this.queryableStoreType = queryableStoreType;
    }

    public StoreQueryParams withPartition(final Integer partition) {
        this.partition = partition;
        return this;
    }

    public StoreQueryParams withIncludeStaleStores(final boolean includeStaleStores) {
        this.includeStaleStores = includeStaleStores;
        return this;
    }

    /**
     * Get the store name for which key is queried by the user.
     *
     * @return String storeName
     */
    public String getStoreName() {
        return storeName;
    }

    /**
     * Get the queryable store type for which key is queried by the user.
     *
     * @return QueryableStoreType queryableStoreType
     */
    public QueryableStoreType<T> getQueryableStoreType() {
        return queryableStoreType;
    }

    /**
     * Get whether the store query params are fetching all partitions or a single partition.
     *
     * @return boolean. True, if all partitions are requests or false if a specific partition is requested
     */
    public boolean getAllLocalPartitions() {
        return partition == null ? true : false;
    }

    @Override
    public boolean equals(final Object obj) {
        if (!(obj instanceof StoreQueryParams)) {
            return false;
        }
        final StoreQueryParams storeQueryParams = (StoreQueryParams) obj;
        return Objects.equals(storeQueryParams.partition, partition)
                && Objects.equals(storeQueryParams.includeStaleStores, includeStaleStores)
                && Objects.equals(storeQueryParams.storeName, storeName)
                && Objects.equals(storeQueryParams.queryableStoreType, queryableStoreType);
    }

    @Override
    public String toString() {
        return "StoreQueryParams {" +
                "partition=" + partition +
                ", includeStaleStores=" + includeStaleStores +
                ", storeName=" + storeName +
                ", queryableStoreType=" + queryableStoreType +
                '}';
    }

    @Override
    public int hashCode() {
        return Objects.hash(partition, includeStaleStores, storeName, queryableStoreType);
    }
}


Deprecating the KafkaStreams#store(final String storeName, final QueryableStoreType<T> queryableStoreType, final boolean includeStaleStores) in favour of the funtion mentioned below.

Code Block
languagejava
titleKafkaStreams.java
collapsetrue
          /**
     * 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.
     *
     * Only permits queries on active replicas of the store (no standbys or restoring replicas).
     * See {@link KafkaStreams#store(StoreQueryParams)}
     * for the option to set {@code StoreQueryParams.withAllPartitionAndStaleStoresEnabled or StoreQueryParams.withPartitionAndStaleStoresEnabled(final Integer partition)} and trade off consistency in favor of availability.
     *
     * @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) {
        return store(new StoreQueryParams<T>(storeName, queryableStoreType));
    }

    /**
     * Get a facade wrapping the local {@link StateStore} instances with the provided {@link StoreQueryParams}.
     * StoreQueryParams need required parameters to be set, which are {@code storeName} and if
     * type is accepted by the provided {@link QueryableStoreType#accepts(StateStore) queryableStoreType}.
     * The optional parameters to the StoreQueryParams include {@code partition} and {@code includeStaleStores}.
     * The returned object can be used to query the {@link StateStore} instances.
     *
     * @param storeQueryParams    If storeQueryParams.withPartition(int partition) is used, it allow queries on the specific partition irrespective if it is a standby
     *                            or a restoring replicas in addition to active ones.
     *                            If storeQueryParams.withIncludeStaleStores() is used, it allow queries on standbys and restoring replicas in addition to active ones for all the local partitions on the instance.
     *                            If StoreQueryParams.withIncludeStaleStores().withPartition(int partition), it allow queries on the specific partition irrespective if it is a standby
     *                            or a restoring replicas in addition to active ones..
     *                            By default, if just storeQueryParams is used, it returns all the local partitions for the store which are in running state.
     * @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 which are provided in {@code storeQueryParams}
     */
    public <T> T store(final StoreQueryParams<T> storeQueryParams) {
        validateIsRunningOrRebalancing();
        return queryableStoreProvider.getStore(storeQueryParams);
    }

...

Compatibility, Deprecation, and Migration Plan:

  • As a part of KIP-535, we overloaded KafkaStreams#store() which will be changed to include the StoreQueryParams rather than includeStaleStores flag, so no deprecation KafkaStreams#store(final String storeName, final QueryableStoreType<T> queryableStoreType, final boolean includeStaleStores) will be changed to the one mentioned in the Public Interfaces changes. Since the mentioned function is not released yet in any version, no deprecation is required.

Rejected Alternatives:

...