Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.kafka.streams.kstream.SlidingWindows;
import org.apache.kafka.streams.kstream.Window;
import org.apache.kafka.streams.kstream.Windows;
import org.apache.kafka.streams.kstream.internals.graph.GracePeriodGraphNode;
import org.apache.kafka.streams.kstream.internals.graph.GraphNode;
import org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode.OptimizableRepartitionNodeBuilder;
import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
Expand Down Expand Up @@ -110,10 +111,11 @@ <KR, W extends Window> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>
"-cogroup-agg-" + counter++,
builder,
CogroupedKStreamImpl.AGGREGATE_NAME);
final ProcessorGraphNode<K, ?> aggProcessorNode =
new ProcessorGraphNode<>(
final GracePeriodGraphNode<K, ?> aggProcessorNode =
new GracePeriodGraphNode<>(
kStreamAggProcessorName,
new ProcessorParameters<>(parentProcessor, kStreamAggProcessorName)
new ProcessorParameters<>(parentProcessor, kStreamAggProcessorName),
windows.gracePeriodMs()
);
processors.add(aggProcessorNode);
builder.addGraphNode(parentNodes.get(kGroupedStream.getKey()), aggProcessorNode);
Expand Down Expand Up @@ -149,10 +151,12 @@ <KR> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? sup
"-cogroup-agg-" + counter++,
builder,
CogroupedKStreamImpl.AGGREGATE_NAME);
final ProcessorGraphNode<K, ?> aggProcessorNode =
new ProcessorGraphNode<>(
final long gracePeriod = sessionWindows.gracePeriodMs() + sessionWindows.inactivityGap();
final GracePeriodGraphNode<K, ?> aggProcessorNode =
new GracePeriodGraphNode<>(
kStreamAggProcessorName,
new ProcessorParameters<>(parentProcessor, kStreamAggProcessorName)
new ProcessorParameters<>(parentProcessor, kStreamAggProcessorName),
gracePeriod
);
processors.add(aggProcessorNode);
builder.addGraphNode(parentNodes.get(kGroupedStream.getKey()), aggProcessorNode);
Expand Down Expand Up @@ -187,10 +191,11 @@ <KR> KTable<KR, VOut> build(final Map<KGroupedStreamImpl<K, ?>, Aggregator<? sup
"-cogroup-agg-" + counter++,
builder,
CogroupedKStreamImpl.AGGREGATE_NAME);
final ProcessorGraphNode<K, ?> aggProcessorNode =
new ProcessorGraphNode<>(
final GracePeriodGraphNode<K, ?> aggProcessorNode =
new GracePeriodGraphNode<>(
kStreamAggProcessorName,
new ProcessorParameters<>(parentProcessor, kStreamAggProcessorName)
new ProcessorParameters<>(parentProcessor, kStreamAggProcessorName),
slidingWindows.gracePeriodMs()
);
processors.add(aggProcessorNode);
builder.addGraphNode(parentNodes.get(kGroupedStream.getKey()), aggProcessorNode);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@
import org.apache.kafka.streams.kstream.Aggregator;
import org.apache.kafka.streams.kstream.Initializer;
import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.internals.graph.GracePeriodGraphNode;
import org.apache.kafka.streams.kstream.internals.graph.GraphNode;
import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
import org.apache.kafka.streams.processor.internals.StoreFactory;

import java.util.Collections;
import java.util.Set;
Expand Down Expand Up @@ -66,23 +66,67 @@ class GroupedStreamAggregateBuilder<K, V> {
this.userProvidedRepartitionTopicName = groupedInternal.name();
}

<KR, VR> KTable<KR, VR> build(final NamedInternal functionName,
final StoreFactory storeFactory,
final KStreamAggProcessorSupplier<K, V, KR, VR> aggregateSupplier,
final String queryableStoreName,
final Serde<KR> keySerde,
final Serde<VR> valueSerde,
final boolean isOutputVersioned) {
assert queryableStoreName == null || queryableStoreName.equals(storeFactory.storeName());
<KR, VR> KTable<KR, VR> buildNonWindowed(final NamedInternal functionName,
final String storeName,
final KStreamAggProcessorSupplier<K, V, KR, VR> aggregateSupplier,
final String queryableStoreName,
final Serde<KR> keySerde,
final Serde<VR> valueSerde,
final boolean isOutputVersioned) {
final String aggFunctionName = functionName.name();

final ProcessorGraphNode<K, V> aggProcessorNode =
new ProcessorGraphNode<>(
aggFunctionName,
new ProcessorParameters<>(aggregateSupplier, aggFunctionName)
);

aggProcessorNode.setOutputVersioned(isOutputVersioned);

return build(aggFunctionName, storeName, aggregateSupplier, aggProcessorNode, queryableStoreName, keySerde, valueSerde);
}

<KR, VR> KTable<KR, VR> buildWindowed(final NamedInternal functionName,
final String storeName,
final long gracePeriod,
final KStreamAggProcessorSupplier<K, V, KR, VR> aggregateSupplier,
final String queryableStoreName,
final Serde<KR> keySerde,
final Serde<VR> valueSerde,
final boolean isOutputVersioned) {
final String aggFunctionName = functionName.name();

final GracePeriodGraphNode<K, V> gracePeriodAggProcessorNode =
new GracePeriodGraphNode<>(
aggFunctionName,
new ProcessorParameters<>(aggregateSupplier, aggFunctionName),
gracePeriod
);

gracePeriodAggProcessorNode.setOutputVersioned(isOutputVersioned);

return build(aggFunctionName, storeName, aggregateSupplier, gracePeriodAggProcessorNode, queryableStoreName, keySerde, valueSerde);
}

private <KR, VR> KTable<KR, VR> build(final String aggFunctionName,
final String storeName,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: misaligned indentation?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

whoops 🙂

final KStreamAggProcessorSupplier<K, V, KR, VR> aggregateSupplier,
final ProcessorGraphNode<K, V> aggProcessorNode,
final String queryableStoreName,
final Serde<KR> keySerde,
final Serde<VR> valueSerde) {
if (!(queryableStoreName == null || queryableStoreName.equals(storeName))) {
throw new IllegalStateException(String.format("queryableStoreName should be null or equal to storeName"
+ " but got storeName='%s' and queryableStoreName='%s'",
storeName, queryableStoreName));
}

String sourceName = this.name;
GraphNode parentNode = graphNode;

if (repartitionRequired) {
final OptimizableRepartitionNodeBuilder<K, V> repartitionNodeBuilder = optimizableRepartitionNodeBuilder();
final String repartitionTopicPrefix = userProvidedRepartitionTopicName != null ? userProvidedRepartitionTopicName : storeFactory.storeName();
final String repartitionTopicPrefix = userProvidedRepartitionTopicName != null ? userProvidedRepartitionTopicName : storeName;
sourceName = createRepartitionSource(repartitionTopicPrefix, repartitionNodeBuilder);

// First time through we need to create a repartition node.
Expand All @@ -97,24 +141,16 @@ <KR, VR> KTable<KR, VR> build(final NamedInternal functionName,
parentNode = repartitionNode;
}

final ProcessorGraphNode<K, V> statefulProcessorNode =
new ProcessorGraphNode<>(
aggFunctionName,
new ProcessorParameters<>(aggregateSupplier, aggFunctionName)
);
statefulProcessorNode.setOutputVersioned(isOutputVersioned);

builder.addGraphNode(parentNode, statefulProcessorNode);
builder.addGraphNode(parentNode, aggProcessorNode);

return new KTableImpl<>(aggFunctionName,
keySerde,
valueSerde,
sourceName.equals(this.name) ? subTopologySourceNodes : Collections.singleton(sourceName),
queryableStoreName,
aggregateSupplier,
statefulProcessorNode,
aggProcessorNode,
builder);

}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -244,9 +244,9 @@ private <T> KTable<K, T> doAggregate(final KStreamAggProcessorSupplier<K, V, K,
final String functionName,
final KeyValueStoreMaterializer<K, T> storeFactory) {

return aggregateBuilder.build(
return aggregateBuilder.buildNonWindowed(
new NamedInternal(functionName),
storeFactory,
storeFactory.storeName(),
aggregateSupplier,
storeFactory.queryableStoreName(),
storeFactory.keySerde(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1225,8 +1225,7 @@ public <KOut, VOut> KStream<KOut, VOut> process(
}

final String name = new NamedInternal(named).name();
final ProcessorToStateConnectorNode<? super K, ? super V> processNode = new
ProcessorToStateConnectorNode<>(
final ProcessorToStateConnectorNode<? super K, ? super V> processNode = new ProcessorToStateConnectorNode<>(
name,
new ProcessorParameters<>(processorSupplier, name),
stateStoreNames);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,10 +110,12 @@ private KTable<Windowed<K>, Long> doCount(final Named named,

final String aggregateName = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME);
final StoreFactory storeFactory = new SessionStoreMaterializer<>(materializedInternal, windows, emitStrategy);
final long gracePeriod = windows.gracePeriodMs() + windows.inactivityGap();

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(aggregateName),
storeFactory,
storeFactory.storeName(),
gracePeriod,
new KStreamSessionWindowAggregate<>(
windows,
storeFactory,
Expand Down Expand Up @@ -162,10 +164,12 @@ public KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,

final String reduceName = new NamedInternal(named).orElseGenerateWithPrefix(builder, REDUCE_NAME);
final StoreFactory storeFactory = new SessionStoreMaterializer<>(materializedInternal, windows, emitStrategy);
final long gracePeriod = windows.gracePeriodMs() + windows.inactivityGap();

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(reduceName),
storeFactory,
storeFactory.storeName(),
gracePeriod,
new KStreamSessionWindowAggregate<>(
windows,
storeFactory,
Expand Down Expand Up @@ -222,10 +226,12 @@ public <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,

final String aggregateName = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME);
final StoreFactory storeFactory = new SessionStoreMaterializer<>(materializedInternal, windows, emitStrategy);
final long gracePeriod = windows.gracePeriodMs() + windows.inactivityGap();

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(aggregateName),
storeFactory,
storeFactory.storeName(),
gracePeriod,
new KStreamSessionWindowAggregate<>(
windows,
storeFactory,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,9 +93,10 @@ private KTable<Windowed<K>, Long> doCount(final Named named,
final String aggregateName = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME);
final StoreFactory storeFactory = new SlidingWindowStoreMaterializer<>(materializedInternal, windows, emitStrategy);

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(aggregateName),
storeFactory,
storeFactory.storeName(),
windows.gracePeriodMs(),
new KStreamSlidingWindowAggregate<>(windows, storeFactory, emitStrategy, aggregateBuilder.countInitializer, aggregateBuilder.countAggregator),
materializedInternal.queryableStoreName(),
materializedInternal.keySerde() != null ? new FullTimeWindowedSerde<>(materializedInternal.keySerde(), windows.timeDifferenceMs()) : null,
Expand Down Expand Up @@ -139,9 +140,10 @@ public <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
final String aggregateName = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME);
final StoreFactory storeFactory = new SlidingWindowStoreMaterializer<>(materializedInternal, windows, emitStrategy);

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(aggregateName),
storeFactory,
storeFactory.storeName(),
windows.gracePeriodMs(),
new KStreamSlidingWindowAggregate<>(windows, storeFactory, emitStrategy, initializer, aggregator),
materializedInternal.queryableStoreName(),
materializedInternal.keySerde() != null ? new FullTimeWindowedSerde<>(materializedInternal.keySerde(), windows.timeDifferenceMs()) : null,
Expand Down Expand Up @@ -186,9 +188,10 @@ public KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
final String reduceName = new NamedInternal(named).orElseGenerateWithPrefix(builder, REDUCE_NAME);
final StoreFactory storeFactory = new SlidingWindowStoreMaterializer<>(materializedInternal, windows, emitStrategy);

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(reduceName),
storeFactory,
storeFactory.storeName(),
windows.gracePeriodMs(),
new KStreamSlidingWindowAggregate<>(windows, storeFactory, emitStrategy, aggregateBuilder.reduceInitializer, aggregatorForReducer(reducer)),
materializedInternal.queryableStoreName(),
materializedInternal.keySerde() != null ? new FullTimeWindowedSerde<>(materializedInternal.keySerde(), windows.timeDifferenceMs()) : null,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,9 +105,10 @@ private KTable<Windowed<K>, Long> doCount(final Named named,
final String aggregateName = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME);
final StoreFactory storeFactory = new WindowStoreMaterializer<>(materializedInternal, windows, emitStrategy);

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(aggregateName),
storeFactory,
storeFactory.storeName(),
windows.gracePeriodMs(),
new KStreamWindowAggregate<>(
windows,
storeFactory,
Expand Down Expand Up @@ -158,9 +159,10 @@ public <VR> KTable<Windowed<K>, VR> aggregate(final Initializer<VR> initializer,
final String aggregateName = new NamedInternal(named).orElseGenerateWithPrefix(builder, AGGREGATE_NAME);
final StoreFactory storeFactory = new WindowStoreMaterializer<>(materializedInternal, windows, emitStrategy);

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(aggregateName),
storeFactory,
storeFactory.storeName(),
windows.gracePeriodMs(),
new KStreamWindowAggregate<>(
windows,
storeFactory,
Expand Down Expand Up @@ -210,9 +212,10 @@ public KTable<Windowed<K>, V> reduce(final Reducer<V> reducer,
final String reduceName = new NamedInternal(named).orElseGenerateWithPrefix(builder, REDUCE_NAME);
final StoreFactory storeFactory = new WindowStoreMaterializer<>(materializedInternal, windows, emitStrategy);

return aggregateBuilder.build(
return aggregateBuilder.buildWindowed(
new NamedInternal(reduceName),
storeFactory,
storeFactory.storeName(),
windows.gracePeriodMs(),
new KStreamWindowAggregate<>(
windows,
storeFactory,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
/*
* 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.kstream.internals.graph;

/**
* Represents a stateful {@link ProcessorGraphNode} where a semantic grace period is defined for the processor
* and its state.
*/
public class GracePeriodGraphNode<K, V> extends ProcessorGraphNode<K, V> {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if it makes sense to add something directly to ProcessorGraphNode such as ProcessorGraphNode#getWindowInformation that returns an Optional<WindowInformation> that contains grace period.

That removes the instanceof GracePeriodGraphNode check, at the cost of adding something empty to all the other nodes. I'm ambivalent as either approach works for me.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah I considered that too. In the end I opted for the smaller change, as there are some other things that work similarly which would be nice to clean up, so I was thinking it might make sense to wait until everything is done and then we can consider the full picture


private final long gracePeriod;

public GracePeriodGraphNode(final String nodeName,
final ProcessorParameters<K, V, ?, ?> processorParameters,
final long gracePeriod) {
super(nodeName, processorParameters);
this.gracePeriod = gracePeriod;
}

public long gracePeriod() {
return gracePeriod;
}
}
Loading