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/10/31 17:39:00 UTC

[1/3] nifi-minifi git commit: MINIFI-117 - Maintainable Configuration Versioning

Repository: nifi-minifi
Updated Branches:
  refs/heads/master 8913a9026 -> 1bbeedf6f


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/main/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMain.java
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/main/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMain.java b/minifi-toolkit/minifi-toolkit-configuration/src/main/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMain.java
index 74e1f7f..251e9a3 100644
--- a/minifi-toolkit/minifi-toolkit-configuration/src/main/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMain.java
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/main/java/org/apache/nifi/minifi/toolkit/configuration/ConfigMain.java
@@ -18,10 +18,11 @@
 package org.apache.nifi.minifi.toolkit.configuration;
 
 import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
 import org.apache.nifi.minifi.commons.schema.common.StringUtil;
+import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
 import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
 import org.apache.nifi.minifi.commons.schema.serialization.SchemaSaver;
-import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
 import org.apache.nifi.minifi.toolkit.configuration.dto.ConfigSchemaFunction;
 import org.apache.nifi.web.api.dto.ConnectableDTO;
 import org.apache.nifi.web.api.dto.ConnectionDTO;
@@ -46,6 +47,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
+import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
@@ -57,12 +59,16 @@ public class ConfigMain {
     public static final int ERR_UNABLE_TO_TRANSFORM_TEMPLATE = 5;
     public static final int ERR_UNABLE_TO_PARSE_CONFIG = 6;
     public static final int ERR_INVALID_CONFIG = 7;
+    public static final int ERR_UNABLE_TO_CLOSE_CONFIG = 8;
+    public static final int ERR_UNABLE_TO_SAVE_CONFIG = 9;
 
     public static final int SUCCESS = 0;
 
     public static final String TRANSFORM = "transform";
     public static final String VALIDATE = "validate";
-    public static final String NO_VALIDATION_ERRORS_FOUND_IN_TEMPLATE = "No validation errors found in template.";
+    public static final String UPGRADE = "upgrade";
+    public static final String THERE_ARE_VALIDATION_ERRORS_WITH_THE_TEMPLATE_STILL_OUTPUTTING_YAML_BUT_IT_WILL_NEED_TO_BE_EDITED =
+            "There are validation errors with the template, still outputting YAML but it will need to be edited.";
 
     private final Map<String, Command> commandMap;
     private final PathInputStreamFactory pathInputStreamFactory;
@@ -96,31 +102,22 @@ public class ConfigMain {
         }
         try (InputStream inputStream = pathInputStreamFactory.create(args[1])) {
             try {
-                ConfigSchema configSchema = SchemaLoader.loadConfigSchemaFromYaml(inputStream);
-                if (!configSchema.isValid()) {
-                    configSchema.getValidationIssues().forEach(s -> System.out.println(s));
-                    System.out.println();
-                    return ERR_INVALID_CONFIG;
-                } else {
-                    System.out.println(NO_VALIDATION_ERRORS_FOUND_IN_TEMPLATE);
-                }
+                return loadAndPrintValidationErrors(inputStream, (configSchema, valid) -> {
+                    if (valid) {
+                        return SUCCESS;
+                    } else {
+                        return ERR_INVALID_CONFIG;
+                    }
+                });
             } catch (IOException|SchemaLoaderException e) {
-                System.out.println("Unable to load configuration. (" + e + ")");
-                System.out.println();
-                printValidateUsage();
-                return ERR_UNABLE_TO_PARSE_CONFIG;
+                return handleErrorLoadingConfiguration(e, ConfigMain::printValidateUsage);
             }
         } catch (FileNotFoundException e) {
-            System.out.println("Unable to open file " + args[1] + " for reading. (" + e + ")");
-            System.out.println();
-            printValidateUsage();
-            return ERR_UNABLE_TO_OPEN_INPUT;
+            return handleErrorOpeningInput(args[1], ConfigMain::printValidateUsage, e);
         } catch (IOException e) {
-            System.out.println("Error closing input. (" + e + ")");
-            System.out.println();
+            handleErrorClosingInput(e);
+            return ERR_UNABLE_TO_CLOSE_CONFIG;
         }
-
-        return SUCCESS;
     }
 
     public static void printTransformUsage() {
@@ -130,6 +127,13 @@ public class ConfigMain {
         System.out.println();
     }
 
+    public static void printUpgradeUsage() {
+        System.out.println("Upgrade Usage:");
+        System.out.println();
+        System.out.println(" upgrade INPUT_FILE OUTPUT_FILE");
+        System.out.println();
+    }
+
     private static void enrichTemplateDTO(TemplateDTO templateDTO) {
         FlowSnippetDTO flowSnippetDTO = templateDTO.getSnippet();
 
@@ -182,32 +186,28 @@ public class ConfigMain {
         }
     }
 
-    public static ConfigSchema transformTemplateToSchema(InputStream source) throws JAXBException, IOException, SchemaLoaderException {
-        try {
-            TemplateDTO templateDTO = (TemplateDTO) JAXBContext.newInstance(TemplateDTO.class).createUnmarshaller().unmarshal(source);
+    public static ConfigSchema transformTemplateToSchema(InputStream source) throws JAXBException, SchemaLoaderException {
+        TemplateDTO templateDTO = (TemplateDTO) JAXBContext.newInstance(TemplateDTO.class).createUnmarshaller().unmarshal(source);
 
-            if (templateDTO.getSnippet().getProcessGroups().size() != 0){
-                throw new SchemaLoaderException("Process Groups are not currently supported in MiNiFi. Please remove any from the template and try again.");
-            }
-
-            if (templateDTO.getSnippet().getOutputPorts().size() != 0){
-                throw new SchemaLoaderException("Output Ports are not currently supported in MiNiFi. Please remove any from the template and try again.");
-            }
+        if (templateDTO.getSnippet().getProcessGroups().size() != 0){
+            throw new SchemaLoaderException("Process Groups are not currently supported in MiNiFi. Please remove any from the template and try again.");
+        }
 
-            if (templateDTO.getSnippet().getInputPorts().size() != 0){
-                throw new SchemaLoaderException("Input Ports are not currently supported in MiNiFi. Please remove any from the template and try again.");
-            }
+        if (templateDTO.getSnippet().getOutputPorts().size() != 0){
+            throw new SchemaLoaderException("Output Ports are not currently supported in MiNiFi. Please remove any from the template and try again.");
+        }
 
-            if (templateDTO.getSnippet().getFunnels().size() != 0){
-                throw new SchemaLoaderException("Funnels are not currently supported in MiNiFi. Please remove any from the template and try again.");
-            }
+        if (templateDTO.getSnippet().getInputPorts().size() != 0){
+            throw new SchemaLoaderException("Input Ports are not currently supported in MiNiFi. Please remove any from the template and try again.");
+        }
 
-            enrichTemplateDTO(templateDTO);
-            ConfigSchema configSchema = new ConfigSchemaFunction().apply(templateDTO);
-            return configSchema;
-        } finally {
-            source.close();
+        if (templateDTO.getSnippet().getFunnels().size() != 0){
+            throw new SchemaLoaderException("Funnels are not currently supported in MiNiFi. Please remove any from the template and try again.");
         }
+
+        enrichTemplateDTO(templateDTO);
+        ConfigSchema configSchema = new ConfigSchemaFunction().apply(templateDTO);
+        return configSchema;
     }
 
     private static void setName(Map<String, String> connectableNameMap, ConnectableDTO connectableDTO) {
@@ -235,62 +235,158 @@ public class ConfigMain {
         }
     }
 
-    public int transform(String[] args) {
+    public int upgrade(String[] args) {
         if (args.length != 3) {
-            printTransformUsage();
+            printUpgradeUsage();
             return ERR_INVALID_ARGS;
         }
+
+        ConfigSchema currentSchema = null;
         try (InputStream inputStream = pathInputStreamFactory.create(args[1])) {
-            try (OutputStream fileOutputStream = pathOutputStreamFactory.create(args[2])) {
-                try {
-                    ConfigSchema configSchema = transformTemplateToSchema(inputStream);
-                    if (!configSchema.isValid()) {
-                        System.out.println("There are validation errors with the template, still outputting YAML but it will need to be edited.");
-                        for (String s : configSchema.getValidationIssues()) {
-                            System.out.println(s);
-                        }
+            try {
+                currentSchema = loadAndPrintValidationErrors(inputStream, (configSchema, valid) -> {
+                    if (!valid) {
+                        System.out.println(THERE_ARE_VALIDATION_ERRORS_WITH_THE_TEMPLATE_STILL_OUTPUTTING_YAML_BUT_IT_WILL_NEED_TO_BE_EDITED);
                         System.out.println();
-                    } else {
-                        System.out.println(NO_VALIDATION_ERRORS_FOUND_IN_TEMPLATE);
                     }
-                    SchemaSaver.saveConfigSchema(configSchema, fileOutputStream);
-                } catch (JAXBException e) {
-                    System.out.println("Error reading template. (" + e + ")");
-                    System.out.println();
-                    printTransformUsage();
-                    return ERR_UNABLE_TO_READ_TEMPLATE;
-                } catch (IOException e) {
-                    System.out.println("Error transforming template to YAML. (" + e + ")");
-                    System.out.println();
-                    printTransformUsage();
-                    return ERR_UNABLE_TO_TRANSFORM_TEMPLATE;
-                } catch (SchemaLoaderException e) {
-                    System.out.println("Error transforming template to YAML. (" + e.getMessage() + ")");
+                    return configSchema;
+                });
+            } catch (IOException|SchemaLoaderException e) {
+                return handleErrorLoadingConfiguration(e, ConfigMain::printUpgradeUsage);
+            }
+        } catch (FileNotFoundException e) {
+            return handleErrorOpeningInput(args[1], ConfigMain::printUpgradeUsage, e);
+        } catch (IOException e) {
+            handleErrorClosingInput(e);
+        }
+
+        try (OutputStream fileOutputStream = pathOutputStreamFactory.create(args[2])) {
+            try {
+                SchemaSaver.saveConfigSchema(currentSchema, fileOutputStream);
+            } catch (IOException e) {
+                return handleErrorSavingCofiguration(e);
+            }
+        } catch (FileNotFoundException e) {
+            return handleErrorOpeningOutput(args[2], ConfigMain::printUpgradeUsage, e);
+        } catch (IOException e) {
+            handleErrorClosingOutput(e);
+        }
+
+        return SUCCESS;
+    }
+
+    public <T> T loadAndPrintValidationErrors(InputStream inputStream, BiFunction<ConfigSchema, Boolean, T> resultHandler) throws IOException, SchemaLoaderException {
+        ConvertableSchema<ConfigSchema> configSchema = SchemaLoader.loadConvertableSchemaFromYaml(inputStream);
+        boolean valid = true;
+        if (!configSchema.isValid()) {
+            System.out.println("Found the following errors when parsing the configuration according to its version. (" + configSchema.getVersion() + ")");
+            configSchema.getValidationIssues().forEach(s -> System.out.println(s));
+            System.out.println();
+            valid = false;
+            configSchema.clearValidationIssues();
+        } else {
+            System.out.println("No errors found when parsing configuration according to its version. (" + configSchema.getVersion() + ")");
+        }
+
+        ConfigSchema currentSchema = configSchema.convert();
+        if (!currentSchema.isValid()) {
+            System.out.println("Found the following errors when converting configuration to latest version. (" + ConfigSchema.CONFIG_VERSION + ")");
+            currentSchema.getValidationIssues().forEach(s -> System.out.println(s));
+            System.out.println();
+            valid = false;
+        } else if (configSchema.getVersion() == currentSchema.getVersion()) {
+            System.out.println("Configuration was already latest version (" + ConfigSchema.CONFIG_VERSION + ") so no conversion was needed.");
+        } else {
+            System.out.println("No errors found when converting configuration to latest version. (" + ConfigSchema.CONFIG_VERSION + ")");
+        }
+        return resultHandler.apply(currentSchema, valid);
+    }
+
+    public int transform(String[] args) {
+        if (args.length != 3) {
+            printTransformUsage();
+            return ERR_INVALID_ARGS;
+        }
+
+        ConfigSchema configSchema = null;
+        try (InputStream inputStream = pathInputStreamFactory.create(args[1])) {
+            try {
+                configSchema = transformTemplateToSchema(inputStream);
+                if (!configSchema.isValid()) {
+                    System.out.println(THERE_ARE_VALIDATION_ERRORS_WITH_THE_TEMPLATE_STILL_OUTPUTTING_YAML_BUT_IT_WILL_NEED_TO_BE_EDITED);
+                    configSchema.getValidationIssues().forEach(System.out::println);
                     System.out.println();
-                    return ERR_UNABLE_TO_TRANSFORM_TEMPLATE;
+                } else {
+                    System.out.println("No validation errors found in converted configuration.");
                 }
-            } catch (FileNotFoundException e) {
-                System.out.println("Unable to open file " + args[2] + " for writing. (" + e + ")");
+            } catch (JAXBException e) {
+                System.out.println("Error reading template. (" + e + ")");
                 System.out.println();
                 printTransformUsage();
-                return ERR_UNABLE_TO_OPEN_OUTPUT;
-            } catch (IOException e) {
-                System.out.println("Error closing output. (" + e + ")");
+                return ERR_UNABLE_TO_READ_TEMPLATE;
+            } catch (SchemaLoaderException e) {
+                System.out.println("Error transforming template to YAML. (" + e.getMessage() + ")");
                 System.out.println();
+                return ERR_UNABLE_TO_TRANSFORM_TEMPLATE;
             }
         } catch (FileNotFoundException e) {
-            System.out.println("Unable to open file " + args[1] + " for reading. (" + e + ")");
-            System.out.println();
-            printTransformUsage();
-            return ERR_UNABLE_TO_OPEN_INPUT;
+            return handleErrorOpeningInput(args[1], ConfigMain::printTransformUsage, e);
         } catch (IOException e) {
-            System.out.println("Error closing input. (" + e + ")");
-            System.out.println();
+            handleErrorClosingInput(e);
+        }
+
+        try (OutputStream fileOutputStream = pathOutputStreamFactory.create(args[2])) {
+            try {
+                SchemaSaver.saveConfigSchema(configSchema, fileOutputStream);
+            } catch (IOException e) {
+                return handleErrorSavingCofiguration(e);
+            }
+        } catch (FileNotFoundException e) {
+            return handleErrorOpeningOutput(args[2], ConfigMain::printTransformUsage, e);
+        } catch (IOException e) {
+            handleErrorClosingOutput(e);
         }
 
         return SUCCESS;
     }
 
+    protected void handleErrorClosingOutput(IOException e) {
+        System.out.println("Error closing output. (" + e + ")");
+        System.out.println();
+    }
+
+    protected void handleErrorClosingInput(IOException e) {
+        System.out.println("Error closing input. (" + e + ")");
+        System.out.println();
+    }
+
+    protected int handleErrorOpeningInput(String fileName, Runnable usagePrinter, FileNotFoundException e) {
+        System.out.println("Unable to open file " + fileName + " for reading. (" + e + ")");
+        System.out.println();
+        usagePrinter.run();
+        return ERR_UNABLE_TO_OPEN_INPUT;
+    }
+
+    protected int handleErrorOpeningOutput(String fileName, Runnable usagePrinter, FileNotFoundException e) {
+        System.out.println("Unable to open file " + fileName + " for writing. (" + e + ")");
+        System.out.println();
+        usagePrinter.run();
+        return ERR_UNABLE_TO_OPEN_OUTPUT;
+    }
+
+    protected int handleErrorLoadingConfiguration(Exception e, Runnable usagePrinter) {
+        System.out.println("Unable to load configuration. (" + e + ")");
+        System.out.println();
+        usagePrinter.run();
+        return ERR_UNABLE_TO_PARSE_CONFIG;
+    }
+
+    protected int handleErrorSavingCofiguration(IOException e) {
+        System.out.println("Unable to save configuration: " + e);
+        System.out.println();
+        return ERR_UNABLE_TO_SAVE_CONFIG;
+    }
+
     public int execute(String[] args) {
         if (args.length < 1 || !commandMap.containsKey(args[0].toLowerCase())) {
             printUsage();
@@ -303,6 +399,7 @@ public class ConfigMain {
         Map<String, Command> result = new TreeMap<>();
         result.put(TRANSFORM, new Command(this::transform, "Transform template xml into MiNiFi config YAML"));
         result.put(VALIDATE, new Command(this::validate, "Validate config YAML"));
+        result.put(UPGRADE, new Command(this::upgrade, "Upgrade config YAML to current version (" + ConfigSchema.CONFIG_VERSION + ")"));
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/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
index 5ca4933..cfdc48f 100644
--- 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
@@ -18,6 +18,11 @@
 package org.apache.nifi.minifi.toolkit.configuration;
 
 import org.apache.commons.io.Charsets;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
+import org.apache.nifi.minifi.commons.schema.ProcessorSchema;
+import org.apache.nifi.minifi.commons.schema.RemoteInputPortSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
 import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
 import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
 import org.junit.Before;
@@ -31,14 +36,18 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.nifi.minifi.toolkit.configuration.ConfigMain.SUCCESS;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.when;
 
@@ -106,7 +115,14 @@ public class ConfigMainTest {
     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}));
+        assertEquals(SUCCESS, configMain.execute(new String[]{ConfigMain.VALIDATE, testInput}));
+    }
+
+    @Test
+    public void testValidateV1Success() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenAnswer(invocation ->
+                ConfigMainTest.class.getClassLoader().getResourceAsStream("config-v1.yml"));
+        assertEquals(SUCCESS, configMain.execute(new String[]{ConfigMain.VALIDATE, testInput}));
     }
 
     @Test
@@ -117,6 +133,8 @@ public class ConfigMainTest {
 
     @Test
     public void testTransformErrorOpeningOutput() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenAnswer(invocation ->
+                ConfigMainTest.class.getClassLoader().getResourceAsStream("CsvToJson.xml"));
         when(pathOutputStreamFactory.create(testOutput)).thenThrow(new FileNotFoundException());
         assertEquals(ConfigMain.ERR_UNABLE_TO_OPEN_OUTPUT, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
     }
@@ -130,13 +148,7 @@ public class ConfigMainTest {
     @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();
-            }
-        });
+                ConfigMainTest.class.getClassLoader().getResourceAsStream("TemplateWithInputPort.xml"));
         assertEquals(ConfigMain.ERR_UNABLE_TO_TRANSFORM_TEMPLATE, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
     }
 
@@ -145,7 +157,7 @@ public class ConfigMainTest {
         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}));
+        assertEquals(SUCCESS, configMain.execute(new String[]{ConfigMain.TRANSFORM, testInput, testOutput}));
     }
 
     @Test
@@ -198,10 +210,106 @@ public class ConfigMainTest {
         ConfigMain.transformTemplateToSchema(getClass().getClassLoader().getResourceAsStream("TemplateWithFunnel.xml")).toMap();
     }
 
+    @Test
+    public void testUpgradeInputFileNotFoundException() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenThrow(new FileNotFoundException());
+        assertEquals(ConfigMain.ERR_UNABLE_TO_OPEN_INPUT, configMain.execute(new String[]{ConfigMain.UPGRADE, testInput, testOutput}));
+    }
+
+    @Test
+    public void testUpgradeCantLoadSchema() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenReturn(new InputStream() {
+            @Override
+            public int read() throws IOException {
+                throw new IOException();
+            }
+        });
+        assertEquals(ConfigMain.ERR_UNABLE_TO_PARSE_CONFIG, configMain.execute(new String[]{ConfigMain.UPGRADE, testInput, testOutput}));
+    }
+
+    @Test
+    public void testUpgradeOutputFileNotFoundException() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenReturn(getClass().getClassLoader().getResourceAsStream("CsvToJson-v1.yml"));
+        when(pathOutputStreamFactory.create(testOutput)).thenThrow(new FileNotFoundException());
+        assertEquals(ConfigMain.ERR_UNABLE_TO_OPEN_OUTPUT, configMain.execute(new String[]{ConfigMain.UPGRADE, testInput, testOutput}));
+    }
+
+    @Test
+    public void testUpgradeCantSaveSchema() throws FileNotFoundException {
+        when(pathInputStreamFactory.create(testInput)).thenReturn(getClass().getClassLoader().getResourceAsStream("CsvToJson-v1.yml"));
+        when(pathOutputStreamFactory.create(testOutput)).thenReturn(new OutputStream() {
+            @Override
+            public void write(int b) throws IOException {
+                throw new IOException();
+            }
+        });
+        assertEquals(ConfigMain.ERR_UNABLE_TO_SAVE_CONFIG, configMain.execute(new String[]{ConfigMain.UPGRADE, testInput, testOutput}));
+    }
+
+    @Test
+    public void testUpgradeInvalidArgs() {
+        assertEquals(ConfigMain.ERR_INVALID_ARGS, configMain.execute(new String[]{ConfigMain.UPGRADE}));
+    }
+
     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);
+        testV1YmlIfPresent(name, yamlMap);
+    }
+
+    private InputStream upgradeAndReturn(String name) throws FileNotFoundException {
+        InputStream yamlV1Stream = getClass().getClassLoader().getResourceAsStream(name + "-v1.yml");
+        if (yamlV1Stream == null) {
+            return null;
+        }
+        when(pathInputStreamFactory.create(testInput)).thenReturn(yamlV1Stream);
+        ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+        when(pathOutputStreamFactory.create(testOutput)).thenReturn(outputStream);
+        assertEquals(SUCCESS, configMain.execute(new String[]{"upgrade", testInput, testOutput}));
+        return new ByteArrayInputStream(outputStream.toByteArray());
+    }
+
+    private void testV1YmlIfPresent(String name, Map<String, Object> yamlMap) throws IOException, SchemaLoaderException {
+        InputStream upgradedInputStream = upgradeAndReturn(name);
+        if (upgradedInputStream != null) {
+            ConvertableSchema<ConfigSchema> configSchemaConvertableSchema = SchemaLoader.loadConvertableSchemaFromYaml(upgradedInputStream);
+            ConfigSchema configSchemaUpgradedFromV1 = configSchemaConvertableSchema.convert();
+            assertTrue(configSchemaUpgradedFromV1.isValid());
+            assertEquals(configSchemaConvertableSchema, configSchemaUpgradedFromV1);
+            ConfigSchema configSchemaFromCurrent = new ConfigSchema(yamlMap);
+            List<ProcessorSchema> currentProcessors = configSchemaFromCurrent.getProcessors();
+            List<ProcessorSchema> v1Processors = configSchemaUpgradedFromV1.getProcessors();
+            assertEquals(currentProcessors.size(), v1Processors.size());
+
+            // V1 doesn't have ids so we need to map the autogenerated ones to the ones from the template
+            Map<String, String> v1IdToCurrentIdMap = new HashMap<>();
+            for (int i = 0; i < currentProcessors.size(); i++) {
+                ProcessorSchema currentProcessor = currentProcessors.get(i);
+                ProcessorSchema v1Processor = v1Processors.get(i);
+                assertEquals(currentProcessor.getName(), v1Processor.getName());
+                v1IdToCurrentIdMap.put(v1Processor.getId(), currentProcessor.getId());
+                v1Processor.setId(currentProcessor.getId());
+            }
+            configSchemaUpgradedFromV1.getRemoteProcessingGroups().stream().flatMap(g -> g.getInputPorts().stream()).map(RemoteInputPortSchema::getId).sequential()
+                    .forEach(id -> v1IdToCurrentIdMap.put(id, id));
+
+            List<ConnectionSchema> currentConnections = configSchemaFromCurrent.getConnections();
+            List<ConnectionSchema> v1Connections = configSchemaUpgradedFromV1.getConnections();
+
+            // Update source and dest ids, can set connection id equal because it isn't referenced elsewhere
+            assertEquals(currentConnections.size(), v1Connections.size());
+            for (int i = 0; i < currentConnections.size(); i++) {
+                ConnectionSchema currentConnection = currentConnections.get(i);
+                ConnectionSchema v1Connection = v1Connections.get(i);
+                assertEquals(currentConnection.getName(), v1Connection.getName());
+                v1Connection.setId(currentConnection.getId());
+                v1Connection.setSourceId(v1IdToCurrentIdMap.get(v1Connection.getSourceId()));
+                v1Connection.setDestinationId(v1IdToCurrentIdMap.get(v1Connection.getDestinationId()));
+            }
+            Map<String, Object> v1YamlMap = configSchemaUpgradedFromV1.toMap();
+            assertNoMapDifferences(v1YamlMap, configSchemaFromCurrent.toMap());
+        }
     }
 
     private void assertNoMapDifferences(Map<String, Object> templateMap, Map<String, Object> yamlMap) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/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
index a8c1908..6ad7e03 100644
--- 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
@@ -17,7 +17,7 @@
 
 package org.apache.nifi.minifi.toolkit.configuration.dto;
 
-import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 import org.junit.Test;
 
 import java.util.Map;
@@ -25,7 +25,7 @@ import java.util.function.Function;
 
 import static org.junit.Assert.assertEquals;
 
-public abstract class BaseSchemaTester<Schema extends BaseSchema, DTO> {
+public abstract class BaseSchemaTester<Schema extends WritableSchema, DTO> {
     protected final Function<DTO, Schema> dtoSchemaFunction;
     protected final Function<Map, Schema> mapSchemaFunction;
     protected DTO dto;

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/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
index b4efa76..6aa77fb 100644
--- 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
@@ -98,7 +98,7 @@ public class ConnectionSchemaTest extends BaseSchemaTester<ConnectionSchema, Con
     public void testNoSourceId() {
         dto.setSource(new ConnectableDTO());
         map.remove(ConnectionSchema.SOURCE_ID_KEY);
-        assertDtoAndMapConstructorAreSame(2);
+        assertDtoAndMapConstructorAreSame(1);
     }
 
     @Test
@@ -123,7 +123,7 @@ public class ConnectionSchemaTest extends BaseSchemaTester<ConnectionSchema, Con
     public void testNoDestinationName() {
         dto.setDestination(new ConnectableDTO());
         map.remove(ConnectionSchema.DESTINATION_ID_KEY);
-        assertDtoAndMapConstructorAreSame(2);
+        assertDtoAndMapConstructorAreSame(1);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson-v1.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson-v1.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson-v1.yml
new file mode 100644
index 0000000..cee775a
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/CsvToJson-v1.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/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow-v1.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow-v1.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow-v1.yml
new file mode 100644
index 0000000..743fdf6
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/DecompressionCircularFlow-v1.yml
@@ -0,0 +1,293 @@
+# 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: DecompressionCircularFlow2
+  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: Compressed?
+  class: org.apache.nifi.processors.standard.RouteOnAttribute
+  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: []
+  Properties:
+    Routing Strategy:
+    gzip: ${mime.type:toUpper():contains("GZIP")}
+- name: GZIP CompressContent
+  class: org.apache.nifi.processors.standard.CompressContent
+  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: []
+  Properties:
+    Compression Format: gzip
+    Compression Level:
+    Mode:
+    Update Filename:
+- name: GZIP CompressContent2
+  class: org.apache.nifi.processors.standard.CompressContent
+  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: []
+  Properties:
+    Compression Format: gzip
+    Compression Level:
+    Mode:
+    Update Filename:
+- name: GZIP CompressContent3
+  class: org.apache.nifi.processors.standard.CompressContent
+  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: []
+  Properties:
+    Compression Format: gzip
+    Compression Level:
+    Mode:
+    Update Filename:
+- name: GZIP CompressContent4
+  class: org.apache.nifi.processors.standard.CompressContent
+  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: []
+  Properties:
+    Compression Format: gzip
+    Compression Level:
+    Mode:
+    Update Filename:
+- name: GenerateFlowFile
+  class: org.apache.nifi.processors.standard.GenerateFlowFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1 min
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    Batch Size:
+    Data Format:
+    File Size: 10kb
+    Unique FlowFiles:
+- name: IdentifyMimeType
+  class: org.apache.nifi.processors.standard.IdentifyMimeType
+  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: []
+  Properties: {}
+- name: LogAttribute
+  class: org.apache.nifi.processors.standard.LogAttribute
+  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:
+    Attributes to Ignore:
+    Attributes to Log:
+    Log Level:
+    Log Payload:
+    Log prefix:
+- name: Uncompress GZIP
+  class: org.apache.nifi.processors.standard.CompressContent
+  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: []
+  Properties:
+    Compression Format: gzip
+    Compression Level:
+    Mode: decompress
+    Update Filename:
+Connections:
+- name: Compressed?/gzip/Uncompress GZIP
+  source name: Compressed?
+  source relationship name: gzip
+  destination name: Uncompress GZIP
+  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: Compressed?/unmatched/LogAttribute
+  source name: Compressed?
+  source relationship name: unmatched
+  destination name: LogAttribute
+  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: GZIP CompressContent/failure/GZIP CompressContent
+  source name: GZIP CompressContent
+  source relationship name: failure
+  destination name: GZIP CompressContent
+  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: GZIP CompressContent/success/GZIP CompressContent2
+  source name: GZIP CompressContent
+  source relationship name: success
+  destination name: GZIP CompressContent2
+  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: GZIP CompressContent2/failure/GZIP CompressContent2
+  source name: GZIP CompressContent2
+  source relationship name: failure
+  destination name: GZIP CompressContent2
+  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: GZIP CompressContent2/success/GZIP CompressContent3
+  source name: GZIP CompressContent2
+  source relationship name: success
+  destination name: GZIP CompressContent3
+  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: GZIP CompressContent3/failure/GZIP CompressContent3
+  source name: GZIP CompressContent3
+  source relationship name: failure
+  destination name: GZIP CompressContent3
+  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: GZIP CompressContent3/success/GZIP CompressContent4
+  source name: GZIP CompressContent3
+  source relationship name: success
+  destination name: GZIP CompressContent4
+  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: GZIP CompressContent4/failure/GZIP CompressContent4
+  source name: GZIP CompressContent4
+  source relationship name: failure
+  destination name: GZIP CompressContent4
+  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: GZIP CompressContent4/success/Uncompress GZIP
+  source name: GZIP CompressContent4
+  source relationship name: success
+  destination name: Uncompress GZIP
+  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/GZIP CompressContent
+  source name: GenerateFlowFile
+  source relationship name: success
+  destination name: GZIP CompressContent
+  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: IdentifyMimeType/success/Compressed?
+  source name: IdentifyMimeType
+  source relationship name: success
+  destination name: Compressed?
+  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: Uncompress GZIP/failure/Uncompress GZIP
+  source name: Uncompress GZIP
+  source relationship name: failure
+  destination name: Uncompress GZIP
+  max work queue size: 0
+  max work queue data size: 0 MB
+  flowfile expiration: 5 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Uncompress GZIP/success/IdentifyMimeType
+  source name: Uncompress GZIP
+  source relationship name: success
+  destination name: IdentifyMimeType
+  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/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/InvokeHttpMiNiFiTemplateTest-v1.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/InvokeHttpMiNiFiTemplateTest-v1.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/InvokeHttpMiNiFiTemplateTest-v1.yml
new file mode 100644
index 0000000..6a07ae4
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/InvokeHttpMiNiFiTemplateTest-v1.yml
@@ -0,0 +1,262 @@
+# 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: InvokeHttpMiNiFiTemplateTest2
+  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: LogAttribute
+  class: org.apache.nifi.processors.standard.LogAttribute
+  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:
+    Attributes to Ignore:
+    Attributes to Log:
+    Log Level: info
+    Log Payload: 'false'
+    Log prefix:
+- name: LogAttribute2
+  class: org.apache.nifi.processors.standard.LogAttribute
+  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:
+    Attributes to Ignore:
+    Attributes to Log:
+    Log Level: info
+    Log Payload: 'false'
+    Log prefix:
+- name: Route On Status Code
+  class: org.apache.nifi.processors.standard.RouteOnAttribute
+  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: []
+  Properties:
+    '200': ${invokehttp.status.code:equals(200)}
+    Routing Strategy: Route to Property name
+- name: Search Google
+  class: org.apache.nifi.processors.standard.InvokeHTTP
+  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: []
+  Properties:
+    Add Response Headers to Request: 'false'
+    Always Output Response: 'false'
+    Attributes to Send:
+    Basic Authentication Password:
+    Basic Authentication Username:
+    Connection Timeout: 5 secs
+    Content-Type: ${mime.type}
+    Digest Authentication: 'false'
+    Follow Redirects: 'True'
+    HTTP Method: GET
+    Include Date Header: 'True'
+    Max Length To Put In Attribute: '256'
+    Penalize on "No Retry": 'false'
+    Proxy Host:
+    Proxy Port:
+    Put Response Body In Attribute:
+    Read Timeout: 15 secs
+    Remote URL: http://www.google.com/search?q=${q}&rct=j
+    SSL Context Service:
+    Trusted Hostname:
+    Use Chunked Encoding: 'false'
+    invokehttp-proxy-password:
+    invokehttp-proxy-user:
+    send-message-body: 'true'
+- name: Source
+  class: org.apache.nifi.processors.standard.GenerateFlowFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 30 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: 10 b
+    Unique FlowFiles: 'false'
+- name: q=nifi
+  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: []
+  Properties:
+    Delete Attributes Expression:
+    q: nifi
+Connections:
+- name: Route On Status Code/200/LogAttribute
+  source name: Route On Status Code
+  source relationship name: '200'
+  destination name: LogAttribute
+  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: Route On Status Code/200/b23a4621-cf19-42e6-967c-ffd3716e6a24
+  source name: Route On Status Code
+  source relationship name: '200'
+  destination name: b23a4621-cf19-42e6-967c-ffd3716e6a24
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Route On Status Code/unmatched/LogAttribute2
+  source name: Route On Status Code
+  source relationship name: unmatched
+  destination name: LogAttribute2
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Route On Status Code/unmatched/b23a4621-cf19-42e6-967c-ffd3716e6a24
+  source name: Route On Status Code
+  source relationship name: unmatched
+  destination name: b23a4621-cf19-42e6-967c-ffd3716e6a24
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Search Google/Failure/Route On Status Code
+  source name: Search Google
+  source relationship name: Failure
+  destination name: Route On Status Code
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Search Google/No Retry/Route On Status Code
+  source name: Search Google
+  source relationship name: No Retry
+  destination name: Route On Status Code
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Search Google/Original/Route On Status Code
+  source name: Search Google
+  source relationship name: Original
+  destination name: Route On Status Code
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Search Google/Response/Route On Status Code
+  source name: Search Google
+  source relationship name: Response
+  destination name: Route On Status Code
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Search Google/Retry/Route On Status Code
+  source name: Search Google
+  source relationship name: Retry
+  destination name: Route On Status Code
+  max work queue size: 10000
+  max work queue data size: 1 GB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: Source/success/q=nifi
+  source name: Source
+  source relationship name: success
+  destination name: q=nifi
+  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: q=nifi/success/Search Google
+  source name: q=nifi
+  source relationship name: success
+  destination name: Search Google
+  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:
+- name: http://nifi:8080/nifi
+  url: http://nifi:8080/nifi
+  comment: ''
+  timeout: 30 sec
+  yield period: 10 sec
+  Input Ports:
+  - id: b23a4621-cf19-42e6-967c-ffd3716e6a24
+    name: response
+    comment: ''
+    max concurrent tasks: 1
+    use compression: false

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/ReplaceTextExpressionLanguageCSVReformatting-v1.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/ReplaceTextExpressionLanguageCSVReformatting-v1.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/ReplaceTextExpressionLanguageCSVReformatting-v1.yml
new file mode 100644
index 0000000..d0f1d66
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/ReplaceTextExpressionLanguageCSVReformatting-v1.yml
@@ -0,0 +1,146 @@
+# 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: ReplaceTextExpressionLanguageCSVReformatting
+  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: Generate Empty File
+  class: org.apache.nifi.processors.standard.GenerateFlowFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 3 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    Batch Size:
+    Data Format:
+    File Size: 0b
+    Unique FlowFiles:
+- name: No-Op Termination
+  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:
+- name: Reformat Date Column
+  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:
+    Evaluation Mode: Line-by-Line
+    Maximum Buffer Size:
+    Regular Expression: (.*?),(.*?),(\d+.*)
+    Replacement Strategy:
+    Replacement Value: $1,$2,${ '$3':toDate('ddMMMyyyy'):format('yyyy/MM/dd') }
+- name: Set CSV Content
+  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:
+    Evaluation Mode:
+    Maximum Buffer Size:
+    Regular Expression:
+    Replacement Strategy:
+    Replacement Value: |-
+      2006,10-01-2004,10may2004
+      2007,15-05-2006,10jun2005
+      2009,8-8-2008,10aug2008
+Connections:
+- name: Generate Empty File/success/Set CSV Content
+  source name: Generate Empty File
+  source relationship name: success
+  destination name: Set CSV Content
+  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: Reformat Date Column/success/No-Op Termination
+  source name: Reformat Date Column
+  source relationship name: success
+  destination name: No-Op Termination
+  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: Set CSV Content/success/Reformat Date Column
+  source name: Set CSV Content
+  source relationship name: success
+  destination name: Reformat Date Column
+  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/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/StressTestFramework-v1.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/StressTestFramework-v1.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/StressTestFramework-v1.yml
new file mode 100644
index 0000000..970552b
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/StressTestFramework-v1.yml
@@ -0,0 +1,114 @@
+# 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: StressTestFramework
+  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: GenerateFlowFile
+  class: org.apache.nifi.processors.standard.GenerateFlowFile
+  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: []
+  Properties:
+    Batch Size: '100'
+    Data Format:
+    File Size: 0 KB
+    Unique FlowFiles:
+- name: RouteOnAttribute
+  class: org.apache.nifi.processors.standard.RouteOnAttribute
+  max concurrent tasks: 2
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 0 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 50000
+  auto-terminated relationships list:
+  - unmatched
+  Properties:
+    Routing Strategy:
+- name: UpdateAttribute
+  class: org.apache.nifi.processors.attributes.UpdateAttribute
+  max concurrent tasks: 2
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 0 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 100000
+  auto-terminated relationships list: []
+  Properties:
+    Delete Attributes Expression:
+    property 1: value 1
+    property 2: value 2 ${nextInt()}
+Connections:
+- name: GenerateFlowFile/success/UpdateAttribute
+  source name: GenerateFlowFile
+  source relationship name: success
+  destination name: UpdateAttribute
+  max work queue size: 200
+  max work queue data size: 0 MB
+  flowfile expiration: 0 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.FirstInFirstOutPrioritizer
+- name: UpdateAttribute/success/RouteOnAttribute
+  source name: UpdateAttribute
+  source relationship name: success
+  destination name: RouteOnAttribute
+  max work queue size: 2000
+  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/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config-v1.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config-v1.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config-v1.yml
new file mode 100644
index 0000000..2af6b9b
--- /dev/null
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config-v1.yml
@@ -0,0 +1,111 @@
+# 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: MiNiFi Flow
+    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: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    destination url: https://localhost:8090/
+    port name: provenance
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config.yml
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config.yml b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config.yml
index 2af6b9b..64a28fe 100644
--- a/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config.yml
+++ b/minifi-toolkit/minifi-toolkit-configuration/src/test/resources/config.yml
@@ -13,99 +13,93 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+MiNiFi Config Version: 2
 Flow Controller:
-    name: MiNiFi Flow
-    comment:
-
+  name: MiNiFi Flow
+  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
-
+  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
-
+  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
-
+  content claim max appendable size: 10 MB
+  content claim max flow files: 100
+  always sync: false
 Provenance Repository:
-    provenance rollover time: 1 min
-
+  provenance rollover time: 1 min
 Component Status Repository:
-    buffer size: 1440
-    snapshot frequency: 1 min
-
+  buffer size: 1440
+  snapshot frequency: 1 min
 Security Properties:
-    keystore: /tmp/ssl/localhost-ks.jks
-    keystore type: JKS
-    keystore password: localtest
-    key password: localtest
-    truststore: /tmp/ssl/localhost-ts.jks
-    truststore type: JKS
-    truststore password: localtest
-    ssl protocol: TLS
-    Sensitive Props:
-        key:
-        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
-        provider: BC
-
+  keystore: /tmp/ssl/localhost-ks.jks
+  keystore type: JKS
+  keystore password: localtest
+  key password: localtest
+  truststore: /tmp/ssl/localhost-ts.jks
+  truststore type: JKS
+  truststore password: localtest
+  ssl protocol: TLS
+  Sensitive Props:
+    key: ''
+    algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+    provider: BC
 Processors:
-    - name: TailFile
-      class: org.apache.nifi.processors.standard.TailFile
-      max concurrent tasks: 1
-      scheduling strategy: TIMER_DRIVEN
-      scheduling period: 1 sec
-      penalization period: 30 sec
-      yield period: 1 sec
-      run duration nanos: 0
-      auto-terminated relationships list:
-      Properties:
-          File to Tail: logs/minifi-app.log
-          Rolling Filename Pattern: minifi-app*
-          Initial Start Position: Beginning of File
-
+- id: TailFile
+  name: TailFile
+  class: org.apache.nifi.processors.standard.TailFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    File to Tail: logs/minifi-app.log
+    Initial Start Position: Beginning of File
+    Rolling Filename Pattern: minifi-app*
 Connections:
-    - name: TailToS2S
-      source name: TailFile
-      source relationship name: success
-      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
-      max work queue size: 0
-      max work queue data size: 1 MB
-      flowfile expiration: 60 sec
-      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
-
+- id: TailToS2S
+  name: TailToS2S
+  source id: TailFile
+  source relationship names:
+  - success
+  destination id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+  max work queue size: 0
+  max work queue data size: 1 MB
+  flowfile expiration: 60 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
 Remote Processing Groups:
-    - name: NiFi Flow
-      comment:
-      url: https://localhost:8090/nifi
-      timeout: 30 secs
-      yield period: 10 sec
-      Input Ports:
-          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
-            name: tailed log
-            comments:
-            max concurrent tasks: 1
-            use compression: false
-
+- name: NiFi Flow
+  url: https://localhost:8090/nifi
+  comment: ''
+  timeout: 30 secs
+  yield period: 10 sec
+  Input Ports:
+  - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+    name: tailed log
+    comment: ''
+    max concurrent tasks: 1
+    use compression: false
 Provenance Reporting:
-    comment:
-    scheduling strategy: TIMER_DRIVEN
-    scheduling period: 30 sec
-    destination url: https://localhost:8090/
-    port name: provenance
-    originating url: http://${hostname(true)}:8081/nifi
-    use compression: true
-    timeout: 30 secs
-    batch size: 1000
\ No newline at end of file
+  comment: ''
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 30 sec
+  destination url: https://localhost:8090/
+  port name: provenance
+  originating url: http://${hostname(true)}:8081/nifi
+  use compression: true
+  timeout: 30 secs
+  batch size: 1000


[3/3] nifi-minifi git commit: MINIFI-117 - Maintainable Configuration Versioning

Posted by jp...@apache.org.
MINIFI-117 - Maintainable Configuration Versioning

This closes #45

Signed-off-by: Joseph Percivall <jo...@yahoo.com>


Project: http://git-wip-us.apache.org/repos/asf/nifi-minifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi-minifi/commit/1bbeedf6
Tree: http://git-wip-us.apache.org/repos/asf/nifi-minifi/tree/1bbeedf6
Diff: http://git-wip-us.apache.org/repos/asf/nifi-minifi/diff/1bbeedf6

Branch: refs/heads/master
Commit: 1bbeedf6fd0d2d28f604b7d6f36ceec15ac66eaf
Parents: 8913a90
Author: Bryan Rosander <br...@gmail.com>
Authored: Fri Oct 7 13:23:21 2016 -0400
Committer: Joseph Percivall <jo...@yahoo.com>
Committed: Mon Oct 31 13:36:00 2016 -0400

----------------------------------------------------------------------
 .../bootstrap/util/ConfigTransformer.java       |   4 +-
 .../bootstrap/util/TestConfigTransformer.java   |  26 +-
 .../src/test/resources/config-v1.yml            | 111 +++++++
 minifi-bootstrap/src/test/resources/config.yml  | 164 +++++------
 .../schema/ComponentStatusRepositorySchema.java |   3 +-
 .../minifi/commons/schema/ConfigSchema.java     | 228 +++------------
 .../minifi/commons/schema/ConnectionSchema.java |  42 +--
 .../commons/schema/ContentRepositorySchema.java |   3 +-
 .../commons/schema/CorePropertiesSchema.java    |   3 +-
 .../commons/schema/FlowControllerSchema.java    |   3 +-
 .../schema/FlowFileRepositorySchema.java        |   3 +-
 .../minifi/commons/schema/ProcessorSchema.java  |   2 +-
 .../schema/ProvenanceReportingSchema.java       |   3 +-
 .../schema/ProvenanceRepositorySchema.java      |   3 +-
 .../schema/RemoteProcessingGroupSchema.java     |   3 +-
 .../schema/SecurityPropertiesSchema.java        |   3 +-
 .../commons/schema/SensitivePropsSchema.java    |   3 +-
 .../nifi/minifi/commons/schema/SwapSchema.java  |   3 +-
 .../commons/schema/common/BaseSchema.java       |  84 +++---
 .../schema/common/BaseSchemaWithIdAndName.java  |  24 +-
 .../schema/common/ConvertableSchema.java        |  40 +++
 .../minifi/commons/schema/common/Schema.java    |  50 ++++
 .../commons/schema/common/WritableSchema.java   |  34 +++
 .../schema/serialization/SchemaLoader.java      |  27 +-
 .../commons/schema/v1/ConfigSchemaV1.java       | 263 +++++++++++++++++
 .../commons/schema/v1/ConnectionSchemaV1.java   | 104 +++++++
 .../commons/schema/v1/ProcessorSchemaV1.java    | 103 +++++++
 .../minifi/commons/schema/ConfigSchemaTest.java |  70 ++---
 .../commons/schema/ConnectionSchemaTest.java    |  92 +-----
 .../schema/serialization/SchemaLoaderTest.java  |  18 +-
 .../commons/schema/v1/ConfigSchemaV1Test.java   |  94 ++++++
 .../schema/v1/ConnectionSchemaV1Test.java       | 192 ++++++++++++
 .../schema/v1/ProcessorSchemaV1Test.java        | 241 +++++++++++++++
 .../src/test/resources/config-minimal-v2.yml    |  38 +++
 .../src/main/markdown/System_Admin_Guide.md     |  12 +
 .../src/main/resources/conf/config.yml          |  86 +++---
 .../src/main/resources/config.sh                |  20 +-
 .../toolkit/configuration/ConfigMain.java       | 257 +++++++++++-----
 .../toolkit/configuration/ConfigMainTest.java   | 126 +++++++-
 .../configuration/dto/BaseSchemaTester.java     |   4 +-
 .../configuration/dto/ConnectionSchemaTest.java |   4 +-
 .../src/test/resources/CsvToJson-v1.yml         | 178 +++++++++++
 .../resources/DecompressionCircularFlow-v1.yml  | 293 +++++++++++++++++++
 .../InvokeHttpMiNiFiTemplateTest-v1.yml         | 262 +++++++++++++++++
 ...TextExpressionLanguageCSVReformatting-v1.yml | 146 +++++++++
 .../test/resources/StressTestFramework-v1.yml   | 114 ++++++++
 .../src/test/resources/config-v1.yml            | 111 +++++++
 .../src/test/resources/config.yml               | 164 +++++------
 48 files changed, 3099 insertions(+), 762 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
index 87659ea..9794415 100644
--- a/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
+++ b/minifi-bootstrap/src/main/java/org/apache/nifi/minifi/bootstrap/util/ConfigTransformer.java
@@ -69,6 +69,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
+import java.util.stream.Collectors;
 import java.util.zip.GZIPOutputStream;
 
 public final class ConfigTransformer {
@@ -89,7 +90,8 @@ public final class ConfigTransformer {
     public static void transformConfigFile(InputStream sourceStream, String destPath) throws Exception {
         ConfigSchema configSchema = SchemaLoader.loadConfigSchemaFromYaml(sourceStream);
         if (!configSchema.isValid()) {
-            throw new InvalidConfigurationException("Failed to transform config file due to:" + configSchema.getValidationIssuesAsString());
+            throw new InvalidConfigurationException("Failed to transform config file due to:["
+                    + configSchema.getValidationIssues().stream().sorted().collect(Collectors.joining("], [")) + "]");
         }
 
         // Create nifi.properties and flow.xml.gz in memory

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java
index 8170d54..617da90 100644
--- a/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java
+++ b/minifi-bootstrap/src/test/java/org/apache/nifi/minifi/bootstrap/util/TestConfigTransformer.java
@@ -18,7 +18,6 @@
 package org.apache.nifi.minifi.bootstrap.util;
 
 import org.apache.nifi.minifi.bootstrap.exception.InvalidConfigurationException;
-import org.apache.nifi.minifi.commons.schema.ConfigSchema;
 import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
 import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
@@ -51,6 +50,23 @@ public class TestConfigTransformer {
     }
 
     @Test
+    public void doesTransformV1File() throws Exception {
+        ConfigTransformer.transformConfigFile("./src/test/resources/config-v1.yml", "./target/");
+        File nifiPropertiesFile = new File("./target/nifi.properties");
+
+        assertTrue(nifiPropertiesFile.exists());
+        assertTrue(nifiPropertiesFile.canRead());
+
+        nifiPropertiesFile.deleteOnExit();
+
+        File flowXml = new File("./target/flow.xml.gz");
+        assertTrue(flowXml.exists());
+        assertTrue(flowXml.canRead());
+
+        flowXml.deleteOnExit();
+    }
+
+    @Test
     public void doesTransformInputStream() throws Exception {
         File inputFile = new File("./src/test/resources/config.yml");
         ConfigTransformer.transformConfigFile(new FileInputStream(inputFile), "./target/");
@@ -199,10 +215,10 @@ public class TestConfigTransformer {
             ConfigTransformer.transformConfigFile("./src/test/resources/config-multiple-problems.yml", "./target/");
             fail("Invalid configuration file was not detected.");
         } catch (InvalidConfigurationException e){
-            assertEquals("Failed to transform config file due to:[" + ConfigSchema.CONNECTIONS_REFER_TO_PROCESSOR_NAMES_THAT_DONT_EXIST
-                    + "null], ['scheduling strategy' in section 'Provenance Reporting' because it is not a valid scheduling strategy], ['class' in section "
-                    + "'Processors' because it was not found and it is required], ['source name' in section 'Connections' because it was not found and it is required], ["
-                    + BaseSchema.getIssueText(ConnectionSchema.SOURCE_ID_KEY, "Connections", BaseSchema.IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED) + "]", e.getMessage());
+            assertEquals("Failed to transform config file due to:['class' in section 'Processors' because it was not found and it is required], " +
+                    "['scheduling strategy' in section 'Provenance Reporting' because it is not a valid scheduling strategy], " +
+                    "[" + BaseSchema.getIssueText(ConnectionSchema.SOURCE_ID_KEY, "Connections", BaseSchema.IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED) + "], " +
+                    "['source name' in section 'Connections' because it was not found and it is required]", e.getMessage());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-bootstrap/src/test/resources/config-v1.yml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/test/resources/config-v1.yml b/minifi-bootstrap/src/test/resources/config-v1.yml
new file mode 100644
index 0000000..2af6b9b
--- /dev/null
+++ b/minifi-bootstrap/src/test/resources/config-v1.yml
@@ -0,0 +1,111 @@
+# 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: MiNiFi Flow
+    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: /tmp/ssl/localhost-ks.jks
+    keystore type: JKS
+    keystore password: localtest
+    key password: localtest
+    truststore: /tmp/ssl/localhost-ts.jks
+    truststore type: JKS
+    truststore password: localtest
+    ssl protocol: TLS
+    Sensitive Props:
+        key:
+        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+        provider: BC
+
+Processors:
+    - name: TailFile
+      class: org.apache.nifi.processors.standard.TailFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 1 sec
+      penalization period: 30 sec
+      yield period: 1 sec
+      run duration nanos: 0
+      auto-terminated relationships list:
+      Properties:
+          File to Tail: logs/minifi-app.log
+          Rolling Filename Pattern: minifi-app*
+          Initial Start Position: Beginning of File
+
+Connections:
+    - name: TailToS2S
+      source name: TailFile
+      source relationship name: success
+      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+      max work queue size: 0
+      max work queue data size: 1 MB
+      flowfile expiration: 60 sec
+      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
+
+Remote Processing Groups:
+    - name: NiFi Flow
+      comment:
+      url: https://localhost:8090/nifi
+      timeout: 30 secs
+      yield period: 10 sec
+      Input Ports:
+          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+            name: tailed log
+            comments:
+            max concurrent tasks: 1
+            use compression: false
+
+Provenance Reporting:
+    comment:
+    scheduling strategy: TIMER_DRIVEN
+    scheduling period: 30 sec
+    destination url: https://localhost:8090/
+    port name: provenance
+    originating url: http://${hostname(true)}:8081/nifi
+    use compression: true
+    timeout: 30 secs
+    batch size: 1000
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-bootstrap/src/test/resources/config.yml
----------------------------------------------------------------------
diff --git a/minifi-bootstrap/src/test/resources/config.yml b/minifi-bootstrap/src/test/resources/config.yml
index 2af6b9b..64a28fe 100644
--- a/minifi-bootstrap/src/test/resources/config.yml
+++ b/minifi-bootstrap/src/test/resources/config.yml
@@ -13,99 +13,93 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+MiNiFi Config Version: 2
 Flow Controller:
-    name: MiNiFi Flow
-    comment:
-
+  name: MiNiFi Flow
+  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
-
+  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
-
+  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
-
+  content claim max appendable size: 10 MB
+  content claim max flow files: 100
+  always sync: false
 Provenance Repository:
-    provenance rollover time: 1 min
-
+  provenance rollover time: 1 min
 Component Status Repository:
-    buffer size: 1440
-    snapshot frequency: 1 min
-
+  buffer size: 1440
+  snapshot frequency: 1 min
 Security Properties:
-    keystore: /tmp/ssl/localhost-ks.jks
-    keystore type: JKS
-    keystore password: localtest
-    key password: localtest
-    truststore: /tmp/ssl/localhost-ts.jks
-    truststore type: JKS
-    truststore password: localtest
-    ssl protocol: TLS
-    Sensitive Props:
-        key:
-        algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
-        provider: BC
-
+  keystore: /tmp/ssl/localhost-ks.jks
+  keystore type: JKS
+  keystore password: localtest
+  key password: localtest
+  truststore: /tmp/ssl/localhost-ts.jks
+  truststore type: JKS
+  truststore password: localtest
+  ssl protocol: TLS
+  Sensitive Props:
+    key: ''
+    algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+    provider: BC
 Processors:
-    - name: TailFile
-      class: org.apache.nifi.processors.standard.TailFile
-      max concurrent tasks: 1
-      scheduling strategy: TIMER_DRIVEN
-      scheduling period: 1 sec
-      penalization period: 30 sec
-      yield period: 1 sec
-      run duration nanos: 0
-      auto-terminated relationships list:
-      Properties:
-          File to Tail: logs/minifi-app.log
-          Rolling Filename Pattern: minifi-app*
-          Initial Start Position: Beginning of File
-
+- id: TailFile
+  name: TailFile
+  class: org.apache.nifi.processors.standard.TailFile
+  max concurrent tasks: 1
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 1 sec
+  penalization period: 30 sec
+  yield period: 1 sec
+  run duration nanos: 0
+  auto-terminated relationships list: []
+  Properties:
+    File to Tail: logs/minifi-app.log
+    Initial Start Position: Beginning of File
+    Rolling Filename Pattern: minifi-app*
 Connections:
-    - name: TailToS2S
-      source name: TailFile
-      source relationship name: success
-      destination name: 8644cbcc-a45c-40e0-964d-5e536e2ada61
-      max work queue size: 0
-      max work queue data size: 1 MB
-      flowfile expiration: 60 sec
-      queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
-
+- id: TailToS2S
+  name: TailToS2S
+  source id: TailFile
+  source relationship names:
+  - success
+  destination id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+  max work queue size: 0
+  max work queue data size: 1 MB
+  flowfile expiration: 60 sec
+  queue prioritizer class: org.apache.nifi.prioritizer.NewestFlowFileFirstPrioritizer
 Remote Processing Groups:
-    - name: NiFi Flow
-      comment:
-      url: https://localhost:8090/nifi
-      timeout: 30 secs
-      yield period: 10 sec
-      Input Ports:
-          - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
-            name: tailed log
-            comments:
-            max concurrent tasks: 1
-            use compression: false
-
+- name: NiFi Flow
+  url: https://localhost:8090/nifi
+  comment: ''
+  timeout: 30 secs
+  yield period: 10 sec
+  Input Ports:
+  - id: 8644cbcc-a45c-40e0-964d-5e536e2ada61
+    name: tailed log
+    comment: ''
+    max concurrent tasks: 1
+    use compression: false
 Provenance Reporting:
-    comment:
-    scheduling strategy: TIMER_DRIVEN
-    scheduling period: 30 sec
-    destination url: https://localhost:8090/
-    port name: provenance
-    originating url: http://${hostname(true)}:8081/nifi
-    use compression: true
-    timeout: 30 secs
-    batch size: 1000
\ No newline at end of file
+  comment: ''
+  scheduling strategy: TIMER_DRIVEN
+  scheduling period: 30 sec
+  destination url: https://localhost:8090/
+  port name: provenance
+  originating url: http://${hostname(true)}:8081/nifi
+  use compression: true
+  timeout: 30 secs
+  batch size: 1000

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ComponentStatusRepositorySchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ComponentStatusRepositorySchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ComponentStatusRepositorySchema.java
index 02f3a78..7f685fc 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ComponentStatusRepositorySchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ComponentStatusRepositorySchema.java
@@ -18,12 +18,13 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
 import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.COMPONENT_STATUS_REPO_KEY;
 
-public class ComponentStatusRepositorySchema extends BaseSchema {
+public class ComponentStatusRepositorySchema extends BaseSchema implements WritableSchema {
     public static final String BUFFER_SIZE_KEY = "buffer size";
     public static final String SNAPSHOT_FREQUENCY_KEY = "snapshot frequency";
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java
index 6ed2dca..8dfd9d4 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConfigSchema.java
@@ -18,17 +18,16 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
 import org.apache.nifi.minifi.commons.schema.common.StringUtil;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
-import java.util.HashMap;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.TreeSet;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
 import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.COMPONENT_STATUS_REPO_KEY;
@@ -46,20 +45,18 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SE
 /**
  *
  */
-public class ConfigSchema extends BaseSchema {
+public class ConfigSchema extends BaseSchema implements WritableSchema, ConvertableSchema<ConfigSchema> {
     public static final String FOUND_THE_FOLLOWING_DUPLICATE_PROCESSOR_IDS = "Found the following duplicate processor ids: ";
     public static final String FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_IDS = "Found the following duplicate connection ids: ";
     public static final String FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_PROCESSING_GROUP_NAMES = "Found the following duplicate remote processing group names: ";
     public static final String FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_INPUT_PORT_IDS = "Found the following duplicate remote input port ids: ";
     public static final String FOUND_THE_FOLLOWING_DUPLICATE_IDS = "Found the following ids that occur both in Processors and Remote Input Ports: ";
-    public static final String CANNOT_LOOK_UP_PROCESSOR_ID_FROM_PROCESSOR_NAME_DUE_TO_DUPLICATE_PROCESSOR_NAMES = "Cannot look up Processor id from Processor name due to duplicate Processor names: ";
     public static final int CONFIG_VERSION = 2;
-    public static final String CONNECTIONS_REFER_TO_PROCESSOR_NAMES_THAT_DONT_EXIST = "Connection(s) refer to Processor names that don't exist: ";
-    public static String TOP_LEVEL_NAME = "top level";
     public static final String VERSION = "MiNiFi Config Version";
-    public static final String EMPTY_NAME = "empty_name";
-    public static final Pattern ID_REPLACE_PATTERN = Pattern.compile("[^A-Za-z0-9_-]");
-
+    public static final String CONNECTION_WITH_ID = "Connection with id ";
+    public static final String HAS_INVALID_SOURCE_ID = " has invalid source id ";
+    public static final String HAS_INVALID_DESTINATION_ID = " has invalid destination id ";
+    public static String TOP_LEVEL_NAME = "top level";
     private FlowControllerSchema flowControllerProperties;
     private CorePropertiesSchema coreProperties;
     private FlowFileRepositorySchema flowfileRepositoryProperties;
@@ -74,6 +71,11 @@ public class ConfigSchema extends BaseSchema {
     private ProvenanceRepositorySchema provenanceRepositorySchema;
 
     public ConfigSchema(Map map) {
+        this(map, Collections.emptyList());
+    }
+
+    public ConfigSchema(Map map, List<String> validationIssues) {
+        validationIssues.stream().forEach(this::addValidationIssue);
         flowControllerProperties = getMapAsType(map, FLOW_CONTROLLER_PROPS_KEY, FlowControllerSchema.class, TOP_LEVEL_NAME, true);
 
         coreProperties = getMapAsType(map, CORE_PROPS_KEY, CorePropertiesSchema.class, TOP_LEVEL_NAME, false);
@@ -83,12 +85,13 @@ public class ConfigSchema extends BaseSchema {
         componentStatusRepositoryProperties = getMapAsType(map, COMPONENT_STATUS_REPO_KEY, ComponentStatusRepositorySchema.class, TOP_LEVEL_NAME, false);
         securityProperties = getMapAsType(map, SECURITY_PROPS_KEY, SecurityPropertiesSchema.class, TOP_LEVEL_NAME, false);
 
-        processors = getProcessorSchemas(getOptionalKeyAsType(map, PROCESSORS_KEY, List.class, TOP_LEVEL_NAME, null));
+        processors = convertListToType(getOptionalKeyAsType(map, PROCESSORS_KEY, List.class, TOP_LEVEL_NAME, new ArrayList<>()), "processor", ProcessorSchema.class, PROCESSORS_KEY);
 
-        remoteProcessingGroups = convertListToType(getOptionalKeyAsType(map, REMOTE_PROCESSING_GROUPS_KEY, List.class, TOP_LEVEL_NAME, null), "remote processing group",
+        remoteProcessingGroups = convertListToType(getOptionalKeyAsType(map, REMOTE_PROCESSING_GROUPS_KEY, List.class, TOP_LEVEL_NAME, new ArrayList<>()), "remote processing group",
                 RemoteProcessingGroupSchema.class, REMOTE_PROCESSING_GROUPS_KEY);
 
-        connections = getConnectionSchemas(getOptionalKeyAsType(map, CONNECTIONS_KEY, List.class, TOP_LEVEL_NAME, null));
+        connections = convertListToType(getOptionalKeyAsType(map, CONNECTIONS_KEY, List.class, TOP_LEVEL_NAME, new ArrayList<>()),
+                "connection", ConnectionSchema.class, CONNECTIONS_KEY);
 
         provenanceReportingProperties = getMapAsType(map, PROVENANCE_REPORTING_KEY, ProvenanceReportingSchema.class, TOP_LEVEL_NAME, false, false);
 
@@ -100,170 +103,49 @@ public class ConfigSchema extends BaseSchema {
         addIssuesIfNotNull(securityProperties);
         addIssuesIfNotNull(provenanceReportingProperties);
         addIssuesIfNotNull(provenanceRepositorySchema);
+        addIssuesIfNotNull(processors);
+        addIssuesIfNotNull(connections);
+        addIssuesIfNotNull(remoteProcessingGroups);
 
         Set<String> processorIds = new HashSet<>();
-        if (processors != null) {
-            List<String> processorIdList = processors.stream().map(ProcessorSchema::getId).collect(Collectors.toList());
-            checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_PROCESSOR_IDS, processorIdList);
-            for (ProcessorSchema processorSchema : processors) {
-                addIssuesIfNotNull(processorSchema);
-            }
-            processorIds.addAll(processorIdList);
-        }
+        List<String> processorIdList = processors.stream().map(ProcessorSchema::getId).collect(Collectors.toList());
+        processorIds.addAll(processorIdList);
 
-        if (connections != null) {
-            List<String> idList = connections.stream().map(ConnectionSchema::getId).filter(s -> !StringUtil.isNullOrEmpty(s)).collect(Collectors.toList());
-            checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_IDS, idList);
-            for (ConnectionSchema connectionSchema : connections) {
-                addIssuesIfNotNull(connectionSchema);
-            }
-        }
+        checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_PROCESSOR_IDS, processorIdList);
+        checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_IDS, connections.stream().map(ConnectionSchema::getId).collect(Collectors.toList()));
+        checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_PROCESSING_GROUP_NAMES,
+                remoteProcessingGroups.stream().map(RemoteProcessingGroupSchema::getName).collect(Collectors.toList()));
 
         Set<String> remoteInputPortIds = new HashSet<>();
-        if (remoteProcessingGroups != null) {
-            checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_PROCESSING_GROUP_NAMES,
-                    remoteProcessingGroups.stream().map(RemoteProcessingGroupSchema::getName).collect(Collectors.toList()));
-            for (RemoteProcessingGroupSchema remoteProcessingGroupSchema : remoteProcessingGroups) {
-                addIssuesIfNotNull(remoteProcessingGroupSchema);
-            }
-            List<RemoteProcessingGroupSchema> remoteProcessingGroups = getRemoteProcessingGroups();
-            if (remoteProcessingGroups != null) {
-                List<String> remoteInputPortIdList = remoteProcessingGroups.stream().filter(r -> r.getInputPorts() != null)
-                        .flatMap(r -> r.getInputPorts().stream()).map(RemoteInputPortSchema::getId).collect(Collectors.toList());
-                checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_INPUT_PORT_IDS, remoteInputPortIdList);
-                remoteInputPortIds.addAll(remoteInputPortIdList);
-            }
-        }
+        List<String> remoteInputPortIdList = remoteProcessingGroups.stream().filter(r -> r.getInputPorts() != null)
+                .flatMap(r -> r.getInputPorts().stream()).map(RemoteInputPortSchema::getId).collect(Collectors.toList());
+        checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_INPUT_PORT_IDS, remoteInputPortIdList);
+        remoteInputPortIds.addAll(remoteInputPortIdList);
 
         Set<String> duplicateIds = new HashSet<>(processorIds);
         duplicateIds.retainAll(remoteInputPortIds);
         if (duplicateIds.size() > 0) {
             addValidationIssue(FOUND_THE_FOLLOWING_DUPLICATE_IDS + duplicateIds.stream().sorted().collect(Collectors.joining(", ")));
         }
-    }
-
-    protected List<ProcessorSchema> getProcessorSchemas(List<Map> processorMaps) {
-        if (processorMaps == null) {
-            return null;
-        }
-        List<ProcessorSchema> processors = convertListToType(processorMaps, "processor", ProcessorSchema.class, PROCESSORS_KEY);
-
-        Map<String, Integer> idMap = processors.stream().map(ProcessorSchema::getId).filter(
-                s -> !StringUtil.isNullOrEmpty(s)).collect(Collectors.toMap(Function.identity(), s -> 2, Integer::compareTo));
-
-        // Set unset ids
-        processors.stream().filter(connection -> StringUtil.isNullOrEmpty(connection.getId())).forEachOrdered(processor -> processor.setId(getUniqueId(idMap, processor.getName())));
-
-        return processors;
-    }
-
-    protected List<ConnectionSchema> getConnectionSchemas(List<Map> connectionMaps) {
-        if (connectionMaps == null) {
-            return null;
-        }
-        List<ConnectionSchema> connections = convertListToType(connectionMaps, "connection", ConnectionSchema.class, CONNECTIONS_KEY);
-        Map<String, Integer> idMap = connections.stream().map(ConnectionSchema::getId).filter(
-                s -> !StringUtil.isNullOrEmpty(s)).collect(Collectors.toMap(Function.identity(), s -> 2, Integer::compareTo));
-
-        Map<String, String> processorNameToIdMap = new HashMap<>();
-
-        // We can't look up id by name for names that appear more than once
-        Set<String> duplicateProcessorNames = new HashSet<>();
-
-        List<ProcessorSchema> processors = getProcessors();
-        if (processors != null) {
-            processors.stream().forEachOrdered(p -> processorNameToIdMap.put(p.getName(), p.getId()));
-
-            Set<String> processorNames = new HashSet<>();
-            processors.stream().map(ProcessorSchema::getName).forEachOrdered(n -> {
-                if (!processorNames.add(n)) {
-                    duplicateProcessorNames.add(n);
-                }
-            });
-        }
 
-        Set<String> remoteInputPortIds = new HashSet<>();
-        List<RemoteProcessingGroupSchema> remoteProcessingGroups = getRemoteProcessingGroups();
-        if (remoteProcessingGroups != null) {
-            remoteInputPortIds.addAll(remoteProcessingGroups.stream().filter(r -> r.getInputPorts() != null)
-                    .flatMap(r -> r.getInputPorts().stream()).map(RemoteInputPortSchema::getId).collect(Collectors.toSet()));
-        }
-
-        Set<String> problematicDuplicateNames = new HashSet<>();
-        Set<String> missingProcessorNames = new HashSet<>();
-        // Set unset ids
-        connections.stream().filter(connection -> StringUtil.isNullOrEmpty(connection.getId())).forEachOrdered(connection -> connection.setId(getUniqueId(idMap, connection.getName())));
-
-        connections.stream().filter(connection -> StringUtil.isNullOrEmpty(connection.getSourceId())).forEach(connection -> {
-            String sourceName = connection.getSourceName();
-            if (remoteInputPortIds.contains(sourceName)) {
-                connection.setSourceId(sourceName);
-            } else {
-                if (duplicateProcessorNames.contains(sourceName)) {
-                    problematicDuplicateNames.add(sourceName);
-                }
-                String sourceId = processorNameToIdMap.get(sourceName);
-                if (StringUtil.isNullOrEmpty(sourceId)) {
-                    missingProcessorNames.add(sourceName);
-                } else {
-                    connection.setSourceId(sourceId);
-                }
+        Set<String> connectableIds = new HashSet<>(processorIds);
+        connectableIds.addAll(remoteInputPortIds);
+        connections.forEach(c -> {
+            String destinationId = c.getDestinationId();
+            if (!StringUtil.isNullOrEmpty(destinationId) && !connectableIds.contains(destinationId)) {
+                addValidationIssue(CONNECTION_WITH_ID + c.getId() + HAS_INVALID_DESTINATION_ID + destinationId);
             }
-        });
-
-        connections.stream().filter(connection -> StringUtil.isNullOrEmpty(connection.getDestinationId()))
-                .forEach(connection -> {
-                    String destinationName = connection.getDestinationName();
-                    if (remoteInputPortIds.contains(destinationName)) {
-                        connection.setDestinationId(destinationName);
-                    } else {
-                        if (duplicateProcessorNames.contains(destinationName)) {
-                            problematicDuplicateNames.add(destinationName);
-                        }
-                        String destinationId = processorNameToIdMap.get(destinationName);
-                        if (StringUtil.isNullOrEmpty(destinationId)) {
-                            missingProcessorNames.add(destinationName);
-                        } else {
-                            connection.setDestinationId(destinationId);
-                        }
-                    }
-                });
-
-        if (problematicDuplicateNames.size() > 0) {
-            addValidationIssue(CANNOT_LOOK_UP_PROCESSOR_ID_FROM_PROCESSOR_NAME_DUE_TO_DUPLICATE_PROCESSOR_NAMES
-                    + problematicDuplicateNames.stream().collect(Collectors.joining(", ")));
-        }
-        if (missingProcessorNames.size() > 0) {
-            addValidationIssue(CONNECTIONS_REFER_TO_PROCESSOR_NAMES_THAT_DONT_EXIST + missingProcessorNames.stream().sorted().collect(Collectors.joining(", ")));
-        }
-        return connections;
-    }
-
-    protected static void checkForDuplicates(Consumer<String> duplicateMessageConsumer, String errorMessagePrefix, List<String> strings) {
-        if (strings != null) {
-            Set<String> seen = new HashSet<>();
-            Set<String> duplicates = new TreeSet<>();
-            for (String string : strings) {
-                if (!seen.add(string)) {
-                    duplicates.add(String.valueOf(string));
-                }
+            String sourceId = c.getSourceId();
+            if (!StringUtil.isNullOrEmpty(sourceId) && !connectableIds.contains(sourceId)) {
+                addValidationIssue(CONNECTION_WITH_ID + c.getId() + HAS_INVALID_SOURCE_ID + sourceId);
             }
-            if (duplicates.size() > 0) {
-                StringBuilder errorMessage = new StringBuilder(errorMessagePrefix);
-                for (String duplicateName : duplicates) {
-                    errorMessage.append(duplicateName);
-                    errorMessage.append(", ");
-                }
-                errorMessage.setLength(errorMessage.length() - 2);
-                duplicateMessageConsumer.accept(errorMessage.toString());
-            }
-        }
+        });
     }
 
     public Map<String, Object> toMap() {
         Map<String, Object> result = mapSupplier.get();
         result.put(VERSION, getVersion());
-        result.put(FLOW_CONTROLLER_PROPS_KEY, flowControllerProperties.toMap());
+        putIfNotNull(result, FLOW_CONTROLLER_PROPS_KEY, flowControllerProperties);
         putIfNotNull(result, CORE_PROPS_KEY, coreProperties);
         putIfNotNull(result, FLOWFILE_REPO_KEY, flowfileRepositoryProperties);
         putIfNotNull(result, CONTENT_REPO_KEY, contentRepositoryProperties);
@@ -321,31 +203,13 @@ public class ConfigSchema extends BaseSchema {
         return provenanceRepositorySchema;
     }
 
+    @Override
     public int getVersion() {
         return CONFIG_VERSION;
     }
 
-    /**
-     * Will replace all characters not in [A-Za-z0-9_] with _
-     * <p>
-     * This has potential for collisions so it will also append numbers as necessary to prevent that
-     *
-     * @param ids  id map of already incremented numbers
-     * @param name the name
-     * @return a unique filesystem-friendly id
-     */
-    protected static String getUniqueId(Map<String, Integer> ids, String name) {
-        String baseId = StringUtil.isNullOrEmpty(name) ? EMPTY_NAME : ID_REPLACE_PATTERN.matcher(name).replaceAll("_");
-        String id = baseId;
-        Integer idNum = ids.get(baseId);
-        while (ids.containsKey(id)) {
-            id = baseId + "_" + idNum++;
-        }
-        // Using != on a string comparison here is intentional.  The two will be reference equal iff the body of the while loop was never executed.
-        if (id != baseId) {
-            ids.put(baseId, idNum);
-        }
-        ids.put(id, 2);
-        return id;
+    @Override
+    public ConfigSchema convert() {
+        return this;
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConnectionSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConnectionSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConnectionSchema.java
index ceba2ca..768213c 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConnectionSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ConnectionSchema.java
@@ -21,7 +21,6 @@ import org.apache.nifi.minifi.commons.schema.common.BaseSchemaWithIdAndName;
 import org.apache.nifi.minifi.commons.schema.common.StringUtil;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
@@ -30,15 +29,12 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.CO
 
 public class ConnectionSchema extends BaseSchemaWithIdAndName {
     public static final String SOURCE_ID_KEY = "source id";
-    public static final String SOURCE_RELATIONSHIP_NAME_KEY = "source relationship name";
     public static final String SOURCE_RELATIONSHIP_NAMES_KEY = "source relationship names";
     public static final String DESTINATION_ID_KEY = "destination id";
     public static final String MAX_WORK_QUEUE_SIZE_KEY = "max work queue size";
     public static final String MAX_WORK_QUEUE_DATA_SIZE_KEY = "max work queue data size";
     public static final String FLOWFILE_EXPIRATION__KEY = "flowfile expiration";
     public static final String QUEUE_PRIORITIZER_CLASS_KEY = "queue prioritizer class";
-    public static final String SOURCE_NAME_KEY = "source name";
-    public static final String DESTINATION_NAME_KEY = "destination name";
 
     public static final long DEFAULT_MAX_WORK_QUEUE_SIZE = 0;
     public static final String DEFAULT_MAX_QUEUE_DATA_SIZE = "0 MB";
@@ -48,9 +44,6 @@ public class ConnectionSchema extends BaseSchemaWithIdAndName {
     private List<String> sourceRelationshipNames;
     private String destinationId;
 
-    private String sourceName;
-    private String destinationName;
-
     private Number maxWorkQueueSize = DEFAULT_MAX_WORK_QUEUE_SIZE;
     private String maxWorkQueueDataSize = DEFAULT_MAX_QUEUE_DATA_SIZE;
     private String flowfileExpiration = DEFAULT_FLOWFILE_EXPIRATION;
@@ -59,30 +52,13 @@ public class ConnectionSchema extends BaseSchemaWithIdAndName {
     public ConnectionSchema(Map map) {
         super(map, CONNECTIONS_KEY);
 
+        // In case of older version, these may not be available until after construction, validated in getValidationIssues()
         sourceId = getOptionalKeyAsType(map, SOURCE_ID_KEY, String.class, CONNECTIONS_KEY, "");
-        if (StringUtil.isNullOrEmpty(sourceId)) {
-            sourceName = getRequiredKeyAsType(map, SOURCE_NAME_KEY, String.class, CONNECTIONS_KEY);
-        }
-
-        String sourceRelationshipName = getOptionalKeyAsType(map, SOURCE_RELATIONSHIP_NAME_KEY, String.class, CONNECTIONS_KEY, null);
-        if (StringUtil.isNullOrEmpty(sourceRelationshipName)) {
-            sourceRelationshipNames = getOptionalKeyAsType(map, SOURCE_RELATIONSHIP_NAMES_KEY, List.class, CONNECTIONS_KEY, new ArrayList());
-            if (sourceRelationshipNames.isEmpty()) {
-                addValidationIssue(getIssueText(SOURCE_RELATIONSHIP_NAMES_KEY, CONNECTIONS_KEY, "expected at least one relationship to be specified"));
-            }
-        } else {
-            if (map.containsKey(SOURCE_RELATIONSHIP_NAMES_KEY)) {
-                addValidationIssue("Only one of " + SOURCE_RELATIONSHIP_NAME_KEY + ", " + SOURCE_RELATIONSHIP_NAMES_KEY + " should be set per connection.  Found both on "
-                        + (StringUtil.isNullOrEmpty(getName()) ? getId() : getName()));
-                sourceRelationshipNames = getRequiredKeyAsType(map, SOURCE_RELATIONSHIP_NAMES_KEY, List.class, CONNECTIONS_KEY);
-            } else {
-                sourceRelationshipNames = new ArrayList<>(Arrays.asList(sourceRelationshipName));
-            }
-        }
-
         destinationId = getOptionalKeyAsType(map, DESTINATION_ID_KEY, String.class, CONNECTIONS_KEY, "");
-        if (StringUtil.isNullOrEmpty(getDestinationId())) {
-            destinationName = getRequiredKeyAsType(map, DESTINATION_NAME_KEY, String.class, CONNECTIONS_KEY);
+
+        sourceRelationshipNames = getOptionalKeyAsType(map, SOURCE_RELATIONSHIP_NAMES_KEY, List.class, CONNECTIONS_KEY, new ArrayList<>());
+        if (sourceRelationshipNames.isEmpty()) {
+            addValidationIssue("Expected at least one value in " + SOURCE_RELATIONSHIP_NAMES_KEY + " for " + CONNECTIONS_KEY + " " + getName());
         }
 
         maxWorkQueueSize = getOptionalKeyAsType(map, MAX_WORK_QUEUE_SIZE_KEY, Number.class, CONNECTIONS_KEY, DEFAULT_MAX_WORK_QUEUE_SIZE);
@@ -141,14 +117,6 @@ public class ConnectionSchema extends BaseSchemaWithIdAndName {
         return queuePrioritizerClass;
     }
 
-    public String getSourceName() {
-        return sourceName;
-    }
-
-    public String getDestinationName() {
-        return destinationName;
-    }
-
     @Override
     public List<String> getValidationIssues() {
         List<String> validationIssues = super.getValidationIssues();

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ContentRepositorySchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ContentRepositorySchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ContentRepositorySchema.java
index 868cb79..e96bb75 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ContentRepositorySchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ContentRepositorySchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
@@ -27,7 +28,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.CO
 /**
  *
  */
-public class ContentRepositorySchema extends BaseSchema {
+public class ContentRepositorySchema extends BaseSchema implements WritableSchema {
     public static final String CONTENT_CLAIM_MAX_APPENDABLE_SIZE_KEY = "content claim max appendable size";
     public static final String CONTENT_CLAIM_MAX_FLOW_FILES_KEY = "content claim max flow files";
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/CorePropertiesSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/CorePropertiesSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/CorePropertiesSchema.java
index ce30d9c..f5f83d8 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/CorePropertiesSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/CorePropertiesSchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
@@ -27,7 +28,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.MA
 /**
  *
  */
-public class CorePropertiesSchema extends BaseSchema {
+public class CorePropertiesSchema extends BaseSchema implements WritableSchema {
 
     public static final String FLOW_CONTROLLER_SHUTDOWN_PERIOD_KEY = "flow controller graceful shutdown period";
     public static final String FLOW_SERVICE_WRITE_DELAY_INTERVAL_KEY = "flow service write delay interval";

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowControllerSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowControllerSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowControllerSchema.java
index 3306029..49bffb9 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowControllerSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowControllerSchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
@@ -28,7 +29,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.NA
 /**
  *
  */
-public class FlowControllerSchema extends BaseSchema {
+public class FlowControllerSchema extends BaseSchema implements WritableSchema {
     private String name;
     private String comment;
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowFileRepositorySchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowFileRepositorySchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowFileRepositorySchema.java
index cd7f456..bf94e4d 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowFileRepositorySchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/FlowFileRepositorySchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
@@ -28,7 +29,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SW
 /**
  *
  */
-public class FlowFileRepositorySchema extends BaseSchema {
+public class FlowFileRepositorySchema extends BaseSchema implements WritableSchema {
     public static final String PARTITIONS_KEY = "partitions";
     public static final String CHECKPOINT_INTERVAL_KEY = "checkpoint interval";
     public static final int DEFAULT_PARTITIONS = 256;

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProcessorSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProcessorSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProcessorSchema.java
index e2bde47..048027c 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProcessorSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProcessorSchema.java
@@ -77,7 +77,7 @@ public class ProcessorSchema extends BaseSchemaWithIdAndName {
         annotationData = getOptionalKeyAsType(map, ANNOTATION_DATA_KEY, String.class, PROCESSORS_KEY, "");
     }
 
-    private static boolean isSchedulingStrategy(String string) {
+    public static boolean isSchedulingStrategy(String string) {
         try {
             SchedulingStrategy.valueOf(string);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceReportingSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceReportingSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceReportingSchema.java
index b12adb7..6490511 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceReportingSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceReportingSchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 import org.apache.nifi.scheduling.SchedulingStrategy;
 
 import java.util.Map;
@@ -32,7 +33,7 @@ import static org.apache.nifi.minifi.commons.schema.RemoteProcessingGroupSchema.
 /**
  *
  */
-public class ProvenanceReportingSchema extends BaseSchema {
+public class ProvenanceReportingSchema extends BaseSchema implements WritableSchema {
     public static final String DESTINATION_URL_KEY = "destination url";
     public static final String PORT_NAME_KEY = "port name";
     public static final String ORIGINATING_URL_KEY = "originating url";

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceRepositorySchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceRepositorySchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceRepositorySchema.java
index ac858ef..1f1d02a 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceRepositorySchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/ProvenanceRepositorySchema.java
@@ -18,12 +18,13 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
 import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.PROVENANCE_REPO_KEY;
 
-public class ProvenanceRepositorySchema extends BaseSchema {
+public class ProvenanceRepositorySchema extends BaseSchema implements WritableSchema {
     public static final String PROVENANCE_REPO_ROLLOVER_TIME_KEY = "provenance rollover time";
 
     public static final String DEFAULT_PROVENANCE_ROLLOVER_TIME = "1 min";

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessingGroupSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessingGroupSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessingGroupSchema.java
index 86ed71e..c1d318e 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessingGroupSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/RemoteProcessingGroupSchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.List;
 import java.util.Map;
@@ -31,7 +32,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.YI
 /**
  *
  */
-public class RemoteProcessingGroupSchema extends BaseSchema {
+public class RemoteProcessingGroupSchema extends BaseSchema implements WritableSchema {
     public static final String URL_KEY = "url";
     public static final String TIMEOUT_KEY = "timeout";
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SecurityPropertiesSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SecurityPropertiesSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SecurityPropertiesSchema.java
index 6adfdfe..8ad68bb 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SecurityPropertiesSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SecurityPropertiesSchema.java
@@ -19,6 +19,7 @@ package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
 import org.apache.nifi.minifi.commons.schema.common.StringUtil;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
@@ -28,7 +29,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SE
 /**
  *
  */
-public class SecurityPropertiesSchema extends BaseSchema {
+public class SecurityPropertiesSchema extends BaseSchema implements WritableSchema {
 
     public static final String KEYSTORE_KEY = "keystore";
     public static final String KEYSTORE_TYPE_KEY = "keystore type";

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SensitivePropsSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SensitivePropsSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SensitivePropsSchema.java
index 93260ea..afbefac 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SensitivePropsSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SensitivePropsSchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
@@ -26,7 +27,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SE
 /**
  *
  */
-public class SensitivePropsSchema extends BaseSchema {
+public class SensitivePropsSchema extends BaseSchema implements WritableSchema {
     public static final String SENSITIVE_PROPS_KEY_KEY = "key";
     public static final String SENSITIVE_PROPS_ALGORITHM_KEY = "algorithm";
     public static final String SENSITIVE_PROPS_PROVIDER_KEY = "provider";

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SwapSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SwapSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SwapSchema.java
index d38ce7a..ee4b8c6 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SwapSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/SwapSchema.java
@@ -18,6 +18,7 @@
 package org.apache.nifi.minifi.commons.schema;
 
 import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.WritableSchema;
 
 import java.util.Map;
 
@@ -26,7 +27,7 @@ import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SW
 /**
  *
  */
-public class SwapSchema extends BaseSchema {
+public class SwapSchema extends BaseSchema implements WritableSchema {
     public static final String THRESHOLD_KEY = "threshold";
     public static final String IN_PERIOD_KEY = "in period";
     public static final String IN_THREADS_KEY = "in threads";

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchema.java
index 0c269cc..7ba322a 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchema.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchema.java
@@ -20,18 +20,25 @@ package org.apache.nifi.minifi.commons.schema.common;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.function.Predicate;
+import java.util.TreeSet;
+import java.util.function.Consumer;
 import java.util.function.Supplier;
+import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
-public abstract class BaseSchema {
+public abstract class BaseSchema implements Schema {
     public static final String IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED = "it was not found and it is required";
+    public static final String EMPTY_NAME = "empty_name";
+
+    public static final Pattern ID_REPLACE_PATTERN = Pattern.compile("[^A-Za-z0-9_-]");
+
     protected final Supplier<Map<String, Object>> mapSupplier;
 
     public BaseSchema() {
@@ -43,41 +50,16 @@ public abstract class BaseSchema {
     }
 
     /******* Validation Issue helper methods *******/
-    private List<String> validationIssues = new LinkedList<>();
+    private Collection<String> validationIssues = new HashSet<>();
 
+    @Override
     public boolean isValid() {
         return getValidationIssues().isEmpty();
     }
 
+    @Override
     public List<String> getValidationIssues() {
-        return new ArrayList<>(validationIssues);
-    }
-
-    public String getValidationIssuesAsString() {
-        StringBuilder stringBuilder = new StringBuilder();
-        boolean first = true;
-        for (String validationIssue : getValidationIssues()) {
-            if (!first) {
-                stringBuilder.append(", ");
-            }
-            stringBuilder.append("[");
-            stringBuilder.append(validationIssue);
-            stringBuilder.append("]");
-            first = false;
-        }
-        return stringBuilder.toString();
-    }
-
-    public <T> T getAndValidateNotNull(Supplier<T> supplier, String keyName, String wrapperName) {
-        return getAndValidate(supplier, t -> t != null, keyName, wrapperName, IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED);
-    }
-
-    public <T> T getAndValidate(Supplier<T> supplier, Predicate<T> predicate, String keyName, String wrapperName, String reason) {
-        T result = supplier.get();
-        if (!predicate.test(result)) {
-            addValidationIssue(keyName, wrapperName, reason);
-        }
-        return result;
+        return validationIssues.stream().sorted().collect(Collectors.toList());
     }
 
     public void addValidationIssue(String issue) {
@@ -98,6 +80,12 @@ public abstract class BaseSchema {
         }
     }
 
+    public void addIssuesIfNotNull(List<? extends BaseSchema> baseSchemas) {
+        if (baseSchemas != null) {
+            baseSchemas.forEach(this::addIssuesIfNotNull);
+        }
+    }
+
     /******* Value Access/Interpretation helper methods *******/
     public <T> T getOptionalKeyAsType(Map valueMap, String key, Class<T> targetClass, String wrapperName, T defaultValue) {
         return getKeyAsType(valueMap, key, targetClass, wrapperName, false, defaultValue);
@@ -138,7 +126,7 @@ public abstract class BaseSchema {
 
     public <InputT, OutputT> List<OutputT> convertListToType(List<InputT> list, String simpleListType, Class<? extends OutputT> targetClass, String wrapperName){
         if (list == null) {
-            return null;
+            return new ArrayList<>();
         }
         List<OutputT> result = new ArrayList<>(list.size());
         for (int i = 0; i < list.size(); i++) {
@@ -182,17 +170,15 @@ public abstract class BaseSchema {
         return null;
     }
 
-    public abstract Map<String, Object> toMap();
-
-    public static void putIfNotNull(Map valueMap, String key, BaseSchema schema) {
+    public static void putIfNotNull(Map valueMap, String key, WritableSchema schema) {
         if (schema != null) {
             valueMap.put(key, schema.toMap());
         }
     }
 
-    public static void putListIfNotNull(Map valueMap, String key, List<? extends BaseSchema> list) {
+    public static void putListIfNotNull(Map valueMap, String key, List<? extends WritableSchema> list) {
         if (list != null) {
-            valueMap.put(key, list.stream().map(BaseSchema::toMap).collect(Collectors.toList()));
+            valueMap.put(key, list.stream().map(WritableSchema::toMap).collect(Collectors.toList()));
         }
     }
 
@@ -207,4 +193,26 @@ public abstract class BaseSchema {
     public static <K, V> Map<K, V> nullToEmpty(Map<K, V> map) {
         return map == null ? Collections.emptyMap() : map;
     }
+
+
+
+    public static void checkForDuplicates(Consumer<String> duplicateMessageConsumer, String errorMessagePrefix, List<String> strings) {
+        if (strings != null) {
+            Set<String> seen = new HashSet<>();
+            Set<String> duplicates = new TreeSet<>();
+            for (String string : strings) {
+                if (!seen.add(string)) {
+                    duplicates.add(String.valueOf(string));
+                }
+            }
+            if (duplicates.size() > 0) {
+                duplicateMessageConsumer.accept(errorMessagePrefix + duplicates.stream().collect(Collectors.joining(", ")));
+            }
+        }
+    }
+
+    @Override
+    public void clearValidationIssues() {
+        validationIssues.clear();
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchemaWithIdAndName.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchemaWithIdAndName.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchemaWithIdAndName.java
index 8acb167..9ab6718 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchemaWithIdAndName.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/BaseSchemaWithIdAndName.java
@@ -26,7 +26,7 @@ import java.util.regex.Pattern;
 import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.ID_KEY;
 import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.NAME_KEY;
 
-public abstract class BaseSchemaWithIdAndName extends BaseSchema {
+public abstract class BaseSchemaWithIdAndName extends BaseSchema implements WritableSchema {
     public static final Pattern VALID_ID_PATTERN = Pattern.compile("[A-Za-z0-9_-]+");
     public static final String ID_DOES_NOT_MATCH_VALID_ID_PATTERN = "Id does not match valid pattern (" + VALID_ID_PATTERN + "): ";
 
@@ -40,35 +40,35 @@ public abstract class BaseSchemaWithIdAndName extends BaseSchema {
         this.wrapperName = wrapperName;
     }
 
-    protected String getName(Map map, String wrapperName) {
-        return getOptionalKeyAsType(map, NAME_KEY, String.class, wrapperName, "");
-    }
-
     protected String getId(Map map, String wrapperName) {
         return getOptionalKeyAsType(map, ID_KEY, String.class, wrapperName, "");
     }
 
-    public void setId(String id) {
-        this.id = id;
-    }
-
-    protected void setName(String name) {
-        this.name = name;
+    protected String getName(Map map, String wrapperName) {
+        return getOptionalKeyAsType(map, NAME_KEY, String.class, wrapperName, "");
     }
 
     public String getId() {
         return id;
     }
 
+    public void setId(String id) {
+        this.id = id;
+    }
+
     public String getName() {
         return name;
     }
 
+    protected void setName(String name) {
+        this.name = name;
+    }
+
     @Override
     public Map<String, Object> toMap() {
         Map<String, Object> map = mapSupplier.get();
-        map.put(NAME_KEY, name);
         map.put(ID_KEY, id);
+        map.put(NAME_KEY, name);
         return map;
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/ConvertableSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/ConvertableSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/ConvertableSchema.java
new file mode 100644
index 0000000..61a5b2a
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/ConvertableSchema.java
@@ -0,0 +1,40 @@
+/*
+ *
+ *  * 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.commons.schema.common;
+
+/**
+ * Schema that can be converted to another.  Typically used to upconvert older versions to newer.
+ * @param <T> the type it can be converted to
+ */
+public interface ConvertableSchema<T extends Schema> extends Schema {
+    /**
+     * Converts this instance to the destination type.
+     *
+     * @return the converted instance
+     */
+    T convert();
+
+    /**
+     * Returns the version of this Schema before conversion.
+     *
+     * @return the version of this Schema before conversion.
+     */
+    int getVersion();
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/Schema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/Schema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/Schema.java
new file mode 100644
index 0000000..f038d76
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/Schema.java
@@ -0,0 +1,50 @@
+/*
+ *
+ *  * 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.commons.schema.common;
+
+import java.util.List;
+
+public interface Schema {
+    /**
+     * Returns known problems with the given schema
+     *
+     * @return known problems with the given schema
+     */
+    List<String> getValidationIssues();
+
+    /**
+     * Adds a validation issue
+     *
+     * @param issue the issue to add
+     */
+    void addValidationIssue(String issue);
+
+    /**
+     * Returns a boolean indicating whether the schema is valid
+     *
+     * @return a boolean indicating whether the schema is valid
+     */
+    boolean isValid();
+
+    /**
+     * Clears validation issues
+     */
+    void clearValidationIssues();
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/WritableSchema.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/WritableSchema.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/WritableSchema.java
new file mode 100644
index 0000000..2119e85
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/common/WritableSchema.java
@@ -0,0 +1,34 @@
+/*
+ *
+ *  * 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.commons.schema.common;
+
+import java.util.Map;
+
+/**
+ * Schema that can be serialized to a Map
+ */
+public interface WritableSchema extends Schema {
+    /**
+     * Serialize the schema to a Map
+     *
+     * @return the output map
+     */
+    Map<String, Object> toMap();
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoader.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoader.java
index 9d73bb7..331f40d 100644
--- a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoader.java
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoader.java
@@ -18,7 +18,10 @@
 package org.apache.nifi.minifi.commons.schema.serialization;
 
 import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.common.StringUtil;
 import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
+import org.apache.nifi.minifi.commons.schema.v1.ConfigSchemaV1;
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.error.YAMLException;
 
@@ -30,13 +33,13 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 
 public class SchemaLoader {
-    private static final Map<String, Function<Map, ConfigSchema>> configSchemaFactories = initConfigSchemaFactories();
+    private static final Map<String, Function<Map, ConvertableSchema<ConfigSchema>>> configSchemaFactories = initConfigSchemaFactories();
 
-    private static Map<String, Function<Map, ConfigSchema>> initConfigSchemaFactories() {
-        Map<String, Function<Map, ConfigSchema>> result = new HashMap<>();
-        result.put(String.valueOf((Object)null), ConfigSchema::new);
-        result.put("", ConfigSchema::new);
-        result.put("1", ConfigSchema::new);
+    private static Map<String, Function<Map, ConvertableSchema<ConfigSchema>>> initConfigSchemaFactories() {
+        Map<String, Function<Map, ConvertableSchema<ConfigSchema>>> result = new HashMap<>();
+        result.put(String.valueOf((Object) null), ConfigSchemaV1::new);
+        result.put("", ConfigSchemaV1::new);
+        result.put(Integer.toString(ConfigSchemaV1.CONFIG_VERSION), ConfigSchemaV1::new);
         result.put(Integer.toString(ConfigSchema.CONFIG_VERSION), ConfigSchema::new);
         return result;
     }
@@ -67,11 +70,19 @@ public class SchemaLoader {
     }
 
     public static ConfigSchema loadConfigSchemaFromYaml(Map<String, Object> yamlAsMap) throws SchemaLoaderException {
+        return loadConvertableSchemaFromYaml(yamlAsMap).convert();
+    }
+
+    public static ConvertableSchema<ConfigSchema> loadConvertableSchemaFromYaml(InputStream inputStream) throws SchemaLoaderException, IOException {
+        return loadConvertableSchemaFromYaml(loadYamlAsMap(inputStream));
+    }
+
+    public static ConvertableSchema<ConfigSchema> loadConvertableSchemaFromYaml(Map<String, Object> yamlAsMap) throws SchemaLoaderException {
         String version = String.valueOf(yamlAsMap.get(ConfigSchema.VERSION));
-        Function<Map, ConfigSchema> schemaFactory = configSchemaFactories.get(version);
+        Function<Map, ConvertableSchema<ConfigSchema>> schemaFactory = configSchemaFactories.get(version);
         if (schemaFactory == null) {
             throw new SchemaLoaderException("YAML configuration version " + version + " not supported.  Supported versions: "
-                    + configSchemaFactories.keySet().stream().sorted().collect(Collectors.joining(", ")));
+                    + configSchemaFactories.keySet().stream().filter(s -> !StringUtil.isNullOrEmpty(s) && !String.valueOf((Object) null).equals(s)).sorted().collect(Collectors.joining(", ")));
         }
         return schemaFactory.apply(yamlAsMap);
     }


[2/3] nifi-minifi git commit: MINIFI-117 - Maintainable Configuration Versioning

Posted by jp...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1.java
new file mode 100644
index 0000000..2ebf28a
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1.java
@@ -0,0 +1,263 @@
+/*
+ *
+ *  * 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.commons.schema.v1;
+
+import org.apache.nifi.minifi.commons.schema.ComponentStatusRepositorySchema;
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
+import org.apache.nifi.minifi.commons.schema.ContentRepositorySchema;
+import org.apache.nifi.minifi.commons.schema.CorePropertiesSchema;
+import org.apache.nifi.minifi.commons.schema.FlowControllerSchema;
+import org.apache.nifi.minifi.commons.schema.FlowFileRepositorySchema;
+import org.apache.nifi.minifi.commons.schema.ProcessorSchema;
+import org.apache.nifi.minifi.commons.schema.ProvenanceReportingSchema;
+import org.apache.nifi.minifi.commons.schema.ProvenanceRepositorySchema;
+import org.apache.nifi.minifi.commons.schema.RemoteInputPortSchema;
+import org.apache.nifi.minifi.commons.schema.RemoteProcessingGroupSchema;
+import org.apache.nifi.minifi.commons.schema.SecurityPropertiesSchema;
+import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.common.StringUtil;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.minifi.commons.schema.ConfigSchema.TOP_LEVEL_NAME;
+import static org.apache.nifi.minifi.commons.schema.ConfigSchema.VERSION;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.COMPONENT_STATUS_REPO_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.CONNECTIONS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.CONTENT_REPO_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.CORE_PROPS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.FLOWFILE_REPO_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.FLOW_CONTROLLER_PROPS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.PROCESSORS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.PROVENANCE_REPORTING_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.PROVENANCE_REPO_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.REMOTE_PROCESSING_GROUPS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SECURITY_PROPS_KEY;
+
+public class ConfigSchemaV1 extends BaseSchema implements ConvertableSchema<ConfigSchema> {
+    public static final String FOUND_THE_FOLLOWING_DUPLICATE_PROCESSOR_NAMES = "Found the following duplicate processor names: ";
+    public static final String FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_NAMES = "Found the following duplicate connection names: ";
+    public static final String FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_PROCESSING_GROUP_NAMES = "Found the following duplicate remote processing group names: ";
+    public static final String CANNOT_LOOK_UP_PROCESSOR_ID_FROM_PROCESSOR_NAME_DUE_TO_DUPLICATE_PROCESSOR_NAMES = "Cannot look up Processor id from Processor name due to duplicate Processor names: ";
+    public static final int CONFIG_VERSION = 1;
+    public static final String CONNECTION_WITH_NAME = "Connection with name ";
+    public static final String HAS_INVALID_DESTINATION_NAME = " has invalid destination name ";
+    public static final String HAS_INVALID_SOURCE_NAME = " has invalid source name ";
+    private FlowControllerSchema flowControllerProperties;
+    private CorePropertiesSchema coreProperties;
+    private FlowFileRepositorySchema flowfileRepositoryProperties;
+    private ContentRepositorySchema contentRepositoryProperties;
+    private ComponentStatusRepositorySchema componentStatusRepositoryProperties;
+    private SecurityPropertiesSchema securityProperties;
+    private List<ProcessorSchemaV1> processors;
+    private List<ConnectionSchemaV1> connections;
+    private List<RemoteProcessingGroupSchema> remoteProcessingGroups;
+    private ProvenanceReportingSchema provenanceReportingProperties;
+
+    private ProvenanceRepositorySchema provenanceRepositorySchema;
+
+    public ConfigSchemaV1(Map map) {
+        flowControllerProperties = getMapAsType(map, FLOW_CONTROLLER_PROPS_KEY, FlowControllerSchema.class, TOP_LEVEL_NAME, true);
+
+        coreProperties = getMapAsType(map, CORE_PROPS_KEY, CorePropertiesSchema.class, TOP_LEVEL_NAME, false);
+        flowfileRepositoryProperties = getMapAsType(map, FLOWFILE_REPO_KEY, FlowFileRepositorySchema.class, TOP_LEVEL_NAME, false);
+        contentRepositoryProperties = getMapAsType(map, CONTENT_REPO_KEY, ContentRepositorySchema.class, TOP_LEVEL_NAME, false);
+        provenanceRepositorySchema = getMapAsType(map, PROVENANCE_REPO_KEY, ProvenanceRepositorySchema.class, TOP_LEVEL_NAME, false);
+        componentStatusRepositoryProperties = getMapAsType(map, COMPONENT_STATUS_REPO_KEY, ComponentStatusRepositorySchema.class, TOP_LEVEL_NAME, false);
+        securityProperties = getMapAsType(map, SECURITY_PROPS_KEY, SecurityPropertiesSchema.class, TOP_LEVEL_NAME, false);
+
+        processors = convertListToType(getOptionalKeyAsType(map, PROCESSORS_KEY, List.class, TOP_LEVEL_NAME, new ArrayList<>()), PROCESSORS_KEY, ProcessorSchemaV1.class, TOP_LEVEL_NAME);
+
+        remoteProcessingGroups = convertListToType(getOptionalKeyAsType(map, REMOTE_PROCESSING_GROUPS_KEY, List.class, TOP_LEVEL_NAME, new ArrayList<>()), "remote processing group",
+                RemoteProcessingGroupSchema.class, REMOTE_PROCESSING_GROUPS_KEY);
+
+        connections = convertListToType(getOptionalKeyAsType(map, CONNECTIONS_KEY, List.class, TOP_LEVEL_NAME, new ArrayList<>()), CONNECTIONS_KEY, ConnectionSchemaV1.class, TOP_LEVEL_NAME);
+
+        provenanceReportingProperties = getMapAsType(map, PROVENANCE_REPORTING_KEY, ProvenanceReportingSchema.class, TOP_LEVEL_NAME, false, false);
+
+        addIssuesIfNotNull(flowControllerProperties);
+        addIssuesIfNotNull(coreProperties);
+        addIssuesIfNotNull(flowfileRepositoryProperties);
+        addIssuesIfNotNull(contentRepositoryProperties);
+        addIssuesIfNotNull(componentStatusRepositoryProperties);
+        addIssuesIfNotNull(securityProperties);
+        addIssuesIfNotNull(provenanceReportingProperties);
+        addIssuesIfNotNull(provenanceRepositorySchema);
+        addIssuesIfNotNull(processors);
+        addIssuesIfNotNull(connections);
+        addIssuesIfNotNull(remoteProcessingGroups);
+
+        List<String> processorNames = processors.stream().map(ProcessorSchemaV1::getName).collect(Collectors.toList());
+
+        checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_PROCESSOR_NAMES, processorNames);
+        checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_NAMES, connections.stream().map(ConnectionSchemaV1::getName).collect(Collectors.toList()));
+        checkForDuplicates(this::addValidationIssue, FOUND_THE_FOLLOWING_DUPLICATE_REMOTE_PROCESSING_GROUP_NAMES, remoteProcessingGroups.stream().map(RemoteProcessingGroupSchema::getName)
+                .collect(Collectors.toList()));
+
+        Set<String> connectableNames = new HashSet<>(processorNames);
+        connectableNames.addAll(remoteProcessingGroups.stream().flatMap(r -> r.getInputPorts().stream()).map(RemoteInputPortSchema::getId).collect(Collectors.toList()));
+        connections.forEach(c -> {
+            String destinationName = c.getDestinationName();
+            if (!StringUtil.isNullOrEmpty(destinationName) && !connectableNames.contains(destinationName)) {
+                addValidationIssue(CONNECTION_WITH_NAME + c.getName() + HAS_INVALID_DESTINATION_NAME + destinationName);
+            }
+            String sourceName = c.getSourceName();
+            if (!StringUtil.isNullOrEmpty(sourceName) && !connectableNames.contains(sourceName)) {
+                addValidationIssue(CONNECTION_WITH_NAME + c.getName() + HAS_INVALID_SOURCE_NAME + sourceName);
+            }
+        });
+    }
+
+    protected List<ProcessorSchema> getProcessorSchemas() {
+        Map<String, Integer> idMap = new HashMap<>();
+        List<ProcessorSchema> processorSchemas = new ArrayList<>(processors.size());
+
+        for (ProcessorSchemaV1 processor : processors) {
+            ProcessorSchema processorSchema = processor.convert();
+            processorSchema.setId(getUniqueId(idMap, processorSchema.getName()));
+            processorSchemas.add(processorSchema);
+        }
+
+        return processorSchemas;
+    }
+
+    protected List<ConnectionSchema> getConnectionSchemas(List<ProcessorSchema> processors, List<String> validationIssues) {
+        Map<String, Integer> idMap = new HashMap<>();
+
+        Map<String, String> processorNameToIdMap = new HashMap<>();
+
+        // We can't look up id by name for names that appear more than once
+        Set<String> duplicateProcessorNames = new HashSet<>();
+
+        if (processors != null) {
+            processors.stream().forEachOrdered(p -> processorNameToIdMap.put(p.getName(), p.getId()));
+
+            Set<String> processorNames = new HashSet<>();
+            processors.stream().map(ProcessorSchema::getName).forEachOrdered(n -> {
+                if (!processorNames.add(n)) {
+                    duplicateProcessorNames.add(n);
+                }
+            });
+        }
+
+        Set<String> remoteInputPortIds = new HashSet<>();
+        if (remoteProcessingGroups != null) {
+            remoteInputPortIds.addAll(remoteProcessingGroups.stream().filter(r -> r.getInputPorts() != null)
+                    .flatMap(r -> r.getInputPorts().stream()).map(RemoteInputPortSchema::getId).collect(Collectors.toSet()));
+        }
+
+        Set<String> problematicDuplicateNames = new HashSet<>();
+
+        List<ConnectionSchema> connectionSchemas = new ArrayList<>(connections.size());
+        for (ConnectionSchemaV1 connection : connections) {
+            ConnectionSchema convert = connection.convert();
+            convert.setId(getUniqueId(idMap, convert.getName()));
+
+            String sourceName = connection.getSourceName();
+            if (remoteInputPortIds.contains(sourceName)) {
+                convert.setSourceId(sourceName);
+            } else {
+                if (duplicateProcessorNames.contains(sourceName)) {
+                    problematicDuplicateNames.add(sourceName);
+                }
+                String sourceId = processorNameToIdMap.get(sourceName);
+                if (!StringUtil.isNullOrEmpty(sourceId)) {
+                    convert.setSourceId(sourceId);
+                }
+            }
+
+            String destinationName = connection.getDestinationName();
+            if (remoteInputPortIds.contains(destinationName)) {
+                convert.setDestinationId(destinationName);
+            } else {
+                if (duplicateProcessorNames.contains(destinationName)) {
+                    problematicDuplicateNames.add(destinationName);
+                }
+                String destinationId = processorNameToIdMap.get(destinationName);
+                if (!StringUtil.isNullOrEmpty(destinationId)) {
+                    convert.setDestinationId(destinationId);
+                }
+            }
+            connectionSchemas.add(convert);
+        }
+
+        if (problematicDuplicateNames.size() > 0) {
+            validationIssues.add(CANNOT_LOOK_UP_PROCESSOR_ID_FROM_PROCESSOR_NAME_DUE_TO_DUPLICATE_PROCESSOR_NAMES
+                    + problematicDuplicateNames.stream().collect(Collectors.joining(", ")));
+        }
+        return connectionSchemas;
+    }
+
+    @Override
+    public ConfigSchema convert() {
+        Map<String, Object> map = new HashMap<>();
+        map.put(VERSION, getVersion());
+        putIfNotNull(map, FLOW_CONTROLLER_PROPS_KEY, flowControllerProperties);
+        putIfNotNull(map, CORE_PROPS_KEY, coreProperties);
+        putIfNotNull(map, FLOWFILE_REPO_KEY, flowfileRepositoryProperties);
+        putIfNotNull(map, CONTENT_REPO_KEY, contentRepositoryProperties);
+        putIfNotNull(map, PROVENANCE_REPO_KEY, provenanceRepositorySchema);
+        putIfNotNull(map, COMPONENT_STATUS_REPO_KEY, componentStatusRepositoryProperties);
+        putIfNotNull(map, SECURITY_PROPS_KEY, securityProperties);
+        List<ProcessorSchema> processorSchemas = getProcessorSchemas();
+        putListIfNotNull(map, PROCESSORS_KEY, processorSchemas);
+        List<String> validationIssues = getValidationIssues();
+        putListIfNotNull(map, CONNECTIONS_KEY, getConnectionSchemas(processorSchemas, validationIssues));
+        putListIfNotNull(map, REMOTE_PROCESSING_GROUPS_KEY, remoteProcessingGroups);
+        putIfNotNull(map, PROVENANCE_REPORTING_KEY, provenanceReportingProperties);
+        return new ConfigSchema(map, validationIssues);
+    }
+
+    /**
+     * Will replace all characters not in [A-Za-z0-9_] with _
+     * <p>
+     * This has potential for collisions so it will also append numbers as necessary to prevent that
+     *
+     * @param ids  id map of already incremented numbers
+     * @param name the name
+     * @return a unique filesystem-friendly id
+     */
+    public static String getUniqueId(Map<String, Integer> ids, String name) {
+        String baseId = StringUtil.isNullOrEmpty(name) ? EMPTY_NAME : ID_REPLACE_PATTERN.matcher(name).replaceAll("_");
+        String id = baseId;
+        Integer idNum = ids.get(baseId);
+        while (ids.containsKey(id)) {
+            id = baseId + "_" + idNum++;
+        }
+        // Using != on a string comparison here is intentional.  The two will be reference equal iff the body of the while loop was never executed.
+        if (id != baseId) {
+            ids.put(baseId, idNum);
+        }
+        ids.put(id, 2);
+        return id;
+    }
+
+    @Override
+    public int getVersion() {
+        return CONFIG_VERSION;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1.java
new file mode 100644
index 0000000..cdf9290
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1.java
@@ -0,0 +1,104 @@
+/*
+ *
+ *  * 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.commons.schema.v1;
+
+import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
+import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+import org.apache.nifi.minifi.commons.schema.common.StringUtil;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.DEFAULT_FLOWFILE_EXPIRATION;
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.DEFAULT_MAX_QUEUE_DATA_SIZE;
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.DEFAULT_MAX_WORK_QUEUE_SIZE;
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.FLOWFILE_EXPIRATION__KEY;
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.MAX_WORK_QUEUE_DATA_SIZE_KEY;
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.MAX_WORK_QUEUE_SIZE_KEY;
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.QUEUE_PRIORITIZER_CLASS_KEY;
+import static org.apache.nifi.minifi.commons.schema.ConnectionSchema.SOURCE_RELATIONSHIP_NAMES_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.CONNECTIONS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.NAME_KEY;
+
+public class ConnectionSchemaV1 extends BaseSchema implements ConvertableSchema<ConnectionSchema> {
+    public static final String SOURCE_RELATIONSHIP_NAME_KEY = "source relationship name";
+    public static final String DESTINATION_NAME_KEY = "destination name";
+    public static final String SOURCE_NAME_KEY = "source name";
+
+    private String name;
+
+    private String sourceRelationshipName;
+    private String destinationName;
+
+    private String sourceName;
+
+    private Number maxWorkQueueSize = DEFAULT_MAX_WORK_QUEUE_SIZE;
+    private String maxWorkQueueDataSize = DEFAULT_MAX_QUEUE_DATA_SIZE;
+    private String flowfileExpiration = DEFAULT_FLOWFILE_EXPIRATION;
+    private String queuePrioritizerClass;
+
+    public ConnectionSchemaV1(Map map) {
+        name = getRequiredKeyAsType(map, NAME_KEY, String.class, CONNECTIONS_KEY);
+        sourceName = getRequiredKeyAsType(map, SOURCE_NAME_KEY, String.class, CONNECTIONS_KEY);
+        sourceRelationshipName = getRequiredKeyAsType(map, SOURCE_RELATIONSHIP_NAME_KEY, String.class, CONNECTIONS_KEY);
+        destinationName = getRequiredKeyAsType(map, DESTINATION_NAME_KEY, String.class, CONNECTIONS_KEY);
+
+        maxWorkQueueSize = getOptionalKeyAsType(map, MAX_WORK_QUEUE_SIZE_KEY, Number.class, CONNECTIONS_KEY, DEFAULT_MAX_WORK_QUEUE_SIZE);
+        maxWorkQueueDataSize = getOptionalKeyAsType(map, MAX_WORK_QUEUE_DATA_SIZE_KEY, String.class, CONNECTIONS_KEY, DEFAULT_MAX_QUEUE_DATA_SIZE);
+        flowfileExpiration = getOptionalKeyAsType(map, FLOWFILE_EXPIRATION__KEY, String.class, CONNECTIONS_KEY, DEFAULT_FLOWFILE_EXPIRATION);
+        queuePrioritizerClass = getOptionalKeyAsType(map, QUEUE_PRIORITIZER_CLASS_KEY, String.class, CONNECTIONS_KEY, "");
+    }
+
+    @Override
+    public ConnectionSchema convert() {
+        Map<String, Object> map = new HashMap<>();
+        map.put(NAME_KEY, name);
+        if (StringUtil.isNullOrEmpty(sourceRelationshipName)) {
+            map.put(SOURCE_RELATIONSHIP_NAMES_KEY, new ArrayList<>());
+        } else {
+            map.put(SOURCE_RELATIONSHIP_NAMES_KEY, new ArrayList<>(Arrays.asList(sourceRelationshipName)));
+        }
+        map.put(MAX_WORK_QUEUE_SIZE_KEY, maxWorkQueueSize);
+        map.put(MAX_WORK_QUEUE_DATA_SIZE_KEY, maxWorkQueueDataSize);
+        map.put(FLOWFILE_EXPIRATION__KEY, flowfileExpiration);
+        map.put(QUEUE_PRIORITIZER_CLASS_KEY, queuePrioritizerClass);
+        return new ConnectionSchema(map);
+    }
+
+    public String getSourceName() {
+        return sourceName;
+    }
+
+    public String getDestinationName() {
+        return destinationName;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public int getVersion() {
+        return ConfigSchemaV1.CONFIG_VERSION;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1.java b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1.java
new file mode 100644
index 0000000..d64c98b
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/main/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1.java
@@ -0,0 +1,103 @@
+/*
+ *
+ *  * 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.commons.schema.v1;
+
+import org.apache.nifi.minifi.commons.schema.ProcessorSchema;
+import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.ConvertableSchema;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.CLASS_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_AUTO_TERMINATED_RELATIONSHIPS_LIST;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_MAX_CONCURRENT_TASKS;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_PENALIZATION_PERIOD;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_PROPERTIES;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_RUN_DURATION_NANOS;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_YIELD_DURATION;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.IT_IS_NOT_A_VALID_SCHEDULING_STRATEGY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.PENALIZATION_PERIOD_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.PROCESSOR_PROPS_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.RUN_DURATION_NANOS_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.isSchedulingStrategy;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.MAX_CONCURRENT_TASKS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.NAME_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.PROCESSORS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_PERIOD_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_STRATEGY_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.YIELD_PERIOD_KEY;
+
+public class ProcessorSchemaV1 extends BaseSchema implements ConvertableSchema<ProcessorSchema> {
+    private String name;
+    private String processorClass;
+    private String schedulingStrategy;
+    private String schedulingPeriod;
+    private Number maxConcurrentTasks = DEFAULT_MAX_CONCURRENT_TASKS;
+    private String penalizationPeriod = DEFAULT_PENALIZATION_PERIOD;
+    private String yieldPeriod = DEFAULT_YIELD_DURATION;
+    private Number runDurationNanos = DEFAULT_RUN_DURATION_NANOS;
+    private List<String> autoTerminatedRelationshipsList = DEFAULT_AUTO_TERMINATED_RELATIONSHIPS_LIST;
+    private Map<String, Object> properties = DEFAULT_PROPERTIES;
+
+    public ProcessorSchemaV1(Map map) {
+        name = getRequiredKeyAsType(map, NAME_KEY, String.class, PROCESSORS_KEY);
+        processorClass = getRequiredKeyAsType(map, CLASS_KEY, String.class, PROCESSORS_KEY);
+        schedulingStrategy = getRequiredKeyAsType(map, SCHEDULING_STRATEGY_KEY, String.class, PROCESSORS_KEY);
+        if (schedulingStrategy != null && !isSchedulingStrategy(schedulingStrategy)) {
+            addValidationIssue(SCHEDULING_STRATEGY_KEY, PROCESSORS_KEY, IT_IS_NOT_A_VALID_SCHEDULING_STRATEGY);
+        }
+        schedulingPeriod = getRequiredKeyAsType(map, SCHEDULING_PERIOD_KEY, String.class, PROCESSORS_KEY);
+
+        maxConcurrentTasks = getOptionalKeyAsType(map, MAX_CONCURRENT_TASKS_KEY, Number.class, PROCESSORS_KEY, DEFAULT_MAX_CONCURRENT_TASKS);
+        penalizationPeriod = getOptionalKeyAsType(map, PENALIZATION_PERIOD_KEY, String.class, PROCESSORS_KEY, DEFAULT_PENALIZATION_PERIOD);
+        yieldPeriod = getOptionalKeyAsType(map, YIELD_PERIOD_KEY, String.class, PROCESSORS_KEY, DEFAULT_YIELD_DURATION);
+        runDurationNanos = getOptionalKeyAsType(map, RUN_DURATION_NANOS_KEY, Number.class, PROCESSORS_KEY, DEFAULT_RUN_DURATION_NANOS);
+        autoTerminatedRelationshipsList = getOptionalKeyAsType(map, AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY, List.class, PROCESSORS_KEY, DEFAULT_AUTO_TERMINATED_RELATIONSHIPS_LIST);
+        properties = getOptionalKeyAsType(map, PROCESSOR_PROPS_KEY, Map.class, PROCESSORS_KEY, DEFAULT_PROPERTIES);
+    }
+
+    @Override
+    public ProcessorSchema convert() {
+        Map<String, Object> map = new HashMap<>();
+        map.put(NAME_KEY, name);
+        map.put(CLASS_KEY, processorClass);
+        map.put(MAX_CONCURRENT_TASKS_KEY, maxConcurrentTasks);
+        map.put(SCHEDULING_STRATEGY_KEY, schedulingStrategy);
+        map.put(SCHEDULING_PERIOD_KEY, schedulingPeriod);
+        map.put(PENALIZATION_PERIOD_KEY, penalizationPeriod);
+        map.put(YIELD_PERIOD_KEY, yieldPeriod);
+        map.put(RUN_DURATION_NANOS_KEY, runDurationNanos);
+        map.put(AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY, autoTerminatedRelationshipsList);
+        map.put(PROCESSOR_PROPS_KEY, new HashMap<>(properties));
+        return new ProcessorSchema(map);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    @Override
+    public int getVersion() {
+        return ConfigSchemaV1.CONFIG_VERSION;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConfigSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConfigSchemaTest.java b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConfigSchemaTest.java
index 2713d15..afca6e0 100644
--- a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConfigSchemaTest.java
+++ b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConfigSchemaTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.nifi.minifi.commons.schema;
 
-import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
 import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
 import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
 import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
@@ -26,75 +25,53 @@ import org.junit.Test;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.ID_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.fail;
 
 public class ConfigSchemaTest {
     @Test
-    public void testGetUniqueIdEmptySet() {
-        String testId = "testId";
-        assertEquals(testId + "___", ConfigSchema.getUniqueId(new HashMap<>(), testId + "/ $"));
-    }
-
-    @Test
-    public void testConnectionGeneratedIds() {
-        List<Map<String, Object>> listWithKeyValues = getListWithKeyValues(CommonPropertyKeys.NAME_KEY, "test", "test", "test_2");
-
-        // These ids should be honored even though they're last
-        listWithKeyValues.addAll(getListWithKeyValues(CommonPropertyKeys.ID_KEY, "test", "test_2"));
-
-        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.CONNECTIONS_KEY, listWithKeyValues));
-        assertMessageDoesNotExist(configSchema, ConfigSchema.FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_IDS);
-        List<ConnectionSchema> connections = configSchema.getConnections();
-        assertEquals(5, connections.size());
-
-        // Generated unique ids
-        assertEquals("test_3", connections.get(0).getId());
-        assertEquals("test_4", connections.get(1).getId());
-        assertEquals("test_2_2", connections.get(2).getId());
-
-        // Specified ids
-        assertEquals("test", connections.get(3).getId());
-        assertEquals("test_2", connections.get(4).getId());
+    public void testValid() throws IOException, SchemaLoaderException {
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(ConfigSchemaTest.class.getClassLoader().getResourceAsStream("config-minimal-v2.yml"));
+        ConfigSchema configSchema = new ConfigSchema(yamlAsMap);
+        List<String> validationIssues = configSchema.getValidationIssues();
+        assertEquals(0, validationIssues.size());
     }
 
     @Test
-    public void testGetUniqueIdConflicts() {
-        Map<String, Integer> ids = new HashMap<>();
-        assertEquals("test_id", ConfigSchema.getUniqueId(ids, "test/id"));
-        assertEquals("test_id_2", ConfigSchema.getUniqueId(ids, "test$id"));
-        assertEquals("test_id_3", ConfigSchema.getUniqueId(ids, "test$id"));
-        assertEquals("test_id_4", ConfigSchema.getUniqueId(ids, "test$id"));
-        assertEquals("test_id_5", ConfigSchema.getUniqueId(ids, "test$id"));
-        assertEquals("test_id_2_2", ConfigSchema.getUniqueId(ids, "test_id_2"));
+    public void testValidationIssuesFromOlder() throws IOException, SchemaLoaderException {
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(ConfigSchemaTest.class.getClassLoader().getResourceAsStream("config-minimal.yml"));
+        ConfigSchema configSchema = new ConfigSchema(yamlAsMap);
+        List<String> validationIssues = configSchema.getValidationIssues();
+        assertNotEquals(0, validationIssues.size());
     }
 
     @Test
     public void testProcessorDuplicateValidationNegativeCase() {
-        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.PROCESSORS_KEY, getListWithKeyValues(CommonPropertyKeys.ID_KEY, "testId1", "testId2")));
+        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.PROCESSORS_KEY, getListWithKeyValues(ID_KEY, "testId1", "testId2")));
         assertMessageDoesNotExist(configSchema, ConfigSchema.FOUND_THE_FOLLOWING_DUPLICATE_PROCESSOR_IDS);
     }
 
     @Test
     public void testProcessorDuplicateValidationPositiveCase() {
-        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.PROCESSORS_KEY, getListWithKeyValues(CommonPropertyKeys.ID_KEY, "testId1", "testId1")));
+        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.PROCESSORS_KEY, getListWithKeyValues(ID_KEY, "testId1", "testId1")));
         assertMessageDoesExist(configSchema, ConfigSchema.FOUND_THE_FOLLOWING_DUPLICATE_PROCESSOR_IDS);
     }
 
     @Test
     public void testConnectionDuplicateValidationNegativeCase() {
-        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.CONNECTIONS_KEY, getListWithKeyValues(CommonPropertyKeys.ID_KEY, "testId1", "testId2")));
+        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.CONNECTIONS_KEY, getListWithKeyValues(ID_KEY, "testId1", "testId2")));
         assertMessageDoesNotExist(configSchema, ConfigSchema.FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_IDS);
     }
 
     @Test
     public void testConnectionDuplicateValidationPositiveCase() {
-        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.CONNECTIONS_KEY, getListWithKeyValues(CommonPropertyKeys.ID_KEY, "testId1", "testId1")));
+        ConfigSchema configSchema = new ConfigSchema(Collections.singletonMap(CommonPropertyKeys.CONNECTIONS_KEY, getListWithKeyValues(ID_KEY, "testId1", "testId1")));
         assertMessageDoesExist(configSchema, ConfigSchema.FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_IDS);
     }
 
@@ -111,8 +88,8 @@ public class ConfigSchemaTest {
     }
 
     @Test
-    public void testInvalidSourceAndDestinationNames() throws IOException, SchemaLoaderException {
-        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(ConfigSchemaTest.class.getClassLoader().getResourceAsStream("config-minimal.yml"));
+    public void testInvalidSourceAndDestinationIds() throws IOException, SchemaLoaderException {
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(ConfigSchemaTest.class.getClassLoader().getResourceAsStream("config-minimal-v2.yml"));
         List<Map<String, Object>> connections = (List<Map<String, Object>>) yamlAsMap.get(CommonPropertyKeys.CONNECTIONS_KEY);
         assertEquals(1, connections.size());
 
@@ -120,15 +97,14 @@ public class ConfigSchemaTest {
         String fakeDestination = "fakeDestination";
 
         Map<String, Object> connection = connections.get(0);
-        connection.put(ConnectionSchema.SOURCE_NAME_KEY, fakeSource);
-        connection.put(ConnectionSchema.DESTINATION_NAME_KEY, fakeDestination);
+        connection.put(ConnectionSchema.SOURCE_ID_KEY, fakeSource);
+        connection.put(ConnectionSchema.DESTINATION_ID_KEY, fakeDestination);
 
         ConfigSchema configSchema = new ConfigSchema(yamlAsMap);
         List<String> validationIssues = configSchema.getValidationIssues();
-        assertEquals(3, validationIssues.size());
-        assertEquals(ConfigSchema.CONNECTIONS_REFER_TO_PROCESSOR_NAMES_THAT_DONT_EXIST + fakeDestination + ", " + fakeSource, validationIssues.get(0));
-        assertEquals(BaseSchema.getIssueText(ConnectionSchema.SOURCE_ID_KEY, CommonPropertyKeys.CONNECTIONS_KEY, BaseSchema.IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED), validationIssues.get(1));
-        assertEquals(BaseSchema.getIssueText(ConnectionSchema.DESTINATION_ID_KEY, CommonPropertyKeys.CONNECTIONS_KEY, BaseSchema.IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED), validationIssues.get(2));
+        assertEquals(2, validationIssues.size());
+        assertEquals(ConfigSchema.CONNECTION_WITH_ID + connection.get(ID_KEY) + ConfigSchema.HAS_INVALID_DESTINATION_ID + fakeDestination, validationIssues.get(0));
+        assertEquals(ConfigSchema.CONNECTION_WITH_ID + connection.get(ID_KEY) + ConfigSchema.HAS_INVALID_SOURCE_ID + fakeSource, validationIssues.get(1));
     }
 
     public static List<Map<String, Object>> getListWithNames(String... names) {

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConnectionSchemaTest.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConnectionSchemaTest.java b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConnectionSchemaTest.java
index 0d37df9..61f9e96 100644
--- a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConnectionSchemaTest.java
+++ b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/ConnectionSchemaTest.java
@@ -30,8 +30,6 @@ import java.util.List;
 import java.util.Map;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 
 public class ConnectionSchemaTest {
     private String testId;
@@ -41,8 +39,6 @@ public class ConnectionSchemaTest {
     private String testSourceRelationShip2;
     private List<String> testSourceRelationships;
     private String testDestinationId;
-    private String testSourceName;
-    private String testDestinationName;
     private int testMaxWorkQueueSize;
     private String testMaxWorkQueueDataSize;
     private String testFlowfileExpiration;
@@ -57,8 +53,6 @@ public class ConnectionSchemaTest {
         testSourceRelationShip2 = "testSourceRelationShip2";
         testSourceRelationships = Arrays.asList(testSourceRelationShip1, testSourceRelationShip2);
         testDestinationId = "testDestinationId";
-        testSourceName = "testSourceName";
-        testDestinationName = "testDestinationName";
         testMaxWorkQueueSize = 55;
         testMaxWorkQueueDataSize = "testMaxWorkQueueDataSize";
         testFlowfileExpiration = "testFlowfileExpiration";
@@ -71,7 +65,7 @@ public class ConnectionSchemaTest {
 
     private ConnectionSchema createSchema(Map<String, Object> map, int expectedValidationIssues) {
         ConnectionSchema connectionSchema = new ConnectionSchema(map);
-        assertEquals(expectedValidationIssues, connectionSchema.getValidationIssues().size());
+        assertEquals(connectionSchema.getValidationIssues().toString(), expectedValidationIssues, connectionSchema.getValidationIssues().size());
         return connectionSchema;
     }
 
@@ -82,8 +76,6 @@ public class ConnectionSchemaTest {
         map.put(ConnectionSchema.SOURCE_ID_KEY, testSourceId);
         map.put(ConnectionSchema.SOURCE_RELATIONSHIP_NAMES_KEY, testSourceRelationships);
         map.put(ConnectionSchema.DESTINATION_ID_KEY, testDestinationId);
-        map.put(ConnectionSchema.SOURCE_NAME_KEY, testSourceName);
-        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);
@@ -140,64 +132,13 @@ public class ConnectionSchemaTest {
     }
 
     @Test
-    public void testNoSourceNameWithId() {
-        ConnectionSchema schema = createSchema(0);
-        assertNull(schema.getSourceName());
-        assertFalse(schema.toMap().containsKey(ConnectionSchema.SOURCE_NAME_KEY));
-    }
-
-    @Test
-    public void testSourceNameNoId() {
-        Map<String, Object> map = createMap();
-        map.remove(ConnectionSchema.SOURCE_ID_KEY);
-        ConnectionSchema schema = createSchema(map, 1);
-        assertEquals("", schema.getSourceId());
-        assertEquals(testSourceName, schema.getSourceName());
-        Map<String, Object> outputMap = schema.toMap();
-        assertEquals(schema.getSourceId(), outputMap.get(ConnectionSchema.SOURCE_ID_KEY));
-        assertFalse(schema.toMap().containsKey(ConnectionSchema.SOURCE_NAME_KEY));
-    }
-
-    @Test
-    public void testNoSourceIdOrSourceName() {
-        Map<String, Object> map = createMap();
-        map.remove(ConnectionSchema.SOURCE_ID_KEY);
-        map.remove(ConnectionSchema.SOURCE_NAME_KEY);
-        ConnectionSchema schema = createSchema(map, 2);
-        assertEquals("", schema.getSourceId());
-        assertNull(schema.getSourceName());
-        Map<String, Object> outputMap = schema.toMap();
-        assertEquals(schema.getSourceId(), outputMap.get(ConnectionSchema.SOURCE_ID_KEY));
-        assertFalse(schema.toMap().containsKey(ConnectionSchema.SOURCE_NAME_KEY));
-    }
-
-    @Test
-    public void testSourceRelationShipNames() {
+    public void testSourceRelationshipNames() {
         ConnectionSchema schema = createSchema(0);
         assertEquals(testSourceRelationships, schema.getSourceRelationshipNames());
         assertEquals(schema.getSourceRelationshipNames(), schema.toMap().get(ConnectionSchema.SOURCE_RELATIONSHIP_NAMES_KEY));
     }
 
     @Test
-    public void testSourceRelationshipName() {
-        Map<String, Object> map = createMap();
-        map.remove(ConnectionSchema.SOURCE_RELATIONSHIP_NAMES_KEY);
-        map.put(ConnectionSchema.SOURCE_RELATIONSHIP_NAME_KEY, testSourceRelationShip1);
-        ConnectionSchema schema = createSchema(map, 0);
-        assertEquals(new ArrayList<>(Arrays.asList(testSourceRelationShip1)), schema.getSourceRelationshipNames());
-        assertEquals(schema.getSourceRelationshipNames(), schema.toMap().get(ConnectionSchema.SOURCE_RELATIONSHIP_NAMES_KEY));
-    }
-
-    @Test
-    public void testSourceRelationshipNameAndSourceRelationshipNames() {
-        Map<String, Object> map = createMap();
-        map.put(ConnectionSchema.SOURCE_RELATIONSHIP_NAME_KEY, testSourceRelationShip1);
-        ConnectionSchema schema = createSchema(map, 1);
-        assertEquals(testSourceRelationships, schema.getSourceRelationshipNames());
-        assertEquals(schema.getSourceRelationshipNames(), schema.toMap().get(ConnectionSchema.SOURCE_RELATIONSHIP_NAMES_KEY));
-    }
-
-    @Test
     public void testNoSourceRelationshipNames() {
         Map<String, Object> map = createMap();
         map.remove(ConnectionSchema.SOURCE_RELATIONSHIP_NAMES_KEY);
@@ -223,35 +164,6 @@ public class ConnectionSchemaTest {
     }
 
     @Test
-    public void testDestinationNameWithId() {
-        ConnectionSchema schema = createSchema(0);
-        assertNull(schema.getDestinationName());
-        assertFalse(schema.toMap().containsKey(ConnectionSchema.DESTINATION_NAME_KEY));
-    }
-
-    @Test
-    public void testDestinationNameNoId() {
-        Map<String, Object> map = createMap();
-        map.remove(ConnectionSchema.DESTINATION_ID_KEY);
-        ConnectionSchema schema = createSchema(map, 1);
-        assertEquals(testDestinationName, schema.getDestinationName());
-        assertFalse(schema.toMap().containsKey(ConnectionSchema.DESTINATION_NAME_KEY));
-    }
-
-    @Test
-    public void testNoDestinationNameNoId() {
-        Map<String, Object> map = createMap();
-        map.remove(ConnectionSchema.DESTINATION_ID_KEY);
-        map.remove(ConnectionSchema.DESTINATION_NAME_KEY);
-        ConnectionSchema schema = createSchema(map, 2);
-        assertEquals("", schema.getDestinationId());
-        assertNull(schema.getDestinationName());
-        Map<String, Object> outputMap = schema.toMap();
-        assertEquals(schema.getDestinationId(), outputMap.get(ConnectionSchema.DESTINATION_ID_KEY));
-        assertFalse(outputMap.containsKey(ConnectionSchema.DESTINATION_NAME_KEY));
-    }
-
-    @Test
     public void testMaxWorkQueueSize() {
         ConnectionSchema schema = createSchema(0);
         assertEquals(testMaxWorkQueueSize, schema.getMaxWorkQueueSize());

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoaderTest.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoaderTest.java b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoaderTest.java
index c1b63d0..5b602ac 100644
--- a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoaderTest.java
+++ b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/serialization/SchemaLoaderTest.java
@@ -21,6 +21,7 @@ import org.apache.nifi.minifi.commons.schema.ConfigSchema;
 import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
 import org.apache.nifi.minifi.commons.schema.ProcessorSchema;
 import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
+import org.apache.nifi.minifi.commons.schema.v1.ConfigSchemaV1;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -30,6 +31,7 @@ import java.util.Map;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 public class SchemaLoaderTest {
     @Test
@@ -49,19 +51,31 @@ public class SchemaLoaderTest {
     @Test
     public void testMinimalConfigV1Version() throws IOException, SchemaLoaderException {
         Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(SchemaLoaderTest.class.getClassLoader().getResourceAsStream("config-minimal.yml"));
-        yamlAsMap.put(ConfigSchema.VERSION, "1");
+        yamlAsMap.put(ConfigSchema.VERSION, ConfigSchemaV1.CONFIG_VERSION);
         ConfigSchema configSchema = SchemaLoader.loadConfigSchemaFromYaml(yamlAsMap);
         validateMinimalConfigVersion1Parse(configSchema);
     }
 
     @Test
     public void testMinimalConfigV2Version() throws IOException, SchemaLoaderException {
-        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(SchemaLoaderTest.class.getClassLoader().getResourceAsStream("config-minimal.yml"));
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(SchemaLoaderTest.class.getClassLoader().getResourceAsStream("config-minimal-v2.yml"));
         yamlAsMap.put(ConfigSchema.VERSION, ConfigSchema.CONFIG_VERSION);
         ConfigSchema configSchema = SchemaLoader.loadConfigSchemaFromYaml(yamlAsMap);
         validateMinimalConfigVersion1Parse(configSchema);
     }
 
+    @Test
+    public void testUnsupportedVersion() throws IOException, SchemaLoaderException {
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(SchemaLoaderTest.class.getClassLoader().getResourceAsStream("config-minimal-v2.yml"));
+        yamlAsMap.put(ConfigSchema.VERSION, "9999999");
+        try {
+            SchemaLoader.loadConfigSchemaFromYaml(yamlAsMap);
+            fail();
+        } catch (SchemaLoaderException e) {
+            assertEquals("YAML configuration version 9999999 not supported.  Supported versions: 1, 2", e.getMessage());
+        }
+    }
+
     private void validateMinimalConfigVersion1Parse(ConfigSchema configSchema) {
         assertTrue(configSchema instanceof ConfigSchema);
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1Test.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1Test.java b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1Test.java
new file mode 100644
index 0000000..56a8103
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConfigSchemaV1Test.java
@@ -0,0 +1,94 @@
+/*
+ *
+ *  * 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.commons.schema.v1;
+
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.ConfigSchemaTest;
+import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
+import org.apache.nifi.minifi.commons.schema.common.BaseSchema;
+import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
+import org.apache.nifi.minifi.commons.schema.exception.SchemaLoaderException;
+import org.apache.nifi.minifi.commons.schema.serialization.SchemaLoader;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.NAME_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class ConfigSchemaV1Test {
+    @Test
+    public void testValid() throws IOException, SchemaLoaderException {
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(ConfigSchemaTest.class.getClassLoader().getResourceAsStream("config-minimal.yml"));
+        ConfigSchema configSchema = new ConfigSchemaV1(yamlAsMap).convert();
+        List<String> validationIssues = configSchema.getValidationIssues();
+        assertEquals(0, validationIssues.size());
+    }
+    @Test
+    public void testValidationIssuesFromNewer() throws IOException, SchemaLoaderException {
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(ConfigSchemaTest.class.getClassLoader().getResourceAsStream("config-minimal-v2.yml"));
+        ConfigSchema configSchema = new ConfigSchemaV1(yamlAsMap).convert();
+        List<String> validationIssues = configSchema.getValidationIssues();
+        assertNotEquals(0, validationIssues.size());
+    }
+
+    @Test
+    public void testInvalidSourceAndDestinationNames() throws IOException, SchemaLoaderException {
+        Map<String, Object> yamlAsMap = SchemaLoader.loadYamlAsMap(ConfigSchemaTest.class.getClassLoader().getResourceAsStream("config-minimal.yml"));
+        List<Map<String, Object>> connections = (List<Map<String, Object>>) yamlAsMap.get(CommonPropertyKeys.CONNECTIONS_KEY);
+        assertEquals(1, connections.size());
+
+        String fakeSource = "fakeSource";
+        String fakeDestination = "fakeDestination";
+
+        Map<String, Object> connection = connections.get(0);
+        connection.put(ConnectionSchemaV1.SOURCE_NAME_KEY, fakeSource);
+        connection.put(ConnectionSchemaV1.DESTINATION_NAME_KEY, fakeDestination);
+
+        ConfigSchema configSchema = new ConfigSchemaV1(yamlAsMap).convert();
+        List<String> validationIssues = configSchema.getValidationIssues();
+        assertEquals(4, validationIssues.size());
+        assertEquals(BaseSchema.getIssueText(ConnectionSchema.DESTINATION_ID_KEY, CommonPropertyKeys.CONNECTIONS_KEY, BaseSchema.IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED), validationIssues.get(0));
+        assertEquals(BaseSchema.getIssueText(ConnectionSchema.SOURCE_ID_KEY, CommonPropertyKeys.CONNECTIONS_KEY, BaseSchema.IT_WAS_NOT_FOUND_AND_IT_IS_REQUIRED), validationIssues.get(1));
+        assertEquals(ConfigSchemaV1.CONNECTION_WITH_NAME + connection.get(NAME_KEY) + ConfigSchemaV1.HAS_INVALID_DESTINATION_NAME + fakeDestination, validationIssues.get(2));
+        assertEquals(ConfigSchemaV1.CONNECTION_WITH_NAME + connection.get(NAME_KEY) + ConfigSchemaV1.HAS_INVALID_SOURCE_NAME + fakeSource, validationIssues.get(3));
+    }
+
+    @Test
+    public void testGetUniqueIdConflicts() {
+        Map<String, Integer> ids = new HashMap<>();
+        assertEquals("test_id", ConfigSchemaV1.getUniqueId(ids, "test/id"));
+        assertEquals("test_id_2", ConfigSchemaV1.getUniqueId(ids, "test$id"));
+        assertEquals("test_id_3", ConfigSchemaV1.getUniqueId(ids, "test$id"));
+        assertEquals("test_id_4", ConfigSchemaV1.getUniqueId(ids, "test$id"));
+        assertEquals("test_id_5", ConfigSchemaV1.getUniqueId(ids, "test$id"));
+        assertEquals("test_id_2_2", ConfigSchemaV1.getUniqueId(ids, "test_id_2"));
+    }
+
+    @Test
+    public void testGetUniqueIdEmptySet() {
+        String testId = "testId";
+        assertEquals(testId + "___", ConfigSchemaV1.getUniqueId(new HashMap<>(), testId + "/ $"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1Test.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1Test.java b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1Test.java
new file mode 100644
index 0000000..e9acf4a
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ConnectionSchemaV1Test.java
@@ -0,0 +1,192 @@
+/*
+ *
+ *  * 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.commons.schema.v1;
+
+import org.apache.nifi.minifi.commons.schema.ConfigSchema;
+import org.apache.nifi.minifi.commons.schema.ConnectionSchema;
+import org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.nifi.minifi.commons.schema.ConfigSchemaTest.assertMessageDoesNotExist;
+import static org.apache.nifi.minifi.commons.schema.ConfigSchemaTest.getListWithKeyValues;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class ConnectionSchemaV1Test {
+    private String testName;
+    private String testSourceRelationship;
+    private String testSourceName;
+    private String testDestinationName;
+    private int testMaxWorkQueueSize;
+    private String testMaxWorkQueueDataSize;
+    private String testFlowfileExpiration;
+    private String testQueuePrioritizerClass;
+
+    @Before
+    public void setup() {
+        testName = "testName";
+        testSourceRelationship = "testSourceRelationship";
+        testSourceName = "testSourceName";
+        testDestinationName = "testDestinationName";
+        testMaxWorkQueueSize = 55;
+        testMaxWorkQueueDataSize = "testMaxWorkQueueDataSize";
+        testFlowfileExpiration = "testFlowfileExpiration";
+        testQueuePrioritizerClass = "testQueuePrioritizerClass";
+    }
+
+    private ConnectionSchemaV1 createSchema(int expectedValidationIssues) {
+        return createSchema(createMap(), expectedValidationIssues);
+    }
+
+    private ConnectionSchemaV1 createSchema(Map<String, Object> map, int expectedValidationIssues) {
+        ConnectionSchemaV1 connectionSchema = new ConnectionSchemaV1(map);
+        assertEquals(expectedValidationIssues, connectionSchema.getValidationIssues().size());
+        return connectionSchema;
+    }
+
+    private Map<String, Object> createMap() {
+        Map<String, Object> map = new HashMap<>();
+        map.put(CommonPropertyKeys.NAME_KEY, testName);
+        map.put(ConnectionSchemaV1.SOURCE_RELATIONSHIP_NAME_KEY, testSourceRelationship);
+        map.put(ConnectionSchemaV1.SOURCE_NAME_KEY, testSourceName);
+        map.put(ConnectionSchemaV1.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);
+        return map;
+    }
+
+    @Test
+    public void testName() {
+        ConnectionSchemaV1 schema = createSchema(0);
+        assertEquals(testName, schema.getName());
+        assertEquals(schema.getName(), schema.convert().getName());
+    }
+
+    @Test
+    public void testNoName() {
+        Map<String, Object> map = createMap();
+        map.remove(CommonPropertyKeys.NAME_KEY);
+        ConnectionSchemaV1 schema = createSchema(map, 1);
+        assertNull(schema.getName());
+        assertEquals("", schema.convert().getName());
+    }
+
+    @Test
+    public void testSourceRelationShipName() {
+        ConnectionSchemaV1 schema = createSchema(0);
+        List<String> sourceRelationshipNames = schema.convert().getSourceRelationshipNames();
+        assertEquals(1, sourceRelationshipNames.size());
+        assertEquals(testSourceRelationship, sourceRelationshipNames.get(0));
+    }
+
+    @Test
+    public void testNoSourceRelationshipName() {
+        Map<String, Object> map = createMap();
+        map.remove(ConnectionSchemaV1.SOURCE_RELATIONSHIP_NAME_KEY);
+        ConnectionSchemaV1 schema = createSchema(map, 1);
+        List<String> sourceRelationshipNames = schema.convert().getSourceRelationshipNames();
+        assertEquals(0, sourceRelationshipNames.size());
+    }
+
+    @Test
+    public void testDestinationName() {
+        assertEquals(testDestinationName, createSchema(0).getDestinationName());
+    }
+
+    @Test
+    public void testNoDestinationName() {
+        Map<String, Object> map = createMap();
+        map.remove(ConnectionSchemaV1.DESTINATION_NAME_KEY);
+        assertNull(createSchema(map, 1).getDestinationName());
+    }
+
+    @Test
+    public void testMaxWorkQueueSize() {
+        assertEquals(testMaxWorkQueueSize, createSchema(0).convert().getMaxWorkQueueSize());
+    }
+
+    @Test
+    public void testNoMaxWorkQueueSize() {
+        Map<String, Object> map = createMap();
+        map.remove(ConnectionSchema.MAX_WORK_QUEUE_SIZE_KEY);
+        assertEquals(ConnectionSchema.DEFAULT_MAX_WORK_QUEUE_SIZE, createSchema(map, 0).convert().getMaxWorkQueueSize());
+    }
+
+    @Test
+    public void testMaxWorkQueueDataSize() {
+        assertEquals(testMaxWorkQueueDataSize, createSchema(0).convert().getMaxWorkQueueDataSize());
+    }
+
+    @Test
+    public void testNoMaxWorkQueueDataSize() {
+        Map<String, Object> map = createMap();
+        map.remove(ConnectionSchema.MAX_WORK_QUEUE_DATA_SIZE_KEY);
+        assertEquals(ConnectionSchema.DEFAULT_MAX_QUEUE_DATA_SIZE, createSchema(map, 0).convert().getMaxWorkQueueDataSize());
+    }
+
+    @Test
+    public void testFlowFileExpiration() {
+        assertEquals(testFlowfileExpiration, createSchema(0).convert().getFlowfileExpiration());
+    }
+
+    @Test
+    public void testNoFlowFileExpiration() {
+        Map<String, Object> map = createMap();
+        map.remove(ConnectionSchema.FLOWFILE_EXPIRATION__KEY);
+        assertEquals(ConnectionSchema.DEFAULT_FLOWFILE_EXPIRATION, createSchema(map, 0).convert().getFlowfileExpiration());
+    }
+
+    @Test
+    public void testQueuePrioritizer() {
+        assertEquals(testQueuePrioritizerClass, createSchema(0).convert().getQueuePrioritizerClass());
+    }
+
+    @Test
+    public void testNoQueuePrioritizer() {
+        Map<String, Object> map = createMap();
+        map.remove(ConnectionSchema.QUEUE_PRIORITIZER_CLASS_KEY);
+        assertEquals("", createSchema(map, 0).convert().getQueuePrioritizerClass());
+    }
+
+    @Test
+    public void testConnectionGeneratedIds() {
+        List<Map<String, Object>> listWithKeyValues = getListWithKeyValues(CommonPropertyKeys.NAME_KEY, "test", "test", "test_2", "test", "test_2");
+
+        ConfigSchema configSchema = new ConfigSchemaV1(Collections.singletonMap(CommonPropertyKeys.CONNECTIONS_KEY, listWithKeyValues)).convert();
+        assertMessageDoesNotExist(configSchema, ConfigSchema.FOUND_THE_FOLLOWING_DUPLICATE_CONNECTION_IDS);
+        List<ConnectionSchema> connections = configSchema.getConnections();
+        assertEquals(5, connections.size());
+
+        // Generated unique ids
+        assertEquals("test", connections.get(0).getId());
+        assertEquals("test_2", connections.get(1).getId());
+        assertEquals("test_2_2", connections.get(2).getId());
+        assertEquals("test_3", connections.get(3).getId());
+        assertEquals("test_2_3", connections.get(4).getId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1Test.java
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1Test.java b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1Test.java
new file mode 100644
index 0000000..3668c0e
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/test/java/org/apache/nifi/minifi/commons/schema/v1/ProcessorSchemaV1Test.java
@@ -0,0 +1,241 @@
+/*
+ *
+ *  * 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.commons.schema.v1;
+
+import org.apache.nifi.scheduling.SchedulingStrategy;
+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.Map;
+
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.CLASS_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_AUTO_TERMINATED_RELATIONSHIPS_LIST;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_MAX_CONCURRENT_TASKS;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_PENALIZATION_PERIOD;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_PROPERTIES;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_RUN_DURATION_NANOS;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.DEFAULT_YIELD_DURATION;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.PENALIZATION_PERIOD_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.PROCESSOR_PROPS_KEY;
+import static org.apache.nifi.minifi.commons.schema.ProcessorSchema.RUN_DURATION_NANOS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.MAX_CONCURRENT_TASKS_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.NAME_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_PERIOD_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.SCHEDULING_STRATEGY_KEY;
+import static org.apache.nifi.minifi.commons.schema.common.CommonPropertyKeys.YIELD_PERIOD_KEY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class ProcessorSchemaV1Test {
+    private String testName;
+    private String testProcessorClass;
+    private String testSchedulingStrategy;
+    private String testSchedulingPeriod;
+    private int testMaxConcurrentTasks;
+    private String testPenalizationPeriod;
+    private String testYieldPeriod;
+    private int testRunDurationNanos;
+    private String testAutoTerminatedRelationship1;
+    private String testAutoTerminatedRelationship2;
+    private List<String> testAutoTerminatedRelationships;
+    private String testKey1;
+    private String testValue1;
+    private String testKey2;
+    private String testValue2;
+    private Map<String, Object> testProperties;
+
+    @Before
+    public void setup() {
+        testName = "testName";
+        testProcessorClass = "testProcessorClass";
+        testSchedulingStrategy = SchedulingStrategy.PRIMARY_NODE_ONLY.toString();
+        testSchedulingPeriod = "testSchedulingPeriod";
+        testMaxConcurrentTasks = 55;
+        testPenalizationPeriod = "testPenalizationPeriod";
+        testYieldPeriod = "testYieldPeriod";
+        testRunDurationNanos = 125;
+        testAutoTerminatedRelationship1 = "testAutoTerminatedRelationship1";
+        testAutoTerminatedRelationship2 = "testAutoTerminatedRelationship2";
+        testAutoTerminatedRelationships = new ArrayList<>(Arrays.asList(testAutoTerminatedRelationship1, testAutoTerminatedRelationship2));
+        testKey1 = "testKey1";
+        testValue1 = "testValue1";
+        testKey2 = "testKey2";
+        testValue2 = "testValue2";
+        testProperties = new HashMap<>();
+        testProperties.put(testKey1, testValue1);
+        testProperties.put(testKey2, testValue2);
+    }
+
+    private ProcessorSchemaV1 createSchema(int expectedValidationIssues) {
+        return createSchema(createMap(), expectedValidationIssues);
+    }
+
+    private ProcessorSchemaV1 createSchema(Map<String, Object> map, int expectedValidationIssues) {
+        ProcessorSchemaV1 processorSchemaV1 = new ProcessorSchemaV1(map);
+        assertEquals(expectedValidationIssues, processorSchemaV1.getValidationIssues().size());
+        return processorSchemaV1;
+    }
+
+    private Map<String, Object> createMap() {
+        Map<String, Object> map = new HashMap<>();
+        map.put(NAME_KEY, testName);
+        map.put(CLASS_KEY, testProcessorClass);
+        map.put(SCHEDULING_STRATEGY_KEY, testSchedulingStrategy);
+        map.put(SCHEDULING_PERIOD_KEY, testSchedulingPeriod);
+        map.put(MAX_CONCURRENT_TASKS_KEY, testMaxConcurrentTasks);
+        map.put(PENALIZATION_PERIOD_KEY, testPenalizationPeriod);
+        map.put(YIELD_PERIOD_KEY, testYieldPeriod);
+        map.put(RUN_DURATION_NANOS_KEY, testRunDurationNanos);
+        map.put(AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY, testAutoTerminatedRelationships);
+        map.put(PROCESSOR_PROPS_KEY, testProperties);
+        return map;
+    }
+
+    @Test
+    public void testName() {
+        assertEquals(testName, createSchema(0).convert().getName());
+    }
+
+    @Test
+    public void testNoName() {
+        Map<String, Object> map = createMap();
+        map.remove(NAME_KEY);
+        assertNull(createSchema(map, 1).getName());
+    }
+
+    @Test
+    public void testProcessorClass() {
+        assertEquals(testProcessorClass, createSchema(0).convert().getProcessorClass());
+    }
+
+    @Test
+    public void testNoProcessorClass() {
+        Map<String, Object> map = createMap();
+        map.remove(CLASS_KEY);
+        assertNull(createSchema(map, 1).convert().getProcessorClass());
+    }
+
+    @Test
+    public void testSchedulingStrategy() {
+        assertEquals(testSchedulingStrategy, createSchema(0).convert().getSchedulingStrategy());
+    }
+
+    @Test
+    public void testNoSchedulingStrategy() {
+        Map<String, Object> map = createMap();
+        map.remove(SCHEDULING_STRATEGY_KEY);
+        assertNull(createSchema(map, 1).convert().getSchedulingStrategy());
+    }
+
+    @Test
+    public void testInvalidSchedulingStrategy() {
+        testSchedulingStrategy = "fake strategy";
+        assertEquals(testSchedulingStrategy, createSchema(1).convert().getSchedulingStrategy());
+    }
+
+    @Test
+    public void testSchedulingPeriod() {
+        assertEquals(testSchedulingPeriod, createSchema(0).convert().getSchedulingPeriod());
+    }
+
+    @Test
+    public void testNoSchedulingPeriod() {
+        Map<String, Object> map = createMap();
+        map.remove(SCHEDULING_PERIOD_KEY);
+        assertNull(createSchema(map, 1).convert().getSchedulingPeriod());
+    }
+
+    @Test
+    public void testMaxConcurrentTasks() {
+        assertEquals(testMaxConcurrentTasks, createSchema(0).convert().getMaxConcurrentTasks());
+    }
+
+    @Test
+    public void testNoMaxConcurrentTasks() {
+        Map<String, Object> map = createMap();
+        map.remove(MAX_CONCURRENT_TASKS_KEY);
+        assertEquals(DEFAULT_MAX_CONCURRENT_TASKS, createSchema(map, 0).convert().getMaxConcurrentTasks());
+    }
+
+    @Test
+    public void testPenalizationPeriod() {
+        assertEquals(testPenalizationPeriod, createSchema(0).convert().getPenalizationPeriod());
+    }
+
+    @Test
+    public void testNoPenalizationPeriod() {
+        Map<String, Object> map = createMap();
+        map.remove(PENALIZATION_PERIOD_KEY);
+        assertEquals(DEFAULT_PENALIZATION_PERIOD, createSchema(map, 0).convert().getPenalizationPeriod());
+    }
+
+    @Test
+    public void testYieldPeriod() {
+        assertEquals(testYieldPeriod, createSchema(0).convert().getYieldPeriod());
+    }
+
+    @Test
+    public void testNoYieldPeriod() {
+        Map<String, Object> map = createMap();
+        map.remove(YIELD_PERIOD_KEY);
+        assertEquals(DEFAULT_YIELD_DURATION, createSchema(map, 0).convert().getYieldPeriod());
+    }
+
+    @Test
+    public void testRunDurationNanos() {
+        assertEquals(testRunDurationNanos, createSchema(0).convert().getRunDurationNanos());
+    }
+
+    @Test
+    public void testNoRunDurationNanos() {
+        Map<String, Object> map = createMap();
+        map.remove(RUN_DURATION_NANOS_KEY);
+        assertEquals(DEFAULT_RUN_DURATION_NANOS, createSchema(map, 0).convert().getRunDurationNanos());
+    }
+
+    @Test
+    public void testAutoTerminatedRelationships() {
+        assertEquals(testAutoTerminatedRelationships, createSchema(0).convert().getAutoTerminatedRelationshipsList());
+    }
+
+    @Test
+    public void testNoAutoTerminatedRelationships() {
+        Map<String, Object> map = createMap();
+        map.remove(AUTO_TERMINATED_RELATIONSHIPS_LIST_KEY);
+        assertEquals(DEFAULT_AUTO_TERMINATED_RELATIONSHIPS_LIST, createSchema(map, 0).convert().getAutoTerminatedRelationshipsList());
+    }
+
+    @Test
+    public void testProperties() {
+        assertEquals(testProperties, createSchema(0).convert().getProperties());
+    }
+
+    @Test
+    public void testNoProperties() {
+        Map<String, Object> map = createMap();
+        map.remove(PROCESSOR_PROPS_KEY);
+        assertEquals(DEFAULT_PROPERTIES, createSchema(map, 0).convert().getProperties());
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-commons/minifi-commons-schema/src/test/resources/config-minimal-v2.yml
----------------------------------------------------------------------
diff --git a/minifi-commons/minifi-commons-schema/src/test/resources/config-minimal-v2.yml b/minifi-commons/minifi-commons-schema/src/test/resources/config-minimal-v2.yml
new file mode 100644
index 0000000..2472a32
--- /dev/null
+++ b/minifi-commons/minifi-commons-schema/src/test/resources/config-minimal-v2.yml
@@ -0,0 +1,38 @@
+# 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.
+
+# This is a minimal V2 config.
+MiNiFi Config Version: 2
+Flow Controller:
+    name: MiNiFi Flow
+
+# When running the Flow (not just doing the transform) these processors will be invalid due to not having the necesary properties/auto-terminated relationships
+Processors:
+    - id: bcd
+      class: org.apache.nifi.processors.standard.TailFile
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 10 sec
+    - id: cde
+      class: org.apache.nifi.processors.standard.PutFile
+      max concurrent tasks: 1
+      scheduling strategy: TIMER_DRIVEN
+      scheduling period: 0 sec
+
+Connections:
+    - id: abc
+      source id: bcd
+      source relationship names:
+      - success
+      destination id: cde

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-docs/src/main/markdown/System_Admin_Guide.md
----------------------------------------------------------------------
diff --git a/minifi-docs/src/main/markdown/System_Admin_Guide.md b/minifi-docs/src/main/markdown/System_Admin_Guide.md
index 3f3d414..38c9588 100644
--- a/minifi-docs/src/main/markdown/System_Admin_Guide.md
+++ b/minifi-docs/src/main/markdown/System_Admin_Guide.md
@@ -237,6 +237,18 @@ available at https://nifi.apache.org/minifi/minifi-toolkit.html.
 NOTE: Note that values for periods of time and data sizes must include the unit of measure,
 for example "10 sec" or "10 MB", not simply "10".
 
+## Versioning
+
+The "MiNiFi Config Version" property is used to indicate to the configuration parser which version of the config file it is looking at.  If the property is empty or missing, version 1 is assumed.
+
+The MiNiFi Toolkit Converter is capable of parsing previous versions (possibly subject to a future deprecation policy) and writing out the current version.  It can also validate that a given config file
+parses and upconverts to the current version without issue.
+
+### Version 1 -> Version 2 changes
+
+1. Use ids instead of names for processors, connections.
+2. Allow multiple source relationships for connections.
+
 
 ## Flow Controller
 

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml
----------------------------------------------------------------------
diff --git a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml
index 89b1011..44d8d4e 100644
--- a/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml
+++ b/minifi-nar-bundles/minifi-framework-bundle/minifi-framework/minifi-resources/src/main/resources/conf/config.yml
@@ -13,58 +13,48 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+MiNiFi Config Version: 2
 Flow Controller:
-    name: MiNiFi Flow
-    comment:
-
+  name: MiNiFi Flow
+  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
-
+  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
-
+  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
-
+  content claim max appendable size: 10 MB
+  content claim max flow files: 100
+  always sync: false
 Provenance Repository:
-    provenance rollover time: 1 min
-
+  provenance rollover time: 1 min
 Component Status Repository:
-    buffer size: 1440
-    snapshot frequency: 1 min
-
+  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:
-
-Connections:
-
-Remote Processing Groups:
-
-Provenance Reporting:
\ No newline at end of file
+  keystore: ''
+  keystore type: ''
+  keystore password: ''
+  key password: ''
+  truststore: ''
+  truststore type: ''
+  truststore password: ''
+  ssl protocol: ''
+  Sensitive Props:
+    key: ''
+    algorithm: PBEWITHMD5AND256BITAES-CBC-OPENSSL
+    provider: BC
+Processors: []
+Connections: []
+Remote Processing Groups: []

http://git-wip-us.apache.org/repos/asf/nifi-minifi/blob/1bbeedf6/minifi-toolkit/minifi-toolkit-assembly/src/main/resources/config.sh
----------------------------------------------------------------------
diff --git a/minifi-toolkit/minifi-toolkit-assembly/src/main/resources/config.sh b/minifi-toolkit/minifi-toolkit-assembly/src/main/resources/config.sh
index ba29fdc..12963cb 100755
--- a/minifi-toolkit/minifi-toolkit-assembly/src/main/resources/config.sh
+++ b/minifi-toolkit/minifi-toolkit-assembly/src/main/resources/config.sh
@@ -93,7 +93,7 @@ init() {
     detectOS
 
     # Locate the Java VM to execute
-    locateJava "$1"
+    locateJava
 }
 
 run() {
@@ -118,19 +118,5 @@ run() {
    return $?
 }
 
-
-case "$1" in
-    transform|validate)
-        init "$1"
-        run "$@"
-        ;;
-    *)
-        echo ""
-        echo "MiNiFi toolkit Usage, valid commands include: {transform|validate}"
-        echo "transform: Transform template xml into MiNiFi config YAML"
-        echo "    config.sh transform INPUT_FILE OUTPUT_FILE"
-        echo "validate: Validate config YAML"
-        echo "    config.sh validate INPUT_FILE"
-        echo ""
-        ;;
-esac
\ No newline at end of file
+init
+run "$@"
\ No newline at end of file