private void validateStreamConfigures(Config config, Map<String, Serde> deserializedSerdes) { if (deserializedSerdes != null) { validateStreamSerdeConfigure(input1Descriptor.getStreamId(), config, deserializedSerdes); validateStreamSerdeConfigure(input2Descriptor.getStreamId(), config, deserializedSerdes); validateStreamSerdeConfigure(outputDescriptor.getStreamId(), config, deserializedSerdes); validateStreamSerdeConfigure(intermediateInputDescriptor.getStreamId(), config, deserializedSerdes); } // generated stream config for intermediate stream String physicalName = intermediateInputDescriptor.getPhysicalName().isPresent() ? intermediateInputDescriptor.getPhysicalName().get() : null; validateIntermediateStreamConfigure(intermediateInputDescriptor.getStreamId(), physicalName, config); }
@Test public void testConfigRewriter() { Map<String, String> configs = new HashMap<>(mockConfig); String streamCfgToOverride = String.format("streams.%s.samza.system", intermediateInputDescriptor.getStreamId()); configs.put(String.format(JobConfig.CONFIG_REWRITER_CLASS(), "mock"), MockConfigRewriter.class.getName()); configs.put(JobConfig.CONFIG_REWRITERS(), "mock"); configs.put(String.format("job.config.rewriter.mock.%s", streamCfgToOverride), "rewritten-system"); mockConfig = spy(new MapConfig(configs)); mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); configureJobNode(mockStreamAppDesc); JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedConfig, jobConfig); assertEquals("rewritten-system", jobConfig.get(streamCfgToOverride)); }
systemDescriptors.add(system1Descriptor); systemDescriptors.add(system2Descriptor); inputDescriptors.put(input1Descriptor.getStreamId(), input1Descriptor); inputDescriptors.put(input2Descriptor.getStreamId(), input2Descriptor); inputDescriptors.put(input3Descriptor.getStreamId(), input3Descriptor); inputDescriptors.put(input4Descriptor.getStreamId(), input4Descriptor); outputDescriptors.put(output1Descriptor.getStreamId(), output1Descriptor); outputDescriptors.put(output2Descriptor.getStreamId(), output2Descriptor);
StreamEdge repartitionEdge = new StreamEdge(repartitionSpec, true, false, mockConfig); Map<String, StreamEdge> inputEdges = new HashMap<>(); inputEdges.put(input1Descriptor.getStreamId(), input1Edge); inputEdges.put(input2Descriptor.getStreamId(), input2Edge); inputEdges.put(repartitionSpec.getId(), repartitionEdge); Map<String, StreamEdge> outputEdges = new HashMap<>();
@Test public void testBroadcastStreamApplication() { // set the application to BroadcastStreamApplication mockStreamAppDesc = new StreamApplicationDescriptorImpl(getBroadcastOnlyStreamApplication(defaultSerde), mockConfig); configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); Map<String, Serde> deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 2); validateStreamSerdeConfigure(broadcastInputDesriptor.getStreamId(), jobConfig, deserializedSerdes); validateIntermediateStreamConfigure(broadcastInputDesriptor.getStreamId(), broadcastInputDesriptor.getPhysicalName().get(), jobConfig); }
@Test public void testStreamApplicationWithTableAndSideInput() { mockStreamAppDesc = new StreamApplicationDescriptorImpl(getRepartitionJoinStreamApplication(), mockConfig); // add table to the RepartitionJoinStreamApplication GenericInputDescriptor<KV<String, Object>> sideInput1 = inputSystemDescriptor.getInputDescriptor("sideInput1", defaultSerde); BaseTableDescriptor mockTableDescriptor = new MockLocalTableDescriptor("testTable", defaultSerde) .withSideInputs(Arrays.asList(sideInput1.getStreamId())) .withSideInputsProcessor(mock(SideInputsProcessor.class, withSettings().serializable())) .withConfig("mock.table.provider.config", "mock.config.value"); // add side input and terminate at table in the appplication mockStreamAppDesc.getInputStream(sideInput1).sendTo(mockStreamAppDesc.getTable(mockTableDescriptor)); StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", inputSystemDescriptor.getSystemName()), false, false, mockConfig); // need to put the sideInput related stream configuration to the original config // TODO: this is confusing since part of the system and stream related configuration is generated outside the JobGraphConfigureGenerator // It would be nice if all system and stream related configuration is generated in one place and only intermediate stream // configuration is generated by JobGraphConfigureGenerator Map<String, String> configs = new HashMap<>(mockConfig); configs.putAll(sideInputEdge.generateConfig()); mockConfig = spy(new MapConfig(configs)); configureJobNode(mockStreamAppDesc); // create the JobGraphConfigureGenerator and generate the jobConfig for the jobNode JobNodeConfigurationGenerator configureGenerator = new JobNodeConfigurationGenerator(); JobConfig jobConfig = configureGenerator.generateJobConfig(mockJobNode, "testJobGraphJson"); Config expectedJobConfig = getExpectedJobConfig(mockConfig, mockJobNode.getInEdges()); validateJobConfig(expectedJobConfig, jobConfig); Map<String, Serde> deserializedSerdes = validateAndGetDeserializedSerdes(jobConfig, 5); validateTableConfigure(jobConfig, deserializedSerdes, mockTableDescriptor); }
@Test public void testPartitionBy() { MapFunction<Object, String> keyFn = m -> m.toString(); MapFunction<Object, Object> valueFn = m -> m; KVSerde<Object, Object> partitionBySerde = KVSerde.of(new NoOpSerde<>(), new NoOpSerde<>()); StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputDescriptor); inputStream.partitionBy(keyFn, valueFn, partitionBySerde, testRepartitionedStreamName); }, getConfig()); assertEquals(2, streamAppDesc.getInputOperators().size()); Map<String, InputOperatorSpec> inputOpSpecs = streamAppDesc.getInputOperators(); assertTrue(inputOpSpecs.keySet().contains(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testRepartitionedStreamName))); InputOperatorSpec inputOpSpec = inputOpSpecs.get(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testRepartitionedStreamName)); assertEquals(String.format("%s-%s-partition_by-%s", testJobName, testJobId, testRepartitionedStreamName), inputOpSpec.getStreamId()); assertTrue(inputOpSpec.getKeySerde() instanceof NoOpSerde); assertTrue(inputOpSpec.getValueSerde() instanceof NoOpSerde); assertTrue(inputOpSpec.isKeyed()); assertNull(inputOpSpec.getScheduledFn()); assertNull(inputOpSpec.getWatermarkFn()); InputOperatorSpec originInputSpec = inputOpSpecs.get(testInputDescriptor.getStreamId()); assertTrue(originInputSpec.getRegisteredOperatorSpecs().toArray()[0] instanceof PartitionByOperatorSpec); PartitionByOperatorSpec reparOpSpec = (PartitionByOperatorSpec) originInputSpec.getRegisteredOperatorSpecs().toArray()[0]; assertEquals(reparOpSpec.getOpId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testRepartitionedStreamName)); assertEquals(reparOpSpec.getKeyFunction(), keyFn); assertEquals(reparOpSpec.getValueFunction(), valueFn); assertEquals(reparOpSpec.getOutputStream().getStreamId(), reparOpSpec.getOpId()); assertNull(reparOpSpec.getScheduledFn()); assertNull(reparOpSpec.getWatermarkFn()); }
.withSideInputs(Arrays.asList(sideInput1.getStreamId())) .withSideInputsProcessor(mock(SideInputsProcessor.class, withSettings().serializable())) .withConfig("mock.table.provider.config", "mock.config.value"); StreamEdge sideInputEdge = new StreamEdge(new StreamSpec(sideInput1.getStreamId(), "sideInput1", inputSystemDescriptor.getSystemName()), false, false, mockConfig);
@Test public void testPartitionByWithNoSerde() { MapFunction<Object, String> keyFn = m -> m.toString(); MapFunction<Object, Object> valueFn = m -> m; StreamApplicationDescriptorImpl streamAppDesc = new StreamApplicationDescriptorImpl(appDesc -> { MessageStream inputStream = appDesc.getInputStream(testInputDescriptor); inputStream.partitionBy(keyFn, valueFn, mock(KVSerde.class), testRepartitionedStreamName); }, getConfig()); InputOperatorSpec inputOpSpec = streamAppDesc.getInputOperators().get( String.format("%s-%s-partition_by-%s", testJobName, testJobId, testRepartitionedStreamName)); assertNotNull(inputOpSpec); assertNull(inputOpSpec.getKeySerde()); assertNull(inputOpSpec.getValueSerde()); assertTrue(inputOpSpec.isKeyed()); assertNull(inputOpSpec.getScheduledFn()); assertNull(inputOpSpec.getWatermarkFn()); InputOperatorSpec originInputSpec = streamAppDesc.getInputOperators().get(testInputDescriptor.getStreamId()); assertTrue(originInputSpec.getRegisteredOperatorSpecs().toArray()[0] instanceof PartitionByOperatorSpec); PartitionByOperatorSpec reparOpSpec = (PartitionByOperatorSpec) originInputSpec.getRegisteredOperatorSpecs().toArray()[0]; assertEquals(reparOpSpec.getOpId(), String.format("%s-%s-partition_by-%s", testJobName, testJobId, testRepartitionedStreamName)); assertEquals(reparOpSpec.getKeyFunction(), keyFn); assertEquals(reparOpSpec.getValueFunction(), valueFn); assertEquals(reparOpSpec.getOutputStream().getStreamId(), reparOpSpec.getOpId()); assertNull(reparOpSpec.getScheduledFn()); assertNull(reparOpSpec.getWatermarkFn()); }