Skip to content

Commit

Permalink
KAFKA-18026: KIP-1112, configure all StoreBuilder & StoreFactory laye…
Browse files Browse the repository at this point in the history
…rs (#18345)

Because of how we have to wrap StoreFactory and StoreBuilder layers on top of each other for various parts of the topology building process, we need to make sure both of these are capable of configuration and will check for & delegate to an underlying layer if it exists

Reviewers: Guozhang Wang <guozhang.wang.us@gmail.com>
  • Loading branch information
ableegoldman authored Dec 31, 2024
1 parent f88cf57 commit 86b5528
Show file tree
Hide file tree
Showing 3 changed files with 40 additions and 6 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,25 @@
/*
* 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.processor.internals;

import org.apache.kafka.streams.StreamsConfig;

public interface ConfigurableStore {

void configure(final StreamsConfig config);

}
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package org.apache.kafka.streams.processor.internals;

import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.internals.SessionStoreBuilder;
import org.apache.kafka.streams.state.internals.TimestampedWindowStoreBuilder;
Expand All @@ -34,6 +35,7 @@
*/
public class StoreBuilderWrapper implements StoreFactory {


private final StoreBuilder<?> builder;
private final Set<String> connectedProcessorNames = new HashSet<>();

Expand All @@ -49,6 +51,13 @@ private StoreBuilderWrapper(final StoreBuilder<?> builder) {
this.builder = builder;
}

@Override
public void configure(final StreamsConfig config) {
if (builder instanceof ConfigurableStore) {
((ConfigurableStore) builder).configure(config);
}
}

@Override
public StoreBuilder<?> builder() {
return builder;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,11 +46,7 @@
* to {@link org.apache.kafka.streams.StreamsBuilder#StreamsBuilder(TopologyConfig)}</li>
* </ul>
*/
public interface StoreFactory {

default void configure(final StreamsConfig config) {
// do nothing
}
public interface StoreFactory extends ConfigurableStore {

StoreBuilder<?> builder();

Expand All @@ -76,7 +72,7 @@ default void configure(final StreamsConfig config) {

boolean isCompatibleWith(StoreFactory storeFactory);

class FactoryWrappingStoreBuilder<T extends StateStore> implements StoreBuilder<T> {
class FactoryWrappingStoreBuilder<T extends StateStore> implements StoreBuilder<T>, ConfigurableStore {

private final StoreFactory storeFactory;

Expand All @@ -88,6 +84,10 @@ public StoreFactory storeFactory() {
return storeFactory;
}

public void configure(final StreamsConfig config) {
storeFactory.configure(config);
}

@Override
public boolean equals(final Object o) {
if (this == o) {
Expand Down

0 comments on commit 86b5528

Please sign in to comment.