mirror of https://github.com/apache/nifi.git
NIFI-8950: Added ability to optionally use a file-system backed Content Repository for Stateless NiFi
This closes #5254 Signed-off-by: David Handermann <exceptionfactory@apache.org>
This commit is contained in:
parent
5106197b3b
commit
fb4edfa0d6
|
@ -89,7 +89,7 @@ public class StreamUtils {
|
|||
len = source.read(destination, bytesRead, destination.length - bytesRead);
|
||||
if (len < 0) {
|
||||
if (ensureCapacity) {
|
||||
throw new EOFException();
|
||||
throw new EOFException("Expected to read " + destination.length + " bytes but encountered EOF after " + bytesRead + " bytes");
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -42,6 +42,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.jar.JarFile;
|
||||
|
@ -277,6 +278,11 @@ public class StatelessKafkaConnectorUtil {
|
|||
return new File(properties.getOrDefault(KRB5_FILE, DEFAULT_KRB5_FILE));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<File> getContentRepositoryDirectory() {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SslContextDefinition getSslContext() {
|
||||
return sslContextDefinition;
|
||||
|
|
|
@ -47,6 +47,7 @@ import java.util.regex.Pattern;
|
|||
public class StatelessNiFiSourceTask extends SourceTask {
|
||||
public static final String STATE_MAP_KEY = "task.index";
|
||||
private static final Logger logger = LoggerFactory.getLogger(StatelessNiFiSourceTask.class);
|
||||
private static final long FAILURE_YIELD_MILLIS = 1000L;
|
||||
|
||||
private StatelessDataflow dataflow;
|
||||
private String outputPortName;
|
||||
|
@ -153,7 +154,7 @@ public class StatelessNiFiSourceTask extends SourceTask {
|
|||
if (!triggerResult.isSuccessful()) {
|
||||
logger.error("Dataflow {} failed to execute properly", dataflowName, triggerResult.getFailureCause().orElse(null));
|
||||
trigger.cancel();
|
||||
failureYieldExpiration = System.currentTimeMillis() + 1000L; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
|
||||
failureYieldExpiration = System.currentTimeMillis() + FAILURE_YIELD_MILLIS; // delay next execution for 1 second to avoid constnatly failing and utilization huge amounts of resources
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -174,10 +175,17 @@ public class StatelessNiFiSourceTask extends SourceTask {
|
|||
partitionMap = clusterStatePartitionMap;
|
||||
}
|
||||
|
||||
for (final FlowFile flowFile : outputFlowFiles) {
|
||||
final byte[] contents = triggerResult.readContent(flowFile);
|
||||
final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);
|
||||
sourceRecords.add(sourceRecord);
|
||||
try {
|
||||
for (final FlowFile flowFile : outputFlowFiles) {
|
||||
final byte[] contents = triggerResult.readContentAsByteArray(flowFile);
|
||||
final SourceRecord sourceRecord = createSourceRecord(flowFile, contents, componentState, partitionMap);
|
||||
sourceRecords.add(sourceRecord);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
logger.error("Failed to obtain contents of Output FlowFiles in order to form Kafka Record", e);
|
||||
triggerResult.abort(e);
|
||||
failureYieldExpiration = System.currentTimeMillis() + FAILURE_YIELD_MILLIS; // delay next execution for 1 second to avoid constantly failing and utilization huge amounts of resources
|
||||
return null;
|
||||
}
|
||||
|
||||
logger.debug("Returning {} records from poll() method (took {} nanos to run dataflow)", sourceRecords.size(), nanos);
|
||||
|
|
|
@ -161,6 +161,7 @@ public interface ContentRepository {
|
|||
* @throws IllegalArgumentException if the given destination is included in
|
||||
* the given claims
|
||||
*/
|
||||
@Deprecated
|
||||
long merge(Collection<ContentClaim> claims, ContentClaim destination, byte[] header, byte[] footer, byte[] demarcator) throws IOException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -128,7 +128,6 @@ public class TestFileSystemRepository {
|
|||
+ NumberFormat.getNumberInstance(Locale.US).format(bytesToWrite) + " bytes) for a write rate of " + mbps + " MB/s");
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testContentNotFoundExceptionThrownIfResourceClaimTooShort() throws IOException {
|
||||
final File contentFile = new File("target/content_repository/0/0.bin");
|
||||
|
|
|
@ -30,6 +30,7 @@ import java.util.ArrayList;
|
|||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Properties;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -41,6 +42,7 @@ public class PropertiesFileEngineConfigurationParser {
|
|||
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";
|
||||
|
||||
private static final String TRUSTSTORE_FILE = PREFIX + "security.truststore";
|
||||
private static final String TRUSTSTORE_TYPE = PREFIX + "security.truststoreType";
|
||||
|
@ -85,6 +87,9 @@ public class PropertiesFileEngineConfigurationParser {
|
|||
throw new StatelessConfigurationException("Extensions Directory " + narDirectory.getAbsolutePath() + " specified in properties file does not exist and could not be created");
|
||||
}
|
||||
|
||||
final String contentRepoDirectoryFilename = properties.getProperty(CONTENT_REPO_DIRECTORY, "");
|
||||
final File contentRepoDirectory = contentRepoDirectoryFilename.isEmpty() ? null : new File(contentRepoDirectoryFilename);
|
||||
|
||||
final String krb5Filename = properties.getProperty(KRB5_FILE, DEFAULT_KRB5_FILENAME);
|
||||
final File krb5File = new File(krb5Filename);
|
||||
|
||||
|
@ -114,6 +119,11 @@ public class PropertiesFileEngineConfigurationParser {
|
|||
return krb5File;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<File> getContentRepositoryDirectory() {
|
||||
return Optional.ofNullable(contentRepoDirectory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SslContextDefinition getSslContext() {
|
||||
return sslContextDefinition;
|
||||
|
|
|
@ -23,4 +23,16 @@ public class DataflowAbortedException extends ProcessException {
|
|||
public DataflowAbortedException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public DataflowAbortedException(final String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public DataflowAbortedException(final String message, final Throwable cause) {
|
||||
super(message, cause);
|
||||
}
|
||||
|
||||
public DataflowAbortedException(final Throwable cause) {
|
||||
super(cause);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.nifi.stateless.config.SslContextDefinition;
|
|||
|
||||
import java.io.File;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
public interface StatelessEngineConfiguration {
|
||||
File getWorkingDirectory();
|
||||
|
@ -32,6 +33,11 @@ public interface StatelessEngineConfiguration {
|
|||
|
||||
File getKrb5File();
|
||||
|
||||
/**
|
||||
* @return the directory to use for storing FlowFile Content, or an empty optional if content is to be stored in memory
|
||||
*/
|
||||
Optional<File> getContentRepositoryDirectory();
|
||||
|
||||
SslContextDefinition getSslContext();
|
||||
|
||||
String getSensitivePropsKey();
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.nifi.stateless.flow;
|
|||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
@ -57,11 +59,27 @@ public interface TriggerResult {
|
|||
* 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);
|
||||
InputStream readContent(FlowFile flowFile) throws IOException;
|
||||
|
||||
/**
|
||||
* Provides the contents of a FlowFile that was obtained by calling {@link #getOutputFlowFiles()} as a byte array
|
||||
*
|
||||
* @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[] readContentAsByteArray(FlowFile flowFile) throws IOException;
|
||||
|
||||
/**
|
||||
* Acknowledges the output of the dataflow and allows the session to be successfully committed.
|
||||
*/
|
||||
void acknowledge();
|
||||
|
||||
/**
|
||||
* Aborts the dataflow
|
||||
* @param cause the cause for aborting the dataflow, or <code>null</code> if no cause is to be specified
|
||||
*/
|
||||
void abort(Throwable cause);
|
||||
}
|
||||
|
|
|
@ -194,6 +194,7 @@ descriptions, and example values:
|
|||
|---------------|-------------|---------------|
|
||||
| nifi.stateless.nar.directory | The location of a directory containing all NiFi Archives (NARs) that are necessary for running the dataflow | /var/lib/nifi/lib |
|
||||
| nifi.stateless.working.directory | The location of a directory where Stateless should store its expanded NAR files and use for temporary storage | /var/lib/nifi/work/stateless |
|
||||
| nifi.stateless.content.repository.directory | The location of a directory where Stateless should store the contents of FlowFiles. If not specified, Stateless will store FlowFile contents only in memory. However, specifying a directory for storing data can allow Stateless NiFi to process data that is larger than is able to be fit into memory. It is important to note that this does not result in persisting state across restarts. The data in the content repository is cleared each time that a dataflow is triggered. | /var/lib/nifi/content
|
||||
|
||||
|
||||
The following properties may be used for configuring security parameters:
|
||||
|
|
|
@ -21,18 +21,21 @@ import org.apache.nifi.components.state.StatelessStateManagerProvider;
|
|||
import org.apache.nifi.connectable.Connection;
|
||||
import org.apache.nifi.connectable.Port;
|
||||
import org.apache.nifi.controller.queue.FlowFileQueue;
|
||||
import org.apache.nifi.controller.repository.ContentRepository;
|
||||
import org.apache.nifi.controller.repository.FlowFileRecord;
|
||||
import org.apache.nifi.controller.repository.claim.ContentClaim;
|
||||
import org.apache.nifi.controller.repository.io.LimitedInputStream;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.groups.ProcessGroup;
|
||||
import org.apache.nifi.stateless.flow.FailurePortEncounteredException;
|
||||
import org.apache.nifi.stateless.flow.TriggerResult;
|
||||
import org.apache.nifi.stateless.queue.DrainableFlowFileQueue;
|
||||
import org.apache.nifi.stateless.repository.ByteArrayContentRepository;
|
||||
import org.apache.nifi.stateless.session.AsynchronousCommitTracker;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
|
@ -45,7 +48,7 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|||
public class StandardExecutionProgress implements ExecutionProgress {
|
||||
private final ProcessGroup rootGroup;
|
||||
private final List<FlowFileQueue> internalFlowFileQueues;
|
||||
private final ByteArrayContentRepository contentRepository;
|
||||
private final ContentRepository contentRepository;
|
||||
private final BlockingQueue<TriggerResult> resultQueue;
|
||||
private final Set<String> failurePortNames;
|
||||
private final AsynchronousCommitTracker commitTracker;
|
||||
|
@ -56,7 +59,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
|
|||
private volatile CompletionAction completionAction = null;
|
||||
|
||||
public StandardExecutionProgress(final ProcessGroup rootGroup, final List<FlowFileQueue> internalFlowFileQueues, final BlockingQueue<TriggerResult> resultQueue,
|
||||
final ByteArrayContentRepository contentRepository, final Set<String> failurePortNames, final AsynchronousCommitTracker commitTracker,
|
||||
final ContentRepository contentRepository, final Set<String> failurePortNames, final AsynchronousCommitTracker commitTracker,
|
||||
final StatelessStateManagerProvider stateManagerProvider) {
|
||||
this.rootGroup = rootGroup;
|
||||
this.internalFlowFileQueues = internalFlowFileQueues;
|
||||
|
@ -118,9 +121,11 @@ public class StandardExecutionProgress implements ExecutionProgress {
|
|||
final boolean canceled = isCanceled();
|
||||
|
||||
return new TriggerResult() {
|
||||
private volatile Throwable abortCause = null;
|
||||
|
||||
@Override
|
||||
public boolean isSuccessful() {
|
||||
return true;
|
||||
return abortCause == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -130,7 +135,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
|
|||
|
||||
@Override
|
||||
public Optional<Throwable> getFailureCause() {
|
||||
return Optional.empty();
|
||||
return Optional.ofNullable(abortCause);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -144,22 +149,43 @@ public class StandardExecutionProgress implements ExecutionProgress {
|
|||
}
|
||||
|
||||
@Override
|
||||
public byte[] readContent(final FlowFile flowFile) {
|
||||
public InputStream 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);
|
||||
final long offset = flowFileRecord.getContentClaimOffset();
|
||||
final long size = flowFileRecord.getSize();
|
||||
|
||||
if (offset == 0 && size == contentClaimContents.length) {
|
||||
return contentClaimContents;
|
||||
final InputStream in = contentRepository.read(contentClaim);
|
||||
final long offset = flowFileRecord.getContentClaimOffset();
|
||||
if (offset > 0) {
|
||||
StreamUtils.skip(in, offset);
|
||||
}
|
||||
|
||||
return new LimitedInputStream(in, flowFile.getSize());
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readContentAsByteArray(final FlowFile flowFile) throws IOException {
|
||||
if (!(flowFile instanceof FlowFileRecord)) {
|
||||
throw new IllegalArgumentException("FlowFile was not created by this flow");
|
||||
}
|
||||
|
||||
if (flowFile.getSize() > Integer.MAX_VALUE) {
|
||||
throw new IOException("Cannot return contents of " + flowFile + " as a byte array because the contents exceed the maximum length supported for byte arrays ("
|
||||
+ Integer.MAX_VALUE + " bytes)");
|
||||
}
|
||||
|
||||
final FlowFileRecord flowFileRecord = (FlowFileRecord) flowFile;
|
||||
|
||||
final long size = flowFileRecord.getSize();
|
||||
final byte[] flowFileContents = new byte[(int) size];
|
||||
|
||||
try (final InputStream in = readContent(flowFile)) {
|
||||
StreamUtils.fillBuffer(in, flowFileContents);
|
||||
}
|
||||
|
||||
final byte[] flowFileContents = Arrays.copyOfRange(contentClaimContents, (int) offset, (int) (size + offset));
|
||||
return flowFileContents;
|
||||
}
|
||||
|
||||
|
@ -168,6 +194,13 @@ public class StandardExecutionProgress implements ExecutionProgress {
|
|||
commitTracker.triggerCallbacks();
|
||||
stateManagerProvider.commitUpdates();
|
||||
completionActionQueue.offer(CompletionAction.COMPLETE);
|
||||
contentRepository.purge();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(final Throwable cause) {
|
||||
abortCause = new DataflowAbortedException("Dataflow was aborted", cause);
|
||||
notifyExecutionFailed(abortCause);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -178,6 +211,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
|
|||
commitTracker.triggerFailureCallbacks(new RuntimeException("Dataflow Canceled"));
|
||||
stateManagerProvider.rollbackUpdates();
|
||||
completionActionQueue.offer(CompletionAction.CANCEL);
|
||||
contentRepository.purge();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -185,6 +219,7 @@ public class StandardExecutionProgress implements ExecutionProgress {
|
|||
commitTracker.triggerFailureCallbacks(cause);
|
||||
stateManagerProvider.rollbackUpdates();
|
||||
completionActionQueue.offer(CompletionAction.CANCEL);
|
||||
contentRepository.purge();
|
||||
}
|
||||
|
||||
public Map<String, List<FlowFile>> drainOutputQueues() {
|
||||
|
|
|
@ -19,6 +19,8 @@ package org.apache.nifi.stateless.flow;
|
|||
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.InputStream;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -51,11 +53,20 @@ public class CanceledTriggerResult implements TriggerResult {
|
|||
}
|
||||
|
||||
@Override
|
||||
public byte[] readContent(final FlowFile flowFile) {
|
||||
public InputStream readContent(final FlowFile flowFile) {
|
||||
return new ByteArrayInputStream(new byte[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readContentAsByteArray(final FlowFile flowFile) {
|
||||
return new byte[0];
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledge() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(final Throwable cause) {
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.nifi.stateless.flow;
|
|||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.exception.TerminatedTaskException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -58,11 +60,23 @@ public class ExceptionalTriggerResult implements TriggerResult {
|
|||
}
|
||||
|
||||
@Override
|
||||
public byte[] readContent(final FlowFile flowFile) {
|
||||
public InputStream readContent(final FlowFile flowFile) {
|
||||
throw new IllegalArgumentException("Unknown FlowFile: " + flowFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] readContentAsByteArray(final FlowFile flowFile) throws IOException {
|
||||
throw new IllegalArgumentException("Unknown FlowFile: " + flowFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void acknowledge() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort(final Throwable cause) {
|
||||
if (cause != null) {
|
||||
failureCause.addSuppressed(cause);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,6 +69,7 @@ import org.apache.nifi.stateless.engine.StatelessProcessContextFactory;
|
|||
import org.apache.nifi.stateless.engine.StatelessProvenanceAuthorizableFactory;
|
||||
import org.apache.nifi.stateless.repository.ByteArrayContentRepository;
|
||||
import org.apache.nifi.stateless.repository.RepositoryContextFactory;
|
||||
import org.apache.nifi.stateless.repository.StatelessFileSystemContentRepository;
|
||||
import org.apache.nifi.stateless.repository.StatelessFlowFileRepository;
|
||||
import org.apache.nifi.stateless.repository.StatelessProvenanceRepository;
|
||||
import org.apache.nifi.stateless.repository.StatelessRepositoryContextFactory;
|
||||
|
@ -85,6 +86,7 @@ import java.util.ArrayList;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
public class StandardStatelessDataflowFactory implements StatelessDataflowFactory<VersionedFlowSnapshot> {
|
||||
private static final Logger logger = LoggerFactory.getLogger(StandardStatelessDataflowFactory.class);
|
||||
|
@ -197,7 +199,7 @@ public class StandardStatelessDataflowFactory implements StatelessDataflowFactor
|
|||
|
||||
final ProcessContextFactory rawProcessContextFactory = new StatelessProcessContextFactory(controllerServiceProvider, lazyInitializedEncryptor, stateManagerProvider);
|
||||
final ProcessContextFactory processContextFactory = new CachingProcessContextFactory(rawProcessContextFactory);
|
||||
contentRepo = new ByteArrayContentRepository();
|
||||
contentRepo = createContentRepository(engineConfiguration);
|
||||
flowFileRepo = new StatelessFlowFileRepository();
|
||||
|
||||
final RepositoryContextFactory repositoryContextFactory = new StatelessRepositoryContextFactory(contentRepo, flowFileRepo, flowFileEventRepo,
|
||||
|
@ -262,6 +264,15 @@ public class StandardStatelessDataflowFactory implements StatelessDataflowFactor
|
|||
}
|
||||
}
|
||||
|
||||
private ContentRepository createContentRepository(final StatelessEngineConfiguration engineConfiguration) {
|
||||
final Optional<File> contentRepoStorageDirectory = engineConfiguration.getContentRepositoryDirectory();
|
||||
if (contentRepoStorageDirectory.isPresent()) {
|
||||
return new StatelessFileSystemContentRepository(contentRepoStorageDirectory.get());
|
||||
} else {
|
||||
return new ByteArrayContentRepository();
|
||||
}
|
||||
}
|
||||
|
||||
private ExtensionClient createExtensionClient(final ExtensionClientDefinition definition, final SslContextDefinition sslContextDefinition) {
|
||||
final String type = definition.getExtensionClientType();
|
||||
if (!isValidExtensionClientType(type)) {
|
||||
|
|
|
@ -52,7 +52,6 @@ import org.apache.nifi.stateless.engine.ExecutionProgress.CompletionAction;
|
|||
import org.apache.nifi.stateless.engine.ProcessContextFactory;
|
||||
import org.apache.nifi.stateless.engine.StandardExecutionProgress;
|
||||
import org.apache.nifi.stateless.queue.DrainableFlowFileQueue;
|
||||
import org.apache.nifi.stateless.repository.ByteArrayContentRepository;
|
||||
import org.apache.nifi.stateless.repository.RepositoryContextFactory;
|
||||
import org.apache.nifi.stateless.session.AsynchronousCommitTracker;
|
||||
import org.apache.nifi.util.Connectables;
|
||||
|
@ -405,7 +404,7 @@ public class StandardStatelessFlow implements StatelessDataflow {
|
|||
final BlockingQueue<TriggerResult> resultQueue = new LinkedBlockingQueue<>();
|
||||
|
||||
final ExecutionProgress executionProgress = new StandardExecutionProgress(rootGroup, internalFlowFileQueues, resultQueue,
|
||||
(ByteArrayContentRepository) repositoryContextFactory.getContentRepository(), dataflowDefinition.getFailurePortNames(), tracker,
|
||||
repositoryContextFactory.getContentRepository(), dataflowDefinition.getFailurePortNames(), tracker,
|
||||
stateManagerProvider);
|
||||
|
||||
final AtomicReference<Future<?>> processFuture = new AtomicReference<>();
|
||||
|
|
|
@ -0,0 +1,368 @@
|
|||
/*
|
||||
* 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.Objects;
|
||||
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 StatelessFileSystemContentRepository implements ContentRepository {
|
||||
private static final Logger logger = LoggerFactory.getLogger(StatelessFileSystemContentRepository.class);
|
||||
private static final String CONTENT_FILE_REGEX = "\\d+\\.nifi\\.bin";
|
||||
|
||||
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 StatelessFileSystemContentRepository(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(CONTENT_FILE_REGEX));
|
||||
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())) {
|
||||
throwInvalidResourceClaim();
|
||||
}
|
||||
|
||||
if (!SECTION.equals(resourceClaim.getSection())) {
|
||||
throwInvalidResourceClaim();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream write(final ContentClaim claim) throws IOException {
|
||||
validateContentClaimForWriting(claim);
|
||||
|
||||
final SynchronizedByteCountingOutputStream out = writableStreamMap.get(claim.getResourceClaim());
|
||||
if (out == null) {
|
||||
throwInvalidContentClaim();
|
||||
}
|
||||
|
||||
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 {
|
||||
Objects.requireNonNull(claim, "ContentClaim cannot be null");
|
||||
|
||||
if (!(claim instanceof StandardContentClaim)) {
|
||||
throwInvalidContentClaim();
|
||||
}
|
||||
|
||||
validateResourceClaim(claim.getResourceClaim());
|
||||
|
||||
if (claim.getLength() >= 0) {
|
||||
throw new IOException("Cannot write to " + claim + " because it has already been written to.");
|
||||
}
|
||||
}
|
||||
|
||||
private void throwInvalidContentClaim() {
|
||||
throw new IllegalArgumentException("The given ContentClaim does not belong to this Content Repository");
|
||||
}
|
||||
|
||||
private void throwInvalidResourceClaim() {
|
||||
throw new IllegalArgumentException("The given ResourceClaim does not belong to this Content Repository");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void purge() {
|
||||
writableClaimQueue.clear();
|
||||
for (final OutputStream out : writableStreamMap.values()) {
|
||||
try {
|
||||
out.close();
|
||||
} catch (final IOException ioe) {
|
||||
logger.warn("Failed to close Content Repository Output Stream", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
for (final ResourceClaim resourceClaim : writableStreamMap.keySet()) {
|
||||
final File file = getFile(resourceClaim);
|
||||
if (!file.delete() && file.exists()) {
|
||||
logger.warn("Failed to remove file from Content Repository: " + file.getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
writableStreamMap.clear();
|
||||
resourceClaimManager.purge();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void cleanup() {
|
||||
purge();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isAccessible(final ContentClaim contentClaim) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
private class ContentOutputStream extends FilterOutputStream {
|
||||
private final StandardContentClaim scc;
|
||||
private final SynchronizedByteCountingOutputStream out;
|
||||
private final long initialOffset;
|
||||
|
||||
public ContentOutputStream(final SynchronizedByteCountingOutputStream out, final StandardContentClaim scc) {
|
||||
super(out);
|
||||
this.scc = scc;
|
||||
this.out = out;
|
||||
this.initialOffset = out.getBytesWritten();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
super.flush();
|
||||
scc.setLength(out.getBytesWritten() - initialOffset);
|
||||
writableClaimQueue.offer(scc.getResourceClaim());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -30,6 +30,7 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
@ -97,6 +98,11 @@ public class TestPropertiesFileFlowDefinitionParser {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<File> getContentRepositoryDirectory() {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SslContextDefinition getSslContext() {
|
||||
return null;
|
||||
|
|
|
@ -0,0 +1,108 @@
|
|||
/*
|
||||
* 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.claim.ContentClaim;
|
||||
import org.apache.nifi.controller.repository.claim.StandardResourceClaimManager;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertThrows;
|
||||
|
||||
public class TestStatelessFileSystemContentRepository {
|
||||
private final File repoDirectory = new File("target/test-stateless-file-system-repository");
|
||||
private StatelessFileSystemContentRepository repository;
|
||||
|
||||
@Before
|
||||
public void setup() throws IOException {
|
||||
repository = new StatelessFileSystemContentRepository(repoDirectory);
|
||||
repository.initialize(new StandardResourceClaimManager());
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup() {
|
||||
repository.cleanup();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteThenRead() throws IOException {
|
||||
final byte[] contents = "Hello, World!".getBytes();
|
||||
final ContentClaim claim = repository.create(true);
|
||||
try (final OutputStream out = repository.write(claim)) {
|
||||
out.write(contents);
|
||||
}
|
||||
|
||||
assertEquals(contents.length, claim.getLength());
|
||||
|
||||
// Ensure we can read multiple times.
|
||||
for (int i=0; i < 5; i++) {
|
||||
final byte[] bytesRead;
|
||||
try (final InputStream in = repository.read(claim);
|
||||
final ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
|
||||
StreamUtils.copy(in, baos);
|
||||
bytesRead = baos.toByteArray();
|
||||
}
|
||||
|
||||
assertEquals(new String(contents), new String(bytesRead));
|
||||
}
|
||||
|
||||
File[] files = repoDirectory.listFiles();
|
||||
assertNotNull(files);
|
||||
assertEquals(1, files.length);
|
||||
|
||||
repository.purge();
|
||||
files = repoDirectory.listFiles();
|
||||
assertNotNull(files);
|
||||
assertEquals(0, files.length);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverwriteFails() throws IOException {
|
||||
final byte[] contents = "Hello, World!".getBytes();
|
||||
final ContentClaim claim = repository.create(true);
|
||||
try (final OutputStream out = repository.write(claim)) {
|
||||
out.write(contents);
|
||||
}
|
||||
|
||||
// An attempt to write to a content claim multiple times should fail
|
||||
assertThrows(IOException.class, () -> repository.write(claim));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOverwriteFailsBeforeClosingOutputStream() throws IOException {
|
||||
final byte[] contents = "Hello, World!".getBytes();
|
||||
final ContentClaim claim = repository.create(true);
|
||||
try (final OutputStream out = repository.write(claim)) {
|
||||
out.write(contents);
|
||||
|
||||
// An attempt to write to a content claim multiple times should fail
|
||||
assertThrows(IOException.class, () -> repository.write(claim));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -17,6 +17,10 @@
|
|||
nifi.stateless.nar.directory=./lib
|
||||
nifi.stateless.working.directory=./work/stateless
|
||||
|
||||
# Content Repository #
|
||||
# If not specified, content is stored in Java's heap. Content is lost upon restart, regardless.
|
||||
nifi.stateless.content.repository.directory=
|
||||
|
||||
# Security Properties #
|
||||
nifi.stateless.security.keystore=
|
||||
nifi.stateless.security.keystoreType=
|
||||
|
|
|
@ -44,6 +44,7 @@ import java.io.InputStream;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -93,6 +94,14 @@ public class StatelessSystemIT {
|
|||
return new File("/etc/krb5.conf");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<File> getContentRepositoryDirectory() {
|
||||
return Optional.empty();
|
||||
|
||||
// Can be used to enable file-based content repository.
|
||||
// return Optional.of(new File("target/nifi-stateless-content-repo"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public SslContextDefinition getSslContext() {
|
||||
return null;
|
||||
|
|
|
@ -38,6 +38,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
|
@ -123,15 +124,20 @@ public class AsyncCommitCallbackIT extends StatelessSystemIT {
|
|||
builder.createConnection(ingestFile, replace, "success");
|
||||
builder.createConnection(replace, outputPort, "success");
|
||||
|
||||
final TriggerResult result = testAsyncCallbackCalledAtFinish(builder.getFlowSnapshot(), inputFile, replacementFile);
|
||||
testAsyncCallbackCalledAtFinish(builder.getFlowSnapshot(), inputFile, replacementFile, result -> {
|
||||
final List<FlowFile> flowFilesOut = result.getOutputFlowFiles("Out");
|
||||
assertEquals(1, flowFilesOut.size());
|
||||
final FlowFile out = flowFilesOut.get(0);
|
||||
assertEquals(replacementFile.getName(), out.getAttribute("filename"));
|
||||
|
||||
final List<FlowFile> flowFilesOut = result.getOutputFlowFiles("Out");
|
||||
assertEquals(1, flowFilesOut.size());
|
||||
final FlowFile out = flowFilesOut.get(0);
|
||||
assertEquals(replacementFile.getName(), out.getAttribute("filename"));
|
||||
|
||||
final byte[] outputContents = result.readContent(out);
|
||||
assertEquals("Good-bye World", new String(outputContents));
|
||||
final byte[] outputContents;
|
||||
try {
|
||||
outputContents = result.readContentAsByteArray(out);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
assertEquals("Good-bye World", new String(outputContents));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -160,10 +166,10 @@ public class AsyncCommitCallbackIT extends StatelessSystemIT {
|
|||
builder.createConnection(ingestFile, replace, "success");
|
||||
builder.createConnection(replace, terminate, "success");
|
||||
|
||||
testAsyncCallbackCalledAtFinish(builder.getFlowSnapshot(), inputFile, replacementFile);
|
||||
testAsyncCallbackCalledAtFinish(builder.getFlowSnapshot(), inputFile, replacementFile, (result) -> {});
|
||||
}
|
||||
|
||||
private TriggerResult testAsyncCallbackCalledAtFinish(final VersionedFlowSnapshot flowSnapshot, final File inputFile, final File replacementFile)
|
||||
private TriggerResult testAsyncCallbackCalledAtFinish(final VersionedFlowSnapshot flowSnapshot, final File inputFile, final File replacementFile, final Consumer<TriggerResult> resultConsumer)
|
||||
throws IOException, InterruptedException, StatelessConfigurationException {
|
||||
final StatelessDataflow dataflow = loadDataflow(flowSnapshot);
|
||||
final DataflowTrigger trigger = dataflow.trigger();
|
||||
|
@ -179,6 +185,9 @@ public class AsyncCommitCallbackIT extends StatelessSystemIT {
|
|||
assertTrue(inputFile.exists());
|
||||
|
||||
assertTrue(result.isSuccessful());
|
||||
|
||||
resultConsumer.accept(result);
|
||||
|
||||
result.acknowledge();
|
||||
|
||||
// When acknowledge() is called, we do not block until the synchronous commits have been "unwound".
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.nifi.stateless.config.StatelessConfigurationException;
|
|||
import org.apache.nifi.stateless.flow.DataflowTrigger;
|
||||
import org.apache.nifi.stateless.flow.StatelessDataflow;
|
||||
import org.apache.nifi.stateless.flow.TriggerResult;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -65,7 +66,6 @@ public class CloneFlowFileIT extends StatelessSystemIT {
|
|||
final DataflowTrigger trigger = dataflow.trigger();
|
||||
final TriggerResult result = trigger.getResult();
|
||||
assertTrue(result.isSuccessful());
|
||||
result.acknowledge();
|
||||
|
||||
final List<FlowFile> flowFiles = result.getOutputFlowFiles("Out");
|
||||
assertEquals(2, flowFiles.size());
|
||||
|
@ -77,15 +77,26 @@ public class CloneFlowFileIT extends StatelessSystemIT {
|
|||
assertEquals("123", second.getAttribute("abc"));
|
||||
|
||||
final long countNormal = flowFiles.stream()
|
||||
.filter(flowFile -> new String(result.readContent(flowFile), StandardCharsets.UTF_8).equals("Hello"))
|
||||
.filter(flowFile -> readContentAsString(result, flowFile).equals("Hello"))
|
||||
.count();
|
||||
|
||||
final long countReversed = flowFiles.stream()
|
||||
.filter(flowFile -> new String(result.readContent(flowFile), StandardCharsets.UTF_8).equals("olleH"))
|
||||
.filter(flowFile -> readContentAsString(result, flowFile).equals("olleH"))
|
||||
.count();
|
||||
|
||||
assertEquals(1L, countNormal);
|
||||
assertEquals(1L, countReversed);
|
||||
|
||||
result.acknowledge();
|
||||
}
|
||||
|
||||
private String readContentAsString(final TriggerResult result, final FlowFile flowFile) {
|
||||
try {
|
||||
return new String(result.readContentAsByteArray(flowFile), StandardCharsets.UTF_8);
|
||||
} catch (final IOException e) {
|
||||
e.printStackTrace();
|
||||
Assert.fail("Could not read content");
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,7 +54,7 @@ public class CreatesFlowFileIT extends StatelessSystemIT {
|
|||
final FlowFile flowFile = flowFiles.get(0);
|
||||
assertEquals("hello", flowFile.getAttribute("greeting"));
|
||||
|
||||
final byte[] bytes = result.readContent(flowFile);
|
||||
final byte[] bytes = result.readContentAsByteArray(flowFile);
|
||||
assertEquals("Hello", new String(bytes, StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
|
|
|
@ -67,13 +67,14 @@ public class MergingIT extends StatelessSystemIT {
|
|||
final DataflowTrigger trigger = dataflow.trigger();
|
||||
final TriggerResult result = trigger.getResult();
|
||||
assertTrue(result.isSuccessful());
|
||||
result.acknowledge();
|
||||
|
||||
final List<FlowFile> flowFiles = result.getOutputFlowFiles("Out");
|
||||
assertEquals(1, flowFiles.size());
|
||||
|
||||
final FlowFile first = flowFiles.get(0);
|
||||
final String outputContent = new String(result.readContent(first));
|
||||
final String outputContent = new String(result.readContentAsByteArray(first));
|
||||
assertEquals("abc123xyz321abc123xyz321abc123xyz321", outputContent);
|
||||
|
||||
result.acknowledge();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -80,7 +80,6 @@ public class RequiresAdditionalInputIT extends StatelessSystemIT {
|
|||
final DataflowTrigger trigger = dataflow.trigger();
|
||||
final TriggerResult result = trigger.getResult();
|
||||
assertTrue(result.isSuccessful());
|
||||
result.acknowledge();
|
||||
|
||||
final List<FlowFile> mergedFlowFiles = result.getOutputFlowFiles().get("merged");
|
||||
assertEquals(1, mergedFlowFiles.size());
|
||||
|
@ -88,7 +87,7 @@ public class RequiresAdditionalInputIT extends StatelessSystemIT {
|
|||
final List<FlowFile> originalFlowFiles = result.getOutputFlowFiles().get("original");
|
||||
assertEquals(flowFileCount, originalFlowFiles.size());
|
||||
|
||||
final String outputText = new String(result.readContent(mergedFlowFiles.get(0)));
|
||||
final String outputText = new String(result.readContentAsByteArray(mergedFlowFiles.get(0)));
|
||||
final StringBuilder expectedTextBuilder = new StringBuilder();
|
||||
for (int i=0; i < flowFileCount; i++) {
|
||||
expectedTextBuilder.append("abc");
|
||||
|
@ -96,6 +95,8 @@ public class RequiresAdditionalInputIT extends StatelessSystemIT {
|
|||
|
||||
final String expectedText = expectedTextBuilder.toString();
|
||||
assertEquals(expectedText, outputText);
|
||||
|
||||
result.acknowledge();
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -144,12 +145,11 @@ public class RequiresAdditionalInputIT extends StatelessSystemIT {
|
|||
final DataflowTrigger trigger = dataflow.trigger();
|
||||
final TriggerResult result = trigger.getResult();
|
||||
assertTrue(result.isSuccessful());
|
||||
result.acknowledge();
|
||||
|
||||
final List<FlowFile> mergedFlowFiles = result.getOutputFlowFiles().get("out");
|
||||
assertEquals(1, mergedFlowFiles.size());
|
||||
|
||||
final String outputText = new String(result.readContent(mergedFlowFiles.get(0)));
|
||||
final String outputText = new String(result.readContentAsByteArray(mergedFlowFiles.get(0)));
|
||||
final StringBuilder expectedTextBuilder = new StringBuilder();
|
||||
for (int i=0; i < 36; i++) {
|
||||
expectedTextBuilder.append("abc");
|
||||
|
@ -157,6 +157,8 @@ public class RequiresAdditionalInputIT extends StatelessSystemIT {
|
|||
|
||||
final String expectedText = expectedTextBuilder.toString();
|
||||
assertEquals(expectedText, outputText);
|
||||
|
||||
result.acknowledge();
|
||||
}
|
||||
|
||||
private TransactionThresholds createTransactionThresholds(final int maxFlowFiles) {
|
||||
|
|
|
@ -73,7 +73,6 @@ public class SplittingIT extends StatelessSystemIT {
|
|||
final DataflowTrigger trigger = dataflow.trigger();
|
||||
final TriggerResult result = trigger.getResult();
|
||||
assertTrue(result.isSuccessful());
|
||||
result.acknowledge();
|
||||
|
||||
final List<FlowFile> flowFiles = result.getOutputFlowFiles("Out");
|
||||
assertEquals(4, flowFiles.size());
|
||||
|
@ -83,8 +82,10 @@ public class SplittingIT extends StatelessSystemIT {
|
|||
final String expected = expectedContent[i];
|
||||
|
||||
final FlowFile flowFile = flowFiles.get(i);
|
||||
final String outputContent = new String(result.readContent(flowFile));
|
||||
final String outputContent = new String(result.readContentAsByteArray(flowFile));
|
||||
assertEquals(expected, outputContent);
|
||||
}
|
||||
|
||||
result.acknowledge();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue