|
25 | 25 | import org.apache.kafka.common.config.ConfigException;
|
26 | 26 | import org.apache.kafka.common.metrics.Sensor;
|
27 | 27 | import org.apache.kafka.common.network.Selectable;
|
| 28 | +import org.apache.kafka.common.serialization.Deserializer; |
| 29 | +import org.apache.kafka.common.serialization.Serde; |
28 | 30 | import org.apache.kafka.common.serialization.Serdes;
|
| 31 | +import org.apache.kafka.common.serialization.Serializer; |
29 | 32 | import org.apache.kafka.common.serialization.StringDeserializer;
|
30 | 33 | import org.apache.kafka.common.serialization.StringSerializer;
|
31 | 34 | import org.apache.kafka.common.utils.Utils;
|
32 | 35 | import org.apache.kafka.streams.errors.StreamsException;
|
33 | 36 | import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
|
34 | 37 | import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
|
35 | 38 | import org.apache.kafka.streams.kstream.Consumed;
|
| 39 | +import org.apache.kafka.streams.kstream.Grouped; |
36 | 40 | import org.apache.kafka.streams.kstream.Materialized;
|
| 41 | +import org.apache.kafka.streams.kstream.Produced; |
37 | 42 | import org.apache.kafka.streams.processor.AbstractProcessor;
|
38 | 43 | import org.apache.kafka.streams.processor.ThreadMetadata;
|
39 | 44 | import org.apache.kafka.streams.processor.internals.GlobalStreamThread;
|
|
74 | 79 | import java.util.stream.Collectors;
|
75 | 80 |
|
76 | 81 | import static java.util.Arrays.asList;
|
| 82 | +import static org.easymock.EasyMock.anyObject; |
| 83 | +import static org.easymock.EasyMock.eq; |
| 84 | +import static org.easymock.EasyMock.expect; |
| 85 | +import static org.easymock.EasyMock.expectLastCall; |
| 86 | +import static org.easymock.EasyMock.mock; |
| 87 | +import static org.easymock.EasyMock.replay; |
| 88 | +import static org.easymock.EasyMock.verify; |
77 | 89 | import static org.junit.Assert.assertEquals;
|
78 | 90 | import static org.junit.Assert.assertFalse;
|
79 | 91 | import static org.junit.Assert.assertNotNull;
|
@@ -725,6 +737,131 @@ public void statefulTopologyShouldCreateStateDirectory() throws Exception {
|
725 | 737 | startStreamsAndCheckDirExists(topology, asList(inputTopic, globalTopicName), outputTopic, true);
|
726 | 738 | }
|
727 | 739 |
|
| 740 | + @SuppressWarnings("unchecked") |
| 741 | + @Test |
| 742 | + public void shouldInitializeUserSerdes() { |
| 743 | + final Deserializer mockSourceKeyDeserialzer = mock(Deserializer.class); |
| 744 | + mockSourceKeyDeserialzer.configure(anyObject(), eq(true)); |
| 745 | + expectLastCall(); |
| 746 | + final Deserializer mockSourceValueDeserialzer = mock(Deserializer.class); |
| 747 | + mockSourceValueDeserialzer.configure(anyObject(), eq(false)); |
| 748 | + expectLastCall(); |
| 749 | + |
| 750 | + final Serde mockSourceKeySerde = mock(Serde.class); |
| 751 | + final Serde mockSourceValueSerde = mock(Serde.class); |
| 752 | + expect(mockSourceKeySerde.deserializer()).andReturn(mockSourceKeyDeserialzer).anyTimes(); |
| 753 | + expect(mockSourceValueSerde.deserializer()).andReturn(mockSourceValueDeserialzer).anyTimes(); |
| 754 | + |
| 755 | + |
| 756 | + |
| 757 | + final Serializer mockThroughKeySerializer = mock(Serializer.class); |
| 758 | + mockThroughKeySerializer.configure(anyObject(), eq(true)); |
| 759 | + expectLastCall(); |
| 760 | + final Serializer mockThroughValueSerializer = mock(Serializer.class); |
| 761 | + mockThroughValueSerializer.configure(anyObject(), eq(false)); |
| 762 | + expectLastCall(); |
| 763 | + final Deserializer mockThroughKeyDeserializer = mock(Deserializer.class); |
| 764 | + mockThroughKeyDeserializer.configure(anyObject(), eq(true)); |
| 765 | + expectLastCall(); |
| 766 | + final Deserializer mockThroughValueDeserializer = mock(Deserializer.class); |
| 767 | + mockThroughValueDeserializer.configure(anyObject(), eq(false)); |
| 768 | + expectLastCall(); |
| 769 | + |
| 770 | + final Serde mockThroughKeySerde = mock(Serde.class); |
| 771 | + final Serde mockThroughValueSerde = mock(Serde.class); |
| 772 | + expect(mockThroughKeySerde.serializer()).andReturn(mockThroughKeySerializer).anyTimes(); |
| 773 | + expect(mockThroughValueSerde.serializer()).andReturn(mockThroughValueSerializer).anyTimes(); |
| 774 | + expect(mockThroughKeySerde.deserializer()).andReturn(mockThroughKeyDeserializer).anyTimes(); |
| 775 | + expect(mockThroughValueSerde.deserializer()).andReturn(mockThroughValueDeserializer).anyTimes(); |
| 776 | + |
| 777 | + |
| 778 | + |
| 779 | + final Serializer mockGroupedKeySerializer = mock(Serializer.class); |
| 780 | + mockGroupedKeySerializer.configure(anyObject(), eq(true)); |
| 781 | + expectLastCall(); |
| 782 | + final Serializer mockGroupedValueSerializer = mock(Serializer.class); |
| 783 | + mockGroupedValueSerializer.configure(anyObject(), eq(false)); |
| 784 | + expectLastCall(); |
| 785 | + final Deserializer mockGroupedKeyDeserializer = mock(Deserializer.class); |
| 786 | + mockGroupedKeyDeserializer.configure(anyObject(), eq(true)); |
| 787 | + expectLastCall(); |
| 788 | + final Deserializer mockGroupedValueDeserializer = mock(Deserializer.class); |
| 789 | + mockGroupedValueDeserializer.configure(anyObject(), eq(false)); |
| 790 | + expectLastCall(); |
| 791 | + |
| 792 | + final Serde mockGroupedKeySerde = mock(Serde.class); |
| 793 | + final Serde mockGroupedValueSerde = mock(Serde.class); |
| 794 | + expect(mockGroupedKeySerde.serializer()).andReturn(mockGroupedKeySerializer).anyTimes(); |
| 795 | + expect(mockGroupedValueSerde.serializer()).andReturn(mockGroupedValueSerializer).anyTimes(); |
| 796 | + expect(mockGroupedKeySerde.deserializer()).andReturn(mockGroupedKeyDeserializer).anyTimes(); |
| 797 | + expect(mockGroupedValueSerde.deserializer()).andReturn(mockGroupedValueDeserializer).anyTimes(); |
| 798 | + |
| 799 | + |
| 800 | + |
| 801 | + final Serializer mockOutputKeySerializer = mock(Serializer.class); |
| 802 | + mockOutputKeySerializer.configure(anyObject(), eq(true)); |
| 803 | + expectLastCall(); |
| 804 | + final Serializer mockOutputValueSerializer = mock(Serializer.class); |
| 805 | + mockOutputValueSerializer.configure(anyObject(), eq(false)); |
| 806 | + expectLastCall(); |
| 807 | + |
| 808 | + final Serde mockOutputKeySerde = mock(Serde.class); |
| 809 | + final Serde mockOutputValueSerde = mock(Serde.class); |
| 810 | + expect(mockOutputKeySerde.serializer()).andReturn(mockOutputKeySerializer).anyTimes(); |
| 811 | + expect(mockOutputValueSerde.serializer()).andReturn(mockOutputValueSerializer).anyTimes(); |
| 812 | + |
| 813 | + |
| 814 | + |
| 815 | + final Deserializer mockGlobalKeyDeserializer = mock(Deserializer.class); |
| 816 | + mockGlobalKeyDeserializer.configure(anyObject(), eq(true)); |
| 817 | + expectLastCall(); |
| 818 | + final Deserializer mockGlobalValueDeserializer = mock(Deserializer.class); |
| 819 | + mockGlobalValueDeserializer.configure(anyObject(), eq(false)); |
| 820 | + expectLastCall(); |
| 821 | + |
| 822 | + final Serde mockGlobalKeySerde = mock(Serde.class); |
| 823 | + final Serde mockGlobalValueSerde = mock(Serde.class); |
| 824 | + expect(mockGlobalKeySerde.deserializer()).andReturn(mockGlobalKeyDeserializer).anyTimes(); |
| 825 | + expect(mockGlobalValueSerde.deserializer()).andReturn(mockGlobalValueDeserializer).anyTimes(); |
| 826 | + |
| 827 | + |
| 828 | + |
| 829 | + builder |
| 830 | + .stream("anyTopic", Consumed.with(mockSourceKeySerde, mockSourceValueSerde)) |
| 831 | + .through("anyOtherTopic", Produced.with(mockThroughKeySerde, mockThroughValueSerde)) |
| 832 | + .selectKey(KeyValue::pair) |
| 833 | + .groupByKey(Grouped.with(mockGroupedKeySerde, mockGroupedValueSerde)) |
| 834 | + .count() |
| 835 | + .toStream() |
| 836 | + .to("anyOutput", Produced.with(mockOutputKeySerde, mockOutputValueSerde)); |
| 837 | + builder.globalTable("anyGlobal", Consumed.with(mockGlobalKeySerde, mockGlobalValueSerde)); |
| 838 | + |
| 839 | + replay( |
| 840 | + mockSourceKeyDeserialzer, mockSourceValueDeserialzer, mockSourceKeySerde, mockSourceValueSerde, |
| 841 | + mockThroughKeySerializer, mockThroughKeyDeserializer, mockThroughKeySerde, |
| 842 | + mockThroughValueSerializer, mockThroughValueDeserializer, mockThroughValueSerde, |
| 843 | + mockGroupedKeySerializer, mockGroupedKeyDeserializer, mockGroupedKeySerde, |
| 844 | + mockGroupedValueSerializer, mockGroupedValueDeserializer, mockGroupedValueSerde, |
| 845 | + mockOutputKeySerializer, mockOutputValueSerializer, mockOutputKeySerde, mockOutputValueSerde, |
| 846 | + mockGlobalKeyDeserializer, mockGlobalValueDeserializer, mockGlobalKeySerde, mockGlobalValueSerde); |
| 847 | + |
| 848 | + KafkaStreams kafkaStreams = null; |
| 849 | + try { |
| 850 | + kafkaStreams = new KafkaStreams(builder.build(), props); |
| 851 | + } finally { |
| 852 | + if (kafkaStreams != null) { |
| 853 | + kafkaStreams.close(); |
| 854 | + } |
| 855 | + } |
| 856 | + |
| 857 | + verify( |
| 858 | + mockSourceKeyDeserialzer, mockSourceValueDeserialzer, |
| 859 | + mockThroughKeySerializer, mockThroughValueSerializer, mockThroughKeyDeserializer, mockThroughValueDeserializer, |
| 860 | + mockGroupedKeySerializer, mockGroupedValueSerializer, mockGroupedKeyDeserializer, mockGroupedValueDeserializer, |
| 861 | + mockOutputKeySerializer, mockOutputValueSerializer, |
| 862 | + mockGlobalKeyDeserializer, mockGlobalValueDeserializer); |
| 863 | + } |
| 864 | + |
728 | 865 | @SuppressWarnings("unchecked")
|
729 | 866 | private Topology getStatefulTopology(final String inputTopic,
|
730 | 867 | final String outputTopic,
|
|
0 commit comments