Skip to content
Closed
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 @@ -23,7 +23,7 @@

/**
* KStream is an abstraction of a stream of key-value pairs.
*
*
* @param <K> the type of keys
* @param <V> the type of values
*/
Expand Down Expand Up @@ -150,11 +150,28 @@ public interface KStream<K, V> {
*/
void to(String topic, Serializer<K> keySerializer, Serializer<V> valSerializer);

/**
* Applies a stateful transformation to all elements in this stream.
*
* @param transformerSupplier the class of TransformerDef
* @return KStream
*/
<K1, V1> KStream<K1, V1> transform(TransformerSupplier<K, V, KeyValue<K1, V1>> transformerSupplier);

/**
* Applies a stateful transformation to all values in this stream.
*
* @param valueTransformerSupplier the class of TransformerDef
* @return KStream
*/
<R> KStream<K, R> transformValues(ValueTransformerSupplier<V, R> valueTransformerSupplier);

/**
* Processes all elements in this stream by applying a processor.
*
* @param processorSupplier the supplier of the Processor to use
* @return the new stream containing the processed output
*/
<K1, V1> KStream<K1, V1> process(ProcessorSupplier<K, V> processorSupplier);
void process(ProcessorSupplier<K, V> processorSupplier);

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/**
* 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;

import org.apache.kafka.streams.processor.ProcessorContext;

public interface Transformer<K, V, R> {

/**
* Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology
* that contains it is initialized.
* <p>
* If this tranformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should
* {@link ProcessorContext#schedule(long) schedule itself} with the provided context.
*
* @param context the context; may not be null
*/
void init(ProcessorContext context);

/**
* Transform the message with the given key and value.
*
* @param key the key for the message
* @param value the value for the message
* @return new value
*/
R transform(K key, V value);

/**
* Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context
* during {@link #init(ProcessorContext) initialization}.
*
* @param timestamp the stream time when this method is being called
*/
void punctuate(long timestamp);

/**
* Close this processor and clean up any resources.
*/
void close();

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/**
* 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;

public interface TransformerSupplier<K, V, R> {

Transformer<K, V, R> get();

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/**
* 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;

import org.apache.kafka.streams.processor.ProcessorContext;

public interface ValueTransformer<V, R> {

/**
* Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology
* that contains it is initialized.
* <p>
* If this tranformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should
* {@link ProcessorContext#schedule(long) schedule itself} with the provided context.
*
* @param context the context; may not be null
*/
void init(ProcessorContext context);

/**
* Transform the message with the given key and value.
*
* @param value the value for the message
* @return new value
*/
R transform(V value);

/**
* Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context
* during {@link #init(ProcessorContext) initialization}.
*
* @param timestamp the stream time when this method is being called
*/
void punctuate(long timestamp);

/**
* Close this processor and clean up any resources.
*/
void close();

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/**
* 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;

public interface ValueTransformerSupplier<V, R> {

ValueTransformer<V, R> get();

}
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,11 @@

import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.streams.kstream.KeyValue;
import org.apache.kafka.streams.kstream.TransformerSupplier;
import org.apache.kafka.streams.kstream.ValueTransformerSupplier;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KStreamWindowed;
import org.apache.kafka.streams.kstream.KeyValue;
import org.apache.kafka.streams.kstream.KeyValueMapper;
import org.apache.kafka.streams.kstream.Predicate;
import org.apache.kafka.streams.kstream.ValueMapper;
Expand All @@ -44,6 +46,10 @@ public class KStreamImpl<K, V> implements KStream<K, V> {

private static final String FLATMAPVALUES_NAME = "KAFKA-FLATMAPVALUES-";

private static final String TRANSFORM_NAME = "KAFKA-TRANSFORM-";

private static final String TRANSFORMVALUES_NAME = "KAFKA-TRANSFORMVALUES-";

private static final String PROCESSOR_NAME = "KAFKA-PROCESSOR-";

private static final String BRANCH_NAME = "KAFKA-BRANCH-";
Expand Down Expand Up @@ -191,11 +197,27 @@ public void to(String topic, Serializer<K> keySerializer, Serializer<V> valSeria
}

@Override
public <K1, V1> KStream<K1, V1> process(final ProcessorSupplier<K, V> processorSupplier) {
String name = PROCESSOR_NAME + INDEX.getAndIncrement();
public <K1, V1> KStream<K1, V1> transform(TransformerSupplier<K, V, KeyValue<K1, V1>> transformerSupplier) {
String name = TRANSFORM_NAME + INDEX.getAndIncrement();

topology.addProcessor(name, processorSupplier, this.name);
topology.addProcessor(name, new KStreamTransform<>(transformerSupplier), this.name);

return new KStreamImpl<>(topology, name);
}

@Override
public <V1> KStream<K, V1> transformValues(ValueTransformerSupplier<V, V1> valueTransformerSupplier) {
String name = TRANSFORMVALUES_NAME + INDEX.getAndIncrement();

topology.addProcessor(name, new KStreamTransformValues<>(valueTransformerSupplier), this.name);

return new KStreamImpl<>(topology, name);
}

@Override
public void process(final ProcessorSupplier<K, V> processorSupplier) {
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.

The functionality of process() now is completely covered by transform: users can define a transform function with return type R be "Void" and add a dummy "return null" in the end of the function. And then in KStream we can add

public void transform(TransformerSupplier<K, V, Void>)

to replace the "process()" call. Having both process() and transform() might be confusing to users, so I would suggest we just remove process() here.

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.

This would require a separate KStreamVoidTransformProcessor, but I feel it worth the internal cost for simpler public APIs.

String name = PROCESSOR_NAME + INDEX.getAndIncrement();

topology.addProcessor(name, processorSupplier, this.name);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/**
* 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;

import org.apache.kafka.streams.kstream.KeyValue;
import org.apache.kafka.streams.kstream.Transformer;
import org.apache.kafka.streams.kstream.TransformerSupplier;
import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.ProcessorSupplier;

public class KStreamTransform<K1, V1, K2, V2> implements ProcessorSupplier<K1, V1> {

private final TransformerSupplier<K1, V1, KeyValue<K2, V2>> transformerSupplier;

public KStreamTransform(TransformerSupplier<K1, V1, KeyValue<K2, V2>> transformerSupplier) {
this.transformerSupplier = transformerSupplier;
}

@Override
public Processor<K1, V1> get() {
return new KStreamTransformProcessor(transformerSupplier.get());
}

public static class KStreamTransformProcessor<K1, V1, K2, V2> implements Processor<K1, V1> {

private final Transformer<K1, V1, KeyValue<K2, V2>> transformer;
private ProcessorContext context;

public KStreamTransformProcessor(Transformer<K1, V1, KeyValue<K2, V2>> transformer) {
this.transformer = transformer;
}

@Override
public void init(ProcessorContext context) {
transformer.init(context);
this.context = context;
}

@Override
public void process(K1 key, V1 value) {
KeyValue<K2, V2> pair = transformer.transform(key, value);
context.forward(pair.key, pair.value);
}

@Override
public void punctuate(long timestamp) {
transformer.punctuate(timestamp);
}

@Override
public void close() {
transformer.close();
}
}
}
Loading