You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jp...@apache.org on 2016/07/05 19:18:35 UTC

[2/5] nifi-minifi git commit: MINIFI-41 - CLI utility for template.xml -> YAML and YAML validation

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMainTest.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMainTest.java b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMainTest.java
new file mode 100644
index 0000000..bb33d8b
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMainTest.java
@@ -0,0 +1,247 @@
+/*
+ * 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.nifi.minifi.toolkit.configuration;
+
+import org.apache.commons.io.Charsets;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import javax.xml.bind.JAXBException;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ConfigMainTest {
+    @Mock
+    PathInputStreamFactory pathInputStreamFactory;
+
+    @Mock
+    PathOutputStreamFactory pathOutputStreamFactory;
+
+    ConfigMain configMain;
+
+    String testInput;
+
+    String testOutput;
+
+    @Before
+    public void setup() {
+        configMain = new ConfigMain(pathInputStreamFactory, pathOutputStreamFactory);
+        testInput = "testInput";
+        testOutput = "testOutput";
+    }
+
+    @Test
+    public void testExecuteNoArgs() {
+        assertEquals(ConfigMain.ERR_INVALID_ARGS, configMain.execute(new String[0]));
+    }
+
+    @Test
+    public void testExecuteInvalidCommand() {
+        assertEquals(ConfigMain.ERR_INVALID_ARGS, configMain.execute(new String[]{"badCommand"}));
+    }
+
+    @Test
+    public void testValidateInvalidCommand() {
+        assertEquals(ConfigMain.ERR_INVALID_ARGS, configMain.execute(new String[]{ConfigMain.VALIDATE}));
+    }
+
+    @Test
+    public void testValidateErrorOpeningInput() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenThrow(new FileNotFoundException());
+        assertEquals(ConfigMain.ERR_UNABLE_TO_OPEN_INPUT, configMain.execute(new String[]{ConfigMain.VALIDATE, testInput}));
+    }
+
+    @Test
+    public void testValidateUnableToParseConfig() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenReturn(new ByteArrayInputStream("!@#$%^&".getBytes(Charsets.UTF_8)));
+        assertEquals(ConfigMain.ERR_UNABLE_TO_PARSE_CONFIG, configMain.execute(new String[]{ConfigMain.VALIDATE, testInput}));
+    }
+
+    @Test
+    public void testValidateInvalidConfig() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenAnswer(invocation ->
+                ConfigMainTest.class.getClassLoader().getResourceAsStream("config-malformed-field.yml"));
+        assertEquals(ConfigMain.ERR_INVALID_CONFIG, configMain.execute(new String[]{ConfigMain.VALIDATE, testInput}));
+    }
+
+    @Test
+    public void testTransformInvalidCommand() {
+        assertEquals(ConfigMain.ERR_INVALID_ARGS, configMain.execute(new String[]{ConfigMain.TRANSFORM}));
+    }
+
+    @Test
+    public void testValidateSuccess() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenAnswer(invocation ->
+                ConfigMainTest.class.getClassLoader().getResourceAsStream("config.yml"));
+        assertEquals(ConfigMain.SUCCESS, configMain.execute(new String[]{ConfigMain.VALIDATE, testInput}));
+    }
+
+    @Test
+    public void testTransformErrorOpeningInput() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenThrow(new FileNotFoundException());
+        assertEquals(ConfigMain.ERR_UNABLE_TO_OPEN_INPUT, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
+    }
+
+    @Test
+    public void testTransformErrorOpeningOutput() throws FileNotFoundException {
+        when(pathOutputStreamFactory.create(testOutput)).thenThrow(new FileNotFoundException());
+        assertEquals(ConfigMain.ERR_UNABLE_TO_OPEN_OUTPUT, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
+    }
+
+    @Test
+    public void testTransformErrorReadingTemplate() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenAnswer(invocation -> new ByteArrayInputStream("malformed xml".getBytes(Charsets.UTF_8)));
+        assertEquals(ConfigMain.ERR_UNABLE_TO_READ_TEMPLATE, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
+    }
+
+    @Test
+    public void testTransformErrorTransformingTemplate() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenAnswer(invocation ->
+                ConfigMainTest.class.getClassLoader().getResourceAsStream("CsvToJson.xml"));
+        when(pathOutputStreamFactory.create(testOutput)).thenAnswer(invocation -> new OutputStream() {
+            @Override
+            public void write(int b) throws IOException {
+                throw new IOException();
+            }
+        });
+        assertEquals(ConfigMain.ERR_UNABLE_TO_TRANFORM_TEMPLATE, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
+    }
+
+    @Test
+    public void testTransformSuccess() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenAnswer(invocation ->
+                ConfigMainTest.class.getClassLoader().getResourceAsStream("CsvToJson.xml"));
+        when(pathOutputStreamFactory.create(testOutput)).thenAnswer(invocation -> new ByteArrayOutputStream());
+        assertEquals(ConfigMain.SUCCESS, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
+    }
+
+    @Test
+    public void testTransformRoundTripCsvToJson() throws IOException, JAXBException, SchemaLoaderException {
+        transformRoundTrip("CsvToJson");
+    }
+
+    @Test
+    public void testTransformRoundTripDecompression() throws IOException, JAXBException, SchemaLoaderException {
+        transformRoundTrip("DecompressionCircularFlow");
+    }
+
+    @Test
+    public void testTransformRoundTripInvokeHttp() throws IOException, JAXBException, SchemaLoaderException {
+        transformRoundTrip("InvokeHttpMiNiFiTemplateTest");
+    }
+
+    @Test
+    public void testTransformRoundTripReplaceText() throws IOException, JAXBException, SchemaLoaderException {
+        transformRoundTrip("ReplaceTextExpressionLanguageCSVReformatting");
+    }
+
+    @Test
+    public void testTransformRoundTripStressTestFramework() throws IOException, JAXBException, SchemaLoaderException {
+        transformRoundTrip("StressTestFramework");
+    }
+
+    private void transformRoundTrip(String name) throws JAXBException, IOException, SchemaLoaderException {
+        Map<String, Object> templateMap = ConfigMain.transformTemplateToSchema(getClass().getClassLoader().getResourceAsStream(name + ".xml")).toMap();
+        Map<String, Object> yamlMap = SchemaLoader.loadYamlAsMap(getClass().getClassLoader().getResourceAsStream(name + ".yml"));
+        assertNoMapDifferences(templateMap, yamlMap);
+    }
+
+    private void assertNoMapDifferences(Map<String, Object> templateMap, Map<String, Object> yamlMap) {
+        List<String> differences = new ArrayList<>();
+        getMapDifferences("", differences, yamlMap, templateMap);
+        if (differences.size() > 0) {
+            fail(String.join("\n", differences.toArray(new String[differences.size()])));
+        }
+    }
+
+    private void getMapDifferences(String path, List<String> differences, Map<String, Object> expected, Map<String, Object> actual) {
+        for (Map.Entry<String, Object> stringObjectEntry : expected.entrySet()) {
+            String key = stringObjectEntry.getKey();
+            String newPath = path.isEmpty() ? key : path + "." + key;
+            if (!actual.containsKey(key)) {
+                differences.add("Missing key: " + newPath);
+            } else {
+                getObjectDifferences(newPath, differences, stringObjectEntry.getValue(), actual.get(key));
+            }
+        }
+
+        Set<String> extraKeys = new HashSet<>(actual.keySet());
+        extraKeys.removeAll(expected.keySet());
+        for (String extraKey : extraKeys) {
+            differences.add("Extra key: " + path + extraKey);
+        }
+    }
+
+    private void getListDifferences(String path, List<String> differences, List<Object> expected, List<Object> actual) {
+        if (expected.size() == actual.size()) {
+            for (int i = 0; i < expected.size(); i++) {
+                getObjectDifferences(path + "[" + i + "]", differences, expected.get(i), actual.get(i));
+            }
+        } else {
+            differences.add("Expect size of " + expected.size() + " for list at " + path + " but got " + actual.size());
+        }
+    }
+
+    private void getObjectDifferences(String path, List<String> differences, Object expectedValue, Object actualValue) {
+        if (expectedValue instanceof Map) {
+            if (actualValue instanceof Map) {
+                getMapDifferences(path, differences, (Map) expectedValue, (Map) actualValue);
+            } else {
+                differences.add("Expected map at " + path + " but got " + actualValue);
+            }
+        } else if (expectedValue instanceof List) {
+            if (actualValue instanceof List) {
+                getListDifferences(path, differences, (List) expectedValue, (List) actualValue);
+            } else {
+                differences.add("Expected map at " + path + " but got " + actualValue);
+            }
+        } else if (expectedValue == null) {
+            if (actualValue != null) {
+                differences.add("Expected null at " + path + " but got " + actualValue);
+            }
+        } else if (expectedValue instanceof Number) {
+            if (actualValue instanceof Number) {
+                if (!expectedValue.toString().equals(actualValue.toString())) {
+                    differences.add("Expected value of " + expectedValue + " at " + path + " but got " + actualValue);
+                }
+            } else {
+                differences.add("Expected Number at " + path + " but got " + actualValue);
+            }
+        } else if (!expectedValue.equals(actualValue)) {
+            differences.add("Expected " + expectedValue + " at " + path + " but got " + actualValue);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/BaseSchemaTester.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/BaseSchemaTester.java b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/BaseSchemaTester.java
new file mode 100644
index 0000000..4dcea13
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/BaseSchemaTester.java
@@ -0,0 +1,55 @@
+/*
+ * 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.nifi.minifi.toolkit.configuration.dto;
+
+import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.junit.Test;
+
+import java.util.Map;
+import java.util.function.Function;
+
+import static org.junit.Assert.assertEquals;
+
+public abstract class BaseSchemaTester<Schema extends BaseSchema, DTO> {
+    protected final Function<DTO, Schema> dtoSchemaFunction;
+    protected final Function<Map, Schema> mapSchemaFunction;
+    protected DTO dto;
+    protected Map<String, Object> map;
+
+    protected BaseSchemaTester(Function<DTO, Schema> dtoSchemaFunction, Function<Map, Schema> mapSchemaFunction) {
+        this.dtoSchemaFunction = dtoSchemaFunction;
+        this.mapSchemaFunction = mapSchemaFunction;
+    }
+
+    protected void assertDtoAndMapConstructorAreSame(int validationErrors) {
+        Schema dtoSchema = dtoSchemaFunction.apply(dto);
+        Schema mapSchema = mapSchemaFunction.apply(map);
+        assertSchemaEquals(dtoSchema, mapSchema);
+        assertEquals(dtoSchema.validationIssues, mapSchema.validationIssues);
+        assertSchemaEquals(dtoSchema, mapSchemaFunction.apply(dtoSchema.toMap()));
+        assertSchemaEquals(mapSchema, mapSchemaFunction.apply(mapSchema.toMap()));
+        assertEquals(validationErrors, dtoSchema.validationIssues.size());
+    }
+
+    public abstract void assertSchemaEquals(Schema one, Schema two);
+
+    @Test
+    public void testFullyPopulatedSame() {
+        assertDtoAndMapConstructorAreSame(0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ConnectionSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ConnectionSchemaTest.java b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ConnectionSchemaTest.java
new file mode 100644
index 0000000..ebd4b2f
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ConnectionSchemaTest.java
@@ -0,0 +1,162 @@
+/*
+ * 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.nifi.minifi.toolkit.configuration.dto;
+
+import org.apache.nifi.connectable.ConnectableType;
+import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
+import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
+import org.apache.nifi.web.api.dto.ConnectableDTO;
+import org.apache.nifi.web.api.dto.ConnectionDTO;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+
+public class ConnectionSchemaTest extends BaseSchemaTester<ConnectionSchema, ConnectionDTO> {
+    private String testName = "testName";
+    private String testSourceName = "testSourceName";
+    private String testSelectedRelationship = "testSelectedRelationship";
+    private String testDestinationName = "testDestinationName";
+    private long testMaxWorkQueueSize = 101L;
+    private String testMaxWorkQueueDataSize = "120 GB";
+    private String testFlowfileExpiration = "1 day";
+    private String testQueuePrioritizerClass = "testQueuePrioritizerClass";
+
+    public ConnectionSchemaTest() {
+        super(new ConnectionSchemaFunction(), ConnectionSchema::new);
+    }
+
+    @Before
+    public void setup() {
+        ConnectableDTO source = new ConnectableDTO();
+        source.setName(testSourceName);
+
+        ConnectableDTO destination = new ConnectableDTO();
+        destination.setName(testDestinationName);
+
+        dto = new ConnectionDTO();
+        dto.setName(testName);
+        dto.setSource(source);
+        dto.setSelectedRelationships(Arrays.asList(testSelectedRelationship).stream().collect(Collectors.toSet()));
+        dto.setDestination(destination);
+        dto.setBackPressureObjectThreshold(testMaxWorkQueueSize);
+        dto.setBackPressureDataSizeThreshold(testMaxWorkQueueDataSize);
+        dto.setFlowFileExpiration(testFlowfileExpiration);
+        dto.setPrioritizers(Arrays.asList(testQueuePrioritizerClass));
+
+        map = new HashMap<>();
+        map.put(CommonPropertyKeys.NAME_KEY, testName);
+        map.put(ConnectionSchema.SOURCE_NAME_KEY, testSourceName);
+        map.put(ConnectionSchema.SOURCE_RELATIONSHIP_NAME_KEY, testSelectedRelationship);
+        map.put(ConnectionSchema.DESTINATION_NAME_KEY, testDestinationName);
+        map.put(ConnectionSchema.MAX_WORK_QUEUE_SIZE_KEY, testMaxWorkQueueSize);
+        map.put(ConnectionSchema.MAX_WORK_QUEUE_DATA_SIZE_KEY, testMaxWorkQueueDataSize);
+        map.put(ConnectionSchema.FLOWFILE_EXPIRATION__KEY, testFlowfileExpiration);
+        map.put(ConnectionSchema.QUEUE_PRIORITIZER_CLASS_KEY, testQueuePrioritizerClass);
+    }
+
+    @Test
+    public void testNoName() {
+        dto.setName(null);
+        map.remove(CommonPropertyKeys.NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoSourceName() {
+        dto.setSource(new ConnectableDTO());
+        map.remove(ConnectionSchema.SOURCE_NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testDtoMultipleSourceRelationships() {
+        dto.setSelectedRelationships(Arrays.asList("one", "two").stream().collect(Collectors.toSet()));
+        assertEquals(1, dtoSchemaFunction.apply(dto).validationIssues.size());
+    }
+
+    @Test
+    public void testNoSelectedRelationshipName() {
+        dto.setSelectedRelationships(null);
+        map.remove(ConnectionSchema.SOURCE_RELATIONSHIP_NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+        dto.setSelectedRelationships(Collections.emptySet());
+        map.remove(ConnectionSchema.SOURCE_RELATIONSHIP_NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoDestinationName() {
+        dto.setDestination(new ConnectableDTO());
+        map.remove(ConnectionSchema.DESTINATION_NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoMaxWorkQueueSize() {
+        dto.setBackPressureObjectThreshold(null);
+        map.remove(ConnectionSchema.MAX_WORK_QUEUE_SIZE_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoMaxWorkQueueDataSize() {
+        dto.setBackPressureDataSizeThreshold(null);
+        map.remove(ConnectionSchema.MAX_WORK_QUEUE_DATA_SIZE_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoFlowFileExpiration() {
+        dto.setFlowFileExpiration(null);
+        map.remove(ConnectionSchema.FLOWFILE_EXPIRATION__KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoQueuePrioritizerClass() {
+        dto.setPrioritizers(null);
+        map.remove(ConnectionSchema.QUEUE_PRIORITIZER_CLASS_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+        dto.setPrioritizers(Collections.emptyList());
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testFunnelValidationMessage() {
+        dto.getSource().setType(ConnectableType.FUNNEL.name());
+        assertEquals(1, dtoSchemaFunction.apply(dto).validationIssues.size());
+    }
+
+    @Override
+    public void assertSchemaEquals(ConnectionSchema one, ConnectionSchema two) {
+        assertEquals(one.getName(), two.getName());
+        assertEquals(one.getSourceName(), two.getSourceName());
+        assertEquals(one.getSourceRelationshipName(), two.getSourceRelationshipName());
+        assertEquals(one.getDestinationName(), two.getDestinationName());
+        assertEquals(one.getMaxWorkQueueSize(), two.getMaxWorkQueueSize());
+        assertEquals(one.getMaxWorkQueueDataSize(), two.getMaxWorkQueueDataSize());
+        assertEquals(one.getFlowfileExpiration(), two.getFlowfileExpiration());
+        assertEquals(one.getQueuePrioritizerClass(), two.getQueuePrioritizerClass());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/FlowControllerSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/FlowControllerSchemaTest.java b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/FlowControllerSchemaTest.java
new file mode 100644
index 0000000..a7f5874
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/FlowControllerSchemaTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.nifi.minifi.toolkit.configuration.dto;
+
+import org.apache.nifi.minifi.commons.schema.FlowControllerSchema;
+import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
+import org.apache.nifi.web.api.dto.TemplateDTO;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+
+public class FlowControllerSchemaTest extends BaseSchemaTester<FlowControllerSchema, TemplateDTO> {
+    private String testName = "testName";
+    private String testComment = "testComment";
+
+    public FlowControllerSchemaTest() {
+        super(new FlowControllerSchemaFunction(), FlowControllerSchema::new);
+    }
+
+    @Before
+    public void setup() {
+        dto = new TemplateDTO();
+
+        dto.setName(testName);
+        dto.setDescription(testComment);
+
+        map = new HashMap<>();
+
+        map.put(CommonPropertyKeys.NAME_KEY, testName);
+        map.put(CommonPropertyKeys.COMMENT_KEY, testComment);
+    }
+
+    @Test
+    public void testNoNameSame() {
+        dto.setName(null);
+        map.remove(CommonPropertyKeys.NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoCommentSame() {
+        dto.setDescription(null);
+        map.remove(CommonPropertyKeys.COMMENT_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Override
+    public void assertSchemaEquals(FlowControllerSchema one, FlowControllerSchema two) {
+        assertEquals(one.getName(), two.getName());
+        assertEquals(one.getComment(), two.getComment());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ProcessorSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ProcessorSchemaTest.java b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ProcessorSchemaTest.java
new file mode 100644
index 0000000..915fd83
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/ProcessorSchemaTest.java
@@ -0,0 +1,180 @@
+/*
+ * 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.nifi.minifi.toolkit.configuration.dto;
+
+import org.apache.nifi.minifi.commons.schema.ProcessorSchema;
+import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.web.api.dto.ProcessorConfigDTO;
+import org.apache.nifi.web.api.dto.ProcessorDTO;
+import org.apache.nifi.web.api.dto.RelationshipDTO;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class ProcessorSchemaTest extends BaseSchemaTester<ProcessorSchema, ProcessorDTO> {
+    private String testName = "testName";
+    private String testProcessorClass = "testProcessorClass";
+    private String testSchedulingStrategy = SchedulingStrategy.PRIMARY_NODE_ONLY.name();
+    private String testSchedulingPeriod = "10 s";
+    private int testMaxConcurrentTasks = 101;
+    private String testYieldDuration = "5 s";
+    private long testRunDurationNanos = 1111000L;
+    private String testRelationship = "testRelationship";
+    private String testKey = "testKey";
+    private String testValue = "testValue";
+    private String testPenalizationPeriod = "55 s";
+    private ProcessorConfigDTO config;
+
+    public ProcessorSchemaTest() {
+        super(new ProcessorSchemaFunction(), ProcessorSchema::new);
+    }
+
+    @Before
+    public void setup() {
+        config = new ProcessorConfigDTO();
+
+        RelationshipDTO relationshipDTO = new RelationshipDTO();
+        relationshipDTO.setName(testRelationship);
+        relationshipDTO.setAutoTerminate(true);
+
+        dto = new ProcessorDTO();
+        dto.setConfig(config);
+        dto.setName(testName);
+        dto.setType(testProcessorClass);
+        config.setSchedulingStrategy(testSchedulingStrategy);
+        config.setSchedulingPeriod(testSchedulingPeriod);
+        config.setConcurrentlySchedulableTaskCount(testMaxConcurrentTasks);
+        config.setPenaltyDuration(testPenalizationPeriod);
+        config.setYieldDuration(testYieldDuration);
+        config.setRunDurationMillis(testRunDurationNanos / 1000);
+        dto.setRelationships(Arrays.asList(relationshipDTO));
+        Map<String, String> properties = new HashMap<>();
+        properties.put(testKey, testValue);
+        config.setProperties(properties);
+
+        map = new HashMap<>();
+        map.put(CommonPropertyKeys.NAME_KEY, testName);
+        map.put(ProcessorSchema.CLASS_KEY, testProcessorClass);
+        map.put(CommonPropertyKeys.SCHEDULING_STRATEGY_KEY, testSchedulingStrategy);
+        map.put(CommonPropertyKeys.SCHEDULING_PERIOD_KEY, testSchedulingPeriod);
+        map.put(CommonPropertyKeys.MAX_CONCURRENT_TASKS_KEY, testMaxConcurrentTasks);
+        map.put(ProcessorSchema.PENALIZATION_PERIOD_KEY, testPenalizationPeriod);
+        map.put(CommonPropertyKeys.YIELD_PERIOD_KEY, testYieldDuration);
+        map.put(ProcessorSchema.RUN_DURATION_NANOS_KEY, testRunDurationNanos);
+        map.put(ProcessorSchema.AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY, Arrays.asList(testRelationship));
+        map.put(ProcessorSchema.PROCESSOR_PROPS_KEY, new HashMap<>(properties));
+    }
+
+    @Test
+    public void testNoName() {
+        dto.setName(null);
+        map.remove(CommonPropertyKeys.NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoProcessorClass() {
+        dto.setType(null);
+        map.remove(ProcessorSchema.CLASS_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoSchedulingStrategy() {
+        config.setSchedulingStrategy(null);
+        map.remove(CommonPropertyKeys.SCHEDULING_STRATEGY_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testInvalidSchedulingStrategy() {
+        String fake = "fake";
+        config.setSchedulingStrategy(fake);
+        map.put(CommonPropertyKeys.SCHEDULING_STRATEGY_KEY, fake);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoSchedulingPeriod() {
+        config.setSchedulingPeriod(null);
+        map.remove(CommonPropertyKeys.SCHEDULING_PERIOD_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoMaxConcurrentTasks() {
+        config.setConcurrentlySchedulableTaskCount(null);
+        map.remove(CommonPropertyKeys.MAX_CONCURRENT_TASKS_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoPenalizationPeriod() {
+        config.setPenaltyDuration(null);
+        map.remove(ProcessorSchema.PENALIZATION_PERIOD_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoYieldPeriod() {
+        config.setYieldDuration(null);
+        map.remove(CommonPropertyKeys.YIELD_PERIOD_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoRunDurationNanos() {
+        config.setRunDurationMillis(null);
+        map.remove(ProcessorSchema.RUN_DURATION_NANOS_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoAutoTerminateRelationships() {
+        dto.setRelationships(null);
+        map.remove(ProcessorSchema.AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoProperties() {
+        config.setProperties(null);
+        map.remove(ProcessorSchema.PROCESSOR_PROPS_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Override
+    public void assertSchemaEquals(ProcessorSchema one, ProcessorSchema two) {
+        assertEquals(one.getName(), two.getName());
+        assertEquals(one.getProcessorClass(), two.getProcessorClass());
+        assertEquals(one.getSchedulingStrategy(), two.getSchedulingStrategy());
+        assertEquals(one.getSchedulingPeriod(), two.getSchedulingPeriod());
+        assertEquals(one.getMaxConcurrentTasks(), two.getMaxConcurrentTasks());
+        assertEquals(one.getPenalizationPeriod(), two.getPenalizationPeriod());
+        assertEquals(one.getYieldPeriod(), two.getYieldPeriod());
+        assertEquals(one.getRunDurationNanos(), two.getRunDurationNanos());
+        assertEquals(one.getAutoTerminatedRelationshipsList(), two.getAutoTerminatedRelationshipsList());
+        assertEquals(one.getProperties(), two.getProperties());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteInputPortSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteInputPortSchemaTest.java b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteInputPortSchemaTest.java
new file mode 100644
index 0000000..e21554a
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteInputPortSchemaTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.nifi.minifi.toolkit.configuration.dto;
+
+import org.apache.nifi.minifi.commons.schema.RemoteInputPortSchema;
+import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
+import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+
+import static org.junit.Assert.assertEquals;
+
+public class RemoteInputPortSchemaTest extends BaseSchemaTester<RemoteInputPortSchema, RemoteProcessGroupPortDTO> {
+
+    private String testId = "testId";
+    private String testName = "testName";
+    private String testComment = "testComment";
+    private int testMaxConcurrentTasks = 111;
+    private boolean testUseCompression = false;
+
+    public RemoteInputPortSchemaTest() {
+        super(new RemoteInputPortSchemaFunction(), RemoteInputPortSchema::new);
+    }
+
+    @Before
+    public void setup() {
+        dto = new RemoteProcessGroupPortDTO();
+        dto.setId(testId);
+        dto.setName(testName);
+        dto.setComments(testComment);
+        dto.setConcurrentlySchedulableTaskCount(testMaxConcurrentTasks);
+        dto.setUseCompression(testUseCompression);
+
+        map = new HashMap<>();
+        map.put(CommonPropertyKeys.ID_KEY, testId);
+        map.put(CommonPropertyKeys.NAME_KEY, testName);
+        map.put(CommonPropertyKeys.COMMENT_KEY, testComment);
+        map.put(CommonPropertyKeys.MAX_CONCURRENT_TASKS_KEY, testMaxConcurrentTasks);
+        map.put(CommonPropertyKeys.USE_COMPRESSION_KEY, testUseCompression);
+    }
+
+    @Test
+    public void testNoId() {
+        dto.setId(null);
+        map.remove(CommonPropertyKeys.ID_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoName() {
+        dto.setName(null);
+        map.remove(CommonPropertyKeys.NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoComment() {
+        dto.setComments(null);
+        map.remove(CommonPropertyKeys.COMMENT_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoMaxConcurrentTasks() {
+        dto.setConcurrentlySchedulableTaskCount(null);
+        map.remove(CommonPropertyKeys.MAX_CONCURRENT_TASKS_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoUseCompression() {
+        dto.setUseCompression(null);
+        map.remove(CommonPropertyKeys.USE_COMPRESSION_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Override
+    public void assertSchemaEquals(RemoteInputPortSchema one, RemoteInputPortSchema two) {
+        assertEquals(one.getId(), two.getId());
+        assertEquals(one.getName(), two.getName());
+        assertEquals(one.getComment(), two.getComment());
+        assertEquals(one.getMax_concurrent_tasks(), two.getMax_concurrent_tasks());
+        assertEquals(one.getUseCompression(), two.getUseCompression());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteProcessingGroupSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteProcessingGroupSchemaTest.java b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteProcessingGroupSchemaTest.java
new file mode 100644
index 0000000..b12e16c
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/java/org/apache/nifi/minifi/toolkit/configuration/dto/RemoteProcessingGroupSchemaTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.nifi.minifi.toolkit.configuration.dto;
+
+import org.apache.nifi.minifi.commons.schema.RemoteInputPortSchema;
+import org.apache.nifi.minifi.commons.schema.RemoteProcessingGroupSchema;
+import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
+import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO;
+import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class RemoteProcessingGroupSchemaTest extends BaseSchemaTester<RemoteProcessingGroupSchema, RemoteProcessGroupDTO> {
+    private final RemoteInputPortSchemaTest remoteInputPortSchemaTest;
+    private String testName = "testName";
+    private String testUrl = "testUrl";
+    private String testComment = "testComment";
+    private String testTimeout = "11 s";
+    private String testYieldPeriod = "22 s";
+
+    public RemoteProcessingGroupSchemaTest() {
+        super(new RemoteProcessingGroupSchemaFunction(new RemoteInputPortSchemaFunction()), RemoteProcessingGroupSchema::new);
+        remoteInputPortSchemaTest = new RemoteInputPortSchemaTest();
+    }
+
+    @Before
+    public void setup() {
+        remoteInputPortSchemaTest.setup();
+
+        dto = new RemoteProcessGroupDTO();
+        dto.setName(testName);
+        dto.setTargetUri(testUrl);
+
+        RemoteProcessGroupContentsDTO contents = new RemoteProcessGroupContentsDTO();
+        contents.setInputPorts(Arrays.asList(remoteInputPortSchemaTest.dto).stream().collect(Collectors.toSet()));
+        dto.setContents(contents);
+
+        dto.setComments(testComment);
+        dto.setCommunicationsTimeout(testTimeout);
+        dto.setYieldDuration(testYieldPeriod);
+
+        map = new HashMap<>();
+
+        map.put(CommonPropertyKeys.NAME_KEY, testName);
+        map.put(RemoteProcessingGroupSchema.URL_KEY, testUrl);
+        map.put(CommonPropertyKeys.INPUT_PORTS_KEY, new ArrayList<>(Arrays.asList(remoteInputPortSchemaTest.map)));
+        map.put(CommonPropertyKeys.COMMENT_KEY, testComment);
+        map.put(RemoteProcessingGroupSchema.TIMEOUT_KEY, testTimeout);
+        map.put(CommonPropertyKeys.YIELD_PERIOD_KEY, testYieldPeriod);
+    }
+
+    @Test
+    public void testNoName() {
+        dto.setName(null);
+        map.remove(CommonPropertyKeys.NAME_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoUrl() {
+        dto.setTargetUri(null);
+        map.remove(RemoteProcessingGroupSchema.URL_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoInputPorts() {
+        dto.getContents().setInputPorts(null);
+        map.remove(CommonPropertyKeys.INPUT_PORTS_KEY);
+        assertDtoAndMapConstructorAreSame(1);
+    }
+
+    @Test
+    public void testNoComment() {
+        dto.setComments(null);
+        map.remove(CommonPropertyKeys.COMMENT_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoTimeout() {
+        dto.setCommunicationsTimeout(null);
+        map.remove(RemoteProcessingGroupSchema.TIMEOUT_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Test
+    public void testNoYield() {
+        dto.setYieldDuration(null);
+        map.remove(CommonPropertyKeys.YIELD_PERIOD_KEY);
+        assertDtoAndMapConstructorAreSame(0);
+    }
+
+    @Override
+    public void assertSchemaEquals(RemoteProcessingGroupSchema one, RemoteProcessingGroupSchema two) {
+        assertEquals(one.getName(), two.getName());
+        assertEquals(one.getUrl(), two.getUrl());
+
+        List<RemoteInputPortSchema> oneInputPorts = one.getInputPorts();
+        List<RemoteInputPortSchema> twoInputPorts = two.getInputPorts();
+        if (oneInputPorts == null) {
+            assertNull(twoInputPorts);
+        } else {
+            assertNotNull(twoInputPorts);
+            assertEquals(oneInputPorts.size(), twoInputPorts.size());
+            for (int i = 0; i < oneInputPorts.size(); i++) {
+                remoteInputPortSchemaTest.assertSchemaEquals(oneInputPorts.get(i), twoInputPorts.get(i));
+            }
+        }
+
+        assertEquals(one.getComment(), two.getComment());
+        assertEquals(one.getTimeout(), two.getTimeout());
+        assertEquals(one.getYieldPeriod(), two.getYieldPeriod());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.xml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.xml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.xml
new file mode 100644
index 0000000..4d11709
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+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.
+ -->
+<template encoding-version="1.0"><description></description><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><name>CsvToJsonWorking</name><snippet><connections><id>0cc34e0a-0ae9-44ba-838f-792ed393a301</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>cdcc2028-238b-42b7-bf95-dd4f301b91fc</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>ffb6d4e9-9d6c-4fbe-ab8a-52bad90ce2e5</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>e32e19cb-6c4c-4c6c-a8db-408447ba2
 ca2</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>5f0ec33c-0a07-4305-a3a9-9a3ba0548ac6</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>cdcc2028-238b-42b7-bf95-dd4f301b91fc</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>b061c170-19da-4e84-9709-327303fef579</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><
 destination><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>1d00089c-78cd-467f-9aa6-31e3bdf90cb0</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>8db2ebc8-12c8-4396-89b6-479d907fbc6b</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>56ef3e2e-ee35-4598-9fbe-ae86050960b0</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>8db2ebc8-12c8-4396-89b6-479d907fbc6b</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelInd
 ex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>matched</selectedRelationships><source><groupId>3d665027-7897-4322-96a1-cabef6b2bcf7</groupId><id>5f0ec33c-0a07-4305-a3a9-9a3ba0548ac6</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><labels><id>a1854f35-469c-41fa-80a2-70ae90cf1ed9</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><position><x>514.0</x><y>431.6599933547974</y></position><height>281.4172668457031</height><label></label><style/><width>388.0479431152344</width></labels><processors><id>cdcc2028-238b-42b7-bf95-dd4f301b91fc</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><position><x>569.3500686645507</x><y>239.77198108673096</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 se
 c</penaltyDuration><properties><entry><key>Regular Expression</key><value>(?s:^.*$)</value></entry><entry><key>Replacement Value</key><value>a,b,c,d</value></entry><entry><key>Character Set</key><value>UTF-8</value></entry><entry><key>Maximum Buffer Size</key><value>1 MB</value></entry><entry><key>Replacement Strategy</key></entry><entry><key>Evaluation Mode</key><value>Entire text</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>ReplaceText</name><relationships><autoTerminate>true</autoTerminate><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.ReplaceText</type></processors><processors><id>ffb6d4e9-9d6c-4fbe-ab8a-52bad90ce2e5</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><p
 osition><x>558.2500778198241</x><y>31.0</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>File Size</key><value>1 b</value></entry><entry><key>Batch Size</key><value>1</value></entry><entry><key>Data Format</key><value>Binary</value></entry><entry><key>Unique FlowFiles</key><value>false</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>10 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GenerateFlowFile</name><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.GenerateFlowFile</type></processors><processors><id>1d00089c-78cd-467f-9aa6-31e3bdf90cb0</id><parentGroupId>3d665027-7897-4322-96a1-cabef
 6b2bcf7</parentGroupId><position><x>585.2500320434569</x><y>866.0879933547974</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Delete Attributes Expression</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>UpdateAttribute</name><relationships><autoTerminate>true</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.attributes.UpdateAttribute</type></processors><processors><id>8db2ebc8-12c8-4396-89b6-479d907fbc6b</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><position><x>590.3500228881835</x><y>658.9239851760865</y></position><config><bulletinLevel>WARN</bulletinLevel>
 <comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Regular Expression</key><value>(?s:^.*$)</value></entry><entry><key>Replacement Value</key><value>{ &quot;field1&quot; : &quot;${csv.1}&quot;, &quot;field2&quot; : &quot;${csv.2}&quot;,
+                            &quot;field3&quot; : &quot;${csv.3}&quot;, &quot;field4&quot; : &quot;${csv.4}&quot; }
+                        </value></entry><entry><key>Character Set</key><value>UTF-8</value></entry><entry><key>Maximum Buffer Size</key><value>1 MB</value></entry><entry><key>Replacement Strategy</key></entry><entry><key>Evaluation Mode</key><value>Entire text</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>ReplaceText2</name><relationships><autoTerminate>true</autoTerminate><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.ReplaceText</type></processors><processors><id>5f0ec33c-0a07-4305-a3a9-9a3ba0548ac6</id><parentGroupId>3d665027-7897-4322-96a1-cabef6b2bcf7</parentGroupId><position><x>580.7500320434569</x><y>450.15196063995364</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></
 comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Character Set</key><value>UTF-8</value></entry><entry><key>Maximum Buffer Size</key><value>1 MB</value></entry><entry><key>Maximum Capture Group Length</key><value>1024</value></entry><entry><key>Enable Canonical Equivalence</key><value>false</value></entry><entry><key>Enable Case-insensitive Matching</key><value>false</value></entry><entry><key>Permit Whitespace and Comments in Pattern</key><value>false</value></entry><entry><key>Enable DOTALL Mode</key><value>false</value></entry><entry><key>Enable Literal Parsing of the Pattern</key><value>false</value></entry><entry><key>Enable Multiline Mode</key><value>false</value></entry><entry><key>Enable Unicode-aware Case Folding</key><value>false</value></entry><entry><key>Enable Unicode Predefined Character Classes</key><value>false</value></entry><entry><key>En
 able Unix Lines Mode</key><value>false</value></entry><entry><key>Include Capture Group 0</key><value>false</value></entry><entry><key>csv</key><value>(.+),(.+),(.+),(.+)</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>ExtractText</name><relationships><autoTerminate>false</autoTerminate><name>matched</name></relationships><relationships><autoTerminate>true</autoTerminate><name>unmatched</name></relationships><style/><type>org.apache.nifi.processors.standard.ExtractText</type></processors></snippet><timestamp>07/04/2016 20:20:45 UTC</timestamp></template>

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.yml
new file mode 100644
index 0000000..cee775a
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson.yml
@@ -0,0 +1,178 @@
+# 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.
+
+Flow Controller:
+  name: CsvToJsonWorking
+  comment: ''
+Core Properties:
+  flow controller graceful shutdown period: 10 sec
+  flow service write delay interval: 500 ms
+  administrative yield duration: 30 sec
+  bored yield duration: 10 millis
+  max concurrent threads: 1
+FlowFile Repository:
+  partitions: 256
+  checkpoint interval: 2 mins
+  always sync: false
+  Swap:
+    threshold: 20000
+    in period: 5 sec
+    in threads: 1
+    out period: 5 sec
+    out threads: 4
+Content Repository:
+  content claim max appendable size: 10 MB
+  content claim max flow files: 100
+  always sync: false
+Provenance Repository:
+  provenance rollover time: 1 min
+Component Status Repository:
+  buffer size: 1440
+  snapshot frequency: 1 min
+Security Properties:
+  keystore: ''
+  keystore type: ''
+  keystore password: ''
+  key password: ''
+  truststore: ''
+  truststore type: ''
+  truststore password: ''
+  ssl protocol: ''
+  Sensitive Props:
+    key:
+    algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+    provider: BC
+Processors:
+- name: ExtractText
+  class: org.apache.nifi.processors.standard.ExtractText
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 0 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - unmatched
+  Properties:
+    Character Set: UTF-8
+    Enable Canonical Equivalence: 'false'
+    Enable Case-insensitive Matching: 'false'
+    Enable DOTALL Mode: 'false'
+    Enable Literal Parsing of the Pattern: 'false'
+    Enable Multiline Mode: 'false'
+    Enable Unicode Predefined Character Classes: 'false'
+    Enable Unicode-aware Case Folding: 'false'
+    Enable Unix Lines Mode: 'false'
+    Include Capture Group 0: 'false'
+    Maximum Buffer Size: 1 MB
+    Maximum Capture Group Length: '1024'
+    Permit Whitespace and Comments in Pattern: 'false'
+    csv: (.+),(.+),(.+),(.+)
+- name: GenerateFlowFile
+  class: org.apache.nifi.processors.standard.GenerateFlowFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 10 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    Batch Size: '1'
+    Data Format: Binary
+    File Size: 1 b
+    Unique FlowFiles: 'false'
+- name: ReplaceText
+  class: org.apache.nifi.processors.standard.ReplaceText
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 0 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - failure
+  Properties:
+    Character Set: UTF-8
+    Evaluation Mode: Entire text
+    Maximum Buffer Size: 1 MB
+    Regular Expression: (?s:^.*$)
+    Replacement Strategy:
+    Replacement Value: a,b,c,d
+- name: ReplaceText2
+  class: org.apache.nifi.processors.standard.ReplaceText
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 0 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - failure
+  Properties:
+    Character Set: UTF-8
+    Evaluation Mode: Entire text
+    Maximum Buffer Size: 1 MB
+    Regular Expression: (?s:^.*$)
+    Replacement Strategy:
+    Replacement Value: "{ \"field1\" : \"${csv.1}\", \"field2\" : \"${csv.2}\",\n\
+      \                            \"field3\" : \"${csv.3}\", \"field4\" : \"${csv.4}\"\
+      \ }\n                        "
+- name: UpdateAttribute
+  class: org.apache.nifi.processors.attributes.UpdateAttribute
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 0 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list:
+  - success
+  Properties:
+    Delete Attributes Expression:
+Connections:
+- name: ExtractText/matched/ReplaceText2
+  source name: ExtractText
+  source relationship name: matched
+  destination name: ReplaceText2
+  max work queue size: 0
+  max work queue data size: 0 MB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: GenerateFlowFile/success/ReplaceText
+  source name: GenerateFlowFile
+  source relationship name: success
+  destination name: ReplaceText
+  max work queue size: 0
+  max work queue data size: 0 MB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: ReplaceText/success/ExtractText
+  source name: ReplaceText
+  source relationship name: success
+  destination name: ExtractText
+  max work queue size: 0
+  max work queue data size: 0 MB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: ReplaceText2/success/UpdateAttribute
+  source name: ReplaceText2
+  source relationship name: success
+  destination name: UpdateAttribute
+  max work queue size: 0
+  max work queue data size: 0 MB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+Remote Processing Groups: []

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/2d1e43e7/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow.xml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow.xml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow.xml
new file mode 100644
index 0000000..37d09e0
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+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.
+ -->
+<template encoding-version="1.0"><description></description><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><name>DecompressionCircularFlow2</name><snippet><connections><id>645ec08f-3d6d-4255-b4d9-0f6d61708974</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>9d33bbc9-2f51-43c8-aa8a-94dcc367b371</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>defa6d98-3e98-41dd-82ac-1bdfd39eacfc</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>bb250fb4-0b15-418f-83f9
 -83a43bd3de9c</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><bends><x>2664.3797325791907</x><y>604.6419422353362</y></bends><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>212097d4-2ea8-497b-b574-b1354fae40f2</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>0bdda776-0c7e-4723-b3ce-969b7d10a922</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>bdfdb568-682d-4d3b-b87b-99199c15505c</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataS
 izeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><bends><x>1701.5</x><y>538.5800204467773</y></bends><bends><x>1701.5</x><y>605.5800204467773</y></bends><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>defa6d98-3e98-41dd-82ac-1bdfd39eacfc</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>failure</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>defa6d98-3e98-41dd-82ac-1bdfd39eacfc</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>f6ff74cf-a30b-4916-86b3-e269266666ff</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><bends><x>1692.5</x><y>326
 .86002044677736</y></bends><bends><x>1692.5</x><y>393.86002044677736</y></bends><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>e227167f-62af-49cc-8fb5-a5b5e5251066</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>failure</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>e227167f-62af-49cc-8fb5-a5b5e5251066</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>ded9891b-4433-4f56-9fc0-1a240146835f</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>1377e3cc-db4e-4832-8a90-c8dc9e08f603</id><type>PROC
 ESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>212097d4-2ea8-497b-b574-b1354fae40f2</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>5cdc2e79-ec8c-4f7d-a3ab-d284c80e3afe</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>e227167f-62af-49cc-8fb5-a5b5e5251066</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selected
 Relationships>success</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>9e28dd28-8116-4199-aeab-bc7a1002aec5</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>42bd2967-d74a-4e22-9b16-9e0a51024658</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><bends><x>1701.5</x><y>767.7200204467774</y></bends><bends><x>1701.5</x><y>834.7200204467774</y></bends><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>9d33bbc9-2f51-43c8-aa8a-94dcc367b371</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>failure</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3<
 /groupId><id>9d33bbc9-2f51-43c8-aa8a-94dcc367b371</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>299f7d1f-e216-43a3-ad0b-7a6d6680dacc</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>9e28dd28-8116-4199-aeab-bc7a1002aec5</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>ad8ff6a3-ba60-4b9b-af9e-0773c8afc1c9</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>6eeca4aa-04c7-4774-aee5-bb6340c3a7b6</id><parentGroupId>ff2213e2-b27e
 -458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>0bdda776-0c7e-4723-b3ce-969b7d10a922</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>9d33bbc9-2f51-43c8-aa8a-94dcc367b371</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>4f98618d-2443-465d-9856-bbf7c1ee49e1</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><bends><x>1695.5</x><y>120.5</y></ben
 ds><bends><x>1695.5</x><y>187.5</y></bends><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>9e28dd28-8116-4199-aeab-bc7a1002aec5</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>failure</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>9e28dd28-8116-4199-aeab-bc7a1002aec5</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>37f15fde-d463-41b1-9fe7-28c72cd6c871</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><bends><x>2647.0997032823157</x><y>701.1219422353362</y></bends><bends><x>2652.689913243253</x><y>883.8309913076018</y></bends><destination><groupId>ff2213e2-b2
 7e-458b-ab3b-1c7a4a50d2e3</groupId><id>0bdda776-0c7e-4723-b3ce-969b7d10a922</id><type>PROCESSOR</type></destination><flowFileExpiration>5 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>failure</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>0bdda776-0c7e-4723-b3ce-969b7d10a922</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>ac822b4f-7d0d-462a-8ac0-03e9032ebefb</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>defa6d98-3e98-41dd-82ac-1bdfd39eacfc</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><p
 rioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>success</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>e227167f-62af-49cc-8fb5-a5b5e5251066</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>a72e8727-9a5c-44dd-a4f5-cf94b3422c1a</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>7209cf79-23ba-421c-b1c3-925ed86c302d</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>unmatched</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id
 >1377e3cc-db4e-4832-8a90-c8dc9e08f603</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><connections><id>5de215d5-9f7e-414b-98aa-2edaa0514d99</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><backPressureDataSizeThreshold>0 MB</backPressureDataSizeThreshold><backPressureObjectThreshold>0</backPressureObjectThreshold><bends><x>2258.299776524503</x><y>1088.9714637197112</y></bends><destination><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>0bdda776-0c7e-4723-b3ce-969b7d10a922</id><type>PROCESSOR</type></destination><flowFileExpiration>0 sec</flowFileExpiration><labelIndex>1</labelIndex><name></name><prioritizers>org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer</prioritizers><selectedRelationships>gzip</selectedRelationships><source><groupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</groupId><id>1377e3cc-db4e-4832-8a90-c8dc9e08f603</id><type>PROCESSOR</type></source><zIndex>0</zIndex></connections><processors><id>1377e3cc-db4e-4832-8a
 90-c8dc9e08f603</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>2864.4202270507812</x><y>1020.8192269897463</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Routing Strategy</key></entry><entry><key>gzip</key><value>${mime.type:toUpper():contains(&quot;GZIP&quot;)}</value></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>Compressed?</name><relationships><autoTerminate>false</autoTerminate><name>gzip</name></relationships><relationships><autoTerminate>false</autoTerminate><name>unmatched</name></relationships><style/><type>org.apache.nifi.processors.standard.RouteOnAttribute</type></processors><proce
 ssors><id>212097d4-2ea8-497b-b574-b1354fae40f2</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>2854.459897460938</x><y>731.99545032959</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties/><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>IdentifyMimeType</name><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.IdentifyMimeType</type></processors><processors><id>9e28dd28-8116-4199-aeab-bc7a1002aec5</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>1082.0</x><y>87.0</y></position><config><bulletinLevel>WARN</bulletinLe
 vel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Mode</key></entry><entry><key>Compression Format</key><value>gzip</value></entry><entry><key>Compression Level</key></entry><entry><key>Update Filename</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GZIP CompressContent</name><relationships><autoTerminate>false</autoTerminate><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.CompressContent</type></processors><processors><id>defa6d98-3e98-41dd-82ac-1bdfd39eacfc</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>1088.0</x><y>505
 .0800204467774</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Mode</key></entry><entry><key>Compression Format</key><value>gzip</value></entry><entry><key>Compression Level</key></entry><entry><key>Update Filename</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GZIP CompressContent3</name><relationships><autoTerminate>false</autoTerminate><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.CompressContent</type></processors><processors><id>0bdda776-0c7e-4723-b3ce-969b7d10a922</id><parentGroupId>ff2213e2-b27
 e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>2025.5</x><y>732.8800204467774</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Mode</key><value>decompress</value></entry><entry><key>Compression Format</key><value>gzip</value></entry><entry><key>Compression Level</key></entry><entry><key>Update Filename</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>Uncompress GZIP</name><relationships><autoTerminate>false</autoTerminate><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.CompressContent</type></process
 ors><processors><id>7209cf79-23ba-421c-b1c3-925ed86c302d</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>2858.3004760742188</x><y>1334.7543026733401</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Log Level</key></entry><entry><key>Log Payload</key></entry><entry><key>Attributes to Log</key></entry><entry><key>Attributes to Ignore</key></entry><entry><key>Log prefix</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>LogAttribute</name><relationships><autoTerminate>true</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.standard.LogAttribute</type></proce
 ssors><processors><id>e227167f-62af-49cc-8fb5-a5b5e5251066</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>1079.0</x><y>293.36002044677736</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Mode</key></entry><entry><key>Compression Format</key><value>gzip</value></entry><entry><key>Compression Level</key></entry><entry><key>Update Filename</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GZIP CompressContent2</name><relationships><autoTerminate>false</autoTerminate><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><
 type>org.apache.nifi.processors.standard.CompressContent</type></processors><processors><id>ad8ff6a3-ba60-4b9b-af9e-0773c8afc1c9</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>416.0</x><y>92.36002044677734</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>File Size</key><value>10kb</value></entry><entry><key>Batch Size</key></entry><entry><key>Data Format</key></entry><entry><key>Unique FlowFiles</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>1 min</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GenerateFlowFile</name><relationships><autoTerminate>false</autoTerminate><name>success</name></relationships><style/><type>org.apache.nifi.processors.
 standard.GenerateFlowFile</type></processors><processors><id>9d33bbc9-2f51-43c8-aa8a-94dcc367b371</id><parentGroupId>ff2213e2-b27e-458b-ab3b-1c7a4a50d2e3</parentGroupId><position><x>1088.0</x><y>734.2200204467774</y></position><config><bulletinLevel>WARN</bulletinLevel><comments></comments><concurrentlySchedulableTaskCount>1</concurrentlySchedulableTaskCount><lossTolerant>false</lossTolerant><penaltyDuration>30 sec</penaltyDuration><properties><entry><key>Mode</key></entry><entry><key>Compression Format</key><value>gzip</value></entry><entry><key>Compression Level</key></entry><entry><key>Update Filename</key></entry></properties><runDurationMillis>0</runDurationMillis><schedulingPeriod>0 sec</schedulingPeriod><schedulingStrategy>TIMER_DRIVEN</schedulingStrategy><yieldDuration>1 sec</yieldDuration></config><name>GZIP CompressContent4</name><relationships><autoTerminate>false</autoTerminate><name>failure</name></relationships><relationships><autoTerminate>false</autoTerminate><name>s
 uccess</name></relationships><style/><type>org.apache.nifi.processors.standard.CompressContent</type></processors></snippet><timestamp>07/04/2016 20:44:19 UTC</timestamp></template>