You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2021/07/28 20:15:56 UTC

[GitHub] [nifi] gresockj commented on a change in pull request #5254: NIFI-8950: Added ability to optionally use a file-system backed Conte…

gresockj commented on a change in pull request #5254:
URL: https://github.com/apache/nifi/pull/5254#discussion_r678391914



##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java
##########
@@ -144,14 +149,23 @@ public boolean isCanceled() {
             }
 
             @Override
-            public byte[] readContent(final FlowFile flowFile) {
+            public byte[] readContent(final FlowFile flowFile) throws IOException {
                 if (!(flowFile instanceof FlowFileRecord)) {
                     throw new IllegalArgumentException("FlowFile was not created by this flow");
                 }
 
                 final FlowFileRecord flowFileRecord = (FlowFileRecord) flowFile;
                 final ContentClaim contentClaim = flowFileRecord.getContentClaim();
-                final byte[] contentClaimContents = contentRepository.getBytes(contentClaim);
+
+                if (contentClaim.getLength() > Integer.MAX_VALUE) {
+                    throw new IOException("Cannot return contents of " + flowFile + " as a byte array because the contents are too large: " + contentClaim.getLength() + " bytes");
+                }
+
+                final byte[] contentClaimContents = new byte[(int) contentClaim.getLength()];

Review comment:
       Just to double check, we can't avoid reading this into memory here, right?  I understand the file-based content repo will make it so the entire contents of the flow doesn't have to be in memory at once, but this struck me as another potential memory improvement.  At first glance, the usage of `byte[] readContent(FlowFile)` couldn't be replaced with a streaming approach, but I thought I'd pose the question. 

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemRepository.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.stateless.repository;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+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.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.io.LimitedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatelessFileSystemRepository implements ContentRepository {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessFileSystemRepository.class);
+
+    private static final String CONTAINER = "stateless";
+    private static final String SECTION = "stateless";
+
+    private final File directory;
+    private final ConcurrentMap<ResourceClaim, SynchronizedByteCountingOutputStream> writableStreamMap = new ConcurrentHashMap<>();
+    private final AtomicLong resourceClaimIndex = new AtomicLong(0L);
+    private final BlockingQueue<ResourceClaim> writableClaimQueue = new LinkedBlockingQueue<>();
+    private ResourceClaimManager resourceClaimManager;
+
+    public StatelessFileSystemRepository(final File directory) {
+        this.directory = directory;
+    }
+
+    @Override
+    public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        this.resourceClaimManager = claimManager;
+        if (!directory.exists() && !directory.mkdirs()) {
+            throw new IOException("Cannot initialize Content Repository because " + directory.getAbsolutePath() + " does not exist and cannot be created");
+        }
+
+        // Check if there are any existing files and if so, purges them.
+        final File[] existingFiles = directory.listFiles(file -> file.getName().matches("\\d+\\.nifi\\.bin"));
+        if (existingFiles == null) {
+            throw new IOException("Cannot initialize Content Repository because failed to list contents of directory " + directory.getAbsolutePath());
+        }
+
+        for (final File existingFile : existingFiles) {
+            logger.info("Found existing file from previous run {}. Removing file.", existingFile.getName());
+            final boolean deleted = existingFile.delete();
+
+            if (!deleted) {
+                logger.warn("Failed to remove existing file from previous run {}", existingFile);
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        purge();
+    }
+
+    @Override
+    public Set<String> getContainerNames() {
+        return Collections.singleton(CONTAINER);
+    }
+
+    @Override
+    public long getContainerCapacity(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public long getContainerUsableSpace(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public String getContainerFileStoreName(final String containerName) {
+        return "container";
+    }
+
+    @Override
+    public ContentClaim create(final boolean lossTolerant) throws IOException {
+        ResourceClaim resourceClaim = writableClaimQueue.poll();
+        long offset;
+
+        if (resourceClaim == null) {
+            resourceClaim = new StandardResourceClaim(resourceClaimManager, CONTAINER, SECTION, String.valueOf(resourceClaimIndex.getAndIncrement()), false);
+            offset = 0L;
+
+            final File resourceClaimFile = getFile(resourceClaim);
+            final OutputStream fos = new FileOutputStream(resourceClaimFile);
+            final SynchronizedByteCountingOutputStream contentOutputStream = new SynchronizedByteCountingOutputStream(fos);
+            writableStreamMap.put(resourceClaim, contentOutputStream);
+        } else {
+            final SynchronizedByteCountingOutputStream contentOutputStream = writableStreamMap.get(resourceClaim);
+            offset = contentOutputStream.getBytesWritten();
+        }
+
+        final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, offset);
+        resourceClaimManager.incrementClaimantCount(contentClaim.getResourceClaim());
+        return contentClaim;
+    }
+
+    @Override
+    public int incrementClaimaintCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.incrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int getClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.getClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int decrementClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.decrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public boolean remove(final ContentClaim claim) {
+        return true;
+    }
+
+    @Override
+    public ContentClaim clone(final ContentClaim original, final boolean lossTolerant) throws IOException {

Review comment:
       Same comment about lossTolerant.

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemRepository.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.stateless.repository;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+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.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.io.LimitedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatelessFileSystemRepository implements ContentRepository {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessFileSystemRepository.class);
+
+    private static final String CONTAINER = "stateless";
+    private static final String SECTION = "stateless";
+
+    private final File directory;
+    private final ConcurrentMap<ResourceClaim, SynchronizedByteCountingOutputStream> writableStreamMap = new ConcurrentHashMap<>();
+    private final AtomicLong resourceClaimIndex = new AtomicLong(0L);
+    private final BlockingQueue<ResourceClaim> writableClaimQueue = new LinkedBlockingQueue<>();
+    private ResourceClaimManager resourceClaimManager;
+
+    public StatelessFileSystemRepository(final File directory) {
+        this.directory = directory;
+    }
+
+    @Override
+    public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        this.resourceClaimManager = claimManager;
+        if (!directory.exists() && !directory.mkdirs()) {
+            throw new IOException("Cannot initialize Content Repository because " + directory.getAbsolutePath() + " does not exist and cannot be created");
+        }
+
+        // Check if there are any existing files and if so, purges them.
+        final File[] existingFiles = directory.listFiles(file -> file.getName().matches("\\d+\\.nifi\\.bin"));
+        if (existingFiles == null) {
+            throw new IOException("Cannot initialize Content Repository because failed to list contents of directory " + directory.getAbsolutePath());
+        }
+
+        for (final File existingFile : existingFiles) {
+            logger.info("Found existing file from previous run {}. Removing file.", existingFile.getName());
+            final boolean deleted = existingFile.delete();
+
+            if (!deleted) {
+                logger.warn("Failed to remove existing file from previous run {}", existingFile);
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        purge();
+    }
+
+    @Override
+    public Set<String> getContainerNames() {
+        return Collections.singleton(CONTAINER);
+    }
+
+    @Override
+    public long getContainerCapacity(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public long getContainerUsableSpace(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public String getContainerFileStoreName(final String containerName) {
+        return "container";
+    }
+
+    @Override
+    public ContentClaim create(final boolean lossTolerant) throws IOException {
+        ResourceClaim resourceClaim = writableClaimQueue.poll();
+        long offset;
+
+        if (resourceClaim == null) {
+            resourceClaim = new StandardResourceClaim(resourceClaimManager, CONTAINER, SECTION, String.valueOf(resourceClaimIndex.getAndIncrement()), false);
+            offset = 0L;
+
+            final File resourceClaimFile = getFile(resourceClaim);
+            final OutputStream fos = new FileOutputStream(resourceClaimFile);
+            final SynchronizedByteCountingOutputStream contentOutputStream = new SynchronizedByteCountingOutputStream(fos);
+            writableStreamMap.put(resourceClaim, contentOutputStream);
+        } else {
+            final SynchronizedByteCountingOutputStream contentOutputStream = writableStreamMap.get(resourceClaim);
+            offset = contentOutputStream.getBytesWritten();
+        }
+
+        final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, offset);
+        resourceClaimManager.incrementClaimantCount(contentClaim.getResourceClaim());
+        return contentClaim;
+    }
+
+    @Override
+    public int incrementClaimaintCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.incrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int getClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.getClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int decrementClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.decrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public boolean remove(final ContentClaim claim) {
+        return true;
+    }
+
+    @Override
+    public ContentClaim clone(final ContentClaim original, final boolean lossTolerant) throws IOException {
+        final ContentClaim clone = create(lossTolerant);
+        try (final InputStream in = read(original);
+             final OutputStream out = write(clone)) {
+            StreamUtils.copy(in, out);
+        }
+
+        return clone;
+    }
+
+    @Override
+    public long merge(final Collection<ContentClaim> claims, final ContentClaim destination, final byte[] header, final byte[] footer, final byte[] demarcator) {
+        throw new UnsupportedOperationException("This never gets used");
+    }
+
+    @Override
+    public long importFrom(final Path content, final ContentClaim claim) throws IOException {
+        try (final InputStream in = Files.newInputStream(content, StandardOpenOption.READ)) {
+            return importFrom(in, claim);
+        }
+    }
+
+    @Override
+    public long importFrom(final InputStream content, final ContentClaim claim) throws IOException {
+        try (final OutputStream out = write(claim)) {
+            return StreamUtils.copy(content, out);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final Path destination, final boolean append) throws IOException {
+        final OpenOption[] openOptions = append ? new StandardOpenOption[] {StandardOpenOption.CREATE, StandardOpenOption.APPEND} :
+            new StandardOpenOption[] {StandardOpenOption.CREATE};
+
+        try (final OutputStream out = Files.newOutputStream(destination, openOptions)) {
+            return exportTo(claim, out);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final Path destination, final boolean append, final long offset, final long length) throws IOException {
+        final OpenOption[] openOptions = append ? new StandardOpenOption[] {StandardOpenOption.CREATE, StandardOpenOption.APPEND} :
+            new StandardOpenOption[] {StandardOpenOption.CREATE};
+
+        try (final OutputStream out = Files.newOutputStream(destination, openOptions)) {
+            return exportTo(claim, out, offset, length);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final OutputStream destination) throws IOException {
+        try (final InputStream in = read(claim)) {
+            return StreamUtils.copy(in, destination);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final OutputStream destination, final long offset, final long length) throws IOException {
+        try (final InputStream in = read(claim)) {
+            StreamUtils.skip(in, offset);
+            StreamUtils.copy(in, destination, length);
+        }
+
+        return length;
+    }
+
+    @Override
+    public long size(final ContentClaim claim) {
+        return claim.getLength();
+    }
+
+    @Override
+    public InputStream read(final ContentClaim claim) throws IOException {
+        if (claim == null) {
+            return new ByteArrayInputStream(new byte[0]);
+        }
+
+        final InputStream resourceClaimIn = read(claim.getResourceClaim());
+        StreamUtils.skip(resourceClaimIn, claim.getOffset());
+
+        final InputStream limitedIn = new LimitedInputStream(resourceClaimIn, claim.getLength());
+        return limitedIn;
+    }
+
+    @Override
+    public InputStream read(final ResourceClaim claim) throws IOException {
+        validateResourceClaim(claim);
+        final File file = getFile(claim);
+        return new FileInputStream(file);
+    }
+
+    private File getFile(final ResourceClaim claim) {
+        return new File(directory, claim.getId() + ".nifi.bin");
+    }
+
+    private void validateResourceClaim(final ResourceClaim resourceClaim) {
+        if (!CONTAINER.equals(resourceClaim.getContainer())) {
+            throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");
+        }
+
+        if (!SECTION.equals(resourceClaim.getSection())) {
+            throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");
+        }
+    }
+
+    @Override
+    public OutputStream write(final ContentClaim claim) throws IOException {
+        validateContentClaimForWriting(claim);
+
+        final SynchronizedByteCountingOutputStream out = writableStreamMap.get(claim.getResourceClaim());
+        if (out == null) {
+            throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");

Review comment:
       Since this specific exception is thrown in several places, what about either making the string a constant, or adding a little method like:
   ```
   private void throwForInvalidResourceClaim() {
       throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");
   }
   ```

##########
File path: nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/config/PropertiesFileEngineConfigurationParser.java
##########
@@ -41,6 +42,7 @@
     private static final String NAR_DIRECTORY = PREFIX + "nar.directory";
     private static final String EXTENSIONS_DIRECTORY = PREFIX + "extensions.directory";
     private static final String WORKING_DIRECTORY = PREFIX + "working.directory";
+    private static final String CONTENT_REPO_DIRECTORY = PREFIX + "content.repository.directory";

Review comment:
       I think you should add this as a blank property to `nifi-stateless/nifi-stateless-resources/src/main/resources/conf/stateless.properties`:
   
   ```
   nifi.stateless.content.repository.directory=
   ```

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemRepository.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.stateless.repository;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+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.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.io.LimitedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatelessFileSystemRepository implements ContentRepository {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessFileSystemRepository.class);
+
+    private static final String CONTAINER = "stateless";
+    private static final String SECTION = "stateless";
+
+    private final File directory;
+    private final ConcurrentMap<ResourceClaim, SynchronizedByteCountingOutputStream> writableStreamMap = new ConcurrentHashMap<>();
+    private final AtomicLong resourceClaimIndex = new AtomicLong(0L);
+    private final BlockingQueue<ResourceClaim> writableClaimQueue = new LinkedBlockingQueue<>();
+    private ResourceClaimManager resourceClaimManager;
+
+    public StatelessFileSystemRepository(final File directory) {
+        this.directory = directory;
+    }
+
+    @Override
+    public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        this.resourceClaimManager = claimManager;
+        if (!directory.exists() && !directory.mkdirs()) {
+            throw new IOException("Cannot initialize Content Repository because " + directory.getAbsolutePath() + " does not exist and cannot be created");
+        }
+
+        // Check if there are any existing files and if so, purges them.
+        final File[] existingFiles = directory.listFiles(file -> file.getName().matches("\\d+\\.nifi\\.bin"));
+        if (existingFiles == null) {
+            throw new IOException("Cannot initialize Content Repository because failed to list contents of directory " + directory.getAbsolutePath());
+        }
+
+        for (final File existingFile : existingFiles) {
+            logger.info("Found existing file from previous run {}. Removing file.", existingFile.getName());
+            final boolean deleted = existingFile.delete();
+
+            if (!deleted) {
+                logger.warn("Failed to remove existing file from previous run {}", existingFile);
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        purge();
+    }
+
+    @Override
+    public Set<String> getContainerNames() {
+        return Collections.singleton(CONTAINER);
+    }
+
+    @Override
+    public long getContainerCapacity(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public long getContainerUsableSpace(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public String getContainerFileStoreName(final String containerName) {
+        return "container";
+    }
+
+    @Override
+    public ContentClaim create(final boolean lossTolerant) throws IOException {

Review comment:
       Maybe a method comment that lossTolerant is unused.

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemRepository.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.stateless.repository;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+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.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.io.LimitedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatelessFileSystemRepository implements ContentRepository {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessFileSystemRepository.class);
+
+    private static final String CONTAINER = "stateless";
+    private static final String SECTION = "stateless";
+
+    private final File directory;
+    private final ConcurrentMap<ResourceClaim, SynchronizedByteCountingOutputStream> writableStreamMap = new ConcurrentHashMap<>();
+    private final AtomicLong resourceClaimIndex = new AtomicLong(0L);
+    private final BlockingQueue<ResourceClaim> writableClaimQueue = new LinkedBlockingQueue<>();
+    private ResourceClaimManager resourceClaimManager;
+
+    public StatelessFileSystemRepository(final File directory) {
+        this.directory = directory;
+    }
+
+    @Override
+    public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        this.resourceClaimManager = claimManager;
+        if (!directory.exists() && !directory.mkdirs()) {
+            throw new IOException("Cannot initialize Content Repository because " + directory.getAbsolutePath() + " does not exist and cannot be created");
+        }
+
+        // Check if there are any existing files and if so, purges them.
+        final File[] existingFiles = directory.listFiles(file -> file.getName().matches("\\d+\\.nifi\\.bin"));
+        if (existingFiles == null) {
+            throw new IOException("Cannot initialize Content Repository because failed to list contents of directory " + directory.getAbsolutePath());
+        }
+
+        for (final File existingFile : existingFiles) {
+            logger.info("Found existing file from previous run {}. Removing file.", existingFile.getName());
+            final boolean deleted = existingFile.delete();
+
+            if (!deleted) {
+                logger.warn("Failed to remove existing file from previous run {}", existingFile);
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        purge();
+    }
+
+    @Override
+    public Set<String> getContainerNames() {
+        return Collections.singleton(CONTAINER);
+    }
+
+    @Override
+    public long getContainerCapacity(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public long getContainerUsableSpace(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public String getContainerFileStoreName(final String containerName) {
+        return "container";
+    }
+
+    @Override
+    public ContentClaim create(final boolean lossTolerant) throws IOException {
+        ResourceClaim resourceClaim = writableClaimQueue.poll();
+        long offset;
+
+        if (resourceClaim == null) {
+            resourceClaim = new StandardResourceClaim(resourceClaimManager, CONTAINER, SECTION, String.valueOf(resourceClaimIndex.getAndIncrement()), false);
+            offset = 0L;
+
+            final File resourceClaimFile = getFile(resourceClaim);
+            final OutputStream fos = new FileOutputStream(resourceClaimFile);
+            final SynchronizedByteCountingOutputStream contentOutputStream = new SynchronizedByteCountingOutputStream(fos);
+            writableStreamMap.put(resourceClaim, contentOutputStream);
+        } else {
+            final SynchronizedByteCountingOutputStream contentOutputStream = writableStreamMap.get(resourceClaim);
+            offset = contentOutputStream.getBytesWritten();
+        }
+
+        final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, offset);
+        resourceClaimManager.incrementClaimantCount(contentClaim.getResourceClaim());
+        return contentClaim;
+    }
+
+    @Override
+    public int incrementClaimaintCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.incrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int getClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.getClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int decrementClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.decrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public boolean remove(final ContentClaim claim) {
+        return true;
+    }
+
+    @Override
+    public ContentClaim clone(final ContentClaim original, final boolean lossTolerant) throws IOException {
+        final ContentClaim clone = create(lossTolerant);
+        try (final InputStream in = read(original);
+             final OutputStream out = write(clone)) {
+            StreamUtils.copy(in, out);
+        }
+
+        return clone;
+    }
+
+    @Override
+    public long merge(final Collection<ContentClaim> claims, final ContentClaim destination, final byte[] header, final byte[] footer, final byte[] demarcator) {
+        throw new UnsupportedOperationException("This never gets used");

Review comment:
       What about "Merge is unused in stateless"?
   
   Also, I noticed you provided an implementation for `ByteArrayContentRepository#merge`.  Should that implementation go away as well, or does something about ByteArrayContentRepository` cause the `merge` method to be used?

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemRepository.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.stateless.repository;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+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.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.io.LimitedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatelessFileSystemRepository implements ContentRepository {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessFileSystemRepository.class);
+
+    private static final String CONTAINER = "stateless";
+    private static final String SECTION = "stateless";
+
+    private final File directory;
+    private final ConcurrentMap<ResourceClaim, SynchronizedByteCountingOutputStream> writableStreamMap = new ConcurrentHashMap<>();
+    private final AtomicLong resourceClaimIndex = new AtomicLong(0L);
+    private final BlockingQueue<ResourceClaim> writableClaimQueue = new LinkedBlockingQueue<>();
+    private ResourceClaimManager resourceClaimManager;
+
+    public StatelessFileSystemRepository(final File directory) {
+        this.directory = directory;
+    }
+
+    @Override
+    public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        this.resourceClaimManager = claimManager;
+        if (!directory.exists() && !directory.mkdirs()) {
+            throw new IOException("Cannot initialize Content Repository because " + directory.getAbsolutePath() + " does not exist and cannot be created");
+        }
+
+        // Check if there are any existing files and if so, purges them.
+        final File[] existingFiles = directory.listFiles(file -> file.getName().matches("\\d+\\.nifi\\.bin"));
+        if (existingFiles == null) {
+            throw new IOException("Cannot initialize Content Repository because failed to list contents of directory " + directory.getAbsolutePath());
+        }
+
+        for (final File existingFile : existingFiles) {
+            logger.info("Found existing file from previous run {}. Removing file.", existingFile.getName());
+            final boolean deleted = existingFile.delete();
+
+            if (!deleted) {
+                logger.warn("Failed to remove existing file from previous run {}", existingFile);
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        purge();
+    }
+
+    @Override
+    public Set<String> getContainerNames() {
+        return Collections.singleton(CONTAINER);
+    }
+
+    @Override
+    public long getContainerCapacity(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public long getContainerUsableSpace(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public String getContainerFileStoreName(final String containerName) {
+        return "container";
+    }
+
+    @Override
+    public ContentClaim create(final boolean lossTolerant) throws IOException {
+        ResourceClaim resourceClaim = writableClaimQueue.poll();
+        long offset;
+
+        if (resourceClaim == null) {
+            resourceClaim = new StandardResourceClaim(resourceClaimManager, CONTAINER, SECTION, String.valueOf(resourceClaimIndex.getAndIncrement()), false);
+            offset = 0L;
+
+            final File resourceClaimFile = getFile(resourceClaim);
+            final OutputStream fos = new FileOutputStream(resourceClaimFile);
+            final SynchronizedByteCountingOutputStream contentOutputStream = new SynchronizedByteCountingOutputStream(fos);
+            writableStreamMap.put(resourceClaim, contentOutputStream);
+        } else {
+            final SynchronizedByteCountingOutputStream contentOutputStream = writableStreamMap.get(resourceClaim);
+            offset = contentOutputStream.getBytesWritten();
+        }
+
+        final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, offset);
+        resourceClaimManager.incrementClaimantCount(contentClaim.getResourceClaim());
+        return contentClaim;
+    }
+
+    @Override
+    public int incrementClaimaintCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.incrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int getClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.getClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int decrementClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.decrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public boolean remove(final ContentClaim claim) {
+        return true;
+    }
+
+    @Override
+    public ContentClaim clone(final ContentClaim original, final boolean lossTolerant) throws IOException {
+        final ContentClaim clone = create(lossTolerant);
+        try (final InputStream in = read(original);
+             final OutputStream out = write(clone)) {
+            StreamUtils.copy(in, out);
+        }
+
+        return clone;
+    }
+
+    @Override
+    public long merge(final Collection<ContentClaim> claims, final ContentClaim destination, final byte[] header, final byte[] footer, final byte[] demarcator) {
+        throw new UnsupportedOperationException("This never gets used");
+    }
+
+    @Override
+    public long importFrom(final Path content, final ContentClaim claim) throws IOException {
+        try (final InputStream in = Files.newInputStream(content, StandardOpenOption.READ)) {
+            return importFrom(in, claim);
+        }
+    }
+
+    @Override
+    public long importFrom(final InputStream content, final ContentClaim claim) throws IOException {
+        try (final OutputStream out = write(claim)) {
+            return StreamUtils.copy(content, out);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final Path destination, final boolean append) throws IOException {
+        final OpenOption[] openOptions = append ? new StandardOpenOption[] {StandardOpenOption.CREATE, StandardOpenOption.APPEND} :
+            new StandardOpenOption[] {StandardOpenOption.CREATE};
+
+        try (final OutputStream out = Files.newOutputStream(destination, openOptions)) {
+            return exportTo(claim, out);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final Path destination, final boolean append, final long offset, final long length) throws IOException {
+        final OpenOption[] openOptions = append ? new StandardOpenOption[] {StandardOpenOption.CREATE, StandardOpenOption.APPEND} :
+            new StandardOpenOption[] {StandardOpenOption.CREATE};
+
+        try (final OutputStream out = Files.newOutputStream(destination, openOptions)) {
+            return exportTo(claim, out, offset, length);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final OutputStream destination) throws IOException {
+        try (final InputStream in = read(claim)) {
+            return StreamUtils.copy(in, destination);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final OutputStream destination, final long offset, final long length) throws IOException {
+        try (final InputStream in = read(claim)) {
+            StreamUtils.skip(in, offset);
+            StreamUtils.copy(in, destination, length);
+        }
+
+        return length;
+    }
+
+    @Override
+    public long size(final ContentClaim claim) {
+        return claim.getLength();
+    }
+
+    @Override
+    public InputStream read(final ContentClaim claim) throws IOException {
+        if (claim == null) {
+            return new ByteArrayInputStream(new byte[0]);
+        }
+
+        final InputStream resourceClaimIn = read(claim.getResourceClaim());
+        StreamUtils.skip(resourceClaimIn, claim.getOffset());
+
+        final InputStream limitedIn = new LimitedInputStream(resourceClaimIn, claim.getLength());
+        return limitedIn;
+    }
+
+    @Override
+    public InputStream read(final ResourceClaim claim) throws IOException {
+        validateResourceClaim(claim);
+        final File file = getFile(claim);
+        return new FileInputStream(file);
+    }
+
+    private File getFile(final ResourceClaim claim) {
+        return new File(directory, claim.getId() + ".nifi.bin");
+    }
+
+    private void validateResourceClaim(final ResourceClaim resourceClaim) {
+        if (!CONTAINER.equals(resourceClaim.getContainer())) {
+            throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");

Review comment:
       That would be alarming, wouldn't it!

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemRepository.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.stateless.repository;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+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.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.io.LimitedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatelessFileSystemRepository implements ContentRepository {
+    private static final Logger logger = LoggerFactory.getLogger(StatelessFileSystemRepository.class);
+
+    private static final String CONTAINER = "stateless";
+    private static final String SECTION = "stateless";
+
+    private final File directory;
+    private final ConcurrentMap<ResourceClaim, SynchronizedByteCountingOutputStream> writableStreamMap = new ConcurrentHashMap<>();
+    private final AtomicLong resourceClaimIndex = new AtomicLong(0L);
+    private final BlockingQueue<ResourceClaim> writableClaimQueue = new LinkedBlockingQueue<>();
+    private ResourceClaimManager resourceClaimManager;
+
+    public StatelessFileSystemRepository(final File directory) {
+        this.directory = directory;
+    }
+
+    @Override
+    public void initialize(final ResourceClaimManager claimManager) throws IOException {
+        this.resourceClaimManager = claimManager;
+        if (!directory.exists() && !directory.mkdirs()) {
+            throw new IOException("Cannot initialize Content Repository because " + directory.getAbsolutePath() + " does not exist and cannot be created");
+        }
+
+        // Check if there are any existing files and if so, purges them.
+        final File[] existingFiles = directory.listFiles(file -> file.getName().matches("\\d+\\.nifi\\.bin"));
+        if (existingFiles == null) {
+            throw new IOException("Cannot initialize Content Repository because failed to list contents of directory " + directory.getAbsolutePath());
+        }
+
+        for (final File existingFile : existingFiles) {
+            logger.info("Found existing file from previous run {}. Removing file.", existingFile.getName());
+            final boolean deleted = existingFile.delete();
+
+            if (!deleted) {
+                logger.warn("Failed to remove existing file from previous run {}", existingFile);
+            }
+        }
+    }
+
+    @Override
+    public void shutdown() {
+        purge();
+    }
+
+    @Override
+    public Set<String> getContainerNames() {
+        return Collections.singleton(CONTAINER);
+    }
+
+    @Override
+    public long getContainerCapacity(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public long getContainerUsableSpace(final String containerName) {
+        return 0;
+    }
+
+    @Override
+    public String getContainerFileStoreName(final String containerName) {
+        return "container";
+    }
+
+    @Override
+    public ContentClaim create(final boolean lossTolerant) throws IOException {
+        ResourceClaim resourceClaim = writableClaimQueue.poll();
+        long offset;
+
+        if (resourceClaim == null) {
+            resourceClaim = new StandardResourceClaim(resourceClaimManager, CONTAINER, SECTION, String.valueOf(resourceClaimIndex.getAndIncrement()), false);
+            offset = 0L;
+
+            final File resourceClaimFile = getFile(resourceClaim);
+            final OutputStream fos = new FileOutputStream(resourceClaimFile);
+            final SynchronizedByteCountingOutputStream contentOutputStream = new SynchronizedByteCountingOutputStream(fos);
+            writableStreamMap.put(resourceClaim, contentOutputStream);
+        } else {
+            final SynchronizedByteCountingOutputStream contentOutputStream = writableStreamMap.get(resourceClaim);
+            offset = contentOutputStream.getBytesWritten();
+        }
+
+        final ContentClaim contentClaim = new StandardContentClaim(resourceClaim, offset);
+        resourceClaimManager.incrementClaimantCount(contentClaim.getResourceClaim());
+        return contentClaim;
+    }
+
+    @Override
+    public int incrementClaimaintCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.incrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int getClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.getClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public int decrementClaimantCount(final ContentClaim claim) {
+        if (claim == null) {
+            return 0;
+        }
+
+        return resourceClaimManager.decrementClaimantCount(claim.getResourceClaim());
+    }
+
+    @Override
+    public boolean remove(final ContentClaim claim) {
+        return true;
+    }
+
+    @Override
+    public ContentClaim clone(final ContentClaim original, final boolean lossTolerant) throws IOException {
+        final ContentClaim clone = create(lossTolerant);
+        try (final InputStream in = read(original);
+             final OutputStream out = write(clone)) {
+            StreamUtils.copy(in, out);
+        }
+
+        return clone;
+    }
+
+    @Override
+    public long merge(final Collection<ContentClaim> claims, final ContentClaim destination, final byte[] header, final byte[] footer, final byte[] demarcator) {
+        throw new UnsupportedOperationException("This never gets used");
+    }
+
+    @Override
+    public long importFrom(final Path content, final ContentClaim claim) throws IOException {
+        try (final InputStream in = Files.newInputStream(content, StandardOpenOption.READ)) {
+            return importFrom(in, claim);
+        }
+    }
+
+    @Override
+    public long importFrom(final InputStream content, final ContentClaim claim) throws IOException {
+        try (final OutputStream out = write(claim)) {
+            return StreamUtils.copy(content, out);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final Path destination, final boolean append) throws IOException {
+        final OpenOption[] openOptions = append ? new StandardOpenOption[] {StandardOpenOption.CREATE, StandardOpenOption.APPEND} :
+            new StandardOpenOption[] {StandardOpenOption.CREATE};
+
+        try (final OutputStream out = Files.newOutputStream(destination, openOptions)) {
+            return exportTo(claim, out);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final Path destination, final boolean append, final long offset, final long length) throws IOException {
+        final OpenOption[] openOptions = append ? new StandardOpenOption[] {StandardOpenOption.CREATE, StandardOpenOption.APPEND} :
+            new StandardOpenOption[] {StandardOpenOption.CREATE};
+
+        try (final OutputStream out = Files.newOutputStream(destination, openOptions)) {
+            return exportTo(claim, out, offset, length);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final OutputStream destination) throws IOException {
+        try (final InputStream in = read(claim)) {
+            return StreamUtils.copy(in, destination);
+        }
+    }
+
+    @Override
+    public long exportTo(final ContentClaim claim, final OutputStream destination, final long offset, final long length) throws IOException {
+        try (final InputStream in = read(claim)) {
+            StreamUtils.skip(in, offset);
+            StreamUtils.copy(in, destination, length);
+        }
+
+        return length;
+    }
+
+    @Override
+    public long size(final ContentClaim claim) {
+        return claim.getLength();
+    }
+
+    @Override
+    public InputStream read(final ContentClaim claim) throws IOException {
+        if (claim == null) {
+            return new ByteArrayInputStream(new byte[0]);
+        }
+
+        final InputStream resourceClaimIn = read(claim.getResourceClaim());
+        StreamUtils.skip(resourceClaimIn, claim.getOffset());
+
+        final InputStream limitedIn = new LimitedInputStream(resourceClaimIn, claim.getLength());
+        return limitedIn;
+    }
+
+    @Override
+    public InputStream read(final ResourceClaim claim) throws IOException {
+        validateResourceClaim(claim);
+        final File file = getFile(claim);
+        return new FileInputStream(file);
+    }
+
+    private File getFile(final ResourceClaim claim) {
+        return new File(directory, claim.getId() + ".nifi.bin");
+    }
+
+    private void validateResourceClaim(final ResourceClaim resourceClaim) {
+        if (!CONTAINER.equals(resourceClaim.getContainer())) {
+            throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");
+        }
+
+        if (!SECTION.equals(resourceClaim.getSection())) {
+            throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");
+        }
+    }
+
+    @Override
+    public OutputStream write(final ContentClaim claim) throws IOException {
+        validateContentClaimForWriting(claim);
+
+        final SynchronizedByteCountingOutputStream out = writableStreamMap.get(claim.getResourceClaim());
+        if (out == null) {
+            throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");
+        }
+
+        final StandardContentClaim scc = (StandardContentClaim) claim;
+        scc.setLength(0); // Set the length to 0. Initially it will be set to -1. By setting it to 0, the repository knows that it has been written to and cannot be written to again.
+        return new ContentOutputStream(out, scc);
+    }
+
+    private void validateContentClaimForWriting(final ContentClaim claim) throws IOException {
+        if (claim == null) {
+            throw new NullPointerException("ContentClaim cannot be null");
+        }

Review comment:
       Could be streamlined with:
   ```suggestion
           Objects.requireNonNull(claim, "ContentClaim cannot be null");
   ```

##########
File path: nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/repository/StatelessFileSystemRepository.java
##########
@@ -0,0 +1,361 @@
+/*
+ * 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.stateless.repository;
+
+import org.apache.nifi.controller.repository.ContentRepository;
+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.controller.repository.claim.StandardResourceClaim;
+import org.apache.nifi.controller.repository.io.LimitedInputStream;
+import org.apache.nifi.stream.io.StreamUtils;
+import org.apache.nifi.stream.io.SynchronizedByteCountingOutputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.OpenOption;
+import java.nio.file.Path;
+import java.nio.file.StandardOpenOption;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class StatelessFileSystemRepository implements ContentRepository {

Review comment:
       Shall we call this StatelessFileSystemContentRepository for clarity?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org