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



##########
File path: 
nifi-stateless/nifi-stateless-api/src/main/java/org/apache/nifi/stateless/flow/TriggerResult.java
##########
@@ -57,11 +58,18 @@
      * Provides the contents of a FlowFile that was obtained by calling {@link 
#getOutputFlowFiles()}.
      * @param flowFile the FlowFile whose contents are to be read
      * @return the contents of the FlowFile
+     * @throws IOException if unable to read the contents of the FlowFile
      */
-    byte[] readContent(FlowFile flowFile);
+    byte[] readContent(FlowFile flowFile) throws IOException;

Review comment:
       If the content is to be read from somewhere, it makes sense for it to 
throw IOException. It should have been there initially, was just an oversight 
in the interface because the implementation didn't need to throw 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");

Review comment:
       I intended to mark this method as @Deprecated but i forgot. Will do so 
now. This method was implemented in ByteArrayContentRepository. I was looking 
to see how it was handled there and realized that the method is never called 
from the framework, so it should just be removed.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to