You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2017/04/20 05:09:41 UTC

[1/5] nifi git commit: NIFI-3273: Added nifi-toolkit-flowfile-repo that contains a simple Java class that is capable of recovering a FlowFile Repository manually in the case of an operating system crash that results in trailing 0's being dumped into the

Repository: nifi
Updated Branches:
  refs/heads/master 141334c3c -> 0f2ac39f6


http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java
new file mode 100644
index 0000000..7e87199
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java
@@ -0,0 +1,122 @@
+/*
+ * 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.controller.repository.claim;
+
+public class StandardResourceClaim implements ResourceClaim, Comparable<ResourceClaim> {
+    private final StandardResourceClaimManager claimManager;
+    private final String id;
+    private final String container;
+    private final String section;
+    private final boolean lossTolerant;
+    private final int hashCode;
+    private volatile boolean writable = true;
+
+    public StandardResourceClaim(final StandardResourceClaimManager claimManager, final String container, final String section, final String id, final boolean lossTolerant) {
+        this.claimManager = claimManager;
+        this.container = container.intern();
+        this.section = section.intern();
+        this.id = id;
+        this.lossTolerant = lossTolerant;
+
+        hashCode = 17 + 19 * id.hashCode() + 19 * container.hashCode() + 19 * section.hashCode();
+    }
+
+    @Override
+    public boolean isLossTolerant() {
+        return lossTolerant;
+    }
+
+    /**
+     * @return the unique identifier for this claim
+     */
+    @Override
+    public String getId() {
+        return id;
+    }
+
+    /**
+     * @return the container identifier in which this claim is held
+     */
+    @Override
+    public String getContainer() {
+        return container;
+    }
+
+    /**
+     * @return the section within a given container the claim is held
+     */
+    @Override
+    public String getSection() {
+        return section;
+    }
+
+    @Override
+    public boolean equals(final Object other) {
+        if (this == other) {
+            return true;
+        }
+
+        if (other == null) {
+            return false;
+        }
+        if (hashCode != other.hashCode()) {
+            // We check hash code before instanceof because instanceof is fairly expensive and for
+            // StandardResourceClaim, calling hashCode() simply returns a pre-calculated value.
+            return false;
+        }
+
+        if (!(other instanceof ResourceClaim)) {
+            return false;
+        }
+        final ResourceClaim otherClaim = (ResourceClaim) other;
+        return id.equals(otherClaim.getId()) && container.equals(otherClaim.getContainer()) && section.equals(otherClaim.getSection());
+    }
+
+    @Override
+    public int hashCode() {
+        return hashCode;
+    }
+
+    @Override
+    public String toString() {
+        return "StandardResourceClaim[id=" + id + ", container=" + container + ", section=" + section + "]";
+    }
+
+    @Override
+    public boolean isWritable() {
+        return writable;
+    }
+
+    /**
+     * Freeze the Resource Claim so that it can now longer be written to
+     */
+    void freeze() {
+        this.writable = false;
+    }
+
+    @Override
+    public boolean isInUse() {
+        // Note that it is critical here that we always check isWritable() BEFORE checking
+        // the claimant count. This is due to the fact that if the claim is in fact writable, the claimant count
+        // could increase. So if we first check claimant count and that is 0, and then we check isWritable, it may be
+        // that the claimant count has changed to 1 before checking isWritable.
+        // However, if isWritable() is false, then the only way that the claimant count can increase is if a FlowFile referencing
+        // the Resource Claim is cloned. In this case, though, the claimant count has not become 0.
+        // Said another way, if isWritable() == false, then the claimant count can never increase from 0.
+        return isWritable() || claimManager.getClaimantCount(this) > 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java
new file mode 100644
index 0000000..e4f060e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java
@@ -0,0 +1,219 @@
+/*
+ * 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.controller.repository.claim;
+
+import java.util.Collection;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class StandardResourceClaimManager implements ResourceClaimManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(StandardResourceClaimManager.class);
+    private final ConcurrentMap<ResourceClaim, ClaimCount> claimantCounts = new ConcurrentHashMap<>();
+    private final BlockingQueue<ResourceClaim> destructableClaims = new LinkedBlockingQueue<>(50000);
+
+    @Override
+    public ResourceClaim newResourceClaim(final String container, final String section, final String id, final boolean lossTolerant, final boolean writable) {
+        final StandardResourceClaim claim = new StandardResourceClaim(this, container, section, id, lossTolerant);
+        if (!writable) {
+            claim.freeze();
+        }
+        return claim;
+    }
+
+    @Override
+    public ResourceClaim getResourceClaim(final String container, final String section, final String id) {
+        final ResourceClaim tempClaim = new StandardResourceClaim(this, container, section, id, false);
+        final ClaimCount count = claimantCounts.get(tempClaim);
+        return (count == null) ? null : count.getClaim();
+    }
+
+    private AtomicInteger getCounter(final ResourceClaim claim) {
+        if (claim == null) {
+            return null;
+        }
+
+        ClaimCount counter = claimantCounts.get(claim);
+        if (counter != null) {
+            return counter.getCount();
+        }
+
+        counter = new ClaimCount(claim, new AtomicInteger(0));
+        final ClaimCount existingCounter = claimantCounts.putIfAbsent(claim, counter);
+        return existingCounter == null ? counter.getCount() : existingCounter.getCount();
+    }
+
+    @Override
+    public int getClaimantCount(final ResourceClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        synchronized (claim) {
+            final ClaimCount counter = claimantCounts.get(claim);
+            return counter == null ? 0 : counter.getCount().get();
+        }
+    }
+
+    @Override
+    public int decrementClaimantCount(final ResourceClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        synchronized (claim) {
+            final ClaimCount counter = claimantCounts.get(claim);
+            if (counter == null) {
+                logger.warn("Decrementing claimant count for {} but claimant count is not known. Returning -1", claim);
+                return -1;
+            }
+
+            final int newClaimantCount = counter.getCount().decrementAndGet();
+            if (newClaimantCount < 0) {
+                logger.error("Decremented claimant count for {} to {}", claim, newClaimantCount);
+            } else {
+                logger.debug("Decrementing claimant count for {} to {}", claim, newClaimantCount);
+            }
+
+            // If the claim is no longer referenced, we want to remove it. We consider the claim to be "no longer referenced"
+            // if the count is 0 and it is no longer writable (if it's writable, it may still be writable by the Content Repository,
+            // even though no existing FlowFile is referencing the claim).
+            if (newClaimantCount == 0 && !claim.isWritable()) {
+                removeClaimantCount(claim);
+            }
+            return newClaimantCount;
+        }
+    }
+
+    // protected so that it can be used in unit tests
+    protected void removeClaimantCount(final ResourceClaim claim) {
+        claimantCounts.remove(claim);
+    }
+
+    @Override
+    public int incrementClaimantCount(final ResourceClaim claim) {
+        return incrementClaimantCount(claim, false);
+    }
+
+    @Override
+    public int incrementClaimantCount(final ResourceClaim claim, final boolean newClaim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        synchronized (claim) {
+            final AtomicInteger counter = getCounter(claim);
+
+            final int newClaimantCount = counter.incrementAndGet();
+            logger.debug("Incrementing claimant count for {} to {}", claim, newClaimantCount);
+
+            // If the claimant count moved from 0 to 1, remove it from the queue of destructable claims.
+            if (!newClaim && newClaimantCount == 1) {
+                destructableClaims.remove(claim);
+            }
+            return newClaimantCount;
+        }
+    }
+
+    @Override
+    public void markDestructable(final ResourceClaim claim) {
+        if (claim == null) {
+            return;
+        }
+
+        synchronized (claim) {
+            if (getClaimantCount(claim) > 0) {
+                return;
+            }
+
+            logger.debug("Marking claim {} as destructable", claim);
+            try {
+                while (!destructableClaims.offer(claim, 30, TimeUnit.MINUTES)) {
+                }
+            } catch (final InterruptedException ie) {
+            }
+        }
+    }
+
+    @Override
+    public void drainDestructableClaims(final Collection<ResourceClaim> destination, final int maxElements) {
+        final int drainedCount = destructableClaims.drainTo(destination, maxElements);
+        logger.debug("Drained {} destructable claims to {}", drainedCount, destination);
+    }
+
+    @Override
+    public void drainDestructableClaims(final Collection<ResourceClaim> destination, final int maxElements, final long timeout, final TimeUnit unit) {
+        try {
+            final ResourceClaim firstClaim = destructableClaims.poll(timeout, unit);
+            if (firstClaim != null) {
+                destination.add(firstClaim);
+                destructableClaims.drainTo(destination, maxElements - 1);
+            }
+        } catch (final InterruptedException e) {
+        }
+    }
+
+    @Override
+    public void purge() {
+        claimantCounts.clear();
+    }
+
+    @Override
+    public void freeze(final ResourceClaim claim) {
+        if (claim == null) {
+            return;
+        }
+
+        if (!(claim instanceof StandardResourceClaim)) {
+            throw new IllegalArgumentException("The given resource claim is not managed by this Resource Claim Manager");
+        }
+
+        ((StandardResourceClaim) claim).freeze();
+
+        synchronized (claim) {
+            if (getClaimantCount(claim) == 0) {
+                claimantCounts.remove(claim);
+            }
+        }
+    }
+
+
+    private static final class ClaimCount {
+        private final ResourceClaim claim;
+        private final AtomicInteger count;
+
+        public ClaimCount(final ResourceClaim claim, final AtomicInteger count) {
+            this.claim = claim;
+            this.count = count;
+        }
+
+        public AtomicInteger getCount() {
+            return count;
+        }
+
+        public ResourceClaim getClaim() {
+            return claim;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
index 45494d4..e185332 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/pom.xml
@@ -29,6 +29,8 @@
         <module>nifi-runtime</module>
         <module>nifi-security</module>
         <module>nifi-site-to-site</module>
+        <module>nifi-repository-models</module>
+        <module>nifi-flowfile-repo-serialization</module>
         <module>nifi-framework-core</module>
         <module>nifi-framework-cluster-protocol</module>
         <module>nifi-framework-cluster</module>

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml b/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml
new file mode 100644
index 0000000..946b195
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml
@@ -0,0 +1,27 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-toolkit</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-toolkit-flowfile-repo</artifactId>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-toolkit/nifi-toolkit-flowfile-repo/src/main/java/org/apache/nifi/toolkit/repos/flowfile/RepairCorruptedFileEndings.java
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-flowfile-repo/src/main/java/org/apache/nifi/toolkit/repos/flowfile/RepairCorruptedFileEndings.java b/nifi-toolkit/nifi-toolkit-flowfile-repo/src/main/java/org/apache/nifi/toolkit/repos/flowfile/RepairCorruptedFileEndings.java
new file mode 100644
index 0000000..d911a9d
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-flowfile-repo/src/main/java/org/apache/nifi/toolkit/repos/flowfile/RepairCorruptedFileEndings.java
@@ -0,0 +1,287 @@
+/*
+ * 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.toolkit.repos.flowfile;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.nifi.stream.io.LimitingInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+
+public class RepairCorruptedFileEndings {
+    private static final Pattern PARTITION_FILE_PATTERN = Pattern.compile("partition\\-\\d+");
+
+    private static void printUsage() {
+        System.out.println("Whenever a sudden power loss occurs, it is common with some operating systems for files that are being written to ");
+        System.out.println("to contain many NUL characters (hex 0) at the end of the file upon restart. If this happens to the FlowFile repository, ");
+        System.out.println("NiFi will be unable to recover, because it cannot properly read the repository. This utility attempts to read the FlowFile ");
+        System.out.println("Repository and write out a new copy of the repository, where the new copy does not contain the trailing NUL characters so ");
+        System.out.println("NiFi can be restarted by pointing at the new FlowFile Repository.");
+        System.out.println("Typically, this problem can be identified by seeing an error in the NiFi logs at startup, indicating either:");
+        System.out.println();
+        System.out.println("Caused by: java.io.IOException: Expected to read a Sentinel Byte of '1' but got a value of '0' instead");
+        System.out.println();
+        System.out.println("or:");
+        System.out.println();
+        System.out.println("Caused by: java.lang.IllegalArgumentException: No enum constant org.wali.UpdateType.");
+        System.out.println();
+        System.out.println();
+        System.out.println("Usage:");
+        System.out.println("java " + RepairCorruptedFileEndings.class.getCanonicalName() + " <repo input directory> <repo destination directory>");
+        System.out.println();
+        System.out.println("<repo input directory>: The existing FlowFile Repository Directory that contains corrupt data");
+        System.out.println("<repo destination directory>: The directory to write the repaired repository to");
+        System.out.println();
+    }
+
+    public static void main(final String[] args) {
+        if (args.length != 2) {
+            printUsage();
+            return;
+        }
+
+        final File inputDir = new File(args[0]);
+        if (!inputDir.exists()) {
+            System.out.println("Input Repository Directory " + inputDir + " does not exist");
+            return;
+        }
+
+        final File[] inputFiles = inputDir.listFiles();
+        if (inputFiles == null) {
+            System.out.println("Could not access files within input Repository Directory " + inputDir);
+            return;
+        }
+
+        final List<File> partitionDirs = Stream.of(inputFiles)
+            .filter(RepairCorruptedFileEndings::isPartitionDirectory)
+            .collect(Collectors.toList());
+
+        if (partitionDirs.isEmpty()) {
+            System.out.println("Found no partitions within input Repository Directory " + inputDir);
+            return;
+        }
+
+        final File outputDir = new File(args[1]);
+        if (outputDir.exists()) {
+            final File[] children = outputDir.listFiles();
+            if (children == null) {
+                System.out.println("Cannot access output Repository Directory " + outputDir);
+                return;
+            }
+
+            if (children.length > 0) {
+                System.out.println("Output Repository Directory " + outputDir + " already exists and has files or sub-directories. "
+                    + "The output directory must either not exist or be empty.");
+                return;
+            }
+        } else if (!outputDir.mkdirs()) {
+            System.out.println("Failed to create output Repository Directory " + outputDir);
+            return;
+        }
+
+        final List<File> nonPartitionDirFiles = Stream.of(inputFiles)
+            .filter(f -> !isPartitionDirectory(f))
+            .filter(f -> !f.getName().equals("wali.lock"))
+            .collect(Collectors.toList());
+
+        for (final File nonPartitionFile : nonPartitionDirFiles) {
+            final File destination = new File(outputDir, nonPartitionFile.getName());
+            try {
+                copy(nonPartitionFile, destination);
+            } catch (final IOException e) {
+                System.out.println("Failed to copy source file " + nonPartitionFile + " to destination file " + destination);
+                e.printStackTrace();
+            }
+        }
+
+        int fullCopies = 0;
+        int partialCopies = 0;
+
+        for (final File partitionDir : partitionDirs) {
+            final File[] partitionFiles = partitionDir.listFiles();
+            if (partitionFiles == null) {
+                System.out.println("Could not access children of input sub-directory " + partitionDir);
+                return;
+            }
+
+            final File outputPartitionDir = new File(outputDir, partitionDir.getName());
+            if (!outputPartitionDir.mkdirs()) {
+                System.out.println("Failed to created output directory " + outputPartitionDir);
+                return;
+            }
+
+            for (final File partitionFile : partitionFiles) {
+                final File destinationFile = new File(outputPartitionDir, partitionFile.getName());
+
+                // All journal files follow the pattern of:
+                // <journal entry> <TRANSACTION_CONTINUE | TRANSACTION_COMMIT> <journal entry> <TRANSACTION_CONTINUE | TRANSACTION_COMMIT> ...
+                // The TRANSACTION_CONTINUE byte is a 1 while the TRANSACTION_COMMIT byte is a 2. So if we have 0's at the end then we know
+                // that we can simply truncate up until the point where we encounter the first of the of the trailing zeroes. At that point,
+                // we know that we are done. It is possible that the repo will still be 'corrupt' in that only part of a transaction was
+                // written out. However, this is okay because the repo will recover from this on restart. What it does NOT properly recover
+                // from on restart is when the file ends with a bunch of 0's because it believes that the Transaction ID is zero and then
+                // it reads in 0 bytes for the "Update Type" and as a result we get an invalid enum name because it thinks that the name of
+                // the UpdateType is an empty string because it's a string of length 0.
+                final int trailingZeroes;
+                try {
+                    trailingZeroes = countTrailingZeroes(partitionFile);
+                } catch (final Exception e) {
+                    System.out.println("Failed to read input file " + partitionFile);
+                    e.printStackTrace();
+                    return;
+                }
+
+                if (trailingZeroes > 0) {
+                    final long goodLength = partitionFile.length() - trailingZeroes;
+
+                    try {
+                        copy(partitionFile, destinationFile, goodLength);
+                        partialCopies++;
+                    } catch (final Exception e) {
+                        System.out.println("Failed to copy " + goodLength + " bytes from " + partitionFile + " to " + destinationFile);
+                        e.printStackTrace();
+                        return;
+                    }
+                } else {
+                    try {
+                        copy(partitionFile, destinationFile);
+                    } catch (final Exception e) {
+                        System.out.println("Failed to copy entire file from " + partitionFile + " to " + destinationFile);
+                        e.printStackTrace();
+                        return;
+                    }
+
+                    fullCopies++;
+                }
+            }
+        }
+
+        System.out.println("Successfully copied " + fullCopies + " journal files fully and truncated " + partialCopies + " journal files in output directory");
+    }
+
+    private static boolean isPartitionDirectory(final File file) {
+        return PARTITION_FILE_PATTERN.matcher(file.getName()).matches();
+    }
+
+    private static void copy(final File input, final File destination) throws IOException {
+        if (input.isFile()) {
+            copyFile(input, destination);
+            return;
+        } else {
+            copyDirectory(input, destination);
+        }
+    }
+
+    private static void copyDirectory(final File input, final File destination) throws IOException {
+        if (!destination.exists() && !destination.mkdirs()) {
+            System.out.println("Failed to copy input directory " + input + " to destination because destination directory " + destination
+                + " does not exist and could not be created");
+            return;
+        }
+
+        final File[] children = input.listFiles();
+        if (children == null) {
+            System.out.println("Failed to copy input directory " + input + " to destination because could not access files of input directory");
+            return;
+        }
+
+        for (final File child : children) {
+            final File destinationChild = new File(destination, child.getName());
+            copy(child, destinationChild);
+        }
+    }
+
+    private static void copyFile(final File input, final File destination) throws IOException {
+        if (!input.exists()) {
+            return;
+        }
+
+        Files.copy(input.toPath(), destination.toPath(), StandardCopyOption.COPY_ATTRIBUTES);
+    }
+
+    private static void copy(final File input, final File destination, final long length) throws IOException {
+        try (final InputStream fis = new FileInputStream(input);
+            final LimitingInputStream in = new LimitingInputStream(fis, length);
+            final OutputStream fos = new FileOutputStream(destination)) {
+            StreamUtils.copy(in, fos);
+        }
+    }
+
+    static int countTrailingZeroes(final File partitionFile) throws IOException {
+        final RandomAccessFile raf = new RandomAccessFile(partitionFile, "r");
+
+        long startPos = partitionFile.length() - 4096;
+
+        int count = 0;
+        boolean reachedStartOfFile = false;
+        while (!reachedStartOfFile) {
+            int bufferLength = 4096;
+
+            if (startPos < 0) {
+                bufferLength = (int) (startPos + 4096);
+                startPos = 0;
+                reachedStartOfFile = true;
+            }
+
+            raf.seek(startPos);
+
+            final byte[] buffer = new byte[bufferLength];
+            final int read = fillBuffer(raf, buffer);
+
+            for (int i = read - 1; i >= 0; i--) {
+                final byte b = buffer[i];
+                if (b == 0) {
+                    count++;
+                } else {
+                    return count;
+                }
+            }
+
+            startPos -= 4096;
+        }
+
+        return count;
+    }
+
+
+    private static int fillBuffer(final RandomAccessFile source, final byte[] destination) throws IOException {
+        int bytesRead = 0;
+        int len;
+        while (bytesRead < destination.length) {
+            len = source.read(destination, bytesRead, destination.length - bytesRead);
+            if (len < 0) {
+                break;
+            }
+
+            bytesRead += len;
+        }
+
+        return bytesRead;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java
----------------------------------------------------------------------
diff --git a/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java b/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java
new file mode 100644
index 0000000..7f8a7a1
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-flowfile-repo/src/test/java/org/apache/nifi/toolkit/repos/flowfile/TestRepairCorruptedFileEndings.java
@@ -0,0 +1,169 @@
+/*
+ * 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.toolkit.repos.flowfile;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+
+import org.junit.Test;
+
+public class TestRepairCorruptedFileEndings {
+    private final File targetFile = new File("target/1.bin");
+
+    @Before
+    @After
+    public void cleanup() {
+        if (targetFile.exists()) {
+            Assert.assertTrue(targetFile.delete());
+        }
+    }
+
+    @Test
+    public void testEndsWithZeroesGreaterThanBufferSize() throws IOException {
+        final byte[] data = new byte[4096 + 8];
+        for (int i=0; i < 4096; i++) {
+            data[i] = 'A';
+        }
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(8, zeroCount);
+    }
+
+    @Test
+    public void testEndsWithZeroesSmallerThanBufferSize() throws IOException {
+        final byte[] data = new byte[1024];
+        for (int i = 0; i < 1020; i++) {
+            data[i] = 'A';
+        }
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(4, zeroCount);
+    }
+
+    @Test
+    public void testEndsWithZeroesEqualToBufferSize() throws IOException {
+        final byte[] data = new byte[4096];
+        for (int i = 0; i < 4090; i++) {
+            data[i] = 'A';
+        }
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(6, zeroCount);
+    }
+
+
+    @Test
+    public void testAllZeroesGreaterThanBufferSize() throws IOException {
+        final byte[] data = new byte[4096 + 8];
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(4096 + 8, zeroCount);
+    }
+
+    @Test
+    public void testAllZeroesEqualToBufferSize() throws IOException {
+        final byte[] data = new byte[4096];
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(4096, zeroCount);
+    }
+
+    @Test
+    public void testAllZeroesSmallerThanBufferSize() throws IOException {
+        final byte[] data = new byte[1024];
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(1024, zeroCount);
+    }
+
+
+    @Test
+    public void testSmallerThanBufferSize() throws IOException {
+        final byte[] data = new byte[1024];
+        for (int i = 0; i < 1020; i++) {
+            data[i] = 'A';
+        }
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(4, zeroCount);
+    }
+
+    @Test
+    public void testSmallerThanBufferSizeNoTrailingZeroes() throws IOException {
+        final byte[] data = new byte[1024];
+        for (int i = 0; i < 1024; i++) {
+            data[i] = 'A';
+        }
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(0, zeroCount);
+    }
+
+
+    @Test
+    public void testLargerThanBufferSizeNoTrailingZeroes() throws IOException {
+        final byte[] data = new byte[8192];
+        for (int i = 0; i < 8192; i++) {
+            data[i] = 'A';
+        }
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(0, zeroCount);
+    }
+
+
+    @Test
+    public void testEqualToBufferSizeNoTrailingZeroes() throws IOException {
+        final byte[] data = new byte[4096];
+        for (int i = 0; i < 4096; i++) {
+            data[i] = 'A';
+        }
+
+        Files.write(targetFile.toPath(), data, StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING);
+
+        final int zeroCount = RepairCorruptedFileEndings.countTrailingZeroes(targetFile);
+        assertEquals(0, zeroCount);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-toolkit/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-toolkit/pom.xml b/nifi-toolkit/pom.xml
index b98325d..75661c7 100644
--- a/nifi-toolkit/pom.xml
+++ b/nifi-toolkit/pom.xml
@@ -27,6 +27,7 @@
         <module>nifi-toolkit-encrypt-config</module>
         <module>nifi-toolkit-s2s</module>
         <module>nifi-toolkit-zookeeper-migrator</module>
+        <module>nifi-toolkit-flowfile-repo</module>
         <module>nifi-toolkit-assembly</module>
     </modules>
     <dependencyManagement>

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 660977d..a735e13 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1,15 +1,16 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <!-- 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. -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
     <modelVersion>4.0.0</modelVersion>
     <parent>
         <groupId>org.apache</groupId>
@@ -87,7 +88,7 @@ language governing permissions and limitations under the License. -->
         <maven.compiler.source>1.8</maven.compiler.source>
         <maven.compiler.target>1.8</maven.compiler.target>
         <maven.min-version>3.1.0</maven.min-version>
-        <maven.surefire.arguments/>
+        <maven.surefire.arguments />
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
         <inceptionYear>2014</inceptionYear>
@@ -330,9 +331,9 @@ language governing permissions and limitations under the License. -->
                 <artifactId>quartz</artifactId>
                 <version>2.2.1</version>
                 <exclusions>
-                    <!-- | Exclude the quartz 2.2.1 bundled version of c3p0 because it is 
-                    lgpl licensed | We also don't use the JDBC related features of quartz for 
-                    which the dependency would matter -->
+                    <!-- | Exclude the quartz 2.2.1 bundled version of c3p0 
+                        because it is lgpl licensed | We also don't use the JDBC related features 
+                        of quartz for which the dependency would matter -->
                     <exclusion>
                         <groupId>c3p0</groupId>
                         <artifactId>c3p0</artifactId>
@@ -401,8 +402,8 @@ language governing permissions and limitations under the License. -->
                 <artifactId>spring-core</artifactId>
                 <version>${spring.version}</version>
                 <exclusions>
-                    <!-- <artifactId>jcl-over-slf4j</artifactId> is used in dependencies 
-                    section -->
+                    <!-- <artifactId>jcl-over-slf4j</artifactId> is used 
+                        in dependencies section -->
                     <exclusion>
                         <groupId>commons-logging</groupId>
                         <artifactId>commons-logging</artifactId>
@@ -818,7 +819,7 @@ language governing permissions and limitations under the License. -->
                 <version>${org.slf4j.version}</version>
             </dependency>
 
-            
+
             <!-- Test Dependencies for testing interactions with ZooKeeper -->
             <dependency>
                 <groupId>org.apache.curator</groupId>
@@ -832,9 +833,9 @@ language governing permissions and limitations under the License. -->
                 <version>6.8.8</version>
                 <scope>test</scope>
             </dependency>
-            
-            
-            <dependency>            
+
+
+            <dependency>
                 <groupId>org.jsoup</groupId>
                 <artifactId>jsoup</artifactId>
                 <version>1.8.3</version>
@@ -872,6 +873,16 @@ language governing permissions and limitations under the License. -->
             </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-flowfile-repo-serialization</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-repository-models</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-custom-ui-utilities</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
             </dependency>
@@ -1253,37 +1264,37 @@ language governing permissions and limitations under the License. -->
                 <artifactId>nifi-elasticsearch-nar</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
-             </dependency>
-             <dependency>
+            </dependency>
+            <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-elasticsearch-5-nar</artifactId>
-                 <version>1.2.0-SNAPSHOT</version>
+                <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
-             </dependency>
-             <dependency>
+            </dependency>
+            <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-lumberjack-nar</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
             </dependency>
-             <dependency>
+            <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-beats-nar</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
             </dependency>
-             <dependency>
+            <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-email-nar</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
             </dependency>
-             <dependency>
-                 <groupId>org.apache.nifi</groupId>
-                 <artifactId>nifi-tcp-nar</artifactId>
-                 <version>1.2.0-SNAPSHOT</version>
-                 <type>nar</type>
-             </dependency>
+            <dependency>
+                <groupId>org.apache.nifi</groupId>
+                <artifactId>nifi-tcp-nar</artifactId>
+                <version>1.2.0-SNAPSHOT</version>
+                <type>nar</type>
+            </dependency>
             <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-splunk-nar</artifactId>
@@ -1308,7 +1319,7 @@ language governing permissions and limitations under the License. -->
                 <version>1.2.0-SNAPSHOT</version>
                 <type>nar</type>
             </dependency>
-	        <dependency>
+            <dependency>
                 <groupId>org.apache.nifi</groupId>
                 <artifactId>nifi-site-to-site-reporting-nar</artifactId>
                 <version>1.2.0-SNAPSHOT</version>
@@ -1600,7 +1611,9 @@ language governing permissions and limitations under the License. -->
                             <include>**/*Spec.class</include>
                         </includes>
                         <redirectTestOutputToFile>true</redirectTestOutputToFile>
-                        <argLine combine.children="append">-Xmx1G -Djava.net.preferIPv4Stack=true ${maven.surefire.arguments}</argLine>
+                        <argLine combine.children="append">-Xmx1G
+                            -Djava.net.preferIPv4Stack=true
+                            ${maven.surefire.arguments}</argLine>
                     </configuration>
                     <dependencies>
                         <dependency>
@@ -1804,71 +1817,100 @@ language governing permissions and limitations under the License. -->
                             <!-- Checks for whitespace -->
                             <!-- See http://checkstyle.sf.net/config_whitespace.html -->
                             <module name="FileTabCharacter">
-                                <property name="eachLine" value="true" />
+                                <property name="eachLine"
+                                    value="true" />
                             </module>
                             <module name="TreeWalker">
                                 <module name="RegexpSinglelineJava">
-                                    <property name="format" value="\s+$" />
-                                    <property name="message" value="Line has trailing whitespace." />
+                                    <property name="format"
+                                        value="\s+$" />
+                                    <property name="message"
+                                        value="Line has trailing whitespace." />
                                 </module>
                                 <module name="RegexpSinglelineJava">
-                                    <property name="format" value="[@]see\s+[{][@]link" />
-                                    <property name="message" value="Javadoc @see does not need @link: pick one or the other." />
+                                    <property name="format"
+                                        value="[@]see\s+[{][@]link" />
+                                    <property name="message"
+                                        value="Javadoc @see does not need @link: pick one or the other." />
                                 </module>
                                 <module name="OuterTypeFilename" />
                                 <module name="LineLength">
-                                    <!-- needs extra, because Eclipse formatter ignores the ending left 
-                                    brace -->
+                                    <!-- needs extra, because Eclipse formatter 
+                                        ignores the ending left brace -->
                                     <property name="max" value="200" />
-                                    <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://" />
+                                    <property name="ignorePattern"
+                                        value="^package.*|^import.*|a href|href|http://|https://|ftp://" />
                                 </module>
                                 <module name="AvoidStarImport" />
                                 <module name="UnusedImports">
-                                    <property name="processJavadoc" value="true" />
+                                    <property name="processJavadoc"
+                                        value="true" />
                                 </module>
                                 <module name="NoLineWrap" />
                                 <module name="LeftCurly">
-                                    <property name="maxLineLength" value="160" />
+                                    <property name="maxLineLength"
+                                        value="160" />
                                 </module>
                                 <module name="RightCurly" />
                                 <module name="RightCurly">
-                                    <property name="option" value="alone" />
-                                    <property name="tokens" value="CLASS_DEF, METHOD_DEF, CTOR_DEF, LITERAL_FOR, LITERAL_WHILE, LITERAL_DO, STATIC_INIT, INSTANCE_INIT" />
+                                    <property name="option"
+                                        value="alone" />
+                                    <property name="tokens"
+                                        value="CLASS_DEF, METHOD_DEF, CTOR_DEF, LITERAL_FOR, LITERAL_WHILE, LITERAL_DO, STATIC_INIT, INSTANCE_INIT" />
                                 </module>
                                 <module name="SeparatorWrap">
-                                    <property name="tokens" value="DOT" />
-                                    <property name="option" value="nl" />
+                                    <property name="tokens"
+                                        value="DOT" />
+                                    <property name="option"
+                                        value="nl" />
                                 </module>
                                 <module name="SeparatorWrap">
-                                    <property name="tokens" value="COMMA" />
-                                    <property name="option" value="EOL" />
+                                    <property name="tokens"
+                                        value="COMMA" />
+                                    <property name="option"
+                                        value="EOL" />
                                 </module>
                                 <module name="PackageName">
-                                    <property name="format" value="^[a-z]+(\.[a-z][a-zA-Z0-9]*)*$" />
+                                    <property name="format"
+                                        value="^[a-z]+(\.[a-z][a-zA-Z0-9]*)*$" />
                                 </module>
                                 <module name="MethodTypeParameterName">
-                                    <property name="format" value="(^[A-Z][0-9]?)$|([A-Z][a-zA-Z0-9]*[T]$)" />
+                                    <property name="format"
+                                        value="(^[A-Z][0-9]?)$|([A-Z][a-zA-Z0-9]*[T]$)" />
                                 </module>
                                 <module name="MethodParamPad" />
                                 <module name="OperatorWrap">
-                                    <property name="option" value="NL" />
-                                    <property name="tokens" value="BAND, BOR, BSR, BXOR, DIV, EQUAL, GE, GT, LAND, LE, LITERAL_INSTANCEOF, LOR, LT, MINUS, MOD, NOT_EQUAL, QUESTION, SL, SR, STAR " />
+                                    <property name="option"
+                                        value="NL" />
+                                    <property name="tokens"
+                                        value="BAND, BOR, BSR, BXOR, DIV, EQUAL, GE, GT, LAND, LE, LITERAL_INSTANCEOF, LOR, LT, MINUS, MOD, NOT_EQUAL, QUESTION, SL, SR, STAR " />
                                 </module>
                                 <module name="AnnotationLocation">
-                                    <property name="tokens" value="CLASS_DEF, INTERFACE_DEF, ENUM_DEF, METHOD_DEF, CTOR_DEF" />
+                                    <property name="tokens"
+                                        value="CLASS_DEF, INTERFACE_DEF, ENUM_DEF, METHOD_DEF, CTOR_DEF" />
                                 </module>
                                 <module name="AnnotationLocation">
-                                    <property name="tokens" value="VARIABLE_DEF" />
-                                    <property name="allowSamelineMultipleAnnotations" value="true" />
+                                    <property name="tokens"
+                                        value="VARIABLE_DEF" />
+                                    <property
+                                        name="allowSamelineMultipleAnnotations"
+                                        value="true" />
                                 </module>
                                 <module name="NonEmptyAtclauseDescription" />
                                 <module name="JavadocMethod">
-                                    <property name="allowMissingJavadoc" value="true" />
-                                    <property name="allowMissingParamTags" value="true" />
-                                    <property name="allowMissingThrowsTags" value="true" />
-                                    <property name="allowMissingReturnTag" value="true" />
-                                    <property name="allowedAnnotations" value="Override,Test,BeforeClass,AfterClass,Before,After" />
-                                    <property name="allowThrowsTagsForSubclasses" value="true" />
+                                    <property name="allowMissingJavadoc"
+                                        value="true" />
+                                    <property name="allowMissingParamTags"
+                                        value="true" />
+                                    <property name="allowMissingThrowsTags"
+                                        value="true" />
+                                    <property name="allowMissingReturnTag"
+                                        value="true" />
+                                    <property name="allowedAnnotations"
+                                        value="Override,Test,BeforeClass,AfterClass,Before,After" />
+                                    <property
+                                        name="allowThrowsTagsForSubclasses"
+                                        value="true" />
                                 </module>
                                 <module name="SingleLineJavadoc" />
                             </module>
@@ -1908,9 +1950,10 @@ language governing permissions and limitations under the License. -->
     </build>
     <profiles>
         <profile>
-            <!-- Performs execution of Integration Tests using the Maven FailSafe Plugin. The view of integration tests in this context
-            are those tests interfacing with external sources and services requiring additional resources or credentials that cannot
-            be explicitly provided. -->
+            <!-- Performs execution of Integration Tests using the Maven 
+                FailSafe Plugin. The view of integration tests in this context are those 
+                tests interfacing with external sources and services requiring additional 
+                resources or credentials that cannot be explicitly provided. -->
             <id>integration-tests</id>
             <build>
                 <plugins>
@@ -1930,12 +1973,12 @@ language governing permissions and limitations under the License. -->
             </build>
         </profile>
         <profile>
-            <!-- Checks style and licensing requirements. This is a good idea to run 
-            for contributions and for the release process. While it would be nice to 
-            run always these plugins can considerably slow the build and have proven 
-            to create unstable builds in our multi-module project and when building using 
-            multiple threads. The stability issues seen with Checkstyle in multi-module 
-            builds include false-positives and false negatives. -->
+            <!-- Checks style and licensing requirements. This is a good 
+                idea to run for contributions and for the release process. While it would 
+                be nice to run always these plugins can considerably slow the build and have 
+                proven to create unstable builds in our multi-module project and when building 
+                using multiple threads. The stability issues seen with Checkstyle in multi-module 
+                builds include false-positives and false negatives. -->
             <id>contrib-check</id>
             <build>
                 <plugins>
@@ -1991,14 +2034,16 @@ language governing permissions and limitations under the License. -->
                 </pluginManagement>
             </build>
         </profile>
-        <!-- The following profiles are here as a convenience for folks that want to build against vendor-specific
-         distributions of the various Hadoop ecosystem libraries.  These will alter which dependencies are sourced
-         in a manner that can adjust the correct LICENSE and NOTICE requirements for any affected jar and the
-         resulting assembly overall.  These L&N impacts are not automatically handled by the build process and are
-         the responsibility of those creating and using the resulting binary artifacts. -->
+        <!-- The following profiles are here as a convenience for folks that 
+            want to build against vendor-specific distributions of the various Hadoop 
+            ecosystem libraries. These will alter which dependencies are sourced in a 
+            manner that can adjust the correct LICENSE and NOTICE requirements for any 
+            affected jar and the resulting assembly overall. These L&N impacts are not 
+            automatically handled by the build process and are the responsibility of 
+            those creating and using the resulting binary artifacts. -->
         <profile>
-            <!-- This profile adds the Hortonworks repository for resolving Hortonworks Data Platform (HDP)
-                 artifacts for the Hadoop bundles -->
+            <!-- This profile adds the Hortonworks repository for resolving 
+                Hortonworks Data Platform (HDP) artifacts for the Hadoop bundles -->
             <id>hortonworks</id>
             <repositories>
                 <repository>
@@ -2033,15 +2078,13 @@ language governing permissions and limitations under the License. -->
                 </repository>
             </repositories>
             <properties>
-                <!-- Vendor-specific version number included here as default, should be overridden on the
-                     command-line
-                <hadoop.version>2.7.1.2.4.0.0-169</hadoop.version>
-                -->
+                <!-- Vendor-specific version number included here as default, 
+                    should be overridden on the command-line <hadoop.version>2.7.1.2.4.0.0-169</hadoop.version> -->
             </properties>
         </profile>
         <profile>
-            <!-- This profile will add the MapR repository for resolving MapR Hadoop
-                 artifacts for the Hadoop bundles -->
+            <!-- This profile will add the MapR repository for resolving 
+                MapR Hadoop artifacts for the Hadoop bundles -->
             <id>mapr</id>
             <repositories>
                 <repository>
@@ -2057,15 +2100,13 @@ language governing permissions and limitations under the License. -->
                 </repository>
             </repositories>
             <properties>
-                <!-- Vendor-specific version number included here as default, should be overridden on the
-                     command-line
-                <hadoop.version>2.7.0-mapr-1602</hadoop.version>
-                -->
+                <!-- Vendor-specific version number included here as default, 
+                    should be overridden on the command-line <hadoop.version>2.7.0-mapr-1602</hadoop.version> -->
             </properties>
         </profile>
         <profile>
-            <!-- This profile will add the Cloudera repository for resolving Cloudera Distribution of Hadoop (CDH)
-                 artifacts for the Hadoop bundles -->
+            <!-- This profile will add the Cloudera repository for resolving 
+                Cloudera Distribution of Hadoop (CDH) artifacts for the Hadoop bundles -->
             <id>cloudera</id>
             <repositories>
                 <repository>
@@ -2081,10 +2122,8 @@ language governing permissions and limitations under the License. -->
                 </repository>
             </repositories>
             <properties>
-                <!-- Vendor-specific version number included here as default, should be overridden on the
-                     command-line
-                <hadoop.version>2.6.0-cdh5.8.1</hadoop.version>
-                -->
+                <!-- Vendor-specific version number included here as default, 
+                    should be overridden on the command-line <hadoop.version>2.6.0-cdh5.8.1</hadoop.version> -->
             </properties>
         </profile>
     </profiles>


[3/5] nifi git commit: NIFI-3273: Added nifi-toolkit-flowfile-repo that contains a simple Java class that is capable of recovering a FlowFile Repository manually in the case of an operating system crash that results in trailing 0's being dumped into the

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java
new file mode 100644
index 0000000..afa19ea
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java
@@ -0,0 +1,85 @@
+/*
+ * 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.controller.repository.schema;
+
+import org.apache.nifi.controller.repository.claim.ResourceClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordSchema;
+
+public class ResourceClaimFieldMap implements Record {
+    private final ResourceClaim resourceClaim;
+    private final RecordSchema schema;
+
+    public ResourceClaimFieldMap(final ResourceClaim resourceClaim, final RecordSchema schema) {
+        this.resourceClaim = resourceClaim;
+        this.schema = schema;
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public Object getFieldValue(final String fieldName) {
+        switch (fieldName) {
+            case ContentClaimSchema.CLAIM_CONTAINER:
+                return resourceClaim.getContainer();
+            case ContentClaimSchema.CLAIM_SECTION:
+                return resourceClaim.getSection();
+            case ContentClaimSchema.CLAIM_IDENTIFIER:
+                return resourceClaim.getId();
+            case ContentClaimSchema.LOSS_TOLERANT:
+                return resourceClaim.isLossTolerant();
+        }
+
+        return null;
+    }
+
+    public static ResourceClaim getResourceClaim(final Record record, final ResourceClaimManager claimManager) {
+        final String container = (String) record.getFieldValue(ContentClaimSchema.CLAIM_CONTAINER);
+        final String section = (String) record.getFieldValue(ContentClaimSchema.CLAIM_SECTION);
+        final String identifier = (String) record.getFieldValue(ContentClaimSchema.CLAIM_IDENTIFIER);
+        final Boolean lossTolerant = (Boolean) record.getFieldValue(ContentClaimSchema.LOSS_TOLERANT);
+
+        return claimManager.newResourceClaim(container, section, identifier, lossTolerant, false);
+    }
+
+    @Override
+    public int hashCode() {
+        return 41 + 91 * resourceClaim.hashCode();
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (obj == this) {
+            return true;
+        }
+
+        if (obj.getClass() != ResourceClaimFieldMap.class) {
+            return false;
+        }
+
+        final ResourceClaimFieldMap other = (ResourceClaimFieldMap) obj;
+        return resourceClaim.equals(other.resourceClaim);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
new file mode 100644
index 0000000..59b0e7b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/test/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerdeTest.java
@@ -0,0 +1,266 @@
+/*
+ * 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.controller.repository;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
+import org.apache.nifi.controller.repository.schema.RepositoryRecordSchema;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.nifi.controller.repository.RepositoryRecordType.SWAP_IN;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class SchemaRepositoryRecordSerdeTest {
+    public static final String TEST_QUEUE_IDENTIFIER = "testQueueIdentifier";
+    private StandardResourceClaimManager resourceClaimManager;
+    private SchemaRepositoryRecordSerde schemaRepositoryRecordSerde;
+    private Map<String, FlowFileQueue> queueMap;
+    private FlowFileQueue flowFileQueue;
+    private ByteArrayOutputStream byteArrayOutputStream;
+    private DataOutputStream dataOutputStream;
+
+    @Before
+    public void setup() {
+        resourceClaimManager = new StandardResourceClaimManager();
+        schemaRepositoryRecordSerde = new SchemaRepositoryRecordSerde(resourceClaimManager);
+        queueMap = new HashMap<>();
+        schemaRepositoryRecordSerde.setQueueMap(queueMap);
+        flowFileQueue = createMockQueue(TEST_QUEUE_IDENTIFIER);
+        byteArrayOutputStream = new ByteArrayOutputStream();
+        dataOutputStream = new DataOutputStream(byteArrayOutputStream);
+    }
+
+    @After
+    public void teardown() {
+        resourceClaimManager.purge();
+    }
+
+    @Test
+    public void testV1CreateCantHandleLongAttributeName() throws IOException {
+        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put(stringBuilder.toString(), "testValue");
+        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
+                RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testV1CreateCantHandleLongAttributeValue() throws IOException {
+        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put("testName", stringBuilder.toString());
+        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
+                RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testV2CreateCanHandleLongAttributeName() throws IOException {
+        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put(stringBuilder.toString(), "testValue");
+        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testV2CreateCanHandleLongAttributeValue() throws IOException {
+        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put("testName", stringBuilder.toString());
+        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testRoundTripCreateV1ToV2() throws IOException {
+        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        attributes.put("testName", "testValue");
+        schemaRepositoryRecordSerde.serializeRecord(createCreateFlowFileRecord(attributes), dataOutputStream,
+                RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testV1SwapInCantHandleLongAttributeName() throws IOException {
+        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put(stringBuilder.toString(), "testValue");
+        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
+        record.setSwapLocation("fake");
+        assertEquals(SWAP_IN, record.getType());
+        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testV1SwapInCantHandleLongAttributeValue() throws IOException {
+        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put("testName", stringBuilder.toString());
+        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
+        record.setSwapLocation("fake");
+        assertEquals(SWAP_IN, record.getType());
+        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertNotEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testV2SwapInCanHandleLongAttributeName() throws IOException {
+        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put(stringBuilder.toString(), "testValue");
+        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
+        record.setSwapLocation("fake");
+        assertEquals(SWAP_IN, record.getType());
+        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testV2SwapInCanHandleLongAttributeValue() throws IOException {
+        schemaRepositoryRecordSerde.writeHeader(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        StringBuilder stringBuilder = new StringBuilder();
+        for (int i = 0; i < 65536; i++) {
+            stringBuilder.append('a');
+        }
+        attributes.put("testName", stringBuilder.toString());
+        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
+        record.setSwapLocation("fake");
+        assertEquals(SWAP_IN, record.getType());
+        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+    }
+
+    @Test
+    public void testRoundTripSwapInV1ToV2() throws IOException {
+        RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1.writeTo(dataOutputStream);
+        Map<String, String> attributes = new HashMap<>();
+        attributes.put("testName", "testValue");
+        StandardRepositoryRecord record = createCreateFlowFileRecord(attributes);
+        record.setSwapLocation("fake");
+        assertEquals(SWAP_IN, record.getType());
+        schemaRepositoryRecordSerde.serializeRecord(record, dataOutputStream, RepositoryRecordSchema.SWAP_IN_SCHEMA_V1, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V1);
+
+        DataInputStream dataInputStream = createDataInputStream();
+        schemaRepositoryRecordSerde.readHeader(dataInputStream);
+        RepositoryRecord repositoryRecord = schemaRepositoryRecordSerde.deserializeRecord(dataInputStream, 2);
+        assertEquals(attributes, repositoryRecord.getCurrent().getAttributes());
+        assertEquals(SWAP_IN, repositoryRecord.getType());
+    }
+
+    private DataInputStream createDataInputStream() throws IOException {
+        dataOutputStream.flush();
+        return new DataInputStream(new ByteArrayInputStream(byteArrayOutputStream.toByteArray()));
+    }
+
+    private StandardRepositoryRecord createCreateFlowFileRecord(Map<String, String> attributes) {
+        StandardRepositoryRecord standardRepositoryRecord = new StandardRepositoryRecord(flowFileQueue);
+        StandardFlowFileRecord.Builder flowFileRecordBuilder = new StandardFlowFileRecord.Builder();
+        flowFileRecordBuilder.addAttributes(attributes);
+        standardRepositoryRecord.setWorking(flowFileRecordBuilder.build());
+        return standardRepositoryRecord;
+    }
+
+    private FlowFileQueue createMockQueue(String identifier) {
+        FlowFileQueue flowFileQueue = mock(FlowFileQueue.class);
+        when(flowFileQueue.getIdentifier()).thenReturn(identifier);
+        queueMap.put(identifier, flowFileQueue);
+        return flowFileQueue;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
index 54d777f..6395e6e 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/pom.xml
@@ -57,6 +57,10 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-repository-models</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
             <artifactId>nifi-properties</artifactId>
         </dependency>
         <dependency>
@@ -136,6 +140,10 @@
             <artifactId>nifi-write-ahead-log</artifactId>
         </dependency>
         <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-flowfile-repo-serialization</artifactId>
+        </dependency>
+        <dependency>
             <groupId>org.apache.zookeeper</groupId>
             <artifactId>zookeeper</artifactId>
         </dependency>

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
deleted file mode 100644
index 44ed62d..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.controller.repository;
-
-import java.util.Map;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.wali.SerDe;
-import org.wali.UpdateType;
-
-public abstract class RepositoryRecordSerde implements SerDe<RepositoryRecord> {
-    private Map<String, FlowFileQueue> flowFileQueueMap = null;
-
-    protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
-        this.flowFileQueueMap = queueMap;
-    }
-
-    protected Map<String, FlowFileQueue> getQueueMap() {
-        return flowFileQueueMap;
-    }
-
-    protected FlowFileQueue getFlowFileQueue(final String queueId) {
-        return flowFileQueueMap.get(queueId);
-    }
-
-    @Override
-    public Long getRecordIdentifier(final RepositoryRecord record) {
-        return record.getCurrent().getId();
-    }
-
-    @Override
-    public UpdateType getUpdateType(final RepositoryRecord record) {
-        switch (record.getType()) {
-            case CONTENTMISSING:
-            case DELETE:
-                return UpdateType.DELETE;
-            case CREATE:
-                return UpdateType.CREATE;
-            case UPDATE:
-                return UpdateType.UPDATE;
-            case SWAP_OUT:
-                return UpdateType.SWAP_OUT;
-            case SWAP_IN:
-                return UpdateType.SWAP_IN;
-        }
-        return null;
-    }
-
-    @Override
-    public String getLocation(final RepositoryRecord record) {
-        return record.getSwapLocation();
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
deleted file mode 100644
index c19fa94..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * 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.controller.repository;
-
-import java.util.Map;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
-import org.wali.SerDe;
-import org.wali.SerDeFactory;
-import org.wali.UpdateType;
-
-public class RepositoryRecordSerdeFactory implements SerDeFactory<RepositoryRecord> {
-    private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde";
-    private final ResourceClaimManager resourceClaimManager;
-    private Map<String, FlowFileQueue> flowFileQueueMap = null;
-
-    public RepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) {
-        this.resourceClaimManager = claimManager;
-    }
-
-    protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
-        this.flowFileQueueMap = queueMap;
-    }
-
-    protected Map<String, FlowFileQueue> getQueueMap() {
-        return flowFileQueueMap;
-    }
-
-    @Override
-    public SerDe<RepositoryRecord> createSerDe(final String encodingName) {
-        if (encodingName == null || SchemaRepositoryRecordSerde.class.getName().equals(encodingName)) {
-            final SchemaRepositoryRecordSerde serde = new SchemaRepositoryRecordSerde(resourceClaimManager);
-            serde.setQueueMap(flowFileQueueMap);
-            return serde;
-        }
-
-        if (WriteAheadRepositoryRecordSerde.class.getName().equals(encodingName)
-            || LEGACY_SERDE_ENCODING_NAME.equals(encodingName)) {
-            final WriteAheadRepositoryRecordSerde serde = new WriteAheadRepositoryRecordSerde(resourceClaimManager);
-            serde.setQueueMap(flowFileQueueMap);
-            return serde;
-        }
-
-        throw new IllegalArgumentException("Cannot create Deserializer for Repository Records because the encoding '" + encodingName + "' is not known");
-    }
-
-    protected FlowFileQueue getFlowFileQueue(final String queueId) {
-        return flowFileQueueMap.get(queueId);
-    }
-
-    @Override
-    public Long getRecordIdentifier(final RepositoryRecord record) {
-        return record.getCurrent().getId();
-    }
-
-    @Override
-    public UpdateType getUpdateType(final RepositoryRecord record) {
-        switch (record.getType()) {
-            case CONTENTMISSING:
-            case DELETE:
-                return UpdateType.DELETE;
-            case CREATE:
-                return UpdateType.CREATE;
-            case UPDATE:
-                return UpdateType.UPDATE;
-            case SWAP_OUT:
-                return UpdateType.SWAP_OUT;
-            case SWAP_IN:
-                return UpdateType.SWAP_IN;
-        }
-        return null;
-    }
-
-    @Override
-    public String getLocation(final RepositoryRecord record) {
-        return record.getSwapLocation();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
deleted file mode 100644
index 221f8ce..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/*
- * 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.controller.repository;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
-import org.apache.nifi.controller.repository.schema.ContentClaimFieldMap;
-import org.apache.nifi.controller.repository.schema.ContentClaimSchema;
-import org.apache.nifi.controller.repository.schema.FlowFileSchema;
-import org.apache.nifi.controller.repository.schema.RepositoryRecordFieldMap;
-import org.apache.nifi.controller.repository.schema.RepositoryRecordSchema;
-import org.apache.nifi.controller.repository.schema.RepositoryRecordUpdate;
-import org.apache.nifi.repository.schema.FieldType;
-import org.apache.nifi.repository.schema.Record;
-import org.apache.nifi.repository.schema.RecordSchema;
-import org.apache.nifi.repository.schema.Repetition;
-import org.apache.nifi.repository.schema.SchemaRecordReader;
-import org.apache.nifi.repository.schema.SchemaRecordWriter;
-import org.apache.nifi.repository.schema.SimpleRecordField;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.wali.SerDe;
-import org.wali.UpdateType;
-
-public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
-    private static final Logger logger = LoggerFactory.getLogger(SchemaRepositoryRecordSerde.class);
-    private static final int MAX_ENCODING_VERSION = 2;
-
-    private final RecordSchema writeSchema = RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V2;
-    private final RecordSchema contentClaimSchema = ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1;
-
-    private final ResourceClaimManager resourceClaimManager;
-    private volatile RecordSchema recoverySchema;
-
-    public SchemaRepositoryRecordSerde(final ResourceClaimManager resourceClaimManager) {
-        this.resourceClaimManager = resourceClaimManager;
-    }
-
-    @Override
-    public void writeHeader(final DataOutputStream out) throws IOException {
-        writeSchema.writeTo(out);
-    }
-
-    @Override
-    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord newRecordState, final DataOutputStream out) throws IOException {
-        serializeRecord(newRecordState, out);
-    }
-
-    @Override
-    public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
-        final RecordSchema schema;
-        switch (record.getType()) {
-            case CREATE:
-            case UPDATE:
-                schema = RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V2;
-                break;
-            case CONTENTMISSING:
-            case DELETE:
-                schema = RepositoryRecordSchema.DELETE_SCHEMA_V2;
-                break;
-            case SWAP_IN:
-                schema = RepositoryRecordSchema.SWAP_IN_SCHEMA_V2;
-                break;
-            case SWAP_OUT:
-                schema = RepositoryRecordSchema.SWAP_OUT_SCHEMA_V2;
-                break;
-            default:
-                throw new IllegalArgumentException("Received Repository Record with unknown Update Type: " + record.getType()); // won't happen.
-        }
-
-        serializeRecord(record, out, schema, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V2);
-    }
-
-
-    protected void serializeRecord(final RepositoryRecord record, final DataOutputStream out, RecordSchema schema, RecordSchema repositoryRecordSchema) throws IOException {
-        final RepositoryRecordFieldMap fieldMap = new RepositoryRecordFieldMap(record, schema, contentClaimSchema);
-        final RepositoryRecordUpdate update = new RepositoryRecordUpdate(fieldMap, repositoryRecordSchema);
-        new SchemaRecordWriter().writeRecord(update, out);
-    }
-
-    @Override
-    public void readHeader(final DataInputStream in) throws IOException {
-        recoverySchema = RecordSchema.readFrom(in);
-    }
-
-    @Override
-    public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
-        return deserializeRecord(in, version);
-    }
-
-    @Override
-    public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
-        final SchemaRecordReader reader = SchemaRecordReader.fromSchema(recoverySchema);
-        final Record updateRecord = reader.readRecord(in);
-        if (updateRecord == null) {
-            // null may be returned by reader.readRecord() if it encounters end-of-stream
-            return null;
-        }
-
-        // Top level is always going to be a "Repository Record Update" record because we need a 'Union' type record at the
-        // top level that indicates which type of record we have.
-        final Record record = (Record) updateRecord.getFieldValue(RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V2);
-
-        final String actionType = (String) record.getFieldValue(RepositoryRecordSchema.ACTION_TYPE_FIELD);
-        final UpdateType updateType = UpdateType.valueOf(actionType);
-        switch (updateType) {
-            case CREATE:
-                return createRecord(record);
-            case DELETE:
-                return deleteRecord(record);
-            case SWAP_IN:
-                return swapInRecord(record);
-            case SWAP_OUT:
-                return swapOutRecord(record);
-            case UPDATE:
-                return updateRecord(record);
-            default:
-                throw new IOException("Found unrecognized Update Type '" + actionType + "'");
-        }
-    }
-
-
-    @SuppressWarnings("unchecked")
-    private StandardRepositoryRecord createRecord(final Record record) {
-        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
-        ffBuilder.id((Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID));
-        ffBuilder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE));
-
-        final Long lastQueueDate = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE);
-        final Long queueDateIndex = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX);
-        ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
-
-        final Long lineageStartDate = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE);
-        final Long lineageStartIndex = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX);
-        ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
-
-        populateContentClaim(ffBuilder, record);
-        ffBuilder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE));
-
-        ffBuilder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES));
-
-        final FlowFileRecord flowFileRecord = ffBuilder.build();
-
-        final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER);
-        final FlowFileQueue queue = getFlowFileQueue(queueId);
-
-        final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord(queue, flowFileRecord);
-        requireFlowFileQueue(repoRecord, queueId);
-        return repoRecord;
-    }
-
-    private void requireFlowFileQueue(final StandardRepositoryRecord repoRecord, final String queueId) {
-        if (queueId == null || queueId.trim().isEmpty()) {
-            logger.warn("{} does not have a Queue associated with it; this record will be discarded", repoRecord.getCurrent());
-            repoRecord.markForAbort();
-        } else if (repoRecord.getOriginalQueue() == null) {
-            logger.warn("{} maps to unknown Queue {}; this record will be discarded", repoRecord.getCurrent(), queueId);
-            repoRecord.markForAbort();
-        }
-    }
-
-    private void populateContentClaim(final StandardFlowFileRecord.Builder ffBuilder, final Record record) {
-        final Object claimMap = record.getFieldValue(FlowFileSchema.CONTENT_CLAIM);
-        if (claimMap == null) {
-            return;
-        }
-
-        final Record claimRecord = (Record) claimMap;
-        final ContentClaim contentClaim = ContentClaimFieldMap.getContentClaim(claimRecord, resourceClaimManager);
-        final Long offset = ContentClaimFieldMap.getContentClaimOffset(claimRecord);
-
-        ffBuilder.contentClaim(contentClaim);
-        ffBuilder.contentClaimOffset(offset);
-    }
-
-    private RepositoryRecord updateRecord(final Record record) {
-        return createRecord(record);
-    }
-
-    private RepositoryRecord deleteRecord(final Record record) {
-        final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
-        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
-        final FlowFileRecord flowFileRecord = ffBuilder.build();
-
-        final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
-        repoRecord.markForDelete();
-        return repoRecord;
-    }
-
-    private RepositoryRecord swapInRecord(final Record record) {
-        final StandardRepositoryRecord repoRecord = createRecord(record);
-        final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        repoRecord.setSwapLocation(swapLocation);
-
-        final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER);
-        requireFlowFileQueue(repoRecord, queueId);
-        return repoRecord;
-    }
-
-    private RepositoryRecord swapOutRecord(final Record record) {
-        final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
-        final String queueId = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
-        final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        final FlowFileQueue queue = getFlowFileQueue(queueId);
-
-        final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
-            .id(recordId)
-            .build();
-
-        return new StandardRepositoryRecord(queue, flowFileRecord, swapLocation);
-    }
-
-    @Override
-    public int getVersion() {
-        return MAX_ENCODING_VERSION;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java
deleted file mode 100644
index a1d5173..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java
+++ /dev/null
@@ -1,341 +0,0 @@
-/*
- * 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.controller.repository;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Pattern;
-
-import org.apache.commons.lang3.builder.CompareToBuilder;
-import org.apache.commons.lang3.builder.EqualsBuilder;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.commons.lang3.builder.ToStringBuilder;
-import org.apache.commons.lang3.builder.ToStringStyle;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.flowfile.FlowFile;
-import org.apache.nifi.flowfile.attributes.CoreAttributes;
-
-/**
- * <p>
- * A flow file is a logical notion of an item in a flow with its associated attributes and identity which can be used as a reference for its actual content.
- * </p>
- *
- * <b>Immutable - Thread Safe</b>
- *
- */
-public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
-
-    private final long id;
-    private final long entryDate;
-    private final long lineageStartDate;
-    private final long lineageStartIndex;
-    private final long size;
-    private final long penaltyExpirationMs;
-    private final Map<String, String> attributes;
-    private final ContentClaim claim;
-    private final long claimOffset;
-    private final long lastQueueDate;
-    private final long queueDateIndex;
-
-    private StandardFlowFileRecord(final Builder builder) {
-        this.id = builder.bId;
-        this.attributes = builder.bAttributes == null ? Collections.emptyMap() : builder.bAttributes;
-        this.entryDate = builder.bEntryDate;
-        this.lineageStartDate = builder.bLineageStartDate;
-        this.lineageStartIndex = builder.bLineageStartIndex;
-        this.penaltyExpirationMs = builder.bPenaltyExpirationMs;
-        this.size = builder.bSize;
-        this.claim = builder.bClaim;
-        this.claimOffset = builder.bClaimOffset;
-        this.lastQueueDate = builder.bLastQueueDate;
-        this.queueDateIndex = builder.bQueueDateIndex;
-    }
-
-    @Override
-    public long getId() {
-        return id;
-    }
-
-    @Override
-    public long getEntryDate() {
-        return entryDate;
-    }
-
-    @Override
-    public long getLineageStartDate() {
-        return lineageStartDate;
-    }
-
-    @Override
-    public Long getLastQueueDate() {
-        return lastQueueDate;
-    }
-
-    @Override
-    public boolean isPenalized() {
-        return penaltyExpirationMs > 0 ? penaltyExpirationMs > System.currentTimeMillis() : false;
-    }
-
-    @Override
-    public String getAttribute(final String key) {
-        return attributes.get(key);
-    }
-
-    @Override
-    public long getSize() {
-        return size;
-    }
-
-    @Override
-    public Map<String, String> getAttributes() {
-        return Collections.unmodifiableMap(this.attributes);
-    }
-
-    @Override
-    public ContentClaim getContentClaim() {
-        return this.claim;
-    }
-
-    @Override
-    public long getContentClaimOffset() {
-        return this.claimOffset;
-    }
-
-    @Override
-    public long getLineageStartIndex() {
-        return lineageStartIndex;
-    }
-
-    @Override
-    public long getQueueDateIndex() {
-        return queueDateIndex;
-    }
-
-    /**
-     * Provides the natural ordering for FlowFile objects which is based on their identifier.
-     *
-     * @param other other
-     * @return standard compare contract
-     */
-    @Override
-    public int compareTo(final FlowFile other) {
-        return new CompareToBuilder().append(id, other.getId()).toComparison();
-    }
-
-    @Override
-    public boolean equals(final Object other) {
-        if (this == other) {
-            return true;
-        }
-        if (!(other instanceof FlowFile)) {
-            return false;
-        }
-        final FlowFile otherRecord = (FlowFile) other;
-        return new EqualsBuilder().append(id, otherRecord.getId()).isEquals();
-    }
-
-    @Override
-    public String toString() {
-        final ToStringBuilder builder = new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE);
-        builder.append("uuid", getAttribute(CoreAttributes.UUID.key()));
-        builder.append("claim", claim == null ? "" : claim.toString());
-        builder.append("offset", claimOffset);
-        builder.append("name", getAttribute(CoreAttributes.FILENAME.key())).append("size", size);
-        return builder.toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return new HashCodeBuilder(7, 13).append(id).toHashCode();
-    }
-
-    public static final class Builder {
-
-        private long bId;
-        private long bEntryDate = System.currentTimeMillis();
-        private long bLineageStartDate = bEntryDate;
-        private long bLineageStartIndex = 0L;
-        private final Set<String> bLineageIdentifiers = new HashSet<>();
-        private long bPenaltyExpirationMs = -1L;
-        private long bSize = 0L;
-        private ContentClaim bClaim = null;
-        private long bClaimOffset = 0L;
-        private long bLastQueueDate = System.currentTimeMillis();
-        private long bQueueDateIndex = 0L;
-        private Map<String, String> bAttributes;
-        private boolean bAttributesCopied = false;
-
-        public Builder id(final long id) {
-            bId = id;
-            return this;
-        }
-
-        public Builder entryDate(final long epochMs) {
-            bEntryDate = epochMs;
-            return this;
-        }
-
-        public Builder lineageStart(final long lineageStartDate, final long lineageStartIndex) {
-            bLineageStartDate = lineageStartDate;
-            bLineageStartIndex = lineageStartIndex;
-            return this;
-        }
-
-        public Builder penaltyExpirationTime(final long epochMilliseconds) {
-            bPenaltyExpirationMs = epochMilliseconds;
-            return this;
-        }
-
-        public Builder size(final long bytes) {
-            if (bytes >= 0) {
-                bSize = bytes;
-            }
-            return this;
-        }
-
-        private Map<String, String> initializeAttributes() {
-            if (bAttributes == null) {
-                bAttributes = new HashMap<>();
-                bAttributesCopied = true;
-            } else if (!bAttributesCopied) {
-                bAttributes = new HashMap<>(bAttributes);
-                bAttributesCopied = true;
-            }
-
-            return bAttributes;
-        }
-
-        public Builder addAttribute(final String key, final String value) {
-            if (key != null && value != null) {
-                initializeAttributes().put(FlowFile.KeyValidator.validateKey(key), value);
-            }
-            return this;
-        }
-
-        public Builder addAttributes(final Map<String, String> attributes) {
-            final Map<String, String> initializedAttributes = initializeAttributes();
-
-            if (null != attributes) {
-                for (final String key : attributes.keySet()) {
-                    FlowFile.KeyValidator.validateKey(key);
-                }
-                for (final Map.Entry<String, String> entry : attributes.entrySet()) {
-                    final String key = entry.getKey();
-                    final String value = entry.getValue();
-                    if (key != null && value != null) {
-                        initializedAttributes.put(key, value);
-                    }
-                }
-            }
-            return this;
-        }
-
-        public Builder removeAttributes(final String... keys) {
-            if (keys != null) {
-                for (final String key : keys) {
-                    if (CoreAttributes.UUID.key().equals(key)) {
-                        continue;
-                    }
-
-                    initializeAttributes().remove(key);
-                }
-            }
-            return this;
-        }
-
-        public Builder removeAttributes(final Set<String> keys) {
-            if (keys != null) {
-                for (final String key : keys) {
-                    if (CoreAttributes.UUID.key().equals(key)) {
-                        continue;
-                    }
-
-                    initializeAttributes().remove(key);
-                }
-            }
-            return this;
-        }
-
-        public Builder removeAttributes(final Pattern keyPattern) {
-            if (keyPattern != null) {
-                final Iterator<String> iterator = initializeAttributes().keySet().iterator();
-                while (iterator.hasNext()) {
-                    final String key = iterator.next();
-
-                    if (CoreAttributes.UUID.key().equals(key)) {
-                        continue;
-                    }
-
-                    if (keyPattern.matcher(key).matches()) {
-                        iterator.remove();
-                    }
-                }
-            }
-            return this;
-        }
-
-        public Builder contentClaim(final ContentClaim claim) {
-            this.bClaim = claim;
-            return this;
-        }
-
-        public Builder contentClaimOffset(final long offset) {
-            this.bClaimOffset = offset;
-            return this;
-        }
-
-        public Builder lastQueued(final long lastQueueDate, final long queueDateIndex) {
-            this.bLastQueueDate = lastQueueDate;
-            this.bQueueDateIndex = queueDateIndex;
-            return this;
-        }
-
-        public Builder fromFlowFile(final FlowFileRecord specFlowFile) {
-            if (specFlowFile == null) {
-                return this;
-            }
-            bId = specFlowFile.getId();
-            bEntryDate = specFlowFile.getEntryDate();
-            bLineageStartDate = specFlowFile.getLineageStartDate();
-            bLineageStartIndex = specFlowFile.getLineageStartIndex();
-            bLineageIdentifiers.clear();
-            bPenaltyExpirationMs = specFlowFile.getPenaltyExpirationMillis();
-            bSize = specFlowFile.getSize();
-            bAttributes = specFlowFile.getAttributes();
-            bAttributesCopied = false;
-            bClaim = specFlowFile.getContentClaim();
-            bClaimOffset = specFlowFile.getContentClaimOffset();
-            bLastQueueDate = specFlowFile.getLastQueueDate();
-            bQueueDateIndex = specFlowFile.getQueueDateIndex();
-
-            return this;
-        }
-
-        public FlowFileRecord build() {
-            return new StandardFlowFileRecord(this);
-        }
-    }
-
-    @Override
-    public long getPenaltyExpirationMillis() {
-        return penaltyExpirationMs;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java
deleted file mode 100644
index 8aa1caf..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/*
- * 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.controller.repository;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.processor.Relationship;
-
-public class StandardRepositoryRecord implements RepositoryRecord {
-
-    private RepositoryRecordType type = null;
-    private FlowFileRecord workingFlowFileRecord = null;
-    private Relationship transferRelationship = null;
-    private FlowFileQueue destination = null;
-    private final FlowFileRecord originalFlowFileRecord;
-    private final FlowFileQueue originalQueue;
-    private String swapLocation;
-    private final Map<String, String> updatedAttributes = new HashMap<>();
-    private final Map<String, String> originalAttributes;
-    private List<ContentClaim> transientClaims;
-
-    /**
-     * Creates a new record which has no original claim or flow file - it is entirely new
-     *
-     * @param originalQueue queue
-     */
-    public StandardRepositoryRecord(final FlowFileQueue originalQueue) {
-        this(originalQueue, null);
-        this.type = RepositoryRecordType.CREATE;
-    }
-
-    /**
-     * Creates a record based on given original items
-     *
-     * @param originalQueue queue
-     * @param originalFlowFileRecord record
-     */
-    public StandardRepositoryRecord(final FlowFileQueue originalQueue, final FlowFileRecord originalFlowFileRecord) {
-        this(originalQueue, originalFlowFileRecord, null);
-        this.type = RepositoryRecordType.UPDATE;
-    }
-
-    public StandardRepositoryRecord(final FlowFileQueue originalQueue, final FlowFileRecord originalFlowFileRecord, final String swapLocation) {
-        this.originalQueue = originalQueue;
-        this.originalFlowFileRecord = originalFlowFileRecord;
-        this.type = RepositoryRecordType.SWAP_OUT;
-        this.swapLocation = swapLocation;
-        this.originalAttributes = originalFlowFileRecord == null ? Collections.<String, String>emptyMap() : originalFlowFileRecord.getAttributes();
-    }
-
-    @Override
-    public FlowFileQueue getDestination() {
-        return destination;
-    }
-
-    public void setDestination(final FlowFileQueue destination) {
-        this.destination = destination;
-    }
-
-    @Override
-    public RepositoryRecordType getType() {
-        return type;
-    }
-
-    FlowFileRecord getOriginal() {
-        return originalFlowFileRecord;
-    }
-
-    @Override
-    public String getSwapLocation() {
-        return swapLocation;
-    }
-
-    public void setSwapLocation(final String swapLocation) {
-        this.swapLocation = swapLocation;
-        if (type != RepositoryRecordType.SWAP_OUT) {
-            type = RepositoryRecordType.SWAP_IN; // we are swapping in a new record
-        }
-    }
-
-    @Override
-    public ContentClaim getOriginalClaim() {
-        return (originalFlowFileRecord == null) ? null : originalFlowFileRecord.getContentClaim();
-    }
-
-    @Override
-    public FlowFileQueue getOriginalQueue() {
-        return originalQueue;
-    }
-
-    public void setWorking(final FlowFileRecord flowFile) {
-        workingFlowFileRecord = flowFile;
-    }
-
-    public void setWorking(final FlowFileRecord flowFile, final String attributeKey, final String attributeValue) {
-        workingFlowFileRecord = flowFile;
-
-        // If setting attribute to same value as original, don't add to updated attributes
-        final String currentValue = originalAttributes.get(attributeKey);
-        if (currentValue == null || !currentValue.equals(attributeValue)) {
-            updatedAttributes.put(attributeKey, attributeValue);
-        }
-    }
-
-    public void setWorking(final FlowFileRecord flowFile, final Map<String, String> updatedAttribs) {
-        workingFlowFileRecord = flowFile;
-
-        for (final Map.Entry<String, String> entry : updatedAttribs.entrySet()) {
-            final String currentValue = originalAttributes.get(entry.getKey());
-            if (currentValue == null || !currentValue.equals(entry.getValue())) {
-                updatedAttributes.put(entry.getKey(), entry.getValue());
-            }
-        }
-    }
-
-    @Override
-    public boolean isAttributesChanged() {
-        return !updatedAttributes.isEmpty();
-    }
-
-    public void markForAbort() {
-        type = RepositoryRecordType.CONTENTMISSING;
-    }
-
-    @Override
-    public boolean isMarkedForAbort() {
-        return RepositoryRecordType.CONTENTMISSING.equals(type);
-    }
-
-    public void markForDelete() {
-        type = RepositoryRecordType.DELETE;
-    }
-
-    public boolean isMarkedForDelete() {
-        return RepositoryRecordType.DELETE.equals(type);
-    }
-
-    public void setTransferRelationship(final Relationship relationship) {
-        transferRelationship = relationship;
-    }
-
-    public Relationship getTransferRelationship() {
-        return transferRelationship;
-    }
-
-    FlowFileRecord getWorking() {
-        return workingFlowFileRecord;
-    }
-
-    ContentClaim getWorkingClaim() {
-        return (workingFlowFileRecord == null) ? null : workingFlowFileRecord.getContentClaim();
-    }
-
-    @Override
-    public FlowFileRecord getCurrent() {
-        return (workingFlowFileRecord == null) ? originalFlowFileRecord : workingFlowFileRecord;
-    }
-
-    @Override
-    public ContentClaim getCurrentClaim() {
-        return (getCurrent() == null) ? null : getCurrent().getContentClaim();
-    }
-
-    @Override
-    public long getCurrentClaimOffset() {
-        return (getCurrent() == null) ? 0L : getCurrent().getContentClaimOffset();
-    }
-
-    boolean isWorking() {
-        return (workingFlowFileRecord != null);
-    }
-
-    Map<String, String> getOriginalAttributes() {
-        return originalAttributes;
-    }
-
-    Map<String, String> getUpdatedAttributes() {
-        return updatedAttributes;
-    }
-
-    @Override
-    public String toString() {
-        return "StandardRepositoryRecord[UpdateType=" + getType() + ",Record=" + getCurrent() + "]";
-    }
-
-    @Override
-    public List<ContentClaim> getTransientClaims() {
-        return transientClaims == null ? Collections.<ContentClaim> emptyList() : Collections.unmodifiableList(transientClaims);
-    }
-
-    void addTransientClaim(final ContentClaim claim) {
-        if (claim == null) {
-            return;
-        }
-
-        if (transientClaims == null) {
-            transientClaims = new ArrayList<>();
-        }
-        transientClaims.add(claim);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
deleted file mode 100644
index e8ce44e..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
+++ /dev/null
@@ -1,517 +0,0 @@
-/*
- * 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.controller.repository;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
-import org.apache.nifi.controller.repository.claim.StandardContentClaim;
-import org.apache.nifi.flowfile.FlowFile;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.wali.SerDe;
-import org.wali.UpdateType;
-
-public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
-    private static final Logger logger = LoggerFactory.getLogger(WriteAheadRepositoryRecordSerde.class);
-
-    private static final int CURRENT_ENCODING_VERSION = 9;
-
-    public static final byte ACTION_CREATE = 0;
-    public static final byte ACTION_UPDATE = 1;
-    public static final byte ACTION_DELETE = 2;
-    public static final byte ACTION_SWAPPED_OUT = 3;
-    public static final byte ACTION_SWAPPED_IN = 4;
-
-    private long recordsRestored = 0L;
-    private final ResourceClaimManager claimManager;
-
-    public WriteAheadRepositoryRecordSerde(final ResourceClaimManager claimManager) {
-        this.claimManager = claimManager;
-    }
-
-    @Override
-    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out) throws IOException {
-        serializeEdit(previousRecordState, record, out, false);
-    }
-
-    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out, final boolean forceAttributesWritten) throws IOException {
-        if (record.isMarkedForAbort()) {
-            logger.warn("Repository Record {} is marked to be aborted; it will be persisted in the FlowFileRepository as a DELETE record", record);
-            out.write(ACTION_DELETE);
-            out.writeLong(getRecordIdentifier(record));
-            serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
-            return;
-        }
-
-        final UpdateType updateType = getUpdateType(record);
-
-        if (updateType.equals(UpdateType.DELETE)) {
-            out.write(ACTION_DELETE);
-            out.writeLong(getRecordIdentifier(record));
-            serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
-            return;
-        }
-
-        // If there's a Destination Connection, that's the one that we want to associated with this record.
-        // However, on restart, we will restore the FlowFile and set this connection to its "originalConnection".
-        // If we then serialize the FlowFile again before it's transferred, it's important to allow this to happen,
-        // so we use the originalConnection instead
-        FlowFileQueue associatedQueue = record.getDestination();
-        if (associatedQueue == null) {
-            associatedQueue = record.getOriginalQueue();
-        }
-
-        if (updateType.equals(UpdateType.SWAP_OUT)) {
-            out.write(ACTION_SWAPPED_OUT);
-            out.writeLong(getRecordIdentifier(record));
-            out.writeUTF(associatedQueue.getIdentifier());
-            out.writeUTF(getLocation(record));
-            return;
-        }
-
-        final FlowFile flowFile = record.getCurrent();
-        final ContentClaim claim = record.getCurrentClaim();
-
-        switch (updateType) {
-            case UPDATE:
-                out.write(ACTION_UPDATE);
-                break;
-            case CREATE:
-                out.write(ACTION_CREATE);
-                break;
-            case SWAP_IN:
-                out.write(ACTION_SWAPPED_IN);
-                break;
-            default:
-                throw new AssertionError();
-        }
-
-        out.writeLong(getRecordIdentifier(record));
-        out.writeLong(flowFile.getEntryDate());
-        out.writeLong(flowFile.getLineageStartDate());
-        out.writeLong(flowFile.getLineageStartIndex());
-
-        final Long queueDate = flowFile.getLastQueueDate();
-        out.writeLong(queueDate == null ? System.currentTimeMillis() : queueDate);
-        out.writeLong(flowFile.getQueueDateIndex());
-        out.writeLong(flowFile.getSize());
-
-        if (associatedQueue == null) {
-            logger.warn("{} Repository Record {} has no Connection associated with it; it will be destroyed on restart",
-                new Object[] {this, record});
-            writeString("", out);
-        } else {
-            writeString(associatedQueue.getIdentifier(), out);
-        }
-
-        serializeContentClaim(claim, record.getCurrentClaimOffset(), out);
-
-        if (forceAttributesWritten || record.isAttributesChanged() || updateType == UpdateType.CREATE || updateType == UpdateType.SWAP_IN) {
-            out.write(1);   // indicate attributes changed
-            final Map<String, String> attributes = flowFile.getAttributes();
-            out.writeInt(attributes.size());
-            for (final Map.Entry<String, String> entry : attributes.entrySet()) {
-                writeString(entry.getKey(), out);
-                writeString(entry.getValue(), out);
-            }
-        } else {
-            out.write(0);   // indicate attributes did not change
-        }
-
-        if (updateType == UpdateType.SWAP_IN) {
-            out.writeUTF(record.getSwapLocation());
-        }
-    }
-
-    @Override
-    public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
-        final int action = in.read();
-        final long recordId = in.readLong();
-        if (action == ACTION_DELETE) {
-            final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
-
-            if (version > 4) {
-                deserializeClaim(in, version, ffBuilder);
-            }
-
-            final FlowFileRecord flowFileRecord = ffBuilder.build();
-            final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
-            record.markForDelete();
-
-            return record;
-        }
-
-        if (action == ACTION_SWAPPED_OUT) {
-            final String queueId = in.readUTF();
-            final String location = in.readUTF();
-            final FlowFileQueue queue = getFlowFileQueue(queueId);
-
-            final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
-                .id(recordId)
-                .build();
-
-            return new StandardRepositoryRecord(queue, flowFileRecord, location);
-        }
-
-        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
-        final RepositoryRecord record = currentRecordStates.get(recordId);
-        ffBuilder.id(recordId);
-        if (record != null) {
-            ffBuilder.fromFlowFile(record.getCurrent());
-        }
-        ffBuilder.entryDate(in.readLong());
-
-        if (version > 1) {
-            // read the lineage identifiers and lineage start date, which were added in version 2.
-            if (version < 9) {
-                final int numLineageIds = in.readInt();
-                for (int i = 0; i < numLineageIds; i++) {
-                    in.readUTF(); //skip identifiers
-                }
-            }
-            final long lineageStartDate = in.readLong();
-            final long lineageStartIndex;
-            if (version > 7) {
-                lineageStartIndex = in.readLong();
-            } else {
-                lineageStartIndex = 0L;
-            }
-            ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
-
-            if (version > 5) {
-                final long lastQueueDate = in.readLong();
-                final long queueDateIndex;
-                if (version > 7) {
-                    queueDateIndex = in.readLong();
-                } else {
-                    queueDateIndex = 0L;
-                }
-
-                ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
-            }
-        }
-
-        ffBuilder.size(in.readLong());
-        final String connectionId = readString(in);
-
-        logger.debug("{} -> {}", new Object[] {recordId, connectionId});
-
-        deserializeClaim(in, version, ffBuilder);
-
-        // recover new attributes, if they changed
-        final int attributesChanged = in.read();
-        if (attributesChanged == -1) {
-            throw new EOFException();
-        } else if (attributesChanged == 1) {
-            final int numAttributes = in.readInt();
-            final Map<String, String> attributes = new HashMap<>();
-            for (int i = 0; i < numAttributes; i++) {
-                final String key = readString(in);
-                final String value = readString(in);
-                attributes.put(key, value);
-            }
-
-            ffBuilder.addAttributes(attributes);
-        } else if (attributesChanged != 0) {
-            throw new IOException("Attribute Change Qualifier not found in stream; found value: "
-                + attributesChanged + " after successfully restoring " + recordsRestored + " records. The FlowFile Repository appears to be corrupt!");
-        }
-
-        final FlowFileRecord flowFile = ffBuilder.build();
-        String swapLocation = null;
-        if (action == ACTION_SWAPPED_IN) {
-            swapLocation = in.readUTF();
-        }
-
-        final FlowFileQueue queue = getFlowFileQueue(connectionId);
-        final StandardRepositoryRecord standardRepoRecord = new StandardRepositoryRecord(queue, flowFile);
-        if (swapLocation != null) {
-            standardRepoRecord.setSwapLocation(swapLocation);
-        }
-
-        if (connectionId.isEmpty()) {
-            logger.warn("{} does not have a Queue associated with it; this record will be discarded", flowFile);
-            standardRepoRecord.markForAbort();
-        } else if (queue == null) {
-            logger.warn("{} maps to unknown Queue {}; this record will be discarded", flowFile, connectionId);
-            standardRepoRecord.markForAbort();
-        }
-
-        recordsRestored++;
-        return standardRepoRecord;
-    }
-
-    @Override
-    public StandardRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
-        final int action = in.read();
-        if (action == -1) {
-            return null;
-        }
-
-        final long recordId = in.readLong();
-        if (action == ACTION_DELETE) {
-            final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
-
-            if (version > 4) {
-                deserializeClaim(in, version, ffBuilder);
-            }
-
-            final FlowFileRecord flowFileRecord = ffBuilder.build();
-            final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
-            record.markForDelete();
-            return record;
-        }
-
-        // if action was not delete, it must be create/swap in
-        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
-        final long entryDate = in.readLong();
-
-        if (version > 1) {
-            // read the lineage identifiers and lineage start date, which were added in version 2.
-            if (version < 9) {
-                final int numLineageIds = in.readInt();
-                for (int i = 0; i < numLineageIds; i++) {
-                    in.readUTF(); //skip identifiers
-                }
-            }
-
-            final long lineageStartDate = in.readLong();
-            final long lineageStartIndex;
-            if (version > 7) {
-                lineageStartIndex = in.readLong();
-            } else {
-                lineageStartIndex = 0L;
-            }
-            ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
-
-            if (version > 5) {
-                final long lastQueueDate = in.readLong();
-                final long queueDateIndex;
-                if (version > 7) {
-                    queueDateIndex = in.readLong();
-                } else {
-                    queueDateIndex = 0L;
-                }
-
-                ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
-            }
-        }
-
-        final long size = in.readLong();
-        final String connectionId = readString(in);
-
-        logger.debug("{} -> {}", new Object[] {recordId, connectionId});
-
-        ffBuilder.id(recordId);
-        ffBuilder.entryDate(entryDate);
-        ffBuilder.size(size);
-
-        deserializeClaim(in, version, ffBuilder);
-
-        final int attributesChanged = in.read();
-        if (attributesChanged == 1) {
-            final int numAttributes = in.readInt();
-            final Map<String, String> attributes = new HashMap<>();
-            for (int i = 0; i < numAttributes; i++) {
-                final String key = readString(in);
-                final String value = readString(in);
-                attributes.put(key, value);
-            }
-
-            ffBuilder.addAttributes(attributes);
-        } else if (attributesChanged == -1) {
-            throw new EOFException();
-        } else if (attributesChanged != 0) {
-            throw new IOException("Attribute Change Qualifier not found in stream; found value: "
-                + attributesChanged + " after successfully restoring " + recordsRestored + " records");
-        }
-
-        final FlowFileRecord flowFile = ffBuilder.build();
-        String swapLocation = null;
-        if (action == ACTION_SWAPPED_IN) {
-            swapLocation = in.readUTF();
-        }
-
-        final StandardRepositoryRecord record;
-        final FlowFileQueue queue = getFlowFileQueue(connectionId);
-        record = new StandardRepositoryRecord(queue, flowFile);
-        if (swapLocation != null) {
-            record.setSwapLocation(swapLocation);
-        }
-
-        if (connectionId.isEmpty()) {
-            logger.warn("{} does not have a FlowFile Queue associated with it; this record will be discarded", flowFile);
-            record.markForAbort();
-        } else if (queue == null) {
-            logger.warn("{} maps to unknown FlowFile Queue {}; this record will be discarded", flowFile, connectionId);
-            record.markForAbort();
-        }
-
-        recordsRestored++;
-        return record;
-    }
-
-    @Override
-    public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
-        serializeEdit(null, record, out, true);
-    }
-
-    private void serializeContentClaim(final ContentClaim claim, final long offset, final DataOutputStream out) throws IOException {
-        if (claim == null) {
-            out.write(0);
-        } else {
-            out.write(1);
-
-            final ResourceClaim resourceClaim = claim.getResourceClaim();
-            writeString(resourceClaim.getId(), out);
-            writeString(resourceClaim.getContainer(), out);
-            writeString(resourceClaim.getSection(), out);
-            out.writeLong(claim.getOffset());
-            out.writeLong(claim.getLength());
-
-            out.writeLong(offset);
-            out.writeBoolean(resourceClaim.isLossTolerant());
-        }
-    }
-
-    private void deserializeClaim(final DataInputStream in, final int serializationVersion, final StandardFlowFileRecord.Builder ffBuilder) throws IOException {
-        // determine current Content Claim.
-        final int claimExists = in.read();
-        if (claimExists == 1) {
-            final String claimId;
-            if (serializationVersion < 4) {
-                claimId = String.valueOf(in.readLong());
-            } else {
-                claimId = readString(in);
-            }
-
-            final String container = readString(in);
-            final String section = readString(in);
-
-            final long resourceOffset;
-            final long resourceLength;
-            if (serializationVersion < 7) {
-                resourceOffset = 0L;
-                resourceLength = -1L;
-            } else {
-                resourceOffset = in.readLong();
-                resourceLength = in.readLong();
-            }
-
-            final long claimOffset = in.readLong();
-
-            final boolean lossTolerant;
-            if (serializationVersion >= 3) {
-                lossTolerant = in.readBoolean();
-            } else {
-                lossTolerant = false;
-            }
-
-            final ResourceClaim resourceClaim = claimManager.newResourceClaim(container, section, claimId, lossTolerant, false);
-            final StandardContentClaim contentClaim = new StandardContentClaim(resourceClaim, resourceOffset);
-            contentClaim.setLength(resourceLength);
-
-            ffBuilder.contentClaim(contentClaim);
-            ffBuilder.contentClaimOffset(claimOffset);
-        } else if (claimExists == -1) {
-            throw new EOFException();
-        } else if (claimExists != 0) {
-            throw new IOException("Claim Existence Qualifier not found in stream; found value: "
-                + claimExists + " after successfully restoring " + recordsRestored + " records");
-        }
-    }
-
-    private void writeString(final String toWrite, final OutputStream out) throws IOException {
-        final byte[] bytes = toWrite.getBytes("UTF-8");
-        final int utflen = bytes.length;
-
-        if (utflen < 65535) {
-            out.write(utflen >>> 8);
-            out.write(utflen);
-            out.write(bytes);
-        } else {
-            out.write(255);
-            out.write(255);
-            out.write(utflen >>> 24);
-            out.write(utflen >>> 16);
-            out.write(utflen >>> 8);
-            out.write(utflen);
-            out.write(bytes);
-        }
-    }
-
-    private String readString(final InputStream in) throws IOException {
-        final Integer numBytes = readFieldLength(in);
-        if (numBytes == null) {
-            throw new EOFException();
-        }
-        final byte[] bytes = new byte[numBytes];
-        fillBuffer(in, bytes, numBytes);
-        return new String(bytes, "UTF-8");
-    }
-
-    private Integer readFieldLength(final InputStream in) throws IOException {
-        final int firstValue = in.read();
-        final int secondValue = in.read();
-        if (firstValue < 0) {
-            return null;
-        }
-        if (secondValue < 0) {
-            throw new EOFException();
-        }
-        if (firstValue == 0xff && secondValue == 0xff) {
-            final int ch1 = in.read();
-            final int ch2 = in.read();
-            final int ch3 = in.read();
-            final int ch4 = in.read();
-            if ((ch1 | ch2 | ch3 | ch4) < 0) {
-                throw new EOFException();
-            }
-            return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4;
-        } else {
-            return (firstValue << 8) + secondValue;
-        }
-    }
-
-    private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
-        int bytesRead;
-        int totalBytesRead = 0;
-        while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
-            totalBytesRead += bytesRead;
-        }
-        if (totalBytesRead != length) {
-            throw new EOFException();
-        }
-    }
-
-    @Override
-    public int getVersion() {
-        return CURRENT_ENCODING_VERSION;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
deleted file mode 100644
index 39a2591..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * 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.controller.repository.claim;
-
-
-/**
- * <p>
- * A ContentClaim is a reference to a given flow file's content. Multiple flow files may reference the same content by both having the same content claim.</p>
- *
- * <p>
- * Must be thread safe</p>
- *
- */
-public final class StandardContentClaim implements ContentClaim, Comparable<ContentClaim> {
-
-    private final ResourceClaim resourceClaim;
-    private final long offset;
-    private volatile long length;
-
-    public StandardContentClaim(final ResourceClaim resourceClaim, final long offset) {
-        this.resourceClaim = resourceClaim;
-        this.offset = offset;
-        this.length = -1L;
-    }
-
-    public void setLength(final long length) {
-        this.length = length;
-    }
-
-    @Override
-    public int hashCode() {
-        final int prime = 31;
-        int result = 1;
-        result = prime * result;
-        result = prime * result + (int) (offset ^ offset >>> 32);
-        result = prime * result + (resourceClaim == null ? 0 : resourceClaim.hashCode());
-        return result;
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (this == obj) {
-            return true;
-        }
-
-        if (obj == null) {
-            return false;
-        }
-
-        if (!(obj instanceof ContentClaim)) {
-            return false;
-        }
-
-        final ContentClaim other = (ContentClaim) obj;
-        if (offset != other.getOffset()) {
-            return false;
-        }
-
-        return resourceClaim.equals(other.getResourceClaim());
-    }
-
-    @Override
-    public int compareTo(final ContentClaim o) {
-        final int resourceComp = resourceClaim.compareTo(o.getResourceClaim());
-        if (resourceComp != 0) {
-            return resourceComp;
-        }
-
-        return Long.compare(offset, o.getOffset());
-    }
-
-    @Override
-    public ResourceClaim getResourceClaim() {
-        return resourceClaim;
-    }
-
-    @Override
-    public long getOffset() {
-        return offset;
-    }
-
-    @Override
-    public long getLength() {
-        return length;
-    }
-
-    @Override
-    public String toString() {
-        return "StandardContentClaim [resourceClaim=" + resourceClaim + ", offset=" + offset + ", length=" + length + "]";
-    }
-}


[2/5] nifi git commit: NIFI-3273: Added nifi-toolkit-flowfile-repo that contains a simple Java class that is capable of recovering a FlowFile Repository manually in the case of an operating system crash that results in trailing 0's being dumped into the

Posted by jo...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java
deleted file mode 100644
index 7e87199..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaim.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * 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.controller.repository.claim;
-
-public class StandardResourceClaim implements ResourceClaim, Comparable<ResourceClaim> {
-    private final StandardResourceClaimManager claimManager;
-    private final String id;
-    private final String container;
-    private final String section;
-    private final boolean lossTolerant;
-    private final int hashCode;
-    private volatile boolean writable = true;
-
-    public StandardResourceClaim(final StandardResourceClaimManager claimManager, final String container, final String section, final String id, final boolean lossTolerant) {
-        this.claimManager = claimManager;
-        this.container = container.intern();
-        this.section = section.intern();
-        this.id = id;
-        this.lossTolerant = lossTolerant;
-
-        hashCode = 17 + 19 * id.hashCode() + 19 * container.hashCode() + 19 * section.hashCode();
-    }
-
-    @Override
-    public boolean isLossTolerant() {
-        return lossTolerant;
-    }
-
-    /**
-     * @return the unique identifier for this claim
-     */
-    @Override
-    public String getId() {
-        return id;
-    }
-
-    /**
-     * @return the container identifier in which this claim is held
-     */
-    @Override
-    public String getContainer() {
-        return container;
-    }
-
-    /**
-     * @return the section within a given container the claim is held
-     */
-    @Override
-    public String getSection() {
-        return section;
-    }
-
-    @Override
-    public boolean equals(final Object other) {
-        if (this == other) {
-            return true;
-        }
-
-        if (other == null) {
-            return false;
-        }
-        if (hashCode != other.hashCode()) {
-            // We check hash code before instanceof because instanceof is fairly expensive and for
-            // StandardResourceClaim, calling hashCode() simply returns a pre-calculated value.
-            return false;
-        }
-
-        if (!(other instanceof ResourceClaim)) {
-            return false;
-        }
-        final ResourceClaim otherClaim = (ResourceClaim) other;
-        return id.equals(otherClaim.getId()) && container.equals(otherClaim.getContainer()) && section.equals(otherClaim.getSection());
-    }
-
-    @Override
-    public int hashCode() {
-        return hashCode;
-    }
-
-    @Override
-    public String toString() {
-        return "StandardResourceClaim[id=" + id + ", container=" + container + ", section=" + section + "]";
-    }
-
-    @Override
-    public boolean isWritable() {
-        return writable;
-    }
-
-    /**
-     * Freeze the Resource Claim so that it can now longer be written to
-     */
-    void freeze() {
-        this.writable = false;
-    }
-
-    @Override
-    public boolean isInUse() {
-        // Note that it is critical here that we always check isWritable() BEFORE checking
-        // the claimant count. This is due to the fact that if the claim is in fact writable, the claimant count
-        // could increase. So if we first check claimant count and that is 0, and then we check isWritable, it may be
-        // that the claimant count has changed to 1 before checking isWritable.
-        // However, if isWritable() is false, then the only way that the claimant count can increase is if a FlowFile referencing
-        // the Resource Claim is cloned. In this case, though, the claimant count has not become 0.
-        // Said another way, if isWritable() == false, then the claimant count can never increase from 0.
-        return isWritable() || claimManager.getClaimantCount(this) > 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java
deleted file mode 100644
index e4f060e..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/claim/StandardResourceClaimManager.java
+++ /dev/null
@@ -1,219 +0,0 @@
-/*
- * 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.controller.repository.claim;
-
-import java.util.Collection;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class StandardResourceClaimManager implements ResourceClaimManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(StandardResourceClaimManager.class);
-    private final ConcurrentMap<ResourceClaim, ClaimCount> claimantCounts = new ConcurrentHashMap<>();
-    private final BlockingQueue<ResourceClaim> destructableClaims = new LinkedBlockingQueue<>(50000);
-
-    @Override
-    public ResourceClaim newResourceClaim(final String container, final String section, final String id, final boolean lossTolerant, final boolean writable) {
-        final StandardResourceClaim claim = new StandardResourceClaim(this, container, section, id, lossTolerant);
-        if (!writable) {
-            claim.freeze();
-        }
-        return claim;
-    }
-
-    @Override
-    public ResourceClaim getResourceClaim(final String container, final String section, final String id) {
-        final ResourceClaim tempClaim = new StandardResourceClaim(this, container, section, id, false);
-        final ClaimCount count = claimantCounts.get(tempClaim);
-        return (count == null) ? null : count.getClaim();
-    }
-
-    private AtomicInteger getCounter(final ResourceClaim claim) {
-        if (claim == null) {
-            return null;
-        }
-
-        ClaimCount counter = claimantCounts.get(claim);
-        if (counter != null) {
-            return counter.getCount();
-        }
-
-        counter = new ClaimCount(claim, new AtomicInteger(0));
-        final ClaimCount existingCounter = claimantCounts.putIfAbsent(claim, counter);
-        return existingCounter == null ? counter.getCount() : existingCounter.getCount();
-    }
-
-    @Override
-    public int getClaimantCount(final ResourceClaim claim) {
-        if (claim == null) {
-            return 0;
-        }
-
-        synchronized (claim) {
-            final ClaimCount counter = claimantCounts.get(claim);
-            return counter == null ? 0 : counter.getCount().get();
-        }
-    }
-
-    @Override
-    public int decrementClaimantCount(final ResourceClaim claim) {
-        if (claim == null) {
-            return 0;
-        }
-
-        synchronized (claim) {
-            final ClaimCount counter = claimantCounts.get(claim);
-            if (counter == null) {
-                logger.warn("Decrementing claimant count for {} but claimant count is not known. Returning -1", claim);
-                return -1;
-            }
-
-            final int newClaimantCount = counter.getCount().decrementAndGet();
-            if (newClaimantCount < 0) {
-                logger.error("Decremented claimant count for {} to {}", claim, newClaimantCount);
-            } else {
-                logger.debug("Decrementing claimant count for {} to {}", claim, newClaimantCount);
-            }
-
-            // If the claim is no longer referenced, we want to remove it. We consider the claim to be "no longer referenced"
-            // if the count is 0 and it is no longer writable (if it's writable, it may still be writable by the Content Repository,
-            // even though no existing FlowFile is referencing the claim).
-            if (newClaimantCount == 0 && !claim.isWritable()) {
-                removeClaimantCount(claim);
-            }
-            return newClaimantCount;
-        }
-    }
-
-    // protected so that it can be used in unit tests
-    protected void removeClaimantCount(final ResourceClaim claim) {
-        claimantCounts.remove(claim);
-    }
-
-    @Override
-    public int incrementClaimantCount(final ResourceClaim claim) {
-        return incrementClaimantCount(claim, false);
-    }
-
-    @Override
-    public int incrementClaimantCount(final ResourceClaim claim, final boolean newClaim) {
-        if (claim == null) {
-            return 0;
-        }
-
-        synchronized (claim) {
-            final AtomicInteger counter = getCounter(claim);
-
-            final int newClaimantCount = counter.incrementAndGet();
-            logger.debug("Incrementing claimant count for {} to {}", claim, newClaimantCount);
-
-            // If the claimant count moved from 0 to 1, remove it from the queue of destructable claims.
-            if (!newClaim && newClaimantCount == 1) {
-                destructableClaims.remove(claim);
-            }
-            return newClaimantCount;
-        }
-    }
-
-    @Override
-    public void markDestructable(final ResourceClaim claim) {
-        if (claim == null) {
-            return;
-        }
-
-        synchronized (claim) {
-            if (getClaimantCount(claim) > 0) {
-                return;
-            }
-
-            logger.debug("Marking claim {} as destructable", claim);
-            try {
-                while (!destructableClaims.offer(claim, 30, TimeUnit.MINUTES)) {
-                }
-            } catch (final InterruptedException ie) {
-            }
-        }
-    }
-
-    @Override
-    public void drainDestructableClaims(final Collection<ResourceClaim> destination, final int maxElements) {
-        final int drainedCount = destructableClaims.drainTo(destination, maxElements);
-        logger.debug("Drained {} destructable claims to {}", drainedCount, destination);
-    }
-
-    @Override
-    public void drainDestructableClaims(final Collection<ResourceClaim> destination, final int maxElements, final long timeout, final TimeUnit unit) {
-        try {
-            final ResourceClaim firstClaim = destructableClaims.poll(timeout, unit);
-            if (firstClaim != null) {
-                destination.add(firstClaim);
-                destructableClaims.drainTo(destination, maxElements - 1);
-            }
-        } catch (final InterruptedException e) {
-        }
-    }
-
-    @Override
-    public void purge() {
-        claimantCounts.clear();
-    }
-
-    @Override
-    public void freeze(final ResourceClaim claim) {
-        if (claim == null) {
-            return;
-        }
-
-        if (!(claim instanceof StandardResourceClaim)) {
-            throw new IllegalArgumentException("The given resource claim is not managed by this Resource Claim Manager");
-        }
-
-        ((StandardResourceClaim) claim).freeze();
-
-        synchronized (claim) {
-            if (getClaimantCount(claim) == 0) {
-                claimantCounts.remove(claim);
-            }
-        }
-    }
-
-
-    private static final class ClaimCount {
-        private final ResourceClaim claim;
-        private final AtomicInteger count;
-
-        public ClaimCount(final ResourceClaim claim, final AtomicInteger count) {
-            this.claim = claim;
-            this.count = count;
-        }
-
-        public AtomicInteger getCount() {
-            return count;
-        }
-
-        public ResourceClaim getClaim() {
-            return claim;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
deleted file mode 100644
index b218ee6..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import java.util.List;
-
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
-import org.apache.nifi.controller.repository.claim.StandardContentClaim;
-import org.apache.nifi.repository.schema.Record;
-import org.apache.nifi.repository.schema.RecordField;
-import org.apache.nifi.repository.schema.RecordSchema;
-
-public class ContentClaimFieldMap implements Record {
-    private final ContentClaim contentClaim;
-    private final long contentClaimOffset;
-    private final ResourceClaimFieldMap resourceClaimFieldMap;
-    private final RecordSchema schema;
-
-    public ContentClaimFieldMap(final ContentClaim contentClaim, final long contentClaimOffset, final RecordSchema schema) {
-        this.contentClaim = contentClaim;
-        this.contentClaimOffset = contentClaimOffset;
-        this.schema = schema;
-
-        final List<RecordField> resourceClaimFields = schema.getField(ContentClaimSchema.RESOURCE_CLAIM).getSubFields();
-        final RecordSchema resourceClaimSchema = new RecordSchema(resourceClaimFields);
-        this.resourceClaimFieldMap = new ResourceClaimFieldMap(contentClaim.getResourceClaim(), resourceClaimSchema);
-    }
-
-    @Override
-    public Object getFieldValue(final String fieldName) {
-        switch (fieldName) {
-            case ContentClaimSchema.RESOURCE_CLAIM:
-                return resourceClaimFieldMap;
-            case ContentClaimSchema.CONTENT_CLAIM_LENGTH:
-                return contentClaim.getLength();
-            case ContentClaimSchema.CONTENT_CLAIM_OFFSET:
-                return contentClaimOffset;
-            case ContentClaimSchema.RESOURCE_CLAIM_OFFSET:
-                return contentClaim.getOffset();
-            default:
-                return null;
-        }
-    }
-
-    @Override
-    public RecordSchema getSchema() {
-        return schema;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) (31 + contentClaimOffset + 21 * resourceClaimFieldMap.hashCode());
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (this == obj) {
-            return true;
-        }
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        ContentClaimFieldMap other = (ContentClaimFieldMap) obj;
-        if (contentClaimOffset != other.contentClaimOffset) {
-            return false;
-        }
-
-        if (resourceClaimFieldMap == null) {
-            if (other.resourceClaimFieldMap != null) {
-                return false;
-            }
-        } else if (!resourceClaimFieldMap.equals(other.resourceClaimFieldMap)) {
-            return false;
-        }
-
-        return true;
-    }
-
-    @Override
-    public String toString() {
-        return "ContentClaimFieldMap[" + contentClaim + "]";
-    }
-
-    public static ContentClaim getContentClaim(final Record claimRecord, final ResourceClaimManager resourceClaimManager) {
-        final Record resourceClaimRecord = (Record) claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM);
-        final String container = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_CONTAINER);
-        final String section = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_SECTION);
-        final String identifier = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_IDENTIFIER);
-        final Boolean lossTolerant = (Boolean) resourceClaimRecord.getFieldValue(ContentClaimSchema.LOSS_TOLERANT);
-
-        final Long length = (Long) claimRecord.getFieldValue(ContentClaimSchema.CONTENT_CLAIM_LENGTH);
-        final Long resourceOffset = (Long) claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM_OFFSET);
-
-        final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim(container, section, identifier, lossTolerant, false);
-        final StandardContentClaim contentClaim = new StandardContentClaim(resourceClaim, resourceOffset);
-        contentClaim.setLength(length);
-
-        return contentClaim;
-    }
-
-    public static Long getContentClaimOffset(final Record claimRecord) {
-        return (Long) claimRecord.getFieldValue(ContentClaimSchema.CONTENT_CLAIM_OFFSET);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java
deleted file mode 100644
index c55c758..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.nifi.repository.schema.ComplexRecordField;
-import org.apache.nifi.repository.schema.FieldType;
-import org.apache.nifi.repository.schema.RecordField;
-import org.apache.nifi.repository.schema.RecordSchema;
-import org.apache.nifi.repository.schema.Repetition;
-import org.apache.nifi.repository.schema.SimpleRecordField;
-
-public class ContentClaimSchema {
-
-    // resource claim fields
-    public static final String CLAIM_CONTAINER = "Container";
-    public static final String CLAIM_SECTION = "Section";
-    public static final String CLAIM_IDENTIFIER = "Identifier";
-    public static final String LOSS_TOLERANT = "Loss Tolerant";
-    public static final String RESOURCE_CLAIM = "Resource Claim";
-
-    // content claim fields
-    public static final String RESOURCE_CLAIM_OFFSET = "Resource Claim Offset"; // offset into resource claim where the content claim begins
-    public static final String CONTENT_CLAIM_OFFSET = "Content Claim Offset"; // offset into the content claim where the flowfile begins
-    public static final String CONTENT_CLAIM_LENGTH = "Content Claim Length";
-
-    public static final RecordSchema CONTENT_CLAIM_SCHEMA_V1;
-    public static final RecordSchema RESOURCE_CLAIM_SCHEMA_V1;
-
-    static {
-        final List<RecordField> resourceClaimFields = new ArrayList<>();
-        resourceClaimFields.add(new SimpleRecordField(CLAIM_CONTAINER, FieldType.STRING, Repetition.EXACTLY_ONE));
-        resourceClaimFields.add(new SimpleRecordField(CLAIM_SECTION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        resourceClaimFields.add(new SimpleRecordField(CLAIM_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
-        resourceClaimFields.add(new SimpleRecordField(LOSS_TOLERANT, FieldType.BOOLEAN, Repetition.EXACTLY_ONE));
-        RESOURCE_CLAIM_SCHEMA_V1 = new RecordSchema(Collections.unmodifiableList(resourceClaimFields));
-
-        final List<RecordField> contentClaimFields = new ArrayList<>();
-        contentClaimFields.add(new ComplexRecordField(RESOURCE_CLAIM, Repetition.EXACTLY_ONE, resourceClaimFields));
-        contentClaimFields.add(new SimpleRecordField(RESOURCE_CLAIM_OFFSET, FieldType.LONG, Repetition.EXACTLY_ONE));
-        contentClaimFields.add(new SimpleRecordField(CONTENT_CLAIM_OFFSET, FieldType.LONG, Repetition.EXACTLY_ONE));
-        contentClaimFields.add(new SimpleRecordField(CONTENT_CLAIM_LENGTH, FieldType.LONG, Repetition.EXACTLY_ONE));
-        CONTENT_CLAIM_SCHEMA_V1 = new RecordSchema(Collections.unmodifiableList(contentClaimFields));
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java
deleted file mode 100644
index ff0615f..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import java.util.Map;
-
-import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.controller.repository.StandardFlowFileRecord;
-import org.apache.nifi.controller.repository.claim.ContentClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
-import org.apache.nifi.repository.schema.Record;
-import org.apache.nifi.repository.schema.RecordField;
-import org.apache.nifi.repository.schema.RecordSchema;
-
-public class FlowFileRecordFieldMap implements Record {
-    private final FlowFileRecord flowFile;
-    private final RecordSchema schema;
-    private final RecordSchema contentClaimSchema;
-    private final ContentClaimFieldMap contentClaim;
-
-    public FlowFileRecordFieldMap(final FlowFileRecord flowFile, final RecordSchema schema) {
-        this.flowFile = flowFile;
-        this.schema = schema;
-
-        final RecordField contentClaimField = schema.getField(FlowFileSchema.CONTENT_CLAIM);
-        contentClaimSchema = new RecordSchema(contentClaimField.getSubFields());
-        contentClaim = flowFile.getContentClaim() == null ? null : new ContentClaimFieldMap(flowFile.getContentClaim(), flowFile.getContentClaimOffset(), contentClaimSchema);
-    }
-
-    @Override
-    public RecordSchema getSchema() {
-        return schema;
-    }
-
-    @Override
-    public Object getFieldValue(final String fieldName) {
-        switch (fieldName) {
-            case FlowFileSchema.ATTRIBUTES:
-                return flowFile.getAttributes();
-            case FlowFileSchema.CONTENT_CLAIM:
-                return contentClaim;
-            case FlowFileSchema.ENTRY_DATE:
-                return flowFile.getEntryDate();
-            case FlowFileSchema.FLOWFILE_SIZE:
-                return flowFile.getSize();
-            case FlowFileSchema.LINEAGE_START_DATE:
-                return flowFile.getLineageStartDate();
-            case FlowFileSchema.LINEAGE_START_INDEX:
-                return flowFile.getLineageStartIndex();
-            case FlowFileSchema.QUEUE_DATE:
-                return flowFile.getLastQueueDate();
-            case FlowFileSchema.QUEUE_DATE_INDEX:
-                return flowFile.getQueueDateIndex();
-            case FlowFileSchema.RECORD_ID:
-                return flowFile.getId();
-        }
-
-        return null;
-    }
-
-    @SuppressWarnings("unchecked")
-    public static FlowFileRecord getFlowFile(final Record record, final ResourceClaimManager claimManager) {
-        final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder();
-        builder.id((Long) record.getFieldValue(FlowFileSchema.RECORD_ID));
-        builder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE));
-        builder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE));
-        builder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES));
-        builder.lineageStart((Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE), (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX));
-        builder.lastQueued((Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE), (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX));
-
-        final Record contentClaimRecord = (Record) record.getFieldValue(FlowFileSchema.CONTENT_CLAIM);
-        if (contentClaimRecord != null) {
-            final ContentClaim claim = ContentClaimFieldMap.getContentClaim(contentClaimRecord, claimManager);
-            builder.contentClaim(claim);
-
-            final Long offset = ContentClaimFieldMap.getContentClaimOffset(contentClaimRecord);
-            if (offset != null) {
-                builder.contentClaimOffset(offset);
-            }
-        }
-
-        return builder.build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java
deleted file mode 100644
index 6af3066..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.nifi.repository.schema.ComplexRecordField;
-import org.apache.nifi.repository.schema.FieldType;
-import org.apache.nifi.repository.schema.MapRecordField;
-import org.apache.nifi.repository.schema.RecordField;
-import org.apache.nifi.repository.schema.RecordSchema;
-import org.apache.nifi.repository.schema.Repetition;
-import org.apache.nifi.repository.schema.SimpleRecordField;
-
-public class FlowFileSchema {
-
-    public static final String RECORD_ID = "Record ID";
-    public static final String ENTRY_DATE = "Entry Date";
-    public static final String LINEAGE_START_DATE = "Lineage Start Date";
-    public static final String LINEAGE_START_INDEX = "Lineage Start Index";
-    public static final String QUEUE_DATE = "Queued Date";
-    public static final String QUEUE_DATE_INDEX = "Queued Date Index";
-    public static final String FLOWFILE_SIZE = "FlowFile Size";
-    public static final String CONTENT_CLAIM = "Content Claim";
-    public static final String ATTRIBUTES = "Attributes";
-
-    // attribute fields
-    public static final String ATTRIBUTE_NAME = "Attribute Name";
-    public static final String ATTRIBUTE_VALUE = "Attribute Value";
-
-    public static final RecordSchema FLOWFILE_SCHEMA_V1;
-    public static final RecordSchema FLOWFILE_SCHEMA_V2;
-
-    static {
-        final List<RecordField> flowFileFields = new ArrayList<>();
-
-        final RecordField attributeNameField = new SimpleRecordField(ATTRIBUTE_NAME, FieldType.STRING, Repetition.EXACTLY_ONE);
-        final RecordField attributeValueField = new SimpleRecordField(ATTRIBUTE_VALUE, FieldType.STRING, Repetition.EXACTLY_ONE);
-
-        flowFileFields.add(new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(ENTRY_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(LINEAGE_START_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(LINEAGE_START_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(QUEUE_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(QUEUE_DATE_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(FLOWFILE_SIZE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new ComplexRecordField(CONTENT_CLAIM, Repetition.ZERO_OR_ONE, ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1.getFields()));
-        flowFileFields.add(new MapRecordField(ATTRIBUTES, attributeNameField, attributeValueField, Repetition.ZERO_OR_ONE));
-
-        FLOWFILE_SCHEMA_V1 = new RecordSchema(flowFileFields);
-    }
-
-    static {
-        final List<RecordField> flowFileFields = new ArrayList<>();
-
-        final RecordField attributeNameField = new SimpleRecordField(ATTRIBUTE_NAME, FieldType.LONG_STRING, Repetition.EXACTLY_ONE);
-        final RecordField attributeValueField = new SimpleRecordField(ATTRIBUTE_VALUE, FieldType.LONG_STRING, Repetition.EXACTLY_ONE);
-
-        flowFileFields.add(new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(ENTRY_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(LINEAGE_START_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(LINEAGE_START_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(QUEUE_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(QUEUE_DATE_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new SimpleRecordField(FLOWFILE_SIZE, FieldType.LONG, Repetition.EXACTLY_ONE));
-        flowFileFields.add(new ComplexRecordField(CONTENT_CLAIM, Repetition.ZERO_OR_ONE, ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1.getFields()));
-        flowFileFields.add(new MapRecordField(ATTRIBUTES, attributeNameField, attributeValueField, Repetition.ZERO_OR_ONE));
-
-        FLOWFILE_SCHEMA_V2 = new RecordSchema(flowFileFields);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
deleted file mode 100644
index 5fe4889..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.controller.repository.FlowFileRecord;
-import org.apache.nifi.controller.repository.RepositoryRecord;
-import org.apache.nifi.repository.schema.Record;
-import org.apache.nifi.repository.schema.RecordSchema;
-
-public class RepositoryRecordFieldMap implements Record {
-    private final RepositoryRecord record;
-    private final FlowFileRecord flowFile;
-    private final RecordSchema schema;
-    private final RecordSchema contentClaimSchema;
-
-    public RepositoryRecordFieldMap(final RepositoryRecord record, final RecordSchema repoRecordSchema, final RecordSchema contentClaimSchema) {
-        this.schema = repoRecordSchema;
-        this.contentClaimSchema = contentClaimSchema;
-        this.record = record;
-        this.flowFile = record.getCurrent();
-    }
-
-    @Override
-    public Object getFieldValue(final String fieldName) {
-        switch (fieldName) {
-            case RepositoryRecordSchema.ACTION_TYPE:
-                return record.getType().name();
-            case RepositoryRecordSchema.RECORD_ID:
-                return record.getCurrent().getId();
-            case RepositoryRecordSchema.SWAP_LOCATION:
-                return record.getSwapLocation();
-            case FlowFileSchema.ATTRIBUTES:
-                return flowFile.getAttributes();
-            case FlowFileSchema.ENTRY_DATE:
-                return flowFile.getEntryDate();
-            case FlowFileSchema.FLOWFILE_SIZE:
-                return flowFile.getSize();
-            case FlowFileSchema.LINEAGE_START_DATE:
-                return flowFile.getLineageStartDate();
-            case FlowFileSchema.LINEAGE_START_INDEX:
-                return flowFile.getLineageStartIndex();
-            case FlowFileSchema.QUEUE_DATE:
-                return flowFile.getLastQueueDate();
-            case FlowFileSchema.QUEUE_DATE_INDEX:
-                return flowFile.getQueueDateIndex();
-            case FlowFileSchema.CONTENT_CLAIM:
-                final ContentClaimFieldMap contentClaimFieldMap = record.getCurrentClaim() == null ? null
-                    : new ContentClaimFieldMap(record.getCurrentClaim(), record.getCurrentClaimOffset(), contentClaimSchema);
-                return contentClaimFieldMap;
-            case RepositoryRecordSchema.QUEUE_IDENTIFIER:
-                final FlowFileQueue queue = record.getDestination() == null ? record.getOriginalQueue() : record.getDestination();
-                return queue == null ? null : queue.getIdentifier();
-            default:
-                return null;
-        }
-    }
-
-    @Override
-    public RecordSchema getSchema() {
-        return schema;
-    }
-
-    @Override
-    public String toString() {
-        return "RepositoryRecordFieldMap[" + record + "]";
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java
deleted file mode 100644
index db77c8b..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import org.apache.nifi.repository.schema.ComplexRecordField;
-import org.apache.nifi.repository.schema.FieldType;
-import org.apache.nifi.repository.schema.RecordField;
-import org.apache.nifi.repository.schema.RecordSchema;
-import org.apache.nifi.repository.schema.Repetition;
-import org.apache.nifi.repository.schema.SimpleRecordField;
-import org.apache.nifi.repository.schema.UnionRecordField;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class RepositoryRecordSchema {
-    public static final String REPOSITORY_RECORD_UPDATE_V1 = "Repository Record Update";  // top level field name
-    public static final String REPOSITORY_RECORD_UPDATE_V2 = "Repository Record Update";  // top level field name
-
-    // repository record fields
-    public static final String ACTION_TYPE = "Action";
-    public static final String RECORD_ID = "Record ID";
-    public static final String QUEUE_IDENTIFIER = "Queue Identifier";
-    public static final String SWAP_LOCATION = "Swap Location";
-
-    // Update types
-    public static final String CREATE_OR_UPDATE_ACTION = "Create or Update";
-    public static final String DELETE_ACTION = "Delete";
-    public static final String SWAP_IN_ACTION = "Swap In";
-    public static final String SWAP_OUT_ACTION = "Swap Out";
-
-    public static final RecordSchema REPOSITORY_RECORD_SCHEMA_V1;
-    public static final RecordSchema CREATE_OR_UPDATE_SCHEMA_V1;
-    public static final RecordSchema DELETE_SCHEMA_V1;
-    public static final RecordSchema SWAP_IN_SCHEMA_V1;
-    public static final RecordSchema SWAP_OUT_SCHEMA_V1;
-
-    public static final RecordSchema REPOSITORY_RECORD_SCHEMA_V2;
-    public static final RecordSchema CREATE_OR_UPDATE_SCHEMA_V2;
-    public static final RecordSchema DELETE_SCHEMA_V2;
-    public static final RecordSchema SWAP_IN_SCHEMA_V2;
-    public static final RecordSchema SWAP_OUT_SCHEMA_V2;
-
-    public static final RecordField ACTION_TYPE_FIELD = new SimpleRecordField(ACTION_TYPE, FieldType.STRING, Repetition.EXACTLY_ONE);
-    public static final RecordField RECORD_ID_FIELD = new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE);
-
-    static {
-        // Fields for "Create" or "Update" records
-        final List<RecordField> createOrUpdateFields = new ArrayList<>();
-        createOrUpdateFields.add(ACTION_TYPE_FIELD);
-        createOrUpdateFields.addAll(FlowFileSchema.FLOWFILE_SCHEMA_V1.getFields());
-
-        createOrUpdateFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
-        createOrUpdateFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.ZERO_OR_ONE));
-        final ComplexRecordField createOrUpdate = new ComplexRecordField(CREATE_OR_UPDATE_ACTION, Repetition.EXACTLY_ONE, createOrUpdateFields);
-        CREATE_OR_UPDATE_SCHEMA_V1 = new RecordSchema(createOrUpdateFields);
-
-        // Fields for "Delete" records
-        final List<RecordField> deleteFields = new ArrayList<>();
-        deleteFields.add(ACTION_TYPE_FIELD);
-        deleteFields.add(RECORD_ID_FIELD);
-        final ComplexRecordField delete = new ComplexRecordField(DELETE_ACTION, Repetition.EXACTLY_ONE, deleteFields);
-        DELETE_SCHEMA_V1 = new RecordSchema(deleteFields);
-
-        // Fields for "Swap Out" records
-        final List<RecordField> swapOutFields = new ArrayList<>();
-        swapOutFields.add(ACTION_TYPE_FIELD);
-        swapOutFields.add(RECORD_ID_FIELD);
-        swapOutFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
-        swapOutFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        final ComplexRecordField swapOut = new ComplexRecordField(SWAP_OUT_ACTION, Repetition.EXACTLY_ONE, swapOutFields);
-        SWAP_OUT_SCHEMA_V1 = new RecordSchema(swapOutFields);
-
-        // Fields for "Swap In" records
-        final List<RecordField> swapInFields = new ArrayList<>(createOrUpdateFields);
-        swapInFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        final ComplexRecordField swapIn = new ComplexRecordField(SWAP_IN_ACTION, Repetition.EXACTLY_ONE, swapInFields);
-        SWAP_IN_SCHEMA_V1 = new RecordSchema(swapInFields);
-
-        // Union Field that creates the top-level field type
-        final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V1, Repetition.EXACTLY_ONE, createOrUpdate, delete, swapOut, swapIn);
-        REPOSITORY_RECORD_SCHEMA_V1 = new RecordSchema(Collections.singletonList(repoUpdateField));
-    }
-
-    static {
-        // Fields for "Create" or "Update" records
-        final List<RecordField> createOrUpdateFields = new ArrayList<>();
-        createOrUpdateFields.add(ACTION_TYPE_FIELD);
-        createOrUpdateFields.addAll(FlowFileSchema.FLOWFILE_SCHEMA_V2.getFields());
-
-        createOrUpdateFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
-        createOrUpdateFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.ZERO_OR_ONE));
-        final ComplexRecordField createOrUpdate = new ComplexRecordField(CREATE_OR_UPDATE_ACTION, Repetition.EXACTLY_ONE, createOrUpdateFields);
-        CREATE_OR_UPDATE_SCHEMA_V2 = new RecordSchema(createOrUpdateFields);
-
-        // Fields for "Delete" records
-        final List<RecordField> deleteFields = new ArrayList<>();
-        deleteFields.add(ACTION_TYPE_FIELD);
-        deleteFields.add(RECORD_ID_FIELD);
-        final ComplexRecordField delete = new ComplexRecordField(DELETE_ACTION, Repetition.EXACTLY_ONE, deleteFields);
-        DELETE_SCHEMA_V2 = new RecordSchema(deleteFields);
-
-        // Fields for "Swap Out" records
-        final List<RecordField> swapOutFields = new ArrayList<>();
-        swapOutFields.add(ACTION_TYPE_FIELD);
-        swapOutFields.add(RECORD_ID_FIELD);
-        swapOutFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
-        swapOutFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        final ComplexRecordField swapOut = new ComplexRecordField(SWAP_OUT_ACTION, Repetition.EXACTLY_ONE, swapOutFields);
-        SWAP_OUT_SCHEMA_V2 = new RecordSchema(swapOutFields);
-
-        // Fields for "Swap In" records
-        final List<RecordField> swapInFields = new ArrayList<>(createOrUpdateFields);
-        swapInFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
-        final ComplexRecordField swapIn = new ComplexRecordField(SWAP_IN_ACTION, Repetition.EXACTLY_ONE, swapInFields);
-        SWAP_IN_SCHEMA_V2 = new RecordSchema(swapInFields);
-
-        // Union Field that creates the top-level field type
-        final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V2, Repetition.EXACTLY_ONE, createOrUpdate, delete, swapOut, swapIn);
-        REPOSITORY_RECORD_SCHEMA_V2 = new RecordSchema(Collections.singletonList(repoUpdateField));
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java
deleted file mode 100644
index 93fa4e4..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import org.apache.nifi.controller.repository.RepositoryRecordType;
-import org.apache.nifi.repository.schema.NamedValue;
-import org.apache.nifi.repository.schema.Record;
-import org.apache.nifi.repository.schema.RecordSchema;
-import org.wali.UpdateType;
-
-public class RepositoryRecordUpdate implements Record {
-    private final RecordSchema schema;
-    private final RepositoryRecordFieldMap fieldMap;
-
-    public RepositoryRecordUpdate(final RepositoryRecordFieldMap fieldMap, final RecordSchema schema) {
-        this.schema = schema;
-        this.fieldMap = fieldMap;
-    }
-
-    @Override
-    public RecordSchema getSchema() {
-        return schema;
-    }
-
-    @Override
-    public Object getFieldValue(final String fieldName) {
-        if (RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V2.equals(fieldName)) {
-            String actionType = (String) fieldMap.getFieldValue(RepositoryRecordSchema.ACTION_TYPE);
-            if (RepositoryRecordType.CONTENTMISSING.name().equals(actionType)) {
-                actionType = RepositoryRecordType.DELETE.name();
-            }
-            final UpdateType updateType = UpdateType.valueOf(actionType);
-
-            final String actionName;
-            switch (updateType) {
-                case CREATE:
-                case UPDATE:
-                    actionName = RepositoryRecordSchema.CREATE_OR_UPDATE_ACTION;
-                    break;
-                case DELETE:
-                    actionName = RepositoryRecordSchema.DELETE_ACTION;
-                    break;
-                case SWAP_IN:
-                    actionName = RepositoryRecordSchema.SWAP_IN_ACTION;
-                    break;
-                case SWAP_OUT:
-                    actionName = RepositoryRecordSchema.SWAP_OUT_ACTION;
-                    break;
-                default:
-                    return null;
-            }
-
-            return new NamedValue(actionName, fieldMap);
-        }
-        return null;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java
deleted file mode 100644
index afa19ea..0000000
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/schema/ResourceClaimFieldMap.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * 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.controller.repository.schema;
-
-import org.apache.nifi.controller.repository.claim.ResourceClaim;
-import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
-import org.apache.nifi.repository.schema.Record;
-import org.apache.nifi.repository.schema.RecordSchema;
-
-public class ResourceClaimFieldMap implements Record {
-    private final ResourceClaim resourceClaim;
-    private final RecordSchema schema;
-
-    public ResourceClaimFieldMap(final ResourceClaim resourceClaim, final RecordSchema schema) {
-        this.resourceClaim = resourceClaim;
-        this.schema = schema;
-    }
-
-    @Override
-    public RecordSchema getSchema() {
-        return schema;
-    }
-
-    @Override
-    public Object getFieldValue(final String fieldName) {
-        switch (fieldName) {
-            case ContentClaimSchema.CLAIM_CONTAINER:
-                return resourceClaim.getContainer();
-            case ContentClaimSchema.CLAIM_SECTION:
-                return resourceClaim.getSection();
-            case ContentClaimSchema.CLAIM_IDENTIFIER:
-                return resourceClaim.getId();
-            case ContentClaimSchema.LOSS_TOLERANT:
-                return resourceClaim.isLossTolerant();
-        }
-
-        return null;
-    }
-
-    public static ResourceClaim getResourceClaim(final Record record, final ResourceClaimManager claimManager) {
-        final String container = (String) record.getFieldValue(ContentClaimSchema.CLAIM_CONTAINER);
-        final String section = (String) record.getFieldValue(ContentClaimSchema.CLAIM_SECTION);
-        final String identifier = (String) record.getFieldValue(ContentClaimSchema.CLAIM_IDENTIFIER);
-        final Boolean lossTolerant = (Boolean) record.getFieldValue(ContentClaimSchema.LOSS_TOLERANT);
-
-        return claimManager.newResourceClaim(container, section, identifier, lossTolerant, false);
-    }
-
-    @Override
-    public int hashCode() {
-        return 41 + 91 * resourceClaim.hashCode();
-    }
-
-    @Override
-    public boolean equals(final Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (obj == this) {
-            return true;
-        }
-
-        if (obj.getClass() != ResourceClaimFieldMap.class) {
-            return false;
-        }
-
-        final ResourceClaimFieldMap other = (ResourceClaimFieldMap) obj;
-        return resourceClaim.equals(other.resourceClaim);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml
new file mode 100644
index 0000000..519b95a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/pom.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-framework</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-repository-models</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-utils</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-utils</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java
new file mode 100644
index 0000000..a1d5173
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardFlowFileRecord.java
@@ -0,0 +1,341 @@
+/*
+ * 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.controller.repository;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang3.builder.CompareToBuilder;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+
+/**
+ * <p>
+ * A flow file is a logical notion of an item in a flow with its associated attributes and identity which can be used as a reference for its actual content.
+ * </p>
+ *
+ * <b>Immutable - Thread Safe</b>
+ *
+ */
+public final class StandardFlowFileRecord implements FlowFile, FlowFileRecord {
+
+    private final long id;
+    private final long entryDate;
+    private final long lineageStartDate;
+    private final long lineageStartIndex;
+    private final long size;
+    private final long penaltyExpirationMs;
+    private final Map<String, String> attributes;
+    private final ContentClaim claim;
+    private final long claimOffset;
+    private final long lastQueueDate;
+    private final long queueDateIndex;
+
+    private StandardFlowFileRecord(final Builder builder) {
+        this.id = builder.bId;
+        this.attributes = builder.bAttributes == null ? Collections.emptyMap() : builder.bAttributes;
+        this.entryDate = builder.bEntryDate;
+        this.lineageStartDate = builder.bLineageStartDate;
+        this.lineageStartIndex = builder.bLineageStartIndex;
+        this.penaltyExpirationMs = builder.bPenaltyExpirationMs;
+        this.size = builder.bSize;
+        this.claim = builder.bClaim;
+        this.claimOffset = builder.bClaimOffset;
+        this.lastQueueDate = builder.bLastQueueDate;
+        this.queueDateIndex = builder.bQueueDateIndex;
+    }
+
+    @Override
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public long getEntryDate() {
+        return entryDate;
+    }
+
+    @Override
+    public long getLineageStartDate() {
+        return lineageStartDate;
+    }
+
+    @Override
+    public Long getLastQueueDate() {
+        return lastQueueDate;
+    }
+
+    @Override
+    public boolean isPenalized() {
+        return penaltyExpirationMs > 0 ? penaltyExpirationMs > System.currentTimeMillis() : false;
+    }
+
+    @Override
+    public String getAttribute(final String key) {
+        return attributes.get(key);
+    }
+
+    @Override
+    public long getSize() {
+        return size;
+    }
+
+    @Override
+    public Map<String, String> getAttributes() {
+        return Collections.unmodifiableMap(this.attributes);
+    }
+
+    @Override
+    public ContentClaim getContentClaim() {
+        return this.claim;
+    }
+
+    @Override
+    public long getContentClaimOffset() {
+        return this.claimOffset;
+    }
+
+    @Override
+    public long getLineageStartIndex() {
+        return lineageStartIndex;
+    }
+
+    @Override
+    public long getQueueDateIndex() {
+        return queueDateIndex;
+    }
+
+    /**
+     * Provides the natural ordering for FlowFile objects which is based on their identifier.
+     *
+     * @param other other
+     * @return standard compare contract
+     */
+    @Override
+    public int compareTo(final FlowFile other) {
+        return new CompareToBuilder().append(id, other.getId()).toComparison();
+    }
+
+    @Override
+    public boolean equals(final Object other) {
+        if (this == other) {
+            return true;
+        }
+        if (!(other instanceof FlowFile)) {
+            return false;
+        }
+        final FlowFile otherRecord = (FlowFile) other;
+        return new EqualsBuilder().append(id, otherRecord.getId()).isEquals();
+    }
+
+    @Override
+    public String toString() {
+        final ToStringBuilder builder = new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE);
+        builder.append("uuid", getAttribute(CoreAttributes.UUID.key()));
+        builder.append("claim", claim == null ? "" : claim.toString());
+        builder.append("offset", claimOffset);
+        builder.append("name", getAttribute(CoreAttributes.FILENAME.key())).append("size", size);
+        return builder.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        return new HashCodeBuilder(7, 13).append(id).toHashCode();
+    }
+
+    public static final class Builder {
+
+        private long bId;
+        private long bEntryDate = System.currentTimeMillis();
+        private long bLineageStartDate = bEntryDate;
+        private long bLineageStartIndex = 0L;
+        private final Set<String> bLineageIdentifiers = new HashSet<>();
+        private long bPenaltyExpirationMs = -1L;
+        private long bSize = 0L;
+        private ContentClaim bClaim = null;
+        private long bClaimOffset = 0L;
+        private long bLastQueueDate = System.currentTimeMillis();
+        private long bQueueDateIndex = 0L;
+        private Map<String, String> bAttributes;
+        private boolean bAttributesCopied = false;
+
+        public Builder id(final long id) {
+            bId = id;
+            return this;
+        }
+
+        public Builder entryDate(final long epochMs) {
+            bEntryDate = epochMs;
+            return this;
+        }
+
+        public Builder lineageStart(final long lineageStartDate, final long lineageStartIndex) {
+            bLineageStartDate = lineageStartDate;
+            bLineageStartIndex = lineageStartIndex;
+            return this;
+        }
+
+        public Builder penaltyExpirationTime(final long epochMilliseconds) {
+            bPenaltyExpirationMs = epochMilliseconds;
+            return this;
+        }
+
+        public Builder size(final long bytes) {
+            if (bytes >= 0) {
+                bSize = bytes;
+            }
+            return this;
+        }
+
+        private Map<String, String> initializeAttributes() {
+            if (bAttributes == null) {
+                bAttributes = new HashMap<>();
+                bAttributesCopied = true;
+            } else if (!bAttributesCopied) {
+                bAttributes = new HashMap<>(bAttributes);
+                bAttributesCopied = true;
+            }
+
+            return bAttributes;
+        }
+
+        public Builder addAttribute(final String key, final String value) {
+            if (key != null && value != null) {
+                initializeAttributes().put(FlowFile.KeyValidator.validateKey(key), value);
+            }
+            return this;
+        }
+
+        public Builder addAttributes(final Map<String, String> attributes) {
+            final Map<String, String> initializedAttributes = initializeAttributes();
+
+            if (null != attributes) {
+                for (final String key : attributes.keySet()) {
+                    FlowFile.KeyValidator.validateKey(key);
+                }
+                for (final Map.Entry<String, String> entry : attributes.entrySet()) {
+                    final String key = entry.getKey();
+                    final String value = entry.getValue();
+                    if (key != null && value != null) {
+                        initializedAttributes.put(key, value);
+                    }
+                }
+            }
+            return this;
+        }
+
+        public Builder removeAttributes(final String... keys) {
+            if (keys != null) {
+                for (final String key : keys) {
+                    if (CoreAttributes.UUID.key().equals(key)) {
+                        continue;
+                    }
+
+                    initializeAttributes().remove(key);
+                }
+            }
+            return this;
+        }
+
+        public Builder removeAttributes(final Set<String> keys) {
+            if (keys != null) {
+                for (final String key : keys) {
+                    if (CoreAttributes.UUID.key().equals(key)) {
+                        continue;
+                    }
+
+                    initializeAttributes().remove(key);
+                }
+            }
+            return this;
+        }
+
+        public Builder removeAttributes(final Pattern keyPattern) {
+            if (keyPattern != null) {
+                final Iterator<String> iterator = initializeAttributes().keySet().iterator();
+                while (iterator.hasNext()) {
+                    final String key = iterator.next();
+
+                    if (CoreAttributes.UUID.key().equals(key)) {
+                        continue;
+                    }
+
+                    if (keyPattern.matcher(key).matches()) {
+                        iterator.remove();
+                    }
+                }
+            }
+            return this;
+        }
+
+        public Builder contentClaim(final ContentClaim claim) {
+            this.bClaim = claim;
+            return this;
+        }
+
+        public Builder contentClaimOffset(final long offset) {
+            this.bClaimOffset = offset;
+            return this;
+        }
+
+        public Builder lastQueued(final long lastQueueDate, final long queueDateIndex) {
+            this.bLastQueueDate = lastQueueDate;
+            this.bQueueDateIndex = queueDateIndex;
+            return this;
+        }
+
+        public Builder fromFlowFile(final FlowFileRecord specFlowFile) {
+            if (specFlowFile == null) {
+                return this;
+            }
+            bId = specFlowFile.getId();
+            bEntryDate = specFlowFile.getEntryDate();
+            bLineageStartDate = specFlowFile.getLineageStartDate();
+            bLineageStartIndex = specFlowFile.getLineageStartIndex();
+            bLineageIdentifiers.clear();
+            bPenaltyExpirationMs = specFlowFile.getPenaltyExpirationMillis();
+            bSize = specFlowFile.getSize();
+            bAttributes = specFlowFile.getAttributes();
+            bAttributesCopied = false;
+            bClaim = specFlowFile.getContentClaim();
+            bClaimOffset = specFlowFile.getContentClaimOffset();
+            bLastQueueDate = specFlowFile.getLastQueueDate();
+            bQueueDateIndex = specFlowFile.getQueueDateIndex();
+
+            return this;
+        }
+
+        public FlowFileRecord build() {
+            return new StandardFlowFileRecord(this);
+        }
+    }
+
+    @Override
+    public long getPenaltyExpirationMillis() {
+        return penaltyExpirationMs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java
new file mode 100644
index 0000000..8aa1caf
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/StandardRepositoryRecord.java
@@ -0,0 +1,221 @@
+/*
+ * 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.controller.repository;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.processor.Relationship;
+
+public class StandardRepositoryRecord implements RepositoryRecord {
+
+    private RepositoryRecordType type = null;
+    private FlowFileRecord workingFlowFileRecord = null;
+    private Relationship transferRelationship = null;
+    private FlowFileQueue destination = null;
+    private final FlowFileRecord originalFlowFileRecord;
+    private final FlowFileQueue originalQueue;
+    private String swapLocation;
+    private final Map<String, String> updatedAttributes = new HashMap<>();
+    private final Map<String, String> originalAttributes;
+    private List<ContentClaim> transientClaims;
+
+    /**
+     * Creates a new record which has no original claim or flow file - it is entirely new
+     *
+     * @param originalQueue queue
+     */
+    public StandardRepositoryRecord(final FlowFileQueue originalQueue) {
+        this(originalQueue, null);
+        this.type = RepositoryRecordType.CREATE;
+    }
+
+    /**
+     * Creates a record based on given original items
+     *
+     * @param originalQueue queue
+     * @param originalFlowFileRecord record
+     */
+    public StandardRepositoryRecord(final FlowFileQueue originalQueue, final FlowFileRecord originalFlowFileRecord) {
+        this(originalQueue, originalFlowFileRecord, null);
+        this.type = RepositoryRecordType.UPDATE;
+    }
+
+    public StandardRepositoryRecord(final FlowFileQueue originalQueue, final FlowFileRecord originalFlowFileRecord, final String swapLocation) {
+        this.originalQueue = originalQueue;
+        this.originalFlowFileRecord = originalFlowFileRecord;
+        this.type = RepositoryRecordType.SWAP_OUT;
+        this.swapLocation = swapLocation;
+        this.originalAttributes = originalFlowFileRecord == null ? Collections.<String, String>emptyMap() : originalFlowFileRecord.getAttributes();
+    }
+
+    @Override
+    public FlowFileQueue getDestination() {
+        return destination;
+    }
+
+    public void setDestination(final FlowFileQueue destination) {
+        this.destination = destination;
+    }
+
+    @Override
+    public RepositoryRecordType getType() {
+        return type;
+    }
+
+    FlowFileRecord getOriginal() {
+        return originalFlowFileRecord;
+    }
+
+    @Override
+    public String getSwapLocation() {
+        return swapLocation;
+    }
+
+    public void setSwapLocation(final String swapLocation) {
+        this.swapLocation = swapLocation;
+        if (type != RepositoryRecordType.SWAP_OUT) {
+            type = RepositoryRecordType.SWAP_IN; // we are swapping in a new record
+        }
+    }
+
+    @Override
+    public ContentClaim getOriginalClaim() {
+        return (originalFlowFileRecord == null) ? null : originalFlowFileRecord.getContentClaim();
+    }
+
+    @Override
+    public FlowFileQueue getOriginalQueue() {
+        return originalQueue;
+    }
+
+    public void setWorking(final FlowFileRecord flowFile) {
+        workingFlowFileRecord = flowFile;
+    }
+
+    public void setWorking(final FlowFileRecord flowFile, final String attributeKey, final String attributeValue) {
+        workingFlowFileRecord = flowFile;
+
+        // If setting attribute to same value as original, don't add to updated attributes
+        final String currentValue = originalAttributes.get(attributeKey);
+        if (currentValue == null || !currentValue.equals(attributeValue)) {
+            updatedAttributes.put(attributeKey, attributeValue);
+        }
+    }
+
+    public void setWorking(final FlowFileRecord flowFile, final Map<String, String> updatedAttribs) {
+        workingFlowFileRecord = flowFile;
+
+        for (final Map.Entry<String, String> entry : updatedAttribs.entrySet()) {
+            final String currentValue = originalAttributes.get(entry.getKey());
+            if (currentValue == null || !currentValue.equals(entry.getValue())) {
+                updatedAttributes.put(entry.getKey(), entry.getValue());
+            }
+        }
+    }
+
+    @Override
+    public boolean isAttributesChanged() {
+        return !updatedAttributes.isEmpty();
+    }
+
+    public void markForAbort() {
+        type = RepositoryRecordType.CONTENTMISSING;
+    }
+
+    @Override
+    public boolean isMarkedForAbort() {
+        return RepositoryRecordType.CONTENTMISSING.equals(type);
+    }
+
+    public void markForDelete() {
+        type = RepositoryRecordType.DELETE;
+    }
+
+    public boolean isMarkedForDelete() {
+        return RepositoryRecordType.DELETE.equals(type);
+    }
+
+    public void setTransferRelationship(final Relationship relationship) {
+        transferRelationship = relationship;
+    }
+
+    public Relationship getTransferRelationship() {
+        return transferRelationship;
+    }
+
+    FlowFileRecord getWorking() {
+        return workingFlowFileRecord;
+    }
+
+    ContentClaim getWorkingClaim() {
+        return (workingFlowFileRecord == null) ? null : workingFlowFileRecord.getContentClaim();
+    }
+
+    @Override
+    public FlowFileRecord getCurrent() {
+        return (workingFlowFileRecord == null) ? originalFlowFileRecord : workingFlowFileRecord;
+    }
+
+    @Override
+    public ContentClaim getCurrentClaim() {
+        return (getCurrent() == null) ? null : getCurrent().getContentClaim();
+    }
+
+    @Override
+    public long getCurrentClaimOffset() {
+        return (getCurrent() == null) ? 0L : getCurrent().getContentClaimOffset();
+    }
+
+    boolean isWorking() {
+        return (workingFlowFileRecord != null);
+    }
+
+    Map<String, String> getOriginalAttributes() {
+        return originalAttributes;
+    }
+
+    Map<String, String> getUpdatedAttributes() {
+        return updatedAttributes;
+    }
+
+    @Override
+    public String toString() {
+        return "StandardRepositoryRecord[UpdateType=" + getType() + ",Record=" + getCurrent() + "]";
+    }
+
+    @Override
+    public List<ContentClaim> getTransientClaims() {
+        return transientClaims == null ? Collections.<ContentClaim> emptyList() : Collections.unmodifiableList(transientClaims);
+    }
+
+    void addTransientClaim(final ContentClaim claim) {
+        if (claim == null) {
+            return;
+        }
+
+        if (transientClaims == null) {
+            transientClaims = new ArrayList<>();
+        }
+        transientClaims.add(claim);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
new file mode 100644
index 0000000..39a2591
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-repository-models/src/main/java/org/apache/nifi/controller/repository/claim/StandardContentClaim.java
@@ -0,0 +1,105 @@
+/*
+ * 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.controller.repository.claim;
+
+
+/**
+ * <p>
+ * A ContentClaim is a reference to a given flow file's content. Multiple flow files may reference the same content by both having the same content claim.</p>
+ *
+ * <p>
+ * Must be thread safe</p>
+ *
+ */
+public final class StandardContentClaim implements ContentClaim, Comparable<ContentClaim> {
+
+    private final ResourceClaim resourceClaim;
+    private final long offset;
+    private volatile long length;
+
+    public StandardContentClaim(final ResourceClaim resourceClaim, final long offset) {
+        this.resourceClaim = resourceClaim;
+        this.offset = offset;
+        this.length = -1L;
+    }
+
+    public void setLength(final long length) {
+        this.length = length;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result;
+        result = prime * result + (int) (offset ^ offset >>> 32);
+        result = prime * result + (resourceClaim == null ? 0 : resourceClaim.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (this == obj) {
+            return true;
+        }
+
+        if (obj == null) {
+            return false;
+        }
+
+        if (!(obj instanceof ContentClaim)) {
+            return false;
+        }
+
+        final ContentClaim other = (ContentClaim) obj;
+        if (offset != other.getOffset()) {
+            return false;
+        }
+
+        return resourceClaim.equals(other.getResourceClaim());
+    }
+
+    @Override
+    public int compareTo(final ContentClaim o) {
+        final int resourceComp = resourceClaim.compareTo(o.getResourceClaim());
+        if (resourceComp != 0) {
+            return resourceComp;
+        }
+
+        return Long.compare(offset, o.getOffset());
+    }
+
+    @Override
+    public ResourceClaim getResourceClaim() {
+        return resourceClaim;
+    }
+
+    @Override
+    public long getOffset() {
+        return offset;
+    }
+
+    @Override
+    public long getLength() {
+        return length;
+    }
+
+    @Override
+    public String toString() {
+        return "StandardContentClaim [resourceClaim=" + resourceClaim + ", offset=" + offset + ", length=" + length + "]";
+    }
+}


[5/5] nifi git commit: NIFI-3273 This closes #1611. Handle the case of trailing NUL bytes in MinimalLockingWriteAheadLog

Posted by jo...@apache.org.
NIFI-3273 This closes #1611. Handle the case of trailing NUL bytes in MinimalLockingWriteAheadLog


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

Branch: refs/heads/master
Commit: 0f2ac39f69c1a744f151f0d924c9978f6790b7f7
Parents: 0207f21
Author: Mark Payne <ma...@hotmail.com>
Authored: Mon Apr 17 13:35:10 2017 -0400
Committer: joewitt <jo...@apache.org>
Committed: Wed Apr 19 22:08:59 2017 -0700

----------------------------------------------------------------------
 .../org/wali/MinimalLockingWriteAheadLog.java   |  69 ++++++++-
 .../test/java/org/wali/DummyRecordSerde.java    |  14 +-
 .../wali/TestMinimalLockingWriteAheadLog.java   | 149 +++++++++++++++++++
 3 files changed, 219 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/0f2ac39f/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
index 8949073..0914a79 100644
--- a/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
+++ b/nifi-commons/nifi-write-ahead-log/src/main/java/org/wali/MinimalLockingWriteAheadLog.java
@@ -663,8 +663,8 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
      * @param <S> type of record held in the partitions
      */
     private static class Partition<S> {
-
         public static final String JOURNAL_EXTENSION = ".journal";
+        private static final int NUL_BYTE = 0;
         private static final Pattern JOURNAL_FILENAME_PATTERN = Pattern.compile("\\d+\\.journal");
 
         private final SerDeFactory<S> serdeFactory;
@@ -1013,6 +1013,17 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
                     transactionId = recoveryIn.readLong();
                 } catch (final EOFException e) {
                     continue;
+                } catch (final Exception e) {
+                    // If the stream consists solely of NUL bytes, then we want to treat it
+                    // the same as an EOF because we see this happen when we suddenly lose power
+                    // while writing to a file.
+                    if (remainingBytesAllNul(recoveryIn)) {
+                        logger.warn("Failed to recover data from Write-Ahead Log Partition because encountered trailing NUL bytes. "
+                            + "This will sometimes happen after a sudden power loss. The rest of this journal file will be skipped for recovery purposes.");
+                        continue;
+                    } else {
+                        throw e;
+                    }
                 }
 
                 this.maxTransactionId.set(transactionId);
@@ -1020,6 +1031,27 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
             }
         }
 
+        /**
+         * In the case of a sudden power loss, it is common - at least in a Linux journaling File System -
+         * that the partition file that is being written to will have many trailing "NUL bytes" (0's).
+         * If this happens, then on restart we want to treat this as an incomplete transaction, so we detect
+         * this case explicitly.
+         *
+         * @param in the input stream to scan
+         * @return <code>true</code> if the InputStream contains no data or contains only NUL bytes
+         * @throws IOException if unable to read from the given InputStream
+         */
+        private boolean remainingBytesAllNul(final InputStream in) throws IOException {
+            int nextByte;
+            while ((nextByte = in.read()) != -1) {
+                if (nextByte != NUL_BYTE) {
+                    return false;
+                }
+            }
+
+            return true;
+        }
+
         private boolean hasMoreData(final InputStream in) throws IOException {
             in.mark(1);
             final int nextByte = in.read();
@@ -1059,7 +1091,40 @@ public final class MinimalLockingWriteAheadLog<T> implements WriteAheadRepositor
 
             int transactionFlag;
             do {
-                final S record = serde.deserializeEdit(recoveryIn, currentRecordMap, recoveryVersion);
+                final S record;
+                try {
+                    record = serde.deserializeEdit(recoveryIn, currentRecordMap, recoveryVersion);
+                } catch (final EOFException eof) {
+                    throw eof;
+                } catch (final Exception e) {
+                    // If the stream consists solely of NUL bytes, then we want to treat it
+                    // the same as an EOF because we see this happen when we suddenly lose power
+                    // while writing to a file. We also have logic already in the caller of this
+                    // method to properly handle EOFException's, so we will simply throw an EOFException
+                    // ourselves. However, if that is not the case, then something else has gone wrong.
+                    // In such a case, there is not much that we can do. If we simply skip over the transaction,
+                    // then the transaction may be indicating that a new attribute was added or changed. Or the
+                    // content of the FlowFile changed. A subsequent transaction for the same FlowFile may then
+                    // update the connection that is holding the FlowFile. In this case, if we simply skip over
+                    // the transaction, we end up with a FlowFile in a queue that has the wrong attributes or
+                    // content, and that can result in some very bad behavior - even security vulnerabilities if
+                    // a Route processor, for instance, routes incorrectly due to a missing attribute or content
+                    // is pointing to a previous claim where sensitive values have not been removed, etc. So
+                    // instead of attempting to skip the transaction and move on, we instead just throw the Exception
+                    // indicating that the write-ahead log is corrupt and allow the user to handle it as he/she sees
+                    // fit (likely this will result in deleting the repo, but it's possible that it could be repaired
+                    // manually or through some sort of script).
+                    if (remainingBytesAllNul(recoveryIn)) {
+                        final EOFException eof = new EOFException("Failed to recover data from Write-Ahead Log Partition because encountered trailing NUL bytes. "
+                            + "This will sometimes happen after a sudden power loss. The rest of this journal file will be skipped for recovery purposes.");
+                        eof.addSuppressed(e);
+                        throw eof;
+                    } else {
+                        throw e;
+                    }
+                }
+
+
                 if (logger.isDebugEnabled()) {
                     logger.debug("{} Recovering Transaction {}: {}", new Object[] { this, maxTransactionId.get(), record });
                 }

http://git-wip-us.apache.org/repos/asf/nifi/blob/0f2ac39f/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/DummyRecordSerde.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/DummyRecordSerde.java b/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/DummyRecordSerde.java
index 3a4e79f..e9f3b01 100644
--- a/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/DummyRecordSerde.java
+++ b/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/DummyRecordSerde.java
@@ -18,13 +18,11 @@ package org.wali;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
-import java.io.EOFException;
 import java.io.IOException;
 import java.util.Map;
 
 public class DummyRecordSerde implements SerDe<DummyRecord> {
 
-    public static final int NUM_UPDATE_TYPES = UpdateType.values().length;
     private int throwIOEAfterNserializeEdits = -1;
     private int throwOOMEAfterNserializeEdits = -1;
     private int serializeEditCount = 0;
@@ -38,7 +36,7 @@ public class DummyRecordSerde implements SerDe<DummyRecord> {
             throw new OutOfMemoryError("Serialized " + (serializeEditCount - 1) + " records successfully, so now it's time to throw OOME");
         }
 
-        out.write(record.getUpdateType().ordinal());
+        out.writeUTF(record.getUpdateType().name());
         out.writeUTF(record.getId());
 
         if (record.getUpdateType() != UpdateType.DELETE) {
@@ -58,14 +56,8 @@ public class DummyRecordSerde implements SerDe<DummyRecord> {
 
     @Override
     public DummyRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
-        final int index = in.read();
-        if (index < 0) {
-            throw new EOFException();
-        }
-        if (index >= NUM_UPDATE_TYPES) {
-            throw new IOException("Corrupt stream; got UpdateType value of " + index + " but there are only " + NUM_UPDATE_TYPES + " valid values");
-        }
-        final UpdateType updateType = UpdateType.values()[index];
+        final String updateTypeName = in.readUTF();
+        final UpdateType updateType = UpdateType.valueOf(updateTypeName);
         final String id = in.readUTF();
         final DummyRecord record = new DummyRecord(id, updateType);
 

http://git-wip-us.apache.org/repos/asf/nifi/blob/0f2ac39f/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
----------------------------------------------------------------------
diff --git a/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java b/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
index cbca968..ef33f57 100644
--- a/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
+++ b/nifi-commons/nifi-write-ahead-log/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java
@@ -27,12 +27,15 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.OutputStream;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -457,6 +460,152 @@ public class TestMinimalLockingWriteAheadLog {
         assertTrue(record3);
     }
 
+
+    @Test
+    public void testRecoverFileThatHasTrailingNULBytesAndTruncation() throws IOException {
+        final int numPartitions = 5;
+        final Path path = Paths.get("target/testRecoverFileThatHasTrailingNULBytes");
+        deleteRecursively(path.toFile());
+        Files.createDirectories(path);
+
+        final DummyRecordSerde serde = new DummyRecordSerde();
+        final WriteAheadRepository<DummyRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
+        final Collection<DummyRecord> initialRecs = repo.recoverRecords();
+        assertTrue(initialRecs.isEmpty());
+
+        final List<DummyRecord> firstTransaction = new ArrayList<>();
+        firstTransaction.add(new DummyRecord("1", UpdateType.CREATE));
+        firstTransaction.add(new DummyRecord("2", UpdateType.CREATE));
+        firstTransaction.add(new DummyRecord("3", UpdateType.CREATE));
+
+        final List<DummyRecord> secondTransaction = new ArrayList<>();
+        secondTransaction.add(new DummyRecord("1", UpdateType.UPDATE).setProperty("abc", "123"));
+        secondTransaction.add(new DummyRecord("2", UpdateType.UPDATE).setProperty("cba", "123"));
+        secondTransaction.add(new DummyRecord("3", UpdateType.UPDATE).setProperty("aaa", "123"));
+
+        final List<DummyRecord> thirdTransaction = new ArrayList<>();
+        thirdTransaction.add(new DummyRecord("1", UpdateType.DELETE));
+        thirdTransaction.add(new DummyRecord("2", UpdateType.DELETE));
+
+        repo.update(firstTransaction, true);
+        repo.update(secondTransaction, true);
+        repo.update(thirdTransaction, true);
+
+        repo.shutdown();
+
+        final File partition3Dir = path.resolve("partition-2").toFile();
+        final File journalFile = partition3Dir.listFiles()[0];
+        final byte[] contents = Files.readAllBytes(journalFile.toPath());
+
+        // Truncate the contents of the journal file by 8 bytes. Then replace with 28 trailing NUL bytes,
+        // as this is what we often see when we have a sudden power loss.
+        final byte[] truncated = Arrays.copyOfRange(contents, 0, contents.length - 8);
+        final byte[] withNuls = new byte[truncated.length + 28];
+        System.arraycopy(truncated, 0, withNuls, 0, truncated.length);
+
+        try (final OutputStream fos = new FileOutputStream(journalFile)) {
+            fos.write(withNuls);
+        }
+
+        final WriteAheadRepository<DummyRecord> recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
+        final Collection<DummyRecord> recoveredRecords = recoverRepo.recoverRecords();
+        assertFalse(recoveredRecords.isEmpty());
+        assertEquals(3, recoveredRecords.size());
+
+        boolean record1 = false, record2 = false, record3 = false;
+        for (final DummyRecord record : recoveredRecords) {
+            switch (record.getId()) {
+                case "1":
+                    record1 = true;
+                    assertEquals("123", record.getProperty("abc"));
+                    break;
+                case "2":
+                    record2 = true;
+                    assertEquals("123", record.getProperty("cba"));
+                    break;
+                case "3":
+                    record3 = true;
+                    assertEquals("123", record.getProperty("aaa"));
+                    break;
+            }
+        }
+
+        assertTrue(record1);
+        assertTrue(record2);
+        assertTrue(record3);
+    }
+
+    @Test
+    public void testRecoverFileThatHasTrailingNULBytesNoTruncation() throws IOException {
+        final int numPartitions = 5;
+        final Path path = Paths.get("target/testRecoverFileThatHasTrailingNULBytes");
+        deleteRecursively(path.toFile());
+        Files.createDirectories(path);
+
+        final DummyRecordSerde serde = new DummyRecordSerde();
+        final WriteAheadRepository<DummyRecord> repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
+        final Collection<DummyRecord> initialRecs = repo.recoverRecords();
+        assertTrue(initialRecs.isEmpty());
+
+        final List<DummyRecord> firstTransaction = new ArrayList<>();
+        firstTransaction.add(new DummyRecord("1", UpdateType.CREATE));
+        firstTransaction.add(new DummyRecord("2", UpdateType.CREATE));
+        firstTransaction.add(new DummyRecord("3", UpdateType.CREATE));
+
+        final List<DummyRecord> secondTransaction = new ArrayList<>();
+        secondTransaction.add(new DummyRecord("1", UpdateType.UPDATE).setProperty("abc", "123"));
+        secondTransaction.add(new DummyRecord("2", UpdateType.UPDATE).setProperty("cba", "123"));
+        secondTransaction.add(new DummyRecord("3", UpdateType.UPDATE).setProperty("aaa", "123"));
+
+        final List<DummyRecord> thirdTransaction = new ArrayList<>();
+        thirdTransaction.add(new DummyRecord("1", UpdateType.DELETE));
+        thirdTransaction.add(new DummyRecord("2", UpdateType.DELETE));
+
+        repo.update(firstTransaction, true);
+        repo.update(secondTransaction, true);
+        repo.update(thirdTransaction, true);
+
+        repo.shutdown();
+
+        final File partition3Dir = path.resolve("partition-2").toFile();
+        final File journalFile = partition3Dir.listFiles()[0];
+
+        // Truncate the contents of the journal file by 8 bytes. Then replace with 28 trailing NUL bytes,
+        // as this is what we often see when we have a sudden power loss.
+        final byte[] withNuls = new byte[28];
+
+        try (final OutputStream fos = new FileOutputStream(journalFile, true)) {
+            fos.write(withNuls);
+        }
+
+        final WriteAheadRepository<DummyRecord> recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null);
+        final Collection<DummyRecord> recoveredRecords = recoverRepo.recoverRecords();
+        assertFalse(recoveredRecords.isEmpty());
+        assertEquals(1, recoveredRecords.size());
+
+        boolean record1 = false, record2 = false, record3 = false;
+        for (final DummyRecord record : recoveredRecords) {
+            switch (record.getId()) {
+                case "1":
+                    record1 = record.getUpdateType() != UpdateType.DELETE;
+                    assertEquals("123", record.getProperty("abc"));
+                    break;
+                case "2":
+                    record2 = record.getUpdateType() != UpdateType.DELETE;
+                    assertEquals("123", record.getProperty("cba"));
+                    break;
+                case "3":
+                    record3 = true;
+                    assertEquals("123", record.getProperty("aaa"));
+                    break;
+            }
+        }
+
+        assertFalse(record1);
+        assertFalse(record2);
+        assertTrue(record3);
+    }
+
     @Test
     public void testCannotModifyLogAfterAllAreBlackListed() throws IOException {
         final int numPartitions = 5;


[4/5] nifi git commit: NIFI-3273: Added nifi-toolkit-flowfile-repo that contains a simple Java class that is capable of recovering a FlowFile Repository manually in the case of an operating system crash that results in trailing 0's being dumped into the

Posted by jo...@apache.org.
NIFI-3273: Added nifi-toolkit-flowfile-repo that contains a simple Java class that is capable of recovering a FlowFile Repository manually in the case of an operating system crash that results in trailing 0's being dumped into the edit logs. Also refactored flowfile repo into some independent modules so that it additional capabilities can be added in the future to examine the flowfile repo


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

Branch: refs/heads/master
Commit: 0207f21ce4bb68536843d65ecc9110ec834028cd
Parents: 141334c
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Mar 22 13:08:30 2017 -0400
Committer: joewitt <jo...@apache.org>
Committed: Wed Apr 19 21:44:23 2017 -0700

----------------------------------------------------------------------
 .../nifi-flowfile-repo-serialization/pom.xml    |  44 ++
 .../repository/RepositoryRecordSerde.java       |  68 +++
 .../RepositoryRecordSerdeFactory.java           |  95 ++++
 .../repository/SchemaRepositoryRecordSerde.java | 239 +++++++++
 .../WriteAheadRepositoryRecordSerde.java        | 517 +++++++++++++++++++
 .../repository/schema/ContentClaimFieldMap.java | 125 +++++
 .../repository/schema/ContentClaimSchema.java   |  63 +++
 .../schema/FlowFileRecordFieldMap.java          |  99 ++++
 .../repository/schema/FlowFileSchema.java       |  87 ++++
 .../schema/RepositoryRecordFieldMap.java        |  83 +++
 .../schema/RepositoryRecordSchema.java          | 138 +++++
 .../schema/RepositoryRecordUpdate.java          |  73 +++
 .../schema/ResourceClaimFieldMap.java           |  85 +++
 .../SchemaRepositoryRecordSerdeTest.java        | 266 ++++++++++
 .../nifi-framework/nifi-framework-core/pom.xml  |   8 +
 .../repository/RepositoryRecordSerde.java       |  68 ---
 .../RepositoryRecordSerdeFactory.java           |  95 ----
 .../repository/SchemaRepositoryRecordSerde.java | 239 ---------
 .../repository/StandardFlowFileRecord.java      | 341 ------------
 .../repository/StandardRepositoryRecord.java    | 221 --------
 .../WriteAheadRepositoryRecordSerde.java        | 517 -------------------
 .../repository/claim/StandardContentClaim.java  | 105 ----
 .../repository/claim/StandardResourceClaim.java | 122 -----
 .../claim/StandardResourceClaimManager.java     | 219 --------
 .../repository/schema/ContentClaimFieldMap.java | 125 -----
 .../repository/schema/ContentClaimSchema.java   |  63 ---
 .../schema/FlowFileRecordFieldMap.java          |  99 ----
 .../repository/schema/FlowFileSchema.java       |  87 ----
 .../schema/RepositoryRecordFieldMap.java        |  83 ---
 .../schema/RepositoryRecordSchema.java          | 138 -----
 .../schema/RepositoryRecordUpdate.java          |  73 ---
 .../schema/ResourceClaimFieldMap.java           |  85 ---
 .../nifi-repository-models/pom.xml              |  44 ++
 .../repository/StandardFlowFileRecord.java      | 341 ++++++++++++
 .../repository/StandardRepositoryRecord.java    | 221 ++++++++
 .../repository/claim/StandardContentClaim.java  | 105 ++++
 .../repository/claim/StandardResourceClaim.java | 122 +++++
 .../claim/StandardResourceClaimManager.java     | 219 ++++++++
 .../nifi-framework/pom.xml                      |   2 +
 nifi-toolkit/nifi-toolkit-flowfile-repo/pom.xml |  27 +
 .../flowfile/RepairCorruptedFileEndings.java    | 287 ++++++++++
 .../TestRepairCorruptedFileEndings.java         | 169 ++++++
 nifi-toolkit/pom.xml                            |   1 +
 pom.xml                                         | 231 +++++----
 44 files changed, 3663 insertions(+), 2776 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml
new file mode 100644
index 0000000..02988c0
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/pom.xml
@@ -0,0 +1,44 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-framework</artifactId>
+        <version>1.2.0-SNAPSHOT</version>
+    </parent>
+    <artifactId>nifi-flowfile-repo-serialization</artifactId>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-framework-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-repository-models</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-write-ahead-log</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-schema-utils</artifactId>
+        </dependency>
+    </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
new file mode 100644
index 0000000..44ed62d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerde.java
@@ -0,0 +1,68 @@
+/*
+ * 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.controller.repository;
+
+import java.util.Map;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.wali.SerDe;
+import org.wali.UpdateType;
+
+public abstract class RepositoryRecordSerde implements SerDe<RepositoryRecord> {
+    private Map<String, FlowFileQueue> flowFileQueueMap = null;
+
+    protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
+        this.flowFileQueueMap = queueMap;
+    }
+
+    protected Map<String, FlowFileQueue> getQueueMap() {
+        return flowFileQueueMap;
+    }
+
+    protected FlowFileQueue getFlowFileQueue(final String queueId) {
+        return flowFileQueueMap.get(queueId);
+    }
+
+    @Override
+    public Long getRecordIdentifier(final RepositoryRecord record) {
+        return record.getCurrent().getId();
+    }
+
+    @Override
+    public UpdateType getUpdateType(final RepositoryRecord record) {
+        switch (record.getType()) {
+            case CONTENTMISSING:
+            case DELETE:
+                return UpdateType.DELETE;
+            case CREATE:
+                return UpdateType.CREATE;
+            case UPDATE:
+                return UpdateType.UPDATE;
+            case SWAP_OUT:
+                return UpdateType.SWAP_OUT;
+            case SWAP_IN:
+                return UpdateType.SWAP_IN;
+        }
+        return null;
+    }
+
+    @Override
+    public String getLocation(final RepositoryRecord record) {
+        return record.getSwapLocation();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
new file mode 100644
index 0000000..c19fa94
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/RepositoryRecordSerdeFactory.java
@@ -0,0 +1,95 @@
+/*
+ * 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.controller.repository;
+
+import java.util.Map;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.wali.SerDe;
+import org.wali.SerDeFactory;
+import org.wali.UpdateType;
+
+public class RepositoryRecordSerdeFactory implements SerDeFactory<RepositoryRecord> {
+    private final String LEGACY_SERDE_ENCODING_NAME = "org.apache.nifi.controller.repository.WriteAheadFlowFileRepository$WriteAheadRecordSerde";
+    private final ResourceClaimManager resourceClaimManager;
+    private Map<String, FlowFileQueue> flowFileQueueMap = null;
+
+    public RepositoryRecordSerdeFactory(final ResourceClaimManager claimManager) {
+        this.resourceClaimManager = claimManager;
+    }
+
+    protected void setQueueMap(final Map<String, FlowFileQueue> queueMap) {
+        this.flowFileQueueMap = queueMap;
+    }
+
+    protected Map<String, FlowFileQueue> getQueueMap() {
+        return flowFileQueueMap;
+    }
+
+    @Override
+    public SerDe<RepositoryRecord> createSerDe(final String encodingName) {
+        if (encodingName == null || SchemaRepositoryRecordSerde.class.getName().equals(encodingName)) {
+            final SchemaRepositoryRecordSerde serde = new SchemaRepositoryRecordSerde(resourceClaimManager);
+            serde.setQueueMap(flowFileQueueMap);
+            return serde;
+        }
+
+        if (WriteAheadRepositoryRecordSerde.class.getName().equals(encodingName)
+            || LEGACY_SERDE_ENCODING_NAME.equals(encodingName)) {
+            final WriteAheadRepositoryRecordSerde serde = new WriteAheadRepositoryRecordSerde(resourceClaimManager);
+            serde.setQueueMap(flowFileQueueMap);
+            return serde;
+        }
+
+        throw new IllegalArgumentException("Cannot create Deserializer for Repository Records because the encoding '" + encodingName + "' is not known");
+    }
+
+    protected FlowFileQueue getFlowFileQueue(final String queueId) {
+        return flowFileQueueMap.get(queueId);
+    }
+
+    @Override
+    public Long getRecordIdentifier(final RepositoryRecord record) {
+        return record.getCurrent().getId();
+    }
+
+    @Override
+    public UpdateType getUpdateType(final RepositoryRecord record) {
+        switch (record.getType()) {
+            case CONTENTMISSING:
+            case DELETE:
+                return UpdateType.DELETE;
+            case CREATE:
+                return UpdateType.CREATE;
+            case UPDATE:
+                return UpdateType.UPDATE;
+            case SWAP_OUT:
+                return UpdateType.SWAP_OUT;
+            case SWAP_IN:
+                return UpdateType.SWAP_IN;
+        }
+        return null;
+    }
+
+    @Override
+    public String getLocation(final RepositoryRecord record) {
+        return record.getSwapLocation();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
new file mode 100644
index 0000000..221f8ce
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/SchemaRepositoryRecordSerde.java
@@ -0,0 +1,239 @@
+/*
+ * 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.controller.repository;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.controller.repository.schema.ContentClaimFieldMap;
+import org.apache.nifi.controller.repository.schema.ContentClaimSchema;
+import org.apache.nifi.controller.repository.schema.FlowFileSchema;
+import org.apache.nifi.controller.repository.schema.RepositoryRecordFieldMap;
+import org.apache.nifi.controller.repository.schema.RepositoryRecordSchema;
+import org.apache.nifi.controller.repository.schema.RepositoryRecordUpdate;
+import org.apache.nifi.repository.schema.FieldType;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordSchema;
+import org.apache.nifi.repository.schema.Repetition;
+import org.apache.nifi.repository.schema.SchemaRecordReader;
+import org.apache.nifi.repository.schema.SchemaRecordWriter;
+import org.apache.nifi.repository.schema.SimpleRecordField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.wali.SerDe;
+import org.wali.UpdateType;
+
+public class SchemaRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
+    private static final Logger logger = LoggerFactory.getLogger(SchemaRepositoryRecordSerde.class);
+    private static final int MAX_ENCODING_VERSION = 2;
+
+    private final RecordSchema writeSchema = RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V2;
+    private final RecordSchema contentClaimSchema = ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1;
+
+    private final ResourceClaimManager resourceClaimManager;
+    private volatile RecordSchema recoverySchema;
+
+    public SchemaRepositoryRecordSerde(final ResourceClaimManager resourceClaimManager) {
+        this.resourceClaimManager = resourceClaimManager;
+    }
+
+    @Override
+    public void writeHeader(final DataOutputStream out) throws IOException {
+        writeSchema.writeTo(out);
+    }
+
+    @Override
+    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord newRecordState, final DataOutputStream out) throws IOException {
+        serializeRecord(newRecordState, out);
+    }
+
+    @Override
+    public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
+        final RecordSchema schema;
+        switch (record.getType()) {
+            case CREATE:
+            case UPDATE:
+                schema = RepositoryRecordSchema.CREATE_OR_UPDATE_SCHEMA_V2;
+                break;
+            case CONTENTMISSING:
+            case DELETE:
+                schema = RepositoryRecordSchema.DELETE_SCHEMA_V2;
+                break;
+            case SWAP_IN:
+                schema = RepositoryRecordSchema.SWAP_IN_SCHEMA_V2;
+                break;
+            case SWAP_OUT:
+                schema = RepositoryRecordSchema.SWAP_OUT_SCHEMA_V2;
+                break;
+            default:
+                throw new IllegalArgumentException("Received Repository Record with unknown Update Type: " + record.getType()); // won't happen.
+        }
+
+        serializeRecord(record, out, schema, RepositoryRecordSchema.REPOSITORY_RECORD_SCHEMA_V2);
+    }
+
+
+    protected void serializeRecord(final RepositoryRecord record, final DataOutputStream out, RecordSchema schema, RecordSchema repositoryRecordSchema) throws IOException {
+        final RepositoryRecordFieldMap fieldMap = new RepositoryRecordFieldMap(record, schema, contentClaimSchema);
+        final RepositoryRecordUpdate update = new RepositoryRecordUpdate(fieldMap, repositoryRecordSchema);
+        new SchemaRecordWriter().writeRecord(update, out);
+    }
+
+    @Override
+    public void readHeader(final DataInputStream in) throws IOException {
+        recoverySchema = RecordSchema.readFrom(in);
+    }
+
+    @Override
+    public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
+        return deserializeRecord(in, version);
+    }
+
+    @Override
+    public RepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
+        final SchemaRecordReader reader = SchemaRecordReader.fromSchema(recoverySchema);
+        final Record updateRecord = reader.readRecord(in);
+        if (updateRecord == null) {
+            // null may be returned by reader.readRecord() if it encounters end-of-stream
+            return null;
+        }
+
+        // Top level is always going to be a "Repository Record Update" record because we need a 'Union' type record at the
+        // top level that indicates which type of record we have.
+        final Record record = (Record) updateRecord.getFieldValue(RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V2);
+
+        final String actionType = (String) record.getFieldValue(RepositoryRecordSchema.ACTION_TYPE_FIELD);
+        final UpdateType updateType = UpdateType.valueOf(actionType);
+        switch (updateType) {
+            case CREATE:
+                return createRecord(record);
+            case DELETE:
+                return deleteRecord(record);
+            case SWAP_IN:
+                return swapInRecord(record);
+            case SWAP_OUT:
+                return swapOutRecord(record);
+            case UPDATE:
+                return updateRecord(record);
+            default:
+                throw new IOException("Found unrecognized Update Type '" + actionType + "'");
+        }
+    }
+
+
+    @SuppressWarnings("unchecked")
+    private StandardRepositoryRecord createRecord(final Record record) {
+        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
+        ffBuilder.id((Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID));
+        ffBuilder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE));
+
+        final Long lastQueueDate = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE);
+        final Long queueDateIndex = (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX);
+        ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
+
+        final Long lineageStartDate = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE);
+        final Long lineageStartIndex = (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX);
+        ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
+
+        populateContentClaim(ffBuilder, record);
+        ffBuilder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE));
+
+        ffBuilder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES));
+
+        final FlowFileRecord flowFileRecord = ffBuilder.build();
+
+        final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER);
+        final FlowFileQueue queue = getFlowFileQueue(queueId);
+
+        final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord(queue, flowFileRecord);
+        requireFlowFileQueue(repoRecord, queueId);
+        return repoRecord;
+    }
+
+    private void requireFlowFileQueue(final StandardRepositoryRecord repoRecord, final String queueId) {
+        if (queueId == null || queueId.trim().isEmpty()) {
+            logger.warn("{} does not have a Queue associated with it; this record will be discarded", repoRecord.getCurrent());
+            repoRecord.markForAbort();
+        } else if (repoRecord.getOriginalQueue() == null) {
+            logger.warn("{} maps to unknown Queue {}; this record will be discarded", repoRecord.getCurrent(), queueId);
+            repoRecord.markForAbort();
+        }
+    }
+
+    private void populateContentClaim(final StandardFlowFileRecord.Builder ffBuilder, final Record record) {
+        final Object claimMap = record.getFieldValue(FlowFileSchema.CONTENT_CLAIM);
+        if (claimMap == null) {
+            return;
+        }
+
+        final Record claimRecord = (Record) claimMap;
+        final ContentClaim contentClaim = ContentClaimFieldMap.getContentClaim(claimRecord, resourceClaimManager);
+        final Long offset = ContentClaimFieldMap.getContentClaimOffset(claimRecord);
+
+        ffBuilder.contentClaim(contentClaim);
+        ffBuilder.contentClaimOffset(offset);
+    }
+
+    private RepositoryRecord updateRecord(final Record record) {
+        return createRecord(record);
+    }
+
+    private RepositoryRecord deleteRecord(final Record record) {
+        final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
+        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
+        final FlowFileRecord flowFileRecord = ffBuilder.build();
+
+        final StandardRepositoryRecord repoRecord = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
+        repoRecord.markForDelete();
+        return repoRecord;
+    }
+
+    private RepositoryRecord swapInRecord(final Record record) {
+        final StandardRepositoryRecord repoRecord = createRecord(record);
+        final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
+        repoRecord.setSwapLocation(swapLocation);
+
+        final String queueId = (String) record.getFieldValue(RepositoryRecordSchema.QUEUE_IDENTIFIER);
+        requireFlowFileQueue(repoRecord, queueId);
+        return repoRecord;
+    }
+
+    private RepositoryRecord swapOutRecord(final Record record) {
+        final Long recordId = (Long) record.getFieldValue(RepositoryRecordSchema.RECORD_ID_FIELD);
+        final String queueId = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
+        final String swapLocation = (String) record.getFieldValue(new SimpleRecordField(RepositoryRecordSchema.SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
+        final FlowFileQueue queue = getFlowFileQueue(queueId);
+
+        final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
+            .id(recordId)
+            .build();
+
+        return new StandardRepositoryRecord(queue, flowFileRecord, swapLocation);
+    }
+
+    @Override
+    public int getVersion() {
+        return MAX_ENCODING_VERSION;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
new file mode 100644
index 0000000..e8ce44e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/WriteAheadRepositoryRecordSerde.java
@@ -0,0 +1,517 @@
+/*
+ * 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.controller.repository;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.controller.repository.claim.StandardContentClaim;
+import org.apache.nifi.flowfile.FlowFile;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.wali.SerDe;
+import org.wali.UpdateType;
+
+public class WriteAheadRepositoryRecordSerde extends RepositoryRecordSerde implements SerDe<RepositoryRecord> {
+    private static final Logger logger = LoggerFactory.getLogger(WriteAheadRepositoryRecordSerde.class);
+
+    private static final int CURRENT_ENCODING_VERSION = 9;
+
+    public static final byte ACTION_CREATE = 0;
+    public static final byte ACTION_UPDATE = 1;
+    public static final byte ACTION_DELETE = 2;
+    public static final byte ACTION_SWAPPED_OUT = 3;
+    public static final byte ACTION_SWAPPED_IN = 4;
+
+    private long recordsRestored = 0L;
+    private final ResourceClaimManager claimManager;
+
+    public WriteAheadRepositoryRecordSerde(final ResourceClaimManager claimManager) {
+        this.claimManager = claimManager;
+    }
+
+    @Override
+    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out) throws IOException {
+        serializeEdit(previousRecordState, record, out, false);
+    }
+
+    public void serializeEdit(final RepositoryRecord previousRecordState, final RepositoryRecord record, final DataOutputStream out, final boolean forceAttributesWritten) throws IOException {
+        if (record.isMarkedForAbort()) {
+            logger.warn("Repository Record {} is marked to be aborted; it will be persisted in the FlowFileRepository as a DELETE record", record);
+            out.write(ACTION_DELETE);
+            out.writeLong(getRecordIdentifier(record));
+            serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
+            return;
+        }
+
+        final UpdateType updateType = getUpdateType(record);
+
+        if (updateType.equals(UpdateType.DELETE)) {
+            out.write(ACTION_DELETE);
+            out.writeLong(getRecordIdentifier(record));
+            serializeContentClaim(record.getCurrentClaim(), record.getCurrentClaimOffset(), out);
+            return;
+        }
+
+        // If there's a Destination Connection, that's the one that we want to associated with this record.
+        // However, on restart, we will restore the FlowFile and set this connection to its "originalConnection".
+        // If we then serialize the FlowFile again before it's transferred, it's important to allow this to happen,
+        // so we use the originalConnection instead
+        FlowFileQueue associatedQueue = record.getDestination();
+        if (associatedQueue == null) {
+            associatedQueue = record.getOriginalQueue();
+        }
+
+        if (updateType.equals(UpdateType.SWAP_OUT)) {
+            out.write(ACTION_SWAPPED_OUT);
+            out.writeLong(getRecordIdentifier(record));
+            out.writeUTF(associatedQueue.getIdentifier());
+            out.writeUTF(getLocation(record));
+            return;
+        }
+
+        final FlowFile flowFile = record.getCurrent();
+        final ContentClaim claim = record.getCurrentClaim();
+
+        switch (updateType) {
+            case UPDATE:
+                out.write(ACTION_UPDATE);
+                break;
+            case CREATE:
+                out.write(ACTION_CREATE);
+                break;
+            case SWAP_IN:
+                out.write(ACTION_SWAPPED_IN);
+                break;
+            default:
+                throw new AssertionError();
+        }
+
+        out.writeLong(getRecordIdentifier(record));
+        out.writeLong(flowFile.getEntryDate());
+        out.writeLong(flowFile.getLineageStartDate());
+        out.writeLong(flowFile.getLineageStartIndex());
+
+        final Long queueDate = flowFile.getLastQueueDate();
+        out.writeLong(queueDate == null ? System.currentTimeMillis() : queueDate);
+        out.writeLong(flowFile.getQueueDateIndex());
+        out.writeLong(flowFile.getSize());
+
+        if (associatedQueue == null) {
+            logger.warn("{} Repository Record {} has no Connection associated with it; it will be destroyed on restart",
+                new Object[] {this, record});
+            writeString("", out);
+        } else {
+            writeString(associatedQueue.getIdentifier(), out);
+        }
+
+        serializeContentClaim(claim, record.getCurrentClaimOffset(), out);
+
+        if (forceAttributesWritten || record.isAttributesChanged() || updateType == UpdateType.CREATE || updateType == UpdateType.SWAP_IN) {
+            out.write(1);   // indicate attributes changed
+            final Map<String, String> attributes = flowFile.getAttributes();
+            out.writeInt(attributes.size());
+            for (final Map.Entry<String, String> entry : attributes.entrySet()) {
+                writeString(entry.getKey(), out);
+                writeString(entry.getValue(), out);
+            }
+        } else {
+            out.write(0);   // indicate attributes did not change
+        }
+
+        if (updateType == UpdateType.SWAP_IN) {
+            out.writeUTF(record.getSwapLocation());
+        }
+    }
+
+    @Override
+    public RepositoryRecord deserializeEdit(final DataInputStream in, final Map<Object, RepositoryRecord> currentRecordStates, final int version) throws IOException {
+        final int action = in.read();
+        final long recordId = in.readLong();
+        if (action == ACTION_DELETE) {
+            final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
+
+            if (version > 4) {
+                deserializeClaim(in, version, ffBuilder);
+            }
+
+            final FlowFileRecord flowFileRecord = ffBuilder.build();
+            final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
+            record.markForDelete();
+
+            return record;
+        }
+
+        if (action == ACTION_SWAPPED_OUT) {
+            final String queueId = in.readUTF();
+            final String location = in.readUTF();
+            final FlowFileQueue queue = getFlowFileQueue(queueId);
+
+            final FlowFileRecord flowFileRecord = new StandardFlowFileRecord.Builder()
+                .id(recordId)
+                .build();
+
+            return new StandardRepositoryRecord(queue, flowFileRecord, location);
+        }
+
+        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
+        final RepositoryRecord record = currentRecordStates.get(recordId);
+        ffBuilder.id(recordId);
+        if (record != null) {
+            ffBuilder.fromFlowFile(record.getCurrent());
+        }
+        ffBuilder.entryDate(in.readLong());
+
+        if (version > 1) {
+            // read the lineage identifiers and lineage start date, which were added in version 2.
+            if (version < 9) {
+                final int numLineageIds = in.readInt();
+                for (int i = 0; i < numLineageIds; i++) {
+                    in.readUTF(); //skip identifiers
+                }
+            }
+            final long lineageStartDate = in.readLong();
+            final long lineageStartIndex;
+            if (version > 7) {
+                lineageStartIndex = in.readLong();
+            } else {
+                lineageStartIndex = 0L;
+            }
+            ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
+
+            if (version > 5) {
+                final long lastQueueDate = in.readLong();
+                final long queueDateIndex;
+                if (version > 7) {
+                    queueDateIndex = in.readLong();
+                } else {
+                    queueDateIndex = 0L;
+                }
+
+                ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
+            }
+        }
+
+        ffBuilder.size(in.readLong());
+        final String connectionId = readString(in);
+
+        logger.debug("{} -> {}", new Object[] {recordId, connectionId});
+
+        deserializeClaim(in, version, ffBuilder);
+
+        // recover new attributes, if they changed
+        final int attributesChanged = in.read();
+        if (attributesChanged == -1) {
+            throw new EOFException();
+        } else if (attributesChanged == 1) {
+            final int numAttributes = in.readInt();
+            final Map<String, String> attributes = new HashMap<>();
+            for (int i = 0; i < numAttributes; i++) {
+                final String key = readString(in);
+                final String value = readString(in);
+                attributes.put(key, value);
+            }
+
+            ffBuilder.addAttributes(attributes);
+        } else if (attributesChanged != 0) {
+            throw new IOException("Attribute Change Qualifier not found in stream; found value: "
+                + attributesChanged + " after successfully restoring " + recordsRestored + " records. The FlowFile Repository appears to be corrupt!");
+        }
+
+        final FlowFileRecord flowFile = ffBuilder.build();
+        String swapLocation = null;
+        if (action == ACTION_SWAPPED_IN) {
+            swapLocation = in.readUTF();
+        }
+
+        final FlowFileQueue queue = getFlowFileQueue(connectionId);
+        final StandardRepositoryRecord standardRepoRecord = new StandardRepositoryRecord(queue, flowFile);
+        if (swapLocation != null) {
+            standardRepoRecord.setSwapLocation(swapLocation);
+        }
+
+        if (connectionId.isEmpty()) {
+            logger.warn("{} does not have a Queue associated with it; this record will be discarded", flowFile);
+            standardRepoRecord.markForAbort();
+        } else if (queue == null) {
+            logger.warn("{} maps to unknown Queue {}; this record will be discarded", flowFile, connectionId);
+            standardRepoRecord.markForAbort();
+        }
+
+        recordsRestored++;
+        return standardRepoRecord;
+    }
+
+    @Override
+    public StandardRepositoryRecord deserializeRecord(final DataInputStream in, final int version) throws IOException {
+        final int action = in.read();
+        if (action == -1) {
+            return null;
+        }
+
+        final long recordId = in.readLong();
+        if (action == ACTION_DELETE) {
+            final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder().id(recordId);
+
+            if (version > 4) {
+                deserializeClaim(in, version, ffBuilder);
+            }
+
+            final FlowFileRecord flowFileRecord = ffBuilder.build();
+            final StandardRepositoryRecord record = new StandardRepositoryRecord((FlowFileQueue) null, flowFileRecord);
+            record.markForDelete();
+            return record;
+        }
+
+        // if action was not delete, it must be create/swap in
+        final StandardFlowFileRecord.Builder ffBuilder = new StandardFlowFileRecord.Builder();
+        final long entryDate = in.readLong();
+
+        if (version > 1) {
+            // read the lineage identifiers and lineage start date, which were added in version 2.
+            if (version < 9) {
+                final int numLineageIds = in.readInt();
+                for (int i = 0; i < numLineageIds; i++) {
+                    in.readUTF(); //skip identifiers
+                }
+            }
+
+            final long lineageStartDate = in.readLong();
+            final long lineageStartIndex;
+            if (version > 7) {
+                lineageStartIndex = in.readLong();
+            } else {
+                lineageStartIndex = 0L;
+            }
+            ffBuilder.lineageStart(lineageStartDate, lineageStartIndex);
+
+            if (version > 5) {
+                final long lastQueueDate = in.readLong();
+                final long queueDateIndex;
+                if (version > 7) {
+                    queueDateIndex = in.readLong();
+                } else {
+                    queueDateIndex = 0L;
+                }
+
+                ffBuilder.lastQueued(lastQueueDate, queueDateIndex);
+            }
+        }
+
+        final long size = in.readLong();
+        final String connectionId = readString(in);
+
+        logger.debug("{} -> {}", new Object[] {recordId, connectionId});
+
+        ffBuilder.id(recordId);
+        ffBuilder.entryDate(entryDate);
+        ffBuilder.size(size);
+
+        deserializeClaim(in, version, ffBuilder);
+
+        final int attributesChanged = in.read();
+        if (attributesChanged == 1) {
+            final int numAttributes = in.readInt();
+            final Map<String, String> attributes = new HashMap<>();
+            for (int i = 0; i < numAttributes; i++) {
+                final String key = readString(in);
+                final String value = readString(in);
+                attributes.put(key, value);
+            }
+
+            ffBuilder.addAttributes(attributes);
+        } else if (attributesChanged == -1) {
+            throw new EOFException();
+        } else if (attributesChanged != 0) {
+            throw new IOException("Attribute Change Qualifier not found in stream; found value: "
+                + attributesChanged + " after successfully restoring " + recordsRestored + " records");
+        }
+
+        final FlowFileRecord flowFile = ffBuilder.build();
+        String swapLocation = null;
+        if (action == ACTION_SWAPPED_IN) {
+            swapLocation = in.readUTF();
+        }
+
+        final StandardRepositoryRecord record;
+        final FlowFileQueue queue = getFlowFileQueue(connectionId);
+        record = new StandardRepositoryRecord(queue, flowFile);
+        if (swapLocation != null) {
+            record.setSwapLocation(swapLocation);
+        }
+
+        if (connectionId.isEmpty()) {
+            logger.warn("{} does not have a FlowFile Queue associated with it; this record will be discarded", flowFile);
+            record.markForAbort();
+        } else if (queue == null) {
+            logger.warn("{} maps to unknown FlowFile Queue {}; this record will be discarded", flowFile, connectionId);
+            record.markForAbort();
+        }
+
+        recordsRestored++;
+        return record;
+    }
+
+    @Override
+    public void serializeRecord(final RepositoryRecord record, final DataOutputStream out) throws IOException {
+        serializeEdit(null, record, out, true);
+    }
+
+    private void serializeContentClaim(final ContentClaim claim, final long offset, final DataOutputStream out) throws IOException {
+        if (claim == null) {
+            out.write(0);
+        } else {
+            out.write(1);
+
+            final ResourceClaim resourceClaim = claim.getResourceClaim();
+            writeString(resourceClaim.getId(), out);
+            writeString(resourceClaim.getContainer(), out);
+            writeString(resourceClaim.getSection(), out);
+            out.writeLong(claim.getOffset());
+            out.writeLong(claim.getLength());
+
+            out.writeLong(offset);
+            out.writeBoolean(resourceClaim.isLossTolerant());
+        }
+    }
+
+    private void deserializeClaim(final DataInputStream in, final int serializationVersion, final StandardFlowFileRecord.Builder ffBuilder) throws IOException {
+        // determine current Content Claim.
+        final int claimExists = in.read();
+        if (claimExists == 1) {
+            final String claimId;
+            if (serializationVersion < 4) {
+                claimId = String.valueOf(in.readLong());
+            } else {
+                claimId = readString(in);
+            }
+
+            final String container = readString(in);
+            final String section = readString(in);
+
+            final long resourceOffset;
+            final long resourceLength;
+            if (serializationVersion < 7) {
+                resourceOffset = 0L;
+                resourceLength = -1L;
+            } else {
+                resourceOffset = in.readLong();
+                resourceLength = in.readLong();
+            }
+
+            final long claimOffset = in.readLong();
+
+            final boolean lossTolerant;
+            if (serializationVersion >= 3) {
+                lossTolerant = in.readBoolean();
+            } else {
+                lossTolerant = false;
+            }
+
+            final ResourceClaim resourceClaim = claimManager.newResourceClaim(container, section, claimId, lossTolerant, false);
+            final StandardContentClaim contentClaim = new StandardContentClaim(resourceClaim, resourceOffset);
+            contentClaim.setLength(resourceLength);
+
+            ffBuilder.contentClaim(contentClaim);
+            ffBuilder.contentClaimOffset(claimOffset);
+        } else if (claimExists == -1) {
+            throw new EOFException();
+        } else if (claimExists != 0) {
+            throw new IOException("Claim Existence Qualifier not found in stream; found value: "
+                + claimExists + " after successfully restoring " + recordsRestored + " records");
+        }
+    }
+
+    private void writeString(final String toWrite, final OutputStream out) throws IOException {
+        final byte[] bytes = toWrite.getBytes("UTF-8");
+        final int utflen = bytes.length;
+
+        if (utflen < 65535) {
+            out.write(utflen >>> 8);
+            out.write(utflen);
+            out.write(bytes);
+        } else {
+            out.write(255);
+            out.write(255);
+            out.write(utflen >>> 24);
+            out.write(utflen >>> 16);
+            out.write(utflen >>> 8);
+            out.write(utflen);
+            out.write(bytes);
+        }
+    }
+
+    private String readString(final InputStream in) throws IOException {
+        final Integer numBytes = readFieldLength(in);
+        if (numBytes == null) {
+            throw new EOFException();
+        }
+        final byte[] bytes = new byte[numBytes];
+        fillBuffer(in, bytes, numBytes);
+        return new String(bytes, "UTF-8");
+    }
+
+    private Integer readFieldLength(final InputStream in) throws IOException {
+        final int firstValue = in.read();
+        final int secondValue = in.read();
+        if (firstValue < 0) {
+            return null;
+        }
+        if (secondValue < 0) {
+            throw new EOFException();
+        }
+        if (firstValue == 0xff && secondValue == 0xff) {
+            final int ch1 = in.read();
+            final int ch2 = in.read();
+            final int ch3 = in.read();
+            final int ch4 = in.read();
+            if ((ch1 | ch2 | ch3 | ch4) < 0) {
+                throw new EOFException();
+            }
+            return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4;
+        } else {
+            return (firstValue << 8) + secondValue;
+        }
+    }
+
+    private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException {
+        int bytesRead;
+        int totalBytesRead = 0;
+        while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) {
+            totalBytesRead += bytesRead;
+        }
+        if (totalBytesRead != length) {
+            throw new EOFException();
+        }
+    }
+
+    @Override
+    public int getVersion() {
+        return CURRENT_ENCODING_VERSION;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
new file mode 100644
index 0000000..b218ee6
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimFieldMap.java
@@ -0,0 +1,125 @@
+/*
+ * 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.controller.repository.schema;
+
+import java.util.List;
+
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.controller.repository.claim.StandardContentClaim;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordField;
+import org.apache.nifi.repository.schema.RecordSchema;
+
+public class ContentClaimFieldMap implements Record {
+    private final ContentClaim contentClaim;
+    private final long contentClaimOffset;
+    private final ResourceClaimFieldMap resourceClaimFieldMap;
+    private final RecordSchema schema;
+
+    public ContentClaimFieldMap(final ContentClaim contentClaim, final long contentClaimOffset, final RecordSchema schema) {
+        this.contentClaim = contentClaim;
+        this.contentClaimOffset = contentClaimOffset;
+        this.schema = schema;
+
+        final List<RecordField> resourceClaimFields = schema.getField(ContentClaimSchema.RESOURCE_CLAIM).getSubFields();
+        final RecordSchema resourceClaimSchema = new RecordSchema(resourceClaimFields);
+        this.resourceClaimFieldMap = new ResourceClaimFieldMap(contentClaim.getResourceClaim(), resourceClaimSchema);
+    }
+
+    @Override
+    public Object getFieldValue(final String fieldName) {
+        switch (fieldName) {
+            case ContentClaimSchema.RESOURCE_CLAIM:
+                return resourceClaimFieldMap;
+            case ContentClaimSchema.CONTENT_CLAIM_LENGTH:
+                return contentClaim.getLength();
+            case ContentClaimSchema.CONTENT_CLAIM_OFFSET:
+                return contentClaimOffset;
+            case ContentClaimSchema.RESOURCE_CLAIM_OFFSET:
+                return contentClaim.getOffset();
+            default:
+                return null;
+        }
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) (31 + contentClaimOffset + 21 * resourceClaimFieldMap.hashCode());
+    }
+
+    @Override
+    public boolean equals(final Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+
+        ContentClaimFieldMap other = (ContentClaimFieldMap) obj;
+        if (contentClaimOffset != other.contentClaimOffset) {
+            return false;
+        }
+
+        if (resourceClaimFieldMap == null) {
+            if (other.resourceClaimFieldMap != null) {
+                return false;
+            }
+        } else if (!resourceClaimFieldMap.equals(other.resourceClaimFieldMap)) {
+            return false;
+        }
+
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "ContentClaimFieldMap[" + contentClaim + "]";
+    }
+
+    public static ContentClaim getContentClaim(final Record claimRecord, final ResourceClaimManager resourceClaimManager) {
+        final Record resourceClaimRecord = (Record) claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM);
+        final String container = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_CONTAINER);
+        final String section = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_SECTION);
+        final String identifier = (String) resourceClaimRecord.getFieldValue(ContentClaimSchema.CLAIM_IDENTIFIER);
+        final Boolean lossTolerant = (Boolean) resourceClaimRecord.getFieldValue(ContentClaimSchema.LOSS_TOLERANT);
+
+        final Long length = (Long) claimRecord.getFieldValue(ContentClaimSchema.CONTENT_CLAIM_LENGTH);
+        final Long resourceOffset = (Long) claimRecord.getFieldValue(ContentClaimSchema.RESOURCE_CLAIM_OFFSET);
+
+        final ResourceClaim resourceClaim = resourceClaimManager.newResourceClaim(container, section, identifier, lossTolerant, false);
+        final StandardContentClaim contentClaim = new StandardContentClaim(resourceClaim, resourceOffset);
+        contentClaim.setLength(length);
+
+        return contentClaim;
+    }
+
+    public static Long getContentClaimOffset(final Record claimRecord) {
+        return (Long) claimRecord.getFieldValue(ContentClaimSchema.CONTENT_CLAIM_OFFSET);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java
new file mode 100644
index 0000000..c55c758
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/ContentClaimSchema.java
@@ -0,0 +1,63 @@
+/*
+ * 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.controller.repository.schema;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.nifi.repository.schema.ComplexRecordField;
+import org.apache.nifi.repository.schema.FieldType;
+import org.apache.nifi.repository.schema.RecordField;
+import org.apache.nifi.repository.schema.RecordSchema;
+import org.apache.nifi.repository.schema.Repetition;
+import org.apache.nifi.repository.schema.SimpleRecordField;
+
+public class ContentClaimSchema {
+
+    // resource claim fields
+    public static final String CLAIM_CONTAINER = "Container";
+    public static final String CLAIM_SECTION = "Section";
+    public static final String CLAIM_IDENTIFIER = "Identifier";
+    public static final String LOSS_TOLERANT = "Loss Tolerant";
+    public static final String RESOURCE_CLAIM = "Resource Claim";
+
+    // content claim fields
+    public static final String RESOURCE_CLAIM_OFFSET = "Resource Claim Offset"; // offset into resource claim where the content claim begins
+    public static final String CONTENT_CLAIM_OFFSET = "Content Claim Offset"; // offset into the content claim where the flowfile begins
+    public static final String CONTENT_CLAIM_LENGTH = "Content Claim Length";
+
+    public static final RecordSchema CONTENT_CLAIM_SCHEMA_V1;
+    public static final RecordSchema RESOURCE_CLAIM_SCHEMA_V1;
+
+    static {
+        final List<RecordField> resourceClaimFields = new ArrayList<>();
+        resourceClaimFields.add(new SimpleRecordField(CLAIM_CONTAINER, FieldType.STRING, Repetition.EXACTLY_ONE));
+        resourceClaimFields.add(new SimpleRecordField(CLAIM_SECTION, FieldType.STRING, Repetition.EXACTLY_ONE));
+        resourceClaimFields.add(new SimpleRecordField(CLAIM_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
+        resourceClaimFields.add(new SimpleRecordField(LOSS_TOLERANT, FieldType.BOOLEAN, Repetition.EXACTLY_ONE));
+        RESOURCE_CLAIM_SCHEMA_V1 = new RecordSchema(Collections.unmodifiableList(resourceClaimFields));
+
+        final List<RecordField> contentClaimFields = new ArrayList<>();
+        contentClaimFields.add(new ComplexRecordField(RESOURCE_CLAIM, Repetition.EXACTLY_ONE, resourceClaimFields));
+        contentClaimFields.add(new SimpleRecordField(RESOURCE_CLAIM_OFFSET, FieldType.LONG, Repetition.EXACTLY_ONE));
+        contentClaimFields.add(new SimpleRecordField(CONTENT_CLAIM_OFFSET, FieldType.LONG, Repetition.EXACTLY_ONE));
+        contentClaimFields.add(new SimpleRecordField(CONTENT_CLAIM_LENGTH, FieldType.LONG, Repetition.EXACTLY_ONE));
+        CONTENT_CLAIM_SCHEMA_V1 = new RecordSchema(Collections.unmodifiableList(contentClaimFields));
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java
new file mode 100644
index 0000000..ff0615f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileRecordFieldMap.java
@@ -0,0 +1,99 @@
+/*
+ * 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.controller.repository.schema;
+
+import java.util.Map;
+
+import org.apache.nifi.controller.repository.FlowFileRecord;
+import org.apache.nifi.controller.repository.StandardFlowFileRecord;
+import org.apache.nifi.controller.repository.claim.ContentClaim;
+import org.apache.nifi.controller.repository.claim.ResourceClaimManager;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordField;
+import org.apache.nifi.repository.schema.RecordSchema;
+
+public class FlowFileRecordFieldMap implements Record {
+    private final FlowFileRecord flowFile;
+    private final RecordSchema schema;
+    private final RecordSchema contentClaimSchema;
+    private final ContentClaimFieldMap contentClaim;
+
+    public FlowFileRecordFieldMap(final FlowFileRecord flowFile, final RecordSchema schema) {
+        this.flowFile = flowFile;
+        this.schema = schema;
+
+        final RecordField contentClaimField = schema.getField(FlowFileSchema.CONTENT_CLAIM);
+        contentClaimSchema = new RecordSchema(contentClaimField.getSubFields());
+        contentClaim = flowFile.getContentClaim() == null ? null : new ContentClaimFieldMap(flowFile.getContentClaim(), flowFile.getContentClaimOffset(), contentClaimSchema);
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public Object getFieldValue(final String fieldName) {
+        switch (fieldName) {
+            case FlowFileSchema.ATTRIBUTES:
+                return flowFile.getAttributes();
+            case FlowFileSchema.CONTENT_CLAIM:
+                return contentClaim;
+            case FlowFileSchema.ENTRY_DATE:
+                return flowFile.getEntryDate();
+            case FlowFileSchema.FLOWFILE_SIZE:
+                return flowFile.getSize();
+            case FlowFileSchema.LINEAGE_START_DATE:
+                return flowFile.getLineageStartDate();
+            case FlowFileSchema.LINEAGE_START_INDEX:
+                return flowFile.getLineageStartIndex();
+            case FlowFileSchema.QUEUE_DATE:
+                return flowFile.getLastQueueDate();
+            case FlowFileSchema.QUEUE_DATE_INDEX:
+                return flowFile.getQueueDateIndex();
+            case FlowFileSchema.RECORD_ID:
+                return flowFile.getId();
+        }
+
+        return null;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static FlowFileRecord getFlowFile(final Record record, final ResourceClaimManager claimManager) {
+        final StandardFlowFileRecord.Builder builder = new StandardFlowFileRecord.Builder();
+        builder.id((Long) record.getFieldValue(FlowFileSchema.RECORD_ID));
+        builder.entryDate((Long) record.getFieldValue(FlowFileSchema.ENTRY_DATE));
+        builder.size((Long) record.getFieldValue(FlowFileSchema.FLOWFILE_SIZE));
+        builder.addAttributes((Map<String, String>) record.getFieldValue(FlowFileSchema.ATTRIBUTES));
+        builder.lineageStart((Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_DATE), (Long) record.getFieldValue(FlowFileSchema.LINEAGE_START_INDEX));
+        builder.lastQueued((Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE), (Long) record.getFieldValue(FlowFileSchema.QUEUE_DATE_INDEX));
+
+        final Record contentClaimRecord = (Record) record.getFieldValue(FlowFileSchema.CONTENT_CLAIM);
+        if (contentClaimRecord != null) {
+            final ContentClaim claim = ContentClaimFieldMap.getContentClaim(contentClaimRecord, claimManager);
+            builder.contentClaim(claim);
+
+            final Long offset = ContentClaimFieldMap.getContentClaimOffset(contentClaimRecord);
+            if (offset != null) {
+                builder.contentClaimOffset(offset);
+            }
+        }
+
+        return builder.build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java
new file mode 100644
index 0000000..6af3066
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/FlowFileSchema.java
@@ -0,0 +1,87 @@
+/*
+ * 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.controller.repository.schema;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.nifi.repository.schema.ComplexRecordField;
+import org.apache.nifi.repository.schema.FieldType;
+import org.apache.nifi.repository.schema.MapRecordField;
+import org.apache.nifi.repository.schema.RecordField;
+import org.apache.nifi.repository.schema.RecordSchema;
+import org.apache.nifi.repository.schema.Repetition;
+import org.apache.nifi.repository.schema.SimpleRecordField;
+
+public class FlowFileSchema {
+
+    public static final String RECORD_ID = "Record ID";
+    public static final String ENTRY_DATE = "Entry Date";
+    public static final String LINEAGE_START_DATE = "Lineage Start Date";
+    public static final String LINEAGE_START_INDEX = "Lineage Start Index";
+    public static final String QUEUE_DATE = "Queued Date";
+    public static final String QUEUE_DATE_INDEX = "Queued Date Index";
+    public static final String FLOWFILE_SIZE = "FlowFile Size";
+    public static final String CONTENT_CLAIM = "Content Claim";
+    public static final String ATTRIBUTES = "Attributes";
+
+    // attribute fields
+    public static final String ATTRIBUTE_NAME = "Attribute Name";
+    public static final String ATTRIBUTE_VALUE = "Attribute Value";
+
+    public static final RecordSchema FLOWFILE_SCHEMA_V1;
+    public static final RecordSchema FLOWFILE_SCHEMA_V2;
+
+    static {
+        final List<RecordField> flowFileFields = new ArrayList<>();
+
+        final RecordField attributeNameField = new SimpleRecordField(ATTRIBUTE_NAME, FieldType.STRING, Repetition.EXACTLY_ONE);
+        final RecordField attributeValueField = new SimpleRecordField(ATTRIBUTE_VALUE, FieldType.STRING, Repetition.EXACTLY_ONE);
+
+        flowFileFields.add(new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(ENTRY_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(LINEAGE_START_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(LINEAGE_START_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(QUEUE_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(QUEUE_DATE_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(FLOWFILE_SIZE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new ComplexRecordField(CONTENT_CLAIM, Repetition.ZERO_OR_ONE, ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1.getFields()));
+        flowFileFields.add(new MapRecordField(ATTRIBUTES, attributeNameField, attributeValueField, Repetition.ZERO_OR_ONE));
+
+        FLOWFILE_SCHEMA_V1 = new RecordSchema(flowFileFields);
+    }
+
+    static {
+        final List<RecordField> flowFileFields = new ArrayList<>();
+
+        final RecordField attributeNameField = new SimpleRecordField(ATTRIBUTE_NAME, FieldType.LONG_STRING, Repetition.EXACTLY_ONE);
+        final RecordField attributeValueField = new SimpleRecordField(ATTRIBUTE_VALUE, FieldType.LONG_STRING, Repetition.EXACTLY_ONE);
+
+        flowFileFields.add(new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(ENTRY_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(LINEAGE_START_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(LINEAGE_START_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(QUEUE_DATE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(QUEUE_DATE_INDEX, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new SimpleRecordField(FLOWFILE_SIZE, FieldType.LONG, Repetition.EXACTLY_ONE));
+        flowFileFields.add(new ComplexRecordField(CONTENT_CLAIM, Repetition.ZERO_OR_ONE, ContentClaimSchema.CONTENT_CLAIM_SCHEMA_V1.getFields()));
+        flowFileFields.add(new MapRecordField(ATTRIBUTES, attributeNameField, attributeValueField, Repetition.ZERO_OR_ONE));
+
+        FLOWFILE_SCHEMA_V2 = new RecordSchema(flowFileFields);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
new file mode 100644
index 0000000..5fe4889
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordFieldMap.java
@@ -0,0 +1,83 @@
+/*
+ * 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.controller.repository.schema;
+
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.controller.repository.FlowFileRecord;
+import org.apache.nifi.controller.repository.RepositoryRecord;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordSchema;
+
+public class RepositoryRecordFieldMap implements Record {
+    private final RepositoryRecord record;
+    private final FlowFileRecord flowFile;
+    private final RecordSchema schema;
+    private final RecordSchema contentClaimSchema;
+
+    public RepositoryRecordFieldMap(final RepositoryRecord record, final RecordSchema repoRecordSchema, final RecordSchema contentClaimSchema) {
+        this.schema = repoRecordSchema;
+        this.contentClaimSchema = contentClaimSchema;
+        this.record = record;
+        this.flowFile = record.getCurrent();
+    }
+
+    @Override
+    public Object getFieldValue(final String fieldName) {
+        switch (fieldName) {
+            case RepositoryRecordSchema.ACTION_TYPE:
+                return record.getType().name();
+            case RepositoryRecordSchema.RECORD_ID:
+                return record.getCurrent().getId();
+            case RepositoryRecordSchema.SWAP_LOCATION:
+                return record.getSwapLocation();
+            case FlowFileSchema.ATTRIBUTES:
+                return flowFile.getAttributes();
+            case FlowFileSchema.ENTRY_DATE:
+                return flowFile.getEntryDate();
+            case FlowFileSchema.FLOWFILE_SIZE:
+                return flowFile.getSize();
+            case FlowFileSchema.LINEAGE_START_DATE:
+                return flowFile.getLineageStartDate();
+            case FlowFileSchema.LINEAGE_START_INDEX:
+                return flowFile.getLineageStartIndex();
+            case FlowFileSchema.QUEUE_DATE:
+                return flowFile.getLastQueueDate();
+            case FlowFileSchema.QUEUE_DATE_INDEX:
+                return flowFile.getQueueDateIndex();
+            case FlowFileSchema.CONTENT_CLAIM:
+                final ContentClaimFieldMap contentClaimFieldMap = record.getCurrentClaim() == null ? null
+                    : new ContentClaimFieldMap(record.getCurrentClaim(), record.getCurrentClaimOffset(), contentClaimSchema);
+                return contentClaimFieldMap;
+            case RepositoryRecordSchema.QUEUE_IDENTIFIER:
+                final FlowFileQueue queue = record.getDestination() == null ? record.getOriginalQueue() : record.getDestination();
+                return queue == null ? null : queue.getIdentifier();
+            default:
+                return null;
+        }
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public String toString() {
+        return "RepositoryRecordFieldMap[" + record + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java
new file mode 100644
index 0000000..db77c8b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordSchema.java
@@ -0,0 +1,138 @@
+/*
+ * 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.controller.repository.schema;
+
+import org.apache.nifi.repository.schema.ComplexRecordField;
+import org.apache.nifi.repository.schema.FieldType;
+import org.apache.nifi.repository.schema.RecordField;
+import org.apache.nifi.repository.schema.RecordSchema;
+import org.apache.nifi.repository.schema.Repetition;
+import org.apache.nifi.repository.schema.SimpleRecordField;
+import org.apache.nifi.repository.schema.UnionRecordField;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class RepositoryRecordSchema {
+    public static final String REPOSITORY_RECORD_UPDATE_V1 = "Repository Record Update";  // top level field name
+    public static final String REPOSITORY_RECORD_UPDATE_V2 = "Repository Record Update";  // top level field name
+
+    // repository record fields
+    public static final String ACTION_TYPE = "Action";
+    public static final String RECORD_ID = "Record ID";
+    public static final String QUEUE_IDENTIFIER = "Queue Identifier";
+    public static final String SWAP_LOCATION = "Swap Location";
+
+    // Update types
+    public static final String CREATE_OR_UPDATE_ACTION = "Create or Update";
+    public static final String DELETE_ACTION = "Delete";
+    public static final String SWAP_IN_ACTION = "Swap In";
+    public static final String SWAP_OUT_ACTION = "Swap Out";
+
+    public static final RecordSchema REPOSITORY_RECORD_SCHEMA_V1;
+    public static final RecordSchema CREATE_OR_UPDATE_SCHEMA_V1;
+    public static final RecordSchema DELETE_SCHEMA_V1;
+    public static final RecordSchema SWAP_IN_SCHEMA_V1;
+    public static final RecordSchema SWAP_OUT_SCHEMA_V1;
+
+    public static final RecordSchema REPOSITORY_RECORD_SCHEMA_V2;
+    public static final RecordSchema CREATE_OR_UPDATE_SCHEMA_V2;
+    public static final RecordSchema DELETE_SCHEMA_V2;
+    public static final RecordSchema SWAP_IN_SCHEMA_V2;
+    public static final RecordSchema SWAP_OUT_SCHEMA_V2;
+
+    public static final RecordField ACTION_TYPE_FIELD = new SimpleRecordField(ACTION_TYPE, FieldType.STRING, Repetition.EXACTLY_ONE);
+    public static final RecordField RECORD_ID_FIELD = new SimpleRecordField(RECORD_ID, FieldType.LONG, Repetition.EXACTLY_ONE);
+
+    static {
+        // Fields for "Create" or "Update" records
+        final List<RecordField> createOrUpdateFields = new ArrayList<>();
+        createOrUpdateFields.add(ACTION_TYPE_FIELD);
+        createOrUpdateFields.addAll(FlowFileSchema.FLOWFILE_SCHEMA_V1.getFields());
+
+        createOrUpdateFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
+        createOrUpdateFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.ZERO_OR_ONE));
+        final ComplexRecordField createOrUpdate = new ComplexRecordField(CREATE_OR_UPDATE_ACTION, Repetition.EXACTLY_ONE, createOrUpdateFields);
+        CREATE_OR_UPDATE_SCHEMA_V1 = new RecordSchema(createOrUpdateFields);
+
+        // Fields for "Delete" records
+        final List<RecordField> deleteFields = new ArrayList<>();
+        deleteFields.add(ACTION_TYPE_FIELD);
+        deleteFields.add(RECORD_ID_FIELD);
+        final ComplexRecordField delete = new ComplexRecordField(DELETE_ACTION, Repetition.EXACTLY_ONE, deleteFields);
+        DELETE_SCHEMA_V1 = new RecordSchema(deleteFields);
+
+        // Fields for "Swap Out" records
+        final List<RecordField> swapOutFields = new ArrayList<>();
+        swapOutFields.add(ACTION_TYPE_FIELD);
+        swapOutFields.add(RECORD_ID_FIELD);
+        swapOutFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
+        swapOutFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
+        final ComplexRecordField swapOut = new ComplexRecordField(SWAP_OUT_ACTION, Repetition.EXACTLY_ONE, swapOutFields);
+        SWAP_OUT_SCHEMA_V1 = new RecordSchema(swapOutFields);
+
+        // Fields for "Swap In" records
+        final List<RecordField> swapInFields = new ArrayList<>(createOrUpdateFields);
+        swapInFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
+        final ComplexRecordField swapIn = new ComplexRecordField(SWAP_IN_ACTION, Repetition.EXACTLY_ONE, swapInFields);
+        SWAP_IN_SCHEMA_V1 = new RecordSchema(swapInFields);
+
+        // Union Field that creates the top-level field type
+        final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V1, Repetition.EXACTLY_ONE, createOrUpdate, delete, swapOut, swapIn);
+        REPOSITORY_RECORD_SCHEMA_V1 = new RecordSchema(Collections.singletonList(repoUpdateField));
+    }
+
+    static {
+        // Fields for "Create" or "Update" records
+        final List<RecordField> createOrUpdateFields = new ArrayList<>();
+        createOrUpdateFields.add(ACTION_TYPE_FIELD);
+        createOrUpdateFields.addAll(FlowFileSchema.FLOWFILE_SCHEMA_V2.getFields());
+
+        createOrUpdateFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
+        createOrUpdateFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.ZERO_OR_ONE));
+        final ComplexRecordField createOrUpdate = new ComplexRecordField(CREATE_OR_UPDATE_ACTION, Repetition.EXACTLY_ONE, createOrUpdateFields);
+        CREATE_OR_UPDATE_SCHEMA_V2 = new RecordSchema(createOrUpdateFields);
+
+        // Fields for "Delete" records
+        final List<RecordField> deleteFields = new ArrayList<>();
+        deleteFields.add(ACTION_TYPE_FIELD);
+        deleteFields.add(RECORD_ID_FIELD);
+        final ComplexRecordField delete = new ComplexRecordField(DELETE_ACTION, Repetition.EXACTLY_ONE, deleteFields);
+        DELETE_SCHEMA_V2 = new RecordSchema(deleteFields);
+
+        // Fields for "Swap Out" records
+        final List<RecordField> swapOutFields = new ArrayList<>();
+        swapOutFields.add(ACTION_TYPE_FIELD);
+        swapOutFields.add(RECORD_ID_FIELD);
+        swapOutFields.add(new SimpleRecordField(QUEUE_IDENTIFIER, FieldType.STRING, Repetition.EXACTLY_ONE));
+        swapOutFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
+        final ComplexRecordField swapOut = new ComplexRecordField(SWAP_OUT_ACTION, Repetition.EXACTLY_ONE, swapOutFields);
+        SWAP_OUT_SCHEMA_V2 = new RecordSchema(swapOutFields);
+
+        // Fields for "Swap In" records
+        final List<RecordField> swapInFields = new ArrayList<>(createOrUpdateFields);
+        swapInFields.add(new SimpleRecordField(SWAP_LOCATION, FieldType.STRING, Repetition.EXACTLY_ONE));
+        final ComplexRecordField swapIn = new ComplexRecordField(SWAP_IN_ACTION, Repetition.EXACTLY_ONE, swapInFields);
+        SWAP_IN_SCHEMA_V2 = new RecordSchema(swapInFields);
+
+        // Union Field that creates the top-level field type
+        final UnionRecordField repoUpdateField = new UnionRecordField(REPOSITORY_RECORD_UPDATE_V2, Repetition.EXACTLY_ONE, createOrUpdate, delete, swapOut, swapIn);
+        REPOSITORY_RECORD_SCHEMA_V2 = new RecordSchema(Collections.singletonList(repoUpdateField));
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/0207f21c/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java
new file mode 100644
index 0000000..93fa4e4
--- /dev/null
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-flowfile-repo-serialization/src/main/java/org/apache/nifi/controller/repository/schema/RepositoryRecordUpdate.java
@@ -0,0 +1,73 @@
+/*
+ * 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.controller.repository.schema;
+
+import org.apache.nifi.controller.repository.RepositoryRecordType;
+import org.apache.nifi.repository.schema.NamedValue;
+import org.apache.nifi.repository.schema.Record;
+import org.apache.nifi.repository.schema.RecordSchema;
+import org.wali.UpdateType;
+
+public class RepositoryRecordUpdate implements Record {
+    private final RecordSchema schema;
+    private final RepositoryRecordFieldMap fieldMap;
+
+    public RepositoryRecordUpdate(final RepositoryRecordFieldMap fieldMap, final RecordSchema schema) {
+        this.schema = schema;
+        this.fieldMap = fieldMap;
+    }
+
+    @Override
+    public RecordSchema getSchema() {
+        return schema;
+    }
+
+    @Override
+    public Object getFieldValue(final String fieldName) {
+        if (RepositoryRecordSchema.REPOSITORY_RECORD_UPDATE_V2.equals(fieldName)) {
+            String actionType = (String) fieldMap.getFieldValue(RepositoryRecordSchema.ACTION_TYPE);
+            if (RepositoryRecordType.CONTENTMISSING.name().equals(actionType)) {
+                actionType = RepositoryRecordType.DELETE.name();
+            }
+            final UpdateType updateType = UpdateType.valueOf(actionType);
+
+            final String actionName;
+            switch (updateType) {
+                case CREATE:
+                case UPDATE:
+                    actionName = RepositoryRecordSchema.CREATE_OR_UPDATE_ACTION;
+                    break;
+                case DELETE:
+                    actionName = RepositoryRecordSchema.DELETE_ACTION;
+                    break;
+                case SWAP_IN:
+                    actionName = RepositoryRecordSchema.SWAP_IN_ACTION;
+                    break;
+                case SWAP_OUT:
+                    actionName = RepositoryRecordSchema.SWAP_OUT_ACTION;
+                    break;
+                default:
+                    return null;
+            }
+
+            return new NamedValue(actionName, fieldMap);
+        }
+        return null;
+    }
+
+}