-
Notifications
You must be signed in to change notification settings - Fork 4
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[DRAFT] Automatic async wrapping for stateful processors #392
base: main
Are you sure you want to change the base?
Changes from all commits
6e5baca
90fc5ab
f1884ca
ac5384d
d426ac9
18c64f1
3d1b7d1
a9263d4
5c48b37
405d3d6
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -32,9 +32,10 @@ | |
import org.apache.kafka.streams.Topology; | ||
import org.apache.kafka.streams.kstream.Consumed; | ||
import org.apache.kafka.streams.kstream.Produced; | ||
import org.apache.kafka.streams.kstream.Transformer; | ||
import org.apache.kafka.streams.processor.ProcessorContext; | ||
import org.apache.kafka.streams.processor.PunctuationType; | ||
import org.apache.kafka.streams.processor.api.Processor; | ||
import org.apache.kafka.streams.processor.api.ProcessorContext; | ||
import org.apache.kafka.streams.processor.api.Record; | ||
import org.apache.kafka.streams.state.KeyValueIterator; | ||
import org.apache.kafka.streams.state.KeyValueStore; | ||
import org.apache.kafka.streams.state.Stores; | ||
|
@@ -70,7 +71,7 @@ protected Topology buildTopology() { | |
} | ||
|
||
builder.stream(ORDERS, Consumed.with(Serdes.String(), RegressionSchema.orderSerde())) | ||
.transform(BatchTransformer::new, "grouped-orders-store") | ||
.process(BatchTransformer::new, "grouped-orders-store") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not part of async -- related to the 4.0 upgrade |
||
.peek((k, v) -> { | ||
if (responsive) { | ||
final var random = Math.abs(randomGenerator.nextLong() % 10000); | ||
|
@@ -84,14 +85,13 @@ protected Topology buildTopology() { | |
return builder.build(); | ||
} | ||
|
||
private static class BatchTransformer | ||
implements Transformer<String, Order, KeyValue<String, GroupedOrder>> { | ||
private static class BatchTransformer implements Processor<String, Order, String, GroupedOrder> { | ||
|
||
private ProcessorContext context; | ||
private ProcessorContext<String, GroupedOrder> context; | ||
private KeyValueStore<String, StoredOrder> store; | ||
|
||
@Override | ||
public void init(final ProcessorContext context) { | ||
public void init(final ProcessorContext<String, GroupedOrder> context) { | ||
this.context = context; | ||
this.store = context.getStateStore("grouped-orders-store"); | ||
this.context.schedule( | ||
|
@@ -102,15 +102,18 @@ public void init(final ProcessorContext context) { | |
} | ||
|
||
@Override | ||
public KeyValue<String, GroupedOrder> transform(final String key, final Order value) { | ||
final long ts = context.timestamp(); | ||
public void process(final Record<String, Order> record) { | ||
final long ts = record.timestamp(); | ||
|
||
// first add the order to the list of orders that are stored | ||
store.put(storedKey(key, ts), new StoredOrder(Optional.of(value), Optional.empty())); | ||
store.put( | ||
storedKey(record.key(), ts), | ||
new StoredOrder(Optional.of(record.value()), Optional.empty()) | ||
); | ||
|
||
// next, we need to update the tracked metadata row to | ||
// check whether the value ought to be emitted | ||
final String mKey = metaKey(key); | ||
final String mKey = metaKey(record.key()); | ||
final StoredOrder.Meta meta = Optional.ofNullable(store.get(mKey)) | ||
.orElse(new StoredOrder(Optional.empty(), Optional.of(new StoredOrder.Meta(ts, 0, 0)))) | ||
.meta() | ||
|
@@ -122,17 +125,15 @@ public KeyValue<String, GroupedOrder> transform(final String key, final Order va | |
final StoredOrder.Meta newMeta = new StoredOrder.Meta( | ||
ts, | ||
meta.count() + 1, | ||
meta.size() + (long) value.amount() | ||
meta.size() + (long) record.value().amount() | ||
); | ||
|
||
if (shouldFlush(newMeta, ts)) { | ||
doFlush(key); | ||
doFlush(record.key()); | ||
store.delete(mKey); | ||
} else { | ||
store.put(mKey, new StoredOrder(Optional.empty(), Optional.of(newMeta))); | ||
} | ||
|
||
return null; | ||
} | ||
|
||
private void flushExpired(long ts) { | ||
|
@@ -174,7 +175,7 @@ private void doFlush(final String key) { | |
"Got stored order with no order! %s".formatted(value)))); | ||
} | ||
|
||
context.forward(key, result); | ||
context.forward(new Record<>(key, result, 0L)); | ||
} | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -284,16 +284,16 @@ private static Properties propsWithOverrides( | |
return propsWithOverrides; | ||
} | ||
|
||
final Object o = configs.originals().get(InternalConfig.INTERNAL_TASK_ASSIGNOR_CLASS); | ||
final Object o = configs.originals().get(StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not part of async -- related to the 4.0 upgrade |
||
if (o == null) { | ||
propsWithOverrides.put( | ||
InternalConfig.INTERNAL_TASK_ASSIGNOR_CLASS, | ||
StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG, | ||
TASK_ASSIGNOR_CLASS_OVERRIDE | ||
); | ||
} else if (!TASK_ASSIGNOR_CLASS_OVERRIDE.equals(o.toString())) { | ||
final String errorMsg = String.format( | ||
"Invalid Streams configuration value for '%s': got %s, expected '%s'", | ||
InternalConfig.INTERNAL_TASK_ASSIGNOR_CLASS, | ||
StreamsConfig.TASK_ASSIGNOR_CLASS_CONFIG, | ||
o, | ||
TASK_ASSIGNOR_CLASS_OVERRIDE | ||
); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,6 +28,7 @@ | |
import org.apache.kafka.streams.processor.api.Processor; | ||
import org.apache.kafka.streams.processor.api.ProcessorContext; | ||
import org.apache.kafka.streams.processor.api.ProcessorSupplier; | ||
import org.apache.kafka.streams.processor.api.WrappedProcessorSupplier; | ||
import org.apache.kafka.streams.state.KeyValueStore; | ||
import org.apache.kafka.streams.state.StoreBuilder; | ||
|
||
|
@@ -114,10 +115,11 @@ | |
* {@link Processor} and requires at least one state store be connected. | ||
*/ | ||
public final class AsyncProcessorSupplier<KIn, VIn, KOut, VOut> | ||
implements ProcessorSupplier<KIn, VIn, KOut, VOut> { | ||
implements WrappedProcessorSupplier<KIn, VIn, KOut, VOut> { | ||
|
||
private final ProcessorSupplier<KIn, VIn, KOut, VOut> userProcessorSupplier; | ||
private final Map<String, AbstractAsyncStoreBuilder<?, ?, ?>> asyncStoreBuilders; | ||
|
||
private Map<String, AbstractAsyncStoreBuilder<?>> asyncStoreBuilders = null; | ||
|
||
/** | ||
* Create an AsyncProcessorSupplier that wraps a custom {@link ProcessorSupplier} | ||
|
@@ -132,32 +134,30 @@ public final class AsyncProcessorSupplier<KIn, VIn, KOut, VOut> | |
public static <KIn, VIn, KOut, VOut> AsyncProcessorSupplier<KIn, VIn, KOut, VOut> createAsyncProcessorSupplier( | ||
final ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier | ||
) { | ||
return new AsyncProcessorSupplier<>(processorSupplier, processorSupplier.stores()); | ||
return new AsyncProcessorSupplier<>(processorSupplier); | ||
} | ||
|
||
private AsyncProcessorSupplier( | ||
final ProcessorSupplier<KIn, VIn, KOut, VOut> userProcessorSupplier, | ||
final Set<StoreBuilder<?>> userStoreBuilders | ||
final ProcessorSupplier<KIn, VIn, KOut, VOut> userProcessorSupplier | ||
) { | ||
if (userStoreBuilders == null || userStoreBuilders.isEmpty()) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not related to KIP-1112 -- this check was supposed to be deleted already but I must have missed it before (the corresponding check in FixedKeyProcessorSupplier was correctly removed at that time) |
||
throw new UnsupportedOperationException( | ||
"Async processing currently requires at least one state store be " | ||
+ "connected to the async processor, and that stores be connected " | ||
+ "by implementing the #stores method in your processor supplier"); | ||
} | ||
|
||
this.userProcessorSupplier = userProcessorSupplier; | ||
this.asyncStoreBuilders = initializeAsyncBuilders(userStoreBuilders); | ||
} | ||
|
||
@Override | ||
public AsyncProcessor<KIn, VIn, KOut, VOut> get() { | ||
maybeInitializeAsyncStoreBuilders(); | ||
return createAsyncProcessor(userProcessorSupplier.get(), asyncStoreBuilders); | ||
} | ||
|
||
@Override | ||
public Set<StoreBuilder<?>> stores() { | ||
maybeInitializeAsyncStoreBuilders(); | ||
return new HashSet<>(asyncStoreBuilders.values()); | ||
} | ||
|
||
private void maybeInitializeAsyncStoreBuilders() { | ||
if (asyncStoreBuilders == null) { | ||
asyncStoreBuilders = initializeAsyncBuilders(userProcessorSupplier.stores()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We need to delay the initialization of the async store builders because the call to the inner processor's #stores method might return different things at different points in the build process (for example, if a downstream operator forces materialization, the #stores will return null/empty until that downstream operator is processed) |
||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
/* | ||
* Copyright 2024 Responsive Computing, Inc. | ||
* | ||
* This source code is licensed under the Responsive Business Source License Agreement v1.0 | ||
* available at: | ||
* | ||
* https://www.responsive.dev/legal/responsive-bsl-10 | ||
* | ||
* This software requires a valid Commercial License Key for production use. Trial and commercial | ||
* licenses can be obtained at https://www.responsive.dev | ||
*/ | ||
|
||
package dev.responsive.kafka.api.async; | ||
|
||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier; | ||
import org.apache.kafka.streams.processor.api.ProcessorSupplier; | ||
import org.apache.kafka.streams.processor.api.ProcessorWrapper; | ||
import org.apache.kafka.streams.processor.api.WrappedFixedKeyProcessorSupplier; | ||
import org.apache.kafka.streams.processor.api.WrappedProcessorSupplier; | ||
|
||
// TODO(sophie): only wrap when stores are Responsive (rather than eg rocksdb or in-memory) | ||
public class AsyncProcessorWrapper implements ProcessorWrapper { | ||
|
||
@Override | ||
public <KIn, VIn, KOut, VOut> WrappedProcessorSupplier<KIn, VIn, KOut, VOut> wrapProcessorSupplier( | ||
final String processorName, | ||
final ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier | ||
) { | ||
final var stores = processorSupplier.stores(); | ||
if (stores != null && !stores.isEmpty()) { | ||
return AsyncProcessorSupplier.createAsyncProcessorSupplier(processorSupplier); | ||
} else { | ||
return ProcessorWrapper.asWrapped(processorSupplier); | ||
} | ||
} | ||
|
||
@Override | ||
public <KIn, VIn, VOut> WrappedFixedKeyProcessorSupplier<KIn, VIn, VOut> wrapFixedKeyProcessorSupplier( | ||
final String processorName, | ||
final FixedKeyProcessorSupplier<KIn, VIn, VOut> processorSupplier | ||
) { | ||
final var stores = processorSupplier.stores(); | ||
if (stores != null && !stores.isEmpty()) { | ||
return AsyncFixedKeyProcessorSupplier.createAsyncProcessorSupplier(processorSupplier); | ||
} else { | ||
return ProcessorWrapper.asWrappedFixedKey(processorSupplier); | ||
} | ||
|
||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Had to put this into all the build.gradle files so I could compile from my local AK branch. I can remove all of these before merging if anyone feels strongly about it, but if it doesn't do any harm then I'd rather keep these for the next time I'm working off of a dev branch