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 @@ -405,6 +405,20 @@ void writeAsText(final String filePath,
*/
void foreach(final ForeachAction<? super K, ? super V> action);

/**
* Perform an action on each record of {@code KStream}.
* This is a stateless record-by-record operation (cf. {@link #process(ProcessorSupplier, String...)}).
* <p>
* Peek is a non-terminal operation that triggers a side effect (such as logging or statistics collection)
* and returns an unchanged stream.
* <p>
* Note that since this operation is stateless, it may execute multiple times for a single record in failure cases.
*
* @param action an action to perform on each record
* @see #process(ProcessorSupplier, String...)
*/
KStream<K, V> peek(final ForeachAction<? super K, ? super V> action);

/**
* Creates an array of {@code KStream} from this stream by branching the records in the original stream based on
* the supplied predicates.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@ public class KStreamImpl<K, V> extends AbstractStream<K> implements KStream<K, V

public static final String FILTER_NAME = "KSTREAM-FILTER-";

public static final String PEEK_NAME = "KSTREAM-PEEK-";

private static final String FLATMAP_NAME = "KSTREAM-FLATMAP-";

private static final String FLATMAPVALUES_NAME = "KSTREAM-FLATMAPVALUES-";
Expand Down Expand Up @@ -317,6 +319,16 @@ public void foreach(ForeachAction<? super K, ? super V> action) {
topology.addProcessor(name, new KStreamForeach<>(action), this.name);
}

@Override
public KStream<K, V> peek(final ForeachAction<? super K, ? super V> action) {
Objects.requireNonNull(action, "action can't be null");
final String name = topology.newName(PEEK_NAME);

topology.addProcessor(name, new KStreamPeek<>(action), this.name);

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

@Override
public KStream<K, V> through(Serde<K> keySerde, Serde<V> valSerde, String topic) {
return through(keySerde, valSerde, null, topic);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/**
* 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.ForeachAction;
import org.apache.kafka.streams.processor.AbstractProcessor;
import org.apache.kafka.streams.processor.Processor;
import org.apache.kafka.streams.processor.ProcessorSupplier;

class KStreamPeek<K, V> implements ProcessorSupplier<K, V> {

private final ForeachAction<K, V> action;

public KStreamPeek(final ForeachAction<K, V> action) {
this.action = action;
}

@Override
public Processor<K, V> get() {
return new KStreamPeekProcessor();
}

private class KStreamPeekProcessor extends AbstractProcessor<K, V> {
@Override
public void process(final K key, final V value) {
action.apply(key, value);
context().forward(key, value);
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/**
* 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 static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;

import java.util.ArrayList;
import java.util.List;

import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.kstream.ForeachAction;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KStreamBuilder;
import org.apache.kafka.test.KStreamTestDriver;
import org.junit.After;
import org.junit.Test;

public class KStreamPeekTest {

private final String topicName = "topic";

private KStreamTestDriver driver = null;

@After
public void cleanup() {
if (driver != null) {
driver.close();
}
}

@Test
public void shouldObserveStreamElements() {
final KStreamBuilder builder = new KStreamBuilder();
final KStream<Integer, String> stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName);
final List<KeyValue<Integer, String>> peekObserved = new ArrayList<>(), streamObserved = new ArrayList<>();
stream.peek(collect(peekObserved)).foreach(collect(streamObserved));

driver = new KStreamTestDriver(builder);
final List<KeyValue<Integer, String>> expected = new ArrayList<>();
for (int key = 0; key < 32; key++) {
Copy link
Member

Choose a reason for hiding this comment

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

add final

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I don't think I understand -- how can key be final and still key++ in the loop?

Copy link
Member

Choose a reason for hiding this comment

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

Sorry for that -- You are of course right. final only for iterator loops...

final String value = "V" + key;
driver.process(topicName, key, value);
expected.add(new KeyValue<>(key, value));
}

assertEquals(expected, peekObserved);
assertEquals(expected, streamObserved);
}

@Test
public void shouldNotAllowNullAction() {
final KStreamBuilder builder = new KStreamBuilder();
final KStream<Integer, String> stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName);
try {
stream.peek(null);
fail("expected null action to throw NPE");
} catch (NullPointerException expected) { }
}

private static <K, V> ForeachAction<K, V> collect(final List<KeyValue<K, V>> into) {
return new ForeachAction<K, V>() {
@Override
public void apply(final K key, final V value) {
into.add(new KeyValue<>(key, value));
}
};
}
}