blob: 9d1141bdab36c284b56dc0b3f2a9362b697b8583 [file] [log] [blame]
/**
* 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.common.serialization.Serdes;
import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.KStreamBuilder;
import org.apache.kafka.streams.kstream.ValueMapper;
import org.apache.kafka.test.KStreamTestDriver;
import org.apache.kafka.test.MockProcessorSupplier;
import org.junit.After;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Locale;
import static org.junit.Assert.assertEquals;
public class KStreamFlatMapValuesTest {
private String topicName = "topic";
private KStreamTestDriver driver = null;
@After
public void cleanup() {
if (driver != null) {
driver.close();
}
driver = null;
}
@Test
public void testFlatMapValues() {
KStreamBuilder builder = new KStreamBuilder();
ValueMapper<String, Iterable<String>> mapper =
new ValueMapper<String, Iterable<String>>() {
@Override
public Iterable<String> apply(String value) {
ArrayList<String> result = new ArrayList<String>();
result.add(value.toLowerCase(Locale.ROOT));
result.add(value);
return result;
}
};
final int[] expectedKeys = {0, 1, 2, 3};
KStream<Integer, String> stream;
MockProcessorSupplier<Integer, String> processor;
processor = new MockProcessorSupplier<>();
stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName);
stream.flatMapValues(mapper).process(processor);
driver = new KStreamTestDriver(builder);
for (int i = 0; i < expectedKeys.length; i++) {
driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]);
}
assertEquals(8, processor.processed.size());
String[] expected = {"0:v0", "0:V0", "1:v1", "1:V1", "2:v2", "2:V2", "3:v3", "3:V3"};
for (int i = 0; i < expected.length; i++) {
assertEquals(expected[i], processor.processed.get(i));
}
}
}