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



##########
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:
       I didn't bother because the notion of loss tolerant content claims was 
added to the original API, about 10 years, but it's never been used anywhere in 
the application :) We should probably remove it in the 2.0 release.




-- 
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