mirror of https://github.com/apache/nifi.git
Merge branch 'develop' of http://git-wip-us.apache.org/repos/asf/incubator-nifi into develop
This commit is contained in:
commit
384b2ac253
|
@ -253,7 +253,7 @@
|
||||||
<module name="Checker">
|
<module name="Checker">
|
||||||
<property name="charset" value="UTF-8" />
|
<property name="charset" value="UTF-8" />
|
||||||
<property name="severity" value="warning" />
|
<property name="severity" value="warning" />
|
||||||
<!-- Checks for whitespace -->
|
<!-- Checks for whitespace -->
|
||||||
<!-- See http://checkstyle.sf.net/config_whitespace.html -->
|
<!-- See http://checkstyle.sf.net/config_whitespace.html -->
|
||||||
<module name="FileTabCharacter">
|
<module name="FileTabCharacter">
|
||||||
<property name="eachLine" value="true" />
|
<property name="eachLine" value="true" />
|
||||||
|
@ -267,10 +267,6 @@
|
||||||
<property name="format" value="[@]see\s+[{][@]link" />
|
<property name="format" value="[@]see\s+[{][@]link" />
|
||||||
<property name="message" value="Javadoc @see does not need @link: pick one or the other." />
|
<property name="message" value="Javadoc @see does not need @link: pick one or the other." />
|
||||||
</module>
|
</module>
|
||||||
<module name="RegexpSinglelineJava">
|
|
||||||
<property name="format" value="jline[.]internal[.]Preconditions" />
|
|
||||||
<property name="message" value="Please use Guava Preconditions not JLine" />
|
|
||||||
</module>
|
|
||||||
<module name="OuterTypeFilename" />
|
<module name="OuterTypeFilename" />
|
||||||
<module name="LineLength">
|
<module name="LineLength">
|
||||||
<!-- needs extra, because Eclipse formatter ignores the ending left brace -->
|
<!-- needs extra, because Eclipse formatter ignores the ending left brace -->
|
||||||
|
|
|
@ -1,18 +1,18 @@
|
||||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
<!--
|
<!--
|
||||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
contributor license agreements. See the NOTICE file distributed with
|
contributor license agreements. See the NOTICE file distributed with
|
||||||
this work for additional information regarding copyright ownership.
|
this work for additional information regarding copyright ownership.
|
||||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
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 not use this file except in compliance with
|
||||||
the License. You may obtain a copy of the License at
|
the License. You may obtain a copy of the License at
|
||||||
http://www.apache.org/licenses/LICENSE-2.0
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
Unless required by applicable law or agreed to in writing, software
|
Unless required by applicable law or agreed to in writing, software
|
||||||
distributed under the License is distributed on an "AS IS" BASIS,
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
See the License for the specific language governing permissions and
|
See the License for the specific language governing permissions and
|
||||||
limitations under the License.
|
limitations under the License.
|
||||||
-->
|
-->
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<modelVersion>4.0.0</modelVersion>
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
|
|
|
@ -45,16 +45,15 @@ import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is a base class that is helpful when building processors interacting
|
* This is a base class that is helpful when building processors interacting with HDFS.
|
||||||
* with HDFS.
|
|
||||||
*/
|
*/
|
||||||
public abstract class AbstractHadoopProcessor extends AbstractProcessor {
|
public abstract class AbstractHadoopProcessor extends AbstractProcessor {
|
||||||
|
|
||||||
// properties
|
// properties
|
||||||
public static final PropertyDescriptor HADOOP_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor HADOOP_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder()
|
||||||
.name("Hadoop Configuration Resources")
|
.name("Hadoop Configuration Resources")
|
||||||
.description(
|
.description("A file or comma separated list of files which contains the Hadoop file system configuration. Without this, Hadoop "
|
||||||
"A file or comma separated list of files which contains the Hadoop file system configuration. Without this, Hadoop will search the classpath for a 'core-site.xml' and 'hdfs-site.xml' file or will revert to a default configuration.")
|
+ "will search the classpath for a 'core-site.xml' and 'hdfs-site.xml' file or will revert to a default configuration.")
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(createMultipleFilesExistValidator())
|
.addValidator(createMultipleFilesExistValidator())
|
||||||
.build();
|
.build();
|
||||||
|
|
|
@ -38,31 +38,21 @@ import org.apache.nifi.processors.hadoop.util.SequenceFileWriter;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* This processor is used to create a Hadoop Sequence File, which essentially is
|
* This processor is used to create a Hadoop Sequence File, which essentially is a file of key/value pairs. The key will be a file name and the value will be the flow file content. The processor will
|
||||||
* a file of key/value pairs. The key will be a file name and the value will be
|
* take either a merged (a.k.a. packaged) flow file or a singular flow file. Historically, this processor handled the merging by type and size or time prior to creating a SequenceFile output; it no
|
||||||
* the flow file content. The processor will take either a merged (a.k.a.
|
* longer does this. If creating a SequenceFile that contains multiple files of the same type is desired, precede this processor with a <code>RouteOnAttribute</code> processor to segregate files of
|
||||||
* packaged) flow file or a singular flow file. Historically, this processor
|
* the same type and follow that with a <code>MergeContent</code> processor to bundle up files. If the type of files is not important, just use the <code>MergeContent</code> processor. When using the
|
||||||
* handled the merging by type and size or time prior to creating a SequenceFile
|
* <code>MergeContent</code> processor, the following Merge Formats are supported by this processor:
|
||||||
* output; it no longer does this. If creating a SequenceFile that contains
|
|
||||||
* multiple files of the same type is desired, precede this processor with a
|
|
||||||
* <code>RouteOnAttribute</code> processor to segregate files of the same type
|
|
||||||
* and follow that with a <code>MergeContent</code> processor to bundle up
|
|
||||||
* files. If the type of files is not important, just use the
|
|
||||||
* <code>MergeContent</code> processor. When using the <code>MergeContent</code>
|
|
||||||
* processor, the following Merge Formats are supported by this processor:
|
|
||||||
* <ul>
|
* <ul>
|
||||||
* <li>TAR</li>
|
* <li>TAR</li>
|
||||||
* <li>ZIP</li>
|
* <li>ZIP</li>
|
||||||
* <li>FlowFileStream v3</li>
|
* <li>FlowFileStream v3</li>
|
||||||
* </ul>
|
* </ul>
|
||||||
* The created SequenceFile is named the same as the incoming FlowFile with the
|
* The created SequenceFile is named the same as the incoming FlowFile with the suffix '.sf'. For incoming FlowFiles that are bundled, the keys in the SequenceFile are the individual file names, the
|
||||||
* suffix '.sf'. For incoming FlowFiles that are bundled, the keys in the
|
* values are the contents of each file.
|
||||||
* SequenceFile are the individual file names, the values are the contents of
|
|
||||||
* each file.
|
|
||||||
* </p>
|
* </p>
|
||||||
* NOTE: The value portion of a key/value pair is loaded into memory. While
|
* NOTE: The value portion of a key/value pair is loaded into memory. While there is a max size limit of 2GB, this could cause memory issues if there are too many concurrent tasks and the flow file
|
||||||
* there is a max size limit of 2GB, this could cause memory issues if there are
|
* sizes are large.
|
||||||
* too many concurrent tasks and the flow file sizes are large.
|
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
@SideEffectFree
|
@SideEffectFree
|
||||||
|
|
|
@ -65,8 +65,10 @@ import org.apache.nifi.util.StopWatch;
|
||||||
@Tags({"hadoop", "HDFS", "get", "fetch", "ingest", "source", "filesystem"})
|
@Tags({"hadoop", "HDFS", "get", "fetch", "ingest", "source", "filesystem"})
|
||||||
@CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles")
|
@CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles")
|
||||||
@WritesAttributes({
|
@WritesAttributes({
|
||||||
@WritesAttribute(attribute = "filename", description = "The name of the file that was read from HDFS."),
|
@WritesAttribute(attribute = "filename", description = "The name of the file that was read from HDFS."),
|
||||||
@WritesAttribute(attribute = "path", description = "The path is set to the relative path of the file's directory on HDFS. For example, if the Directory property is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"abc/1/2/3\".") })
|
@WritesAttribute(attribute = "path", description = "The path is set to the relative path of the file's directory on HDFS. For example, if "
|
||||||
|
+ "the Directory property is set to /tmp, then files picked up from /tmp will have the path attribute set to \"./\". If the Recurse "
|
||||||
|
+ "Subdirectories property is set to true and a file is picked up from /tmp/abc/1/2/3, then the path attribute will be set to \"abc/1/2/3\".")})
|
||||||
@SeeAlso(PutHDFS.class)
|
@SeeAlso(PutHDFS.class)
|
||||||
public class GetHDFS extends AbstractHadoopProcessor {
|
public class GetHDFS extends AbstractHadoopProcessor {
|
||||||
|
|
||||||
|
@ -112,16 +114,16 @@ public class GetHDFS extends AbstractHadoopProcessor {
|
||||||
|
|
||||||
public static final PropertyDescriptor FILE_FILTER_REGEX = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor FILE_FILTER_REGEX = new PropertyDescriptor.Builder()
|
||||||
.name("File Filter Regex")
|
.name("File Filter Regex")
|
||||||
.description(
|
.description("A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose names match that Regular "
|
||||||
"A Java Regular Expression for filtering Filenames; if a filter is supplied then only files whose names match that Regular Expression will be fetched, otherwise all files will be fetched")
|
+ "Expression will be fetched, otherwise all files will be fetched")
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
|
.addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
public static final PropertyDescriptor FILTER_MATCH_NAME_ONLY = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor FILTER_MATCH_NAME_ONLY = new PropertyDescriptor.Builder()
|
||||||
.name("Filter Match Name Only")
|
.name("Filter Match Name Only")
|
||||||
.description(
|
.description("If true then File Filter Regex will match on just the filename, otherwise subdirectory names will be included with filename "
|
||||||
"If true then File Filter Regex will match on just the filename, otherwise subdirectory names will be included with filename in the regex comparison")
|
+ "in the regex comparison")
|
||||||
.required(true)
|
.required(true)
|
||||||
.allowableValues("true", "false")
|
.allowableValues("true", "false")
|
||||||
.defaultValue("true")
|
.defaultValue("true")
|
||||||
|
@ -137,21 +139,17 @@ public class GetHDFS extends AbstractHadoopProcessor {
|
||||||
|
|
||||||
public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
|
||||||
.name("Minimum File Age")
|
.name("Minimum File Age")
|
||||||
.description(
|
.description("The minimum age that a file must be in order to be pulled; any file younger than this amount of time (based on last modification date) will be ignored")
|
||||||
"The minimum age that a file must be in order to be pulled; any file younger than this amount of time (based on last modification date) will be ignored")
|
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(
|
.addValidator(StandardValidators.createTimePeriodValidator(0, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
|
||||||
StandardValidators.createTimePeriodValidator(0, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
|
|
||||||
.defaultValue("0 sec")
|
.defaultValue("0 sec")
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
public static final PropertyDescriptor MAX_AGE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor MAX_AGE = new PropertyDescriptor.Builder()
|
||||||
.name("Maximum File Age")
|
.name("Maximum File Age")
|
||||||
.description(
|
.description("The maximum age that a file must be in order to be pulled; any file older than this amount of time (based on last modification date) will be ignored")
|
||||||
"The maximum age that a file must be in order to be pulled; any file older than this amount of time (based on last modification date) will be ignored")
|
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(
|
.addValidator(StandardValidators.createTimePeriodValidator(100, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
|
||||||
StandardValidators.createTimePeriodValidator(100, TimeUnit.MILLISECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||||
|
@ -389,11 +387,11 @@ public class GetHDFS extends AbstractHadoopProcessor {
|
||||||
/**
|
/**
|
||||||
* Do a listing of HDFS if the POLLING_INTERVAL has lapsed.
|
* Do a listing of HDFS if the POLLING_INTERVAL has lapsed.
|
||||||
*
|
*
|
||||||
* Will return null if POLLING_INTERVAL has not lapsed. Will return an empty
|
* Will return null if POLLING_INTERVAL has not lapsed. Will return an empty set if no files were found on HDFS that matched the configured filters.
|
||||||
* set if no files were found on HDFS that matched the configured filters.
|
*
|
||||||
* @param context
|
* @param context context
|
||||||
* @return
|
* @return null if POLLING_INTERVAL has not lapsed. Will return an empty set if no files were found on HDFS that matched the configured filters
|
||||||
* @throws java.io.IOException
|
* @throws java.io.IOException ex
|
||||||
*/
|
*/
|
||||||
protected Set<Path> performListing(final ProcessContext context) throws IOException {
|
protected Set<Path> performListing(final ProcessContext context) throws IOException {
|
||||||
|
|
||||||
|
@ -417,11 +415,12 @@ public class GetHDFS extends AbstractHadoopProcessor {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Poll HDFS for files to process that match the configured file filters.
|
* Poll HDFS for files to process that match the configured file filters.
|
||||||
* @param hdfs
|
*
|
||||||
* @param dir
|
* @param hdfs hdfs
|
||||||
* @param filesVisited
|
* @param dir dir
|
||||||
* @return
|
* @param filesVisited filesVisited
|
||||||
* @throws java.io.IOException
|
* @return files to process
|
||||||
|
* @throws java.io.IOException ex
|
||||||
*/
|
*/
|
||||||
protected Set<Path> selectFiles(final FileSystem hdfs, final Path dir, Set<Path> filesVisited) throws IOException {
|
protected Set<Path> selectFiles(final FileSystem hdfs, final Path dir, Set<Path> filesVisited) throws IOException {
|
||||||
if (null == filesVisited) {
|
if (null == filesVisited) {
|
||||||
|
@ -465,11 +464,11 @@ public class GetHDFS extends AbstractHadoopProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the relative path of the child that does not include the filename
|
* Returns the relative path of the child that does not include the filename or the root path.
|
||||||
* or the root path.
|
*
|
||||||
* @param root
|
* @param root root
|
||||||
* @param child
|
* @param child child
|
||||||
* @return
|
* @return the relative path of the child that does not include the filename or the root path
|
||||||
*/
|
*/
|
||||||
public static String getPathDifference(final Path root, final Path child) {
|
public static String getPathDifference(final Path root, final Path child) {
|
||||||
final int depthDiff = child.depth() - root.depth();
|
final int depthDiff = child.depth() - root.depth();
|
||||||
|
@ -492,8 +491,7 @@ public class GetHDFS extends AbstractHadoopProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Holder for a snapshot in time of some processor properties that are
|
* Holder for a snapshot in time of some processor properties that are passed around.
|
||||||
* passed around.
|
|
||||||
*/
|
*/
|
||||||
protected static class ProcessorConfiguration {
|
protected static class ProcessorConfiguration {
|
||||||
|
|
||||||
|
|
|
@ -40,19 +40,13 @@ import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This processor is used to pull files from HDFS. The files being pulled in
|
* This processor is used to pull files from HDFS. The files being pulled in MUST be SequenceFile formatted files. The processor creates a flow file for each key/value entry in the ingested
|
||||||
* MUST be SequenceFile formatted files. The processor creates a flow file for
|
* SequenceFile. The created flow file's content depends on the value of the optional configuration property FlowFile Content. Currently, there are two choices: VALUE ONLY and KEY VALUE PAIR. With the
|
||||||
* each key/value entry in the ingested SequenceFile. The created flow file's
|
* prior, only the SequenceFile value element is written to the flow file contents. With the latter, the SequenceFile key and value are written to the flow file contents as serialized objects; the
|
||||||
* content depends on the value of the optional configuration property FlowFile
|
* format is key length (int), key(String), value length(int), value(bytes). The default is VALUE ONLY.
|
||||||
* Content. Currently, there are two choices: VALUE ONLY and KEY VALUE PAIR.
|
|
||||||
* With the prior, only the SequenceFile value element is written to the flow
|
|
||||||
* file contents. With the latter, the SequenceFile key and value are written to
|
|
||||||
* the flow file contents as serialized objects; the format is key length (int),
|
|
||||||
* key(String), value length(int), value(bytes). The default is VALUE ONLY.
|
|
||||||
* <p>
|
* <p>
|
||||||
* NOTE: This processor loads the entire value entry into memory. While the size
|
* NOTE: This processor loads the entire value entry into memory. While the size limit for a value entry is 2GB, this will cause memory problems if there are too many concurrent tasks and the data
|
||||||
* limit for a value entry is 2GB, this will cause memory problems if there are
|
* being ingested is large.
|
||||||
* too many concurrent tasks and the data being ingested is large.
|
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
@TriggerWhenEmpty
|
@TriggerWhenEmpty
|
||||||
|
|
|
@ -43,10 +43,8 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class reads a SequenceFile and generates FlowFiles, one per KeyValue
|
* This class reads a SequenceFile and generates FlowFiles, one per KeyValue pair in the SequenceFile. The FlowFile name is based on the the incoming file name with System nanotime appended; the
|
||||||
* pair in the SequenceFile. The FlowFile name is based on the the incoming file
|
* FlowFile content is the key/value pair serialized via Text.
|
||||||
* name with System nanotime appended; the FlowFile content is the key/value
|
|
||||||
* pair serialized via Text.
|
|
||||||
*/
|
*/
|
||||||
public class KeyValueReader implements SequenceFileReader<Set<FlowFile>> {
|
public class KeyValueReader implements SequenceFileReader<Set<FlowFile>> {
|
||||||
|
|
||||||
|
|
|
@ -42,9 +42,8 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class reads a SequenceFile and generates FlowFiles, one per each
|
* This class reads a SequenceFile and generates FlowFiles, one per each KeyValue Pair in the SequenceFile. The FlowFile name is the key, which is typically a file name but may not be; the FlowFile
|
||||||
* KeyValue Pair in the SequenceFile. The FlowFile name is the key, which is
|
* content is the value.
|
||||||
* typically a file name but may not be; the FlowFile content is the value.
|
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
public class ValueReader implements SequenceFileReader<Set<FlowFile>> {
|
public class ValueReader implements SequenceFileReader<Set<FlowFile>> {
|
||||||
|
|
|
@ -24,10 +24,8 @@ import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class allows the user to define byte-array filters or single-byte
|
* This class allows the user to define byte-array filters or single-byte filters that will modify the content that is written to the underlying stream. Each filter can be given a maximum number of
|
||||||
* filters that will modify the content that is written to the underlying
|
* replacements that it should perform.
|
||||||
* stream. Each filter can be given a maximum number of replacements that it
|
|
||||||
* should perform.
|
|
||||||
*/
|
*/
|
||||||
public class ByteFilteringOutputStream extends FilterOutputStream {
|
public class ByteFilteringOutputStream extends FilterOutputStream {
|
||||||
|
|
||||||
|
@ -66,8 +64,7 @@ public class ByteFilteringOutputStream extends FilterOutputStream {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Causes this stream to write <tt>replaceWith</tt> in place of
|
* Causes this stream to write <tt>replaceWith</tt> in place of
|
||||||
* <tt>toReplace</tt> if {@link #write(byte[], int, int)} is called where
|
* <tt>toReplace</tt> if {@link #write(byte[], int, int)} is called where the value to write is equal to
|
||||||
* the value to write is equal to
|
|
||||||
* <tt>toReplace</tt>.
|
* <tt>toReplace</tt>.
|
||||||
* <p/>
|
* <p/>
|
||||||
* @param toReplace the byte array to replace
|
* @param toReplace the byte array to replace
|
||||||
|
@ -79,14 +76,12 @@ public class ByteFilteringOutputStream extends FilterOutputStream {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Causes this stream to write <tt>replaceWith</tt> in place of
|
* Causes this stream to write <tt>replaceWith</tt> in place of
|
||||||
* <tt>toReplace</tt> if {@link #write(byte[], int, int)} is called where
|
* <tt>toReplace</tt> if {@link #write(byte[], int, int)} is called where the value to write is equal to
|
||||||
* the value to write is equal to
|
|
||||||
* <tt>toReplace</tt>.
|
* <tt>toReplace</tt>.
|
||||||
* <p/>
|
* <p/>
|
||||||
* @param toReplace the byte array to replace
|
* @param toReplace the byte array to replace
|
||||||
* @param replaceWith the byte array to be substituted
|
* @param replaceWith the byte array to be substituted
|
||||||
* @param maxReplacements the maximum number of replacements that should be
|
* @param maxReplacements the maximum number of replacements that should be made
|
||||||
* made
|
|
||||||
*/
|
*/
|
||||||
public void addFilter(final byte[] toReplace, final byte[] replaceWith, final int maxReplacements) {
|
public void addFilter(final byte[] toReplace, final byte[] replaceWith, final int maxReplacements) {
|
||||||
multiByteFilters.add(new Filter(toReplace, replaceWith, maxReplacements));
|
multiByteFilters.add(new Filter(toReplace, replaceWith, maxReplacements));
|
||||||
|
@ -94,8 +89,7 @@ public class ByteFilteringOutputStream extends FilterOutputStream {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Causes this stream to write <tt>replaceWith</tt> in place of
|
* Causes this stream to write <tt>replaceWith</tt> in place of
|
||||||
* <tt>toReplace</tt> if {@link #write(int)} is called where the value to
|
* <tt>toReplace</tt> if {@link #write(int)} is called where the value to write is equal to
|
||||||
* write is equal to
|
|
||||||
* <tt>toReplace</tt>.
|
* <tt>toReplace</tt>.
|
||||||
* <p/>
|
* <p/>
|
||||||
* @param toReplace the byte to replace
|
* @param toReplace the byte to replace
|
||||||
|
@ -107,14 +101,12 @@ public class ByteFilteringOutputStream extends FilterOutputStream {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Causes this stream to write <tt>replaceWith</tt> in place of
|
* Causes this stream to write <tt>replaceWith</tt> in place of
|
||||||
* <tt>toReplace</tt> if {@link #write(int)} is called where the value to
|
* <tt>toReplace</tt> if {@link #write(int)} is called where the value to write is equal to
|
||||||
* write is equal to
|
|
||||||
* <tt>toReplace</tt>.
|
* <tt>toReplace</tt>.
|
||||||
* <p/>
|
* <p/>
|
||||||
* @param toReplace the byte to replace
|
* @param toReplace the byte to replace
|
||||||
* @param replaceWith the byte to be substituted
|
* @param replaceWith the byte to be substituted
|
||||||
* @param maxReplacements the maximum number of replacements that should be
|
* @param maxReplacements the maximum number of replacements that should be made
|
||||||
* made
|
|
||||||
*/
|
*/
|
||||||
public void addFilter(final byte toReplace, final byte replaceWith, final int maxReplacements) {
|
public void addFilter(final byte toReplace, final byte replaceWith, final int maxReplacements) {
|
||||||
singleByteFilters.add(new Filter(new byte[]{toReplace}, new byte[]{replaceWith}, maxReplacements));
|
singleByteFilters.add(new Filter(new byte[]{toReplace}, new byte[]{replaceWith}, maxReplacements));
|
||||||
|
|
|
@ -24,10 +24,8 @@ import java.io.InputStream;
|
||||||
import org.apache.hadoop.io.Writable;
|
import org.apache.hadoop.io.Writable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Simple implementation of {@link Writable} that writes data from an
|
* Simple implementation of {@link Writable} that writes data from an InputStream. This class will throw an
|
||||||
* InputStream. This class will throw an
|
* <tt>UnsupportedOperationException</tt> if {@link #readFields(DataInput)} is called.
|
||||||
* <tt>UnsupportedOperationException</tt> if {@link #readFields(DataInput)} is
|
|
||||||
* called.
|
|
||||||
*/
|
*/
|
||||||
public class InputStreamWritable implements Writable {
|
public class InputStreamWritable implements Writable {
|
||||||
|
|
||||||
|
|
|
@ -28,8 +28,7 @@ import org.apache.hadoop.io.DataInputBuffer;
|
||||||
import org.apache.hadoop.io.Writable;
|
import org.apache.hadoop.io.Writable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class will write to an output stream, rather than an in-memory buffer,
|
* This class will write to an output stream, rather than an in-memory buffer, the fields being read.
|
||||||
* the fields being read.
|
|
||||||
*
|
*
|
||||||
* @author unattributed
|
* @author unattributed
|
||||||
*
|
*
|
||||||
|
|
|
@ -25,15 +25,13 @@ import org.apache.hadoop.io.SequenceFile.CompressionType;
|
||||||
public interface SequenceFileWriter {
|
public interface SequenceFileWriter {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a Sequence File by writing the given FlowFile as key/value pairs.
|
* Creates a Sequence File by writing the given FlowFile as key/value pairs. The provided FlowFile may be a package of multiple FlowFiles, or just one. The keys for the Sequence File are the flow
|
||||||
* The provided FlowFile may be a package of multiple FlowFiles, or just
|
* files' logical names. The values are the flow files' content.
|
||||||
* one. The keys for the Sequence File are the flow files' logical names.
|
|
||||||
* The values are the flow files' content.
|
|
||||||
*
|
*
|
||||||
* @param flowFile - the FlowFile to write to the Sequence File.
|
* @param flowFile - the FlowFile to write to the Sequence File.
|
||||||
* @param session
|
* @param session session
|
||||||
* @param configuration
|
* @param configuration configuration
|
||||||
* @param compressionType
|
* @param compressionType compression type
|
||||||
* @return the written to SequenceFile flow file
|
* @return the written to SequenceFile flow file
|
||||||
*/
|
*/
|
||||||
FlowFile writeSequenceFile(FlowFile flowFile, ProcessSession session, Configuration configuration, CompressionType compressionType);
|
FlowFile writeSequenceFile(FlowFile flowFile, ProcessSession session, Configuration configuration, CompressionType compressionType);
|
||||||
|
|
|
@ -225,8 +225,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
|
||||||
try {
|
try {
|
||||||
binAlreadyCommitted = this.processBin(bin, binCopy, context, session);
|
binAlreadyCommitted = this.processBin(bin, binCopy, context, session);
|
||||||
} catch (final ProcessException e) {
|
} catch (final ProcessException e) {
|
||||||
logger.
|
logger.error("Failed to process bundle of {} files due to {}", new Object[]{binCopy.size(), e});
|
||||||
error("Failed to process bundle of {} files due to {}", new Object[]{binCopy.size(), e});
|
|
||||||
|
|
||||||
for (final FlowFileSessionWrapper wrapper : binCopy) {
|
for (final FlowFileSessionWrapper wrapper : binCopy) {
|
||||||
wrapper.getSession().transfer(wrapper.getFlowFile(), REL_FAILURE);
|
wrapper.getSession().transfer(wrapper.getFlowFile(), REL_FAILURE);
|
||||||
|
@ -294,8 +293,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (context.getProperty(MAX_SIZE).isSet()) {
|
if (context.getProperty(MAX_SIZE).isSet()) {
|
||||||
binManager.setMaximumSize(context.getProperty(MAX_SIZE).
|
binManager.setMaximumSize(context.getProperty(MAX_SIZE).asDataSize(DataUnit.B).longValue());
|
||||||
asDataSize(DataUnit.B).longValue());
|
|
||||||
} else {
|
} else {
|
||||||
binManager.setMaximumSize(Long.MAX_VALUE);
|
binManager.setMaximumSize(Long.MAX_VALUE);
|
||||||
}
|
}
|
||||||
|
@ -313,8 +311,7 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected final Collection<ValidationResult> customValidate(final ValidationContext context) {
|
protected final Collection<ValidationResult> customValidate(final ValidationContext context) {
|
||||||
final List<ValidationResult> problems = new ArrayList<>(super.
|
final List<ValidationResult> problems = new ArrayList<>(super.customValidate(context));
|
||||||
customValidate(context));
|
|
||||||
|
|
||||||
final long minBytes = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
|
final long minBytes = context.getProperty(MIN_SIZE).asDataSize(DataUnit.B).longValue();
|
||||||
final Double maxBytes = context.getProperty(MAX_SIZE).asDataSize(DataUnit.B);
|
final Double maxBytes = context.getProperty(MAX_SIZE).asDataSize(DataUnit.B);
|
||||||
|
@ -330,10 +327,8 @@ public abstract class BinFiles extends AbstractSessionFactoryProcessor {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
final Long min = context.getProperty(MIN_ENTRIES).
|
final Long min = context.getProperty(MIN_ENTRIES).asLong();
|
||||||
asLong();
|
final Long max = context.getProperty(MAX_ENTRIES).asLong();
|
||||||
final Long max = context.getProperty(MAX_ENTRIES).
|
|
||||||
asLong();
|
|
||||||
|
|
||||||
if (min != null && max != null) {
|
if (min != null && max != null) {
|
||||||
if (min > max) {
|
if (min > max) {
|
||||||
|
|
|
@ -146,8 +146,7 @@ public class CompressContent extends AbstractProcessor {
|
||||||
mimeTypeMap.put("application/gzip", COMPRESSION_FORMAT_GZIP);
|
mimeTypeMap.put("application/gzip", COMPRESSION_FORMAT_GZIP);
|
||||||
mimeTypeMap.put("application/bzip2", COMPRESSION_FORMAT_BZIP2);
|
mimeTypeMap.put("application/bzip2", COMPRESSION_FORMAT_BZIP2);
|
||||||
mimeTypeMap.put("application/x-lzma", COMPRESSION_FORMAT_LZMA);
|
mimeTypeMap.put("application/x-lzma", COMPRESSION_FORMAT_LZMA);
|
||||||
this.compressionFormatMimeTypeMap = Collections.
|
this.compressionFormatMimeTypeMap = Collections.unmodifiableMap(mimeTypeMap);
|
||||||
unmodifiableMap(mimeTypeMap);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -286,8 +285,7 @@ public class CompressContent extends AbstractProcessor {
|
||||||
|
|
||||||
final long sizeAfterCompression = flowFile.getSize();
|
final long sizeAfterCompression = flowFile.getSize();
|
||||||
if (MODE_DECOMPRESS.equalsIgnoreCase(compressionMode)) {
|
if (MODE_DECOMPRESS.equalsIgnoreCase(compressionMode)) {
|
||||||
flowFile = session.
|
flowFile = session.removeAttribute(flowFile, CoreAttributes.MIME_TYPE.key());
|
||||||
removeAttribute(flowFile, CoreAttributes.MIME_TYPE.key());
|
|
||||||
|
|
||||||
if (context.getProperty(UPDATE_FILENAME).asBoolean()) {
|
if (context.getProperty(UPDATE_FILENAME).asBoolean()) {
|
||||||
final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
||||||
|
@ -296,8 +294,7 @@ public class CompressContent extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
flowFile = session.
|
flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), mimeTypeRef.get());
|
||||||
putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), mimeTypeRef.get());
|
|
||||||
|
|
||||||
if (context.getProperty(UPDATE_FILENAME).asBoolean()) {
|
if (context.getProperty(UPDATE_FILENAME).asBoolean()) {
|
||||||
final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
final String filename = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
||||||
|
|
|
@ -156,10 +156,10 @@ public class ControlRate extends AbstractProcessor {
|
||||||
rateValidator = StandardValidators.POSITIVE_LONG_VALIDATOR;
|
rateValidator = StandardValidators.POSITIVE_LONG_VALIDATOR;
|
||||||
final String rateAttr = context.getProperty(RATE_CONTROL_ATTRIBUTE_NAME).getValue();
|
final String rateAttr = context.getProperty(RATE_CONTROL_ATTRIBUTE_NAME).getValue();
|
||||||
if (rateAttr == null) {
|
if (rateAttr == null) {
|
||||||
validationResults.add(new ValidationResult.Builder().
|
validationResults.add(new ValidationResult.Builder()
|
||||||
subject(RATE_CONTROL_ATTRIBUTE_NAME.getName()).
|
.subject(RATE_CONTROL_ATTRIBUTE_NAME.getName())
|
||||||
explanation("<Rate Controlled Attribute> property must be set if using <Rate Control Criteria> of 'attribute value'").
|
.explanation("<Rate Controlled Attribute> property must be set if using <Rate Control Criteria> of 'attribute value'")
|
||||||
build());
|
.build());
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
case FLOWFILE_RATE:
|
case FLOWFILE_RATE:
|
||||||
|
@ -281,8 +281,7 @@ public class ControlRate extends AbstractProcessor {
|
||||||
throttle.lock();
|
throttle.lock();
|
||||||
try {
|
try {
|
||||||
if (throttle.tryAdd(rateValue)) {
|
if (throttle.tryAdd(rateValue)) {
|
||||||
logger.
|
logger.info("transferring {} to 'success'", new Object[]{flowFile});
|
||||||
info("transferring {} to 'success'", new Object[]{flowFile});
|
|
||||||
session.transfer(flowFile, REL_SUCCESS);
|
session.transfer(flowFile, REL_SUCCESS);
|
||||||
} else {
|
} else {
|
||||||
flowFile = session.penalize(flowFile);
|
flowFile = session.penalize(flowFile);
|
||||||
|
|
|
@ -16,6 +16,13 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.processors.standard;
|
package org.apache.nifi.processors.standard;
|
||||||
|
|
||||||
|
import java.io.BufferedReader;
|
||||||
|
import java.io.BufferedWriter;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.io.InputStreamReader;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
import java.io.OutputStreamWriter;
|
||||||
import org.apache.nifi.processor.ProcessContext;
|
import org.apache.nifi.processor.ProcessContext;
|
||||||
import org.apache.nifi.processor.AbstractProcessor;
|
import org.apache.nifi.processor.AbstractProcessor;
|
||||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||||
|
@ -34,13 +41,16 @@ import org.apache.nifi.processor.io.StreamCallback;
|
||||||
import org.apache.nifi.processor.util.StandardValidators;
|
import org.apache.nifi.processor.util.StandardValidators;
|
||||||
import org.apache.nifi.util.StopWatch;
|
import org.apache.nifi.util.StopWatch;
|
||||||
|
|
||||||
import java.io.*;
|
|
||||||
import java.nio.CharBuffer;
|
import java.nio.CharBuffer;
|
||||||
import java.nio.charset.Charset;
|
import java.nio.charset.Charset;
|
||||||
import java.nio.charset.CharsetDecoder;
|
import java.nio.charset.CharsetDecoder;
|
||||||
import java.nio.charset.CharsetEncoder;
|
import java.nio.charset.CharsetEncoder;
|
||||||
import java.nio.charset.CodingErrorAction;
|
import java.nio.charset.CodingErrorAction;
|
||||||
import java.util.*;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -117,12 +127,8 @@ public class ConvertCharacterSet extends AbstractProcessor {
|
||||||
public void onTrigger(final ProcessContext context, final ProcessSession session) {
|
public void onTrigger(final ProcessContext context, final ProcessSession session) {
|
||||||
final ProcessorLog logger = getLogger();
|
final ProcessorLog logger = getLogger();
|
||||||
|
|
||||||
final Charset inputCharset = Charset.forName(context.
|
final Charset inputCharset = Charset.forName(context.getProperty(INPUT_CHARSET).getValue());
|
||||||
getProperty(INPUT_CHARSET).
|
final Charset outputCharset = Charset.forName(context.getProperty(OUTPUT_CHARSET).getValue());
|
||||||
getValue());
|
|
||||||
final Charset outputCharset = Charset.forName(context.
|
|
||||||
getProperty(OUTPUT_CHARSET).
|
|
||||||
getValue());
|
|
||||||
final CharBuffer charBuffer = CharBuffer.allocate(MAX_BUFFER_SIZE);
|
final CharBuffer charBuffer = CharBuffer.allocate(MAX_BUFFER_SIZE);
|
||||||
|
|
||||||
final CharsetDecoder decoder = inputCharset.newDecoder();
|
final CharsetDecoder decoder = inputCharset.newDecoder();
|
||||||
|
|
|
@ -117,11 +117,11 @@ public class DistributeLoad extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
}).build();
|
}).build();
|
||||||
public static final PropertyDescriptor LOAD_DISTRIBUTION_SERVICE_TEMPLATE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor LOAD_DISTRIBUTION_SERVICE_TEMPLATE = new PropertyDescriptor.Builder()
|
||||||
.name("Load Distribution Service ID").
|
.name("Load Distribution Service ID")
|
||||||
description("The identifier of the Load Distribution Service").
|
.description("The identifier of the Load Distribution Service")
|
||||||
required(true).
|
.required(true)
|
||||||
identifiesControllerService(LoadDistributionService.class).
|
.identifiesControllerService(LoadDistributionService.class)
|
||||||
build();
|
.build();
|
||||||
|
|
||||||
private List<PropertyDescriptor> properties;
|
private List<PropertyDescriptor> properties;
|
||||||
private final AtomicReference<Set<Relationship>> relationshipsRef = new AtomicReference<>();
|
private final AtomicReference<Set<Relationship>> relationshipsRef = new AtomicReference<>();
|
||||||
|
@ -327,8 +327,7 @@ public class DistributeLoad extends AbstractProcessor {
|
||||||
final List<Relationship> relationshipList = new ArrayList<>();
|
final List<Relationship> relationshipList = new ArrayList<>();
|
||||||
for (final Map.Entry<Integer, Integer> entry : weightings.entrySet()) {
|
for (final Map.Entry<Integer, Integer> entry : weightings.entrySet()) {
|
||||||
final String relationshipName = String.valueOf(entry.getKey());
|
final String relationshipName = String.valueOf(entry.getKey());
|
||||||
final Relationship relationship = new Relationship.Builder().
|
final Relationship relationship = new Relationship.Builder().name(relationshipName).build();
|
||||||
name(relationshipName).build();
|
|
||||||
for (int i = 0; i < entry.getValue(); i++) {
|
for (int i = 0; i < entry.getValue(); i++) {
|
||||||
relationshipList.add(relationship);
|
relationshipList.add(relationship);
|
||||||
}
|
}
|
||||||
|
@ -386,8 +385,8 @@ public class DistributeLoad extends AbstractProcessor {
|
||||||
private static interface DistributionStrategy {
|
private static interface DistributionStrategy {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param session session
|
* @param context context
|
||||||
* @param flowFiles flowFile
|
* @param flowFile flowFile
|
||||||
* @return a mapping of FlowFile to Relationship or <code>null</code> if the needed relationships are not available to accept files
|
* @return a mapping of FlowFile to Relationship or <code>null</code> if the needed relationships are not available to accept files
|
||||||
*/
|
*/
|
||||||
Relationship mapToRelationship(ProcessContext context, FlowFile flowFile);
|
Relationship mapToRelationship(ProcessContext context, FlowFile flowFile);
|
||||||
|
|
|
@ -151,8 +151,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Collection<ValidationResult> customValidate(final ValidationContext context) {
|
protected Collection<ValidationResult> customValidate(final ValidationContext context) {
|
||||||
final List<ValidationResult> results = new ArrayList<>(super.
|
final List<ValidationResult> results = new ArrayList<>(super.customValidate(context));
|
||||||
customValidate(context));
|
|
||||||
|
|
||||||
final String destination = context.getProperty(DESTINATION).getValue();
|
final String destination = context.getProperty(DESTINATION).getValue();
|
||||||
if (DESTINATION_CONTENT.equals(destination)) {
|
if (DESTINATION_CONTENT.equals(destination)) {
|
||||||
|
@ -165,8 +164,8 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (jsonPathCount != 1) {
|
if (jsonPathCount != 1) {
|
||||||
results.add(new ValidationResult.Builder().subject("JsonPaths").valid(false).
|
results.add(new ValidationResult.Builder().subject("JsonPaths").valid(false)
|
||||||
explanation("Exactly one JsonPath must be set if using destination of " + DESTINATION_CONTENT).build());
|
.explanation("Exactly one JsonPath must be set if using destination of " + DESTINATION_CONTENT).build());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -185,18 +184,17 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
|
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
|
||||||
return new PropertyDescriptor.Builder().name(propertyDescriptorName).expressionLanguageSupported(false).
|
return new PropertyDescriptor.Builder().name(propertyDescriptorName).expressionLanguageSupported(false).addValidator(new JsonPathValidator() {
|
||||||
addValidator(new JsonPathValidator() {
|
@Override
|
||||||
@Override
|
public void cacheComputedValue(String subject, String input, JsonPath computedJsonPath) {
|
||||||
public void cacheComputedValue(String subject, String input, JsonPath computedJsonPath) {
|
cachedJsonPathMap.put(input, computedJsonPath);
|
||||||
cachedJsonPathMap.put(input, computedJsonPath);
|
}
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isStale(String subject, String input) {
|
public boolean isStale(String subject, String input) {
|
||||||
return cachedJsonPathMap.get(input) == null;
|
return cachedJsonPathMap.get(input) == null;
|
||||||
}
|
}
|
||||||
}).required(false).dynamic(true).build();
|
}).required(false).dynamic(true).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -235,8 +233,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor {
|
||||||
|
|
||||||
final ProcessorLog logger = getLogger();
|
final ProcessorLog logger = getLogger();
|
||||||
|
|
||||||
String representationOption = processContext.
|
String representationOption = processContext.getProperty(NULL_VALUE_DEFAULT_REPRESENTATION).getValue();
|
||||||
getProperty(NULL_VALUE_DEFAULT_REPRESENTATION).getValue();
|
|
||||||
final String nullDefaultValue = NULL_REPRESENTATION_MAP.get(representationOption);
|
final String nullDefaultValue = NULL_REPRESENTATION_MAP.get(representationOption);
|
||||||
|
|
||||||
/* Build the JsonPath expressions from attributes */
|
/* Build the JsonPath expressions from attributes */
|
||||||
|
@ -309,8 +306,7 @@ public class EvaluateJsonPath extends AbstractJsonPathProcessor {
|
||||||
@Override
|
@Override
|
||||||
public void process(final OutputStream out) throws IOException {
|
public void process(final OutputStream out) throws IOException {
|
||||||
try (OutputStream outputStream = new BufferedOutputStream(out)) {
|
try (OutputStream outputStream = new BufferedOutputStream(out)) {
|
||||||
outputStream.write(resultRepresentation.
|
outputStream.write(resultRepresentation.getBytes(StandardCharsets.UTF_8));
|
||||||
getBytes(StandardCharsets.UTF_8));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
|
@ -165,11 +165,9 @@ public class EvaluateXPath extends AbstractProcessor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Collection<ValidationResult> customValidate(final ValidationContext context) {
|
protected Collection<ValidationResult> customValidate(final ValidationContext context) {
|
||||||
final List<ValidationResult> results = new ArrayList<>(super.
|
final List<ValidationResult> results = new ArrayList<>(super.customValidate(context));
|
||||||
customValidate(context));
|
|
||||||
|
|
||||||
final String destination = context.getProperty(DESTINATION).
|
final String destination = context.getProperty(DESTINATION).getValue();
|
||||||
getValue();
|
|
||||||
if (DESTINATION_CONTENT.equals(destination)) {
|
if (DESTINATION_CONTENT.equals(destination)) {
|
||||||
int xpathCount = 0;
|
int xpathCount = 0;
|
||||||
|
|
||||||
|
@ -356,8 +354,7 @@ public class EvaluateXPath extends AbstractProcessor {
|
||||||
@Override
|
@Override
|
||||||
public void process(final OutputStream rawOut) throws IOException {
|
public void process(final OutputStream rawOut) throws IOException {
|
||||||
try (final OutputStream out = new BufferedOutputStream(rawOut)) {
|
try (final OutputStream out = new BufferedOutputStream(rawOut)) {
|
||||||
out.write(resultString.
|
out.write(resultString.getBytes("UTF-8"));
|
||||||
getBytes("UTF-8"));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
|
@ -184,8 +184,7 @@ public class EvaluateXQuery extends AbstractProcessor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Collection<ValidationResult> customValidate(final ValidationContext context) {
|
protected Collection<ValidationResult> customValidate(final ValidationContext context) {
|
||||||
final List<ValidationResult> results = new ArrayList<>(super.
|
final List<ValidationResult> results = new ArrayList<>(super.customValidate(context));
|
||||||
customValidate(context));
|
|
||||||
|
|
||||||
final String destination = context.getProperty(DESTINATION).getValue();
|
final String destination = context.getProperty(DESTINATION).getValue();
|
||||||
if (DESTINATION_CONTENT.equals(destination)) {
|
if (DESTINATION_CONTENT.equals(destination)) {
|
||||||
|
@ -311,8 +310,7 @@ public class EvaluateXQuery extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
} else { // if (DESTINATION_CONTENT.equals(destination)){
|
} else { // if (DESTINATION_CONTENT.equals(destination)){
|
||||||
if (result.size() == 0) {
|
if (result.size() == 0) {
|
||||||
logger.
|
logger.info("Routing {} to 'unmatched'", new Object[]{flowFile});
|
||||||
info("Routing {} to 'unmatched'", new Object[]{flowFile});
|
|
||||||
session.transfer(flowFile, REL_NO_MATCH);
|
session.transfer(flowFile, REL_NO_MATCH);
|
||||||
continue flowFileLoop;
|
continue flowFileLoop;
|
||||||
} else if (result.size() == 1) {
|
} else if (result.size() == 1) {
|
||||||
|
|
|
@ -163,8 +163,7 @@ public class ExecuteProcess extends AbstractProcessor {
|
||||||
if (inQuotes) {
|
if (inQuotes) {
|
||||||
sb.append(c);
|
sb.append(c);
|
||||||
} else {
|
} else {
|
||||||
final String arg = sb.toString().
|
final String arg = sb.toString().trim();
|
||||||
trim();
|
|
||||||
if (!arg.isEmpty()) {
|
if (!arg.isEmpty()) {
|
||||||
args.add(arg);
|
args.add(arg);
|
||||||
}
|
}
|
||||||
|
@ -377,13 +376,11 @@ public class ExecuteProcess extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
final int exitCode;
|
final int exitCode;
|
||||||
final long millis = TimeUnit.NANOSECONDS.
|
final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
|
||||||
toMillis(System.nanoTime() - startNanos);
|
|
||||||
try {
|
try {
|
||||||
exitCode = process.waitFor();
|
exitCode = process.waitFor();
|
||||||
} catch (final InterruptedException ie) {
|
} catch (final InterruptedException ie) {
|
||||||
getLogger().
|
getLogger().warn("Process was interrupted before finishing");
|
||||||
warn("Process was interrupted before finishing");
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -121,10 +121,10 @@ import org.apache.nifi.stream.io.StreamUtils;
|
||||||
@WritesAttribute(attribute = "execution.error", description = "Any error messages returned from executing the command")})
|
@WritesAttribute(attribute = "execution.error", description = "Any error messages returned from executing the command")})
|
||||||
public class ExecuteStreamCommand extends AbstractProcessor {
|
public class ExecuteStreamCommand extends AbstractProcessor {
|
||||||
|
|
||||||
public static final Relationship ORIGINAL_RELATIONSHIP = new Relationship.Builder().
|
public static final Relationship ORIGINAL_RELATIONSHIP = new Relationship.Builder()
|
||||||
name("original").
|
.name("original")
|
||||||
description("FlowFiles that were successfully processed").
|
.description("FlowFiles that were successfully processed")
|
||||||
build();
|
.build();
|
||||||
public static final Relationship OUTPUT_STREAM_RELATIONSHIP = new Relationship.Builder()
|
public static final Relationship OUTPUT_STREAM_RELATIONSHIP = new Relationship.Builder()
|
||||||
.name("output stream")
|
.name("output stream")
|
||||||
.description("The destination path for the flow file created from the command's output")
|
.description("The destination path for the flow file created from the command's output")
|
||||||
|
@ -139,8 +139,8 @@ public class ExecuteStreamCommand extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final Validator ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR = StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true);
|
private static final Validator ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR = StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true);
|
||||||
static final PropertyDescriptor EXECUTION_COMMAND = new PropertyDescriptor.Builder().
|
static final PropertyDescriptor EXECUTION_COMMAND = new PropertyDescriptor.Builder()
|
||||||
name("Command Path")
|
.name("Command Path")
|
||||||
.description("Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.")
|
.description("Specifies the command to be executed; if just the name of an executable is provided, it must be in the user's environment PATH.")
|
||||||
.expressionLanguageSupported(true)
|
.expressionLanguageSupported(true)
|
||||||
.addValidator(ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
|
.addValidator(ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR)
|
||||||
|
@ -158,8 +158,7 @@ public class ExecuteStreamCommand extends AbstractProcessor {
|
||||||
.subject(subject).valid(true).input(input).build();
|
.subject(subject).valid(true).input(input).build();
|
||||||
String[] args = input.split(";");
|
String[] args = input.split(";");
|
||||||
for (String arg : args) {
|
for (String arg : args) {
|
||||||
ValidationResult valResult = ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR.
|
ValidationResult valResult = ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR.validate(subject, arg, context);
|
||||||
validate(subject, arg, context);
|
|
||||||
if (!valResult.isValid()) {
|
if (!valResult.isValid()) {
|
||||||
result = valResult;
|
result = valResult;
|
||||||
break;
|
break;
|
||||||
|
@ -255,8 +254,7 @@ public class ExecuteStreamCommand extends AbstractProcessor {
|
||||||
session.read(flowFile, callback);
|
session.read(flowFile, callback);
|
||||||
outputStreamFlowFile = callback.outputStreamFlowFile;
|
outputStreamFlowFile = callback.outputStreamFlowFile;
|
||||||
exitCode = callback.exitCode;
|
exitCode = callback.exitCode;
|
||||||
logger.
|
logger.debug("Execution complete for command: {}. Exited with code: {}", new Object[]{executeCommand, exitCode});
|
||||||
debug("Execution complete for command: {}. Exited with code: {}", new Object[]{executeCommand, exitCode});
|
|
||||||
|
|
||||||
Map<String, String> attributes = new HashMap<>();
|
Map<String, String> attributes = new HashMap<>();
|
||||||
|
|
||||||
|
@ -328,8 +326,7 @@ public class ExecuteStreamCommand extends AbstractProcessor {
|
||||||
try {
|
try {
|
||||||
StreamUtils.copy(incomingFlowFileIS, stdInWritable);
|
StreamUtils.copy(incomingFlowFileIS, stdInWritable);
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
logger.
|
logger.error("Failed to write flow file to stdIn due to {}", new Object[]{e}, e);
|
||||||
error("Failed to write flow file to stdIn due to {}", new Object[]{e}, e);
|
|
||||||
}
|
}
|
||||||
// MUST close the output stream to the stdIn so that whatever is reading knows
|
// MUST close the output stream to the stdIn so that whatever is reading knows
|
||||||
// there is no more data
|
// there is no more data
|
||||||
|
|
|
@ -272,8 +272,7 @@ public class ExtractText extends AbstractProcessor {
|
||||||
final Map<String, Pattern> patternMap = compiledPattersMapRef.get();
|
final Map<String, Pattern> patternMap = compiledPattersMapRef.get();
|
||||||
for (final Map.Entry<String, Pattern> entry : patternMap.entrySet()) {
|
for (final Map.Entry<String, Pattern> entry : patternMap.entrySet()) {
|
||||||
|
|
||||||
final Matcher matcher = entry.getValue().
|
final Matcher matcher = entry.getValue().matcher(contentString);
|
||||||
matcher(contentString);
|
|
||||||
|
|
||||||
if (matcher.find()) {
|
if (matcher.find()) {
|
||||||
final String baseKey = entry.getKey();
|
final String baseKey = entry.getKey();
|
||||||
|
|
|
@ -149,8 +149,7 @@ public class GenerateFlowFile extends AbstractProcessor {
|
||||||
data = this.data.get();
|
data = this.data.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
for (int i = 0; i < context.getProperty(BATCH_SIZE).
|
for (int i = 0; i < context.getProperty(BATCH_SIZE).asInteger(); i++) {
|
||||||
asInteger(); i++) {
|
|
||||||
FlowFile flowFile = session.create();
|
FlowFile flowFile = session.create();
|
||||||
if (data.length > 0) {
|
if (data.length > 0) {
|
||||||
flowFile = session.write(flowFile, new OutputStreamCallback() {
|
flowFile = session.write(flowFile, new OutputStreamCallback() {
|
||||||
|
|
|
@ -344,8 +344,7 @@ public class GetFile extends AbstractProcessor {
|
||||||
if (store.supportsFileAttributeView("posix")) {
|
if (store.supportsFileAttributeView("posix")) {
|
||||||
try {
|
try {
|
||||||
PosixFileAttributeView view = Files.getFileAttributeView(file, PosixFileAttributeView.class);
|
PosixFileAttributeView view = Files.getFileAttributeView(file, PosixFileAttributeView.class);
|
||||||
attributes.
|
attributes.put(FILE_PERMISSIONS_ATTRIBUTE, PosixFilePermissions.toString(view.readAttributes().permissions()));
|
||||||
put(FILE_PERMISSIONS_ATTRIBUTE, PosixFilePermissions.toString(view.readAttributes().permissions()));
|
|
||||||
attributes.put(FILE_GROUP_ATTRIBUTE, view.readAttributes().group().getName());
|
attributes.put(FILE_GROUP_ATTRIBUTE, view.readAttributes().group().getName());
|
||||||
} catch (Exception ignore) {
|
} catch (Exception ignore) {
|
||||||
} // allow other attributes if these fail
|
} // allow other attributes if these fail
|
||||||
|
@ -425,8 +424,7 @@ public class GetFile extends AbstractProcessor {
|
||||||
|
|
||||||
flowFile = session.create();
|
flowFile = session.create();
|
||||||
final long importStart = System.nanoTime();
|
final long importStart = System.nanoTime();
|
||||||
flowFile = session.
|
flowFile = session.importFrom(filePath, keepingSourceFile, flowFile);
|
||||||
importFrom(filePath, keepingSourceFile, flowFile);
|
|
||||||
final long importNanos = System.nanoTime() - importStart;
|
final long importNanos = System.nanoTime() - importStart;
|
||||||
final long importMillis = TimeUnit.MILLISECONDS.convert(importNanos, TimeUnit.NANOSECONDS);
|
final long importMillis = TimeUnit.MILLISECONDS.convert(importNanos, TimeUnit.NANOSECONDS);
|
||||||
|
|
||||||
|
|
|
@ -268,8 +268,7 @@ public class GetJMSTopic extends JmsConsumer {
|
||||||
final String serverUrl = props.getProperty(URL.getName());
|
final String serverUrl = props.getProperty(URL.getName());
|
||||||
final String username = props.getProperty(USERNAME.getName());
|
final String username = props.getProperty(USERNAME.getName());
|
||||||
final String encryptedPassword = props.getProperty(PASSWORD.getName());
|
final String encryptedPassword = props.getProperty(PASSWORD.getName());
|
||||||
final String subscriptionName = props.
|
final String subscriptionName = props.getProperty(SUBSCRIPTION_NAME_PROPERTY);
|
||||||
getProperty(SUBSCRIPTION_NAME_PROPERTY);
|
|
||||||
final String jmsProvider = props.getProperty(JMS_PROVIDER.getName());
|
final String jmsProvider = props.getProperty(JMS_PROVIDER.getName());
|
||||||
|
|
||||||
final String password = encryptedPassword == null ? null : context.decrypt(encryptedPassword);
|
final String password = encryptedPassword == null ? null : context.decrypt(encryptedPassword);
|
||||||
|
|
|
@ -104,8 +104,7 @@ import com.sun.jersey.api.client.ClientResponse.Status;
|
||||||
public class HandleHttpRequest extends AbstractProcessor {
|
public class HandleHttpRequest extends AbstractProcessor {
|
||||||
|
|
||||||
public static final String HTTP_CONTEXT_ID = "http.context.identifier";
|
public static final String HTTP_CONTEXT_ID = "http.context.identifier";
|
||||||
private static final Pattern URL_QUERY_PARAM_DELIMITER = Pattern.
|
private static final Pattern URL_QUERY_PARAM_DELIMITER = Pattern.compile("&");
|
||||||
compile("&");
|
|
||||||
|
|
||||||
// Allowable values for client auth
|
// Allowable values for client auth
|
||||||
public static final AllowableValue CLIENT_NONE = new AllowableValue("No Authentication", "No Authentication",
|
public static final AllowableValue CLIENT_NONE = new AllowableValue("No Authentication", "No Authentication",
|
||||||
|
@ -174,13 +173,13 @@ public class HandleHttpRequest extends AbstractProcessor {
|
||||||
.allowableValues("true", "false")
|
.allowableValues("true", "false")
|
||||||
.defaultValue("true")
|
.defaultValue("true")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor ALLOW_PUT = new PropertyDescriptor.Builder().
|
public static final PropertyDescriptor ALLOW_PUT = new PropertyDescriptor.Builder()
|
||||||
name("Allow PUT").
|
.name("Allow PUT")
|
||||||
description("Allow HTTP PUT Method").
|
.description("Allow HTTP PUT Method")
|
||||||
required(true).
|
.required(true)
|
||||||
allowableValues("true", "false").
|
.allowableValues("true", "false")
|
||||||
defaultValue("true").
|
.defaultValue("true")
|
||||||
build();
|
.build();
|
||||||
public static final PropertyDescriptor ALLOW_DELETE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor ALLOW_DELETE = new PropertyDescriptor.Builder()
|
||||||
.name("Allow DELETE")
|
.name("Allow DELETE")
|
||||||
.description("Allow HTTP DELETE Method")
|
.description("Allow HTTP DELETE Method")
|
||||||
|
|
|
@ -154,8 +154,7 @@ public class HandleHttpResponse extends AbstractProcessor {
|
||||||
response.flushBuffer();
|
response.flushBuffer();
|
||||||
} catch (final IOException ioe) {
|
} catch (final IOException ioe) {
|
||||||
session.transfer(flowFile, REL_FAILURE);
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
getLogger().
|
getLogger().error("Failed to respond to HTTP request for {} due to {}", new Object[]{flowFile, ioe});
|
||||||
error("Failed to respond to HTTP request for {} due to {}", new Object[]{flowFile, ioe});
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -129,8 +129,7 @@ public class HashAttribute extends AbstractProcessor {
|
||||||
|
|
||||||
private Set<Relationship> relationships;
|
private Set<Relationship> relationships;
|
||||||
private List<PropertyDescriptor> properties;
|
private List<PropertyDescriptor> properties;
|
||||||
private final AtomicReference<Map<String, Pattern>> regexMapRef = new AtomicReference<>(Collections.
|
private final AtomicReference<Map<String, Pattern>> regexMapRef = new AtomicReference<>(Collections.<String, Pattern>emptyMap());
|
||||||
<String, Pattern>emptyMap());
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void init(final ProcessorInitializationContext context) {
|
protected void init(final ProcessorInitializationContext context) {
|
||||||
|
@ -157,12 +156,7 @@ public class HashAttribute extends AbstractProcessor {
|
||||||
@Override
|
@Override
|
||||||
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
|
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
|
||||||
return new PropertyDescriptor.Builder()
|
return new PropertyDescriptor.Builder()
|
||||||
.name(propertyDescriptorName).
|
.name(propertyDescriptorName).addValidator(StandardValidators.createRegexValidator(0, 1, false)).required(false).dynamic(true).build();
|
||||||
addValidator(StandardValidators.
|
|
||||||
createRegexValidator(0, 1, false)).
|
|
||||||
required(false).
|
|
||||||
dynamic(true).
|
|
||||||
build();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -244,17 +244,16 @@ public final class InvokeHTTP extends AbstractProcessor {
|
||||||
.identifiesControllerService(SSLContextService.class)
|
.identifiesControllerService(SSLContextService.class)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
public static final List<PropertyDescriptor> PROPERTIES = Collections.
|
public static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
|
||||||
unmodifiableList(Arrays.asList(
|
PROP_METHOD,
|
||||||
PROP_METHOD,
|
PROP_URL,
|
||||||
PROP_URL,
|
PROP_SSL_CONTEXT_SERVICE,
|
||||||
PROP_SSL_CONTEXT_SERVICE,
|
PROP_CONNECT_TIMEOUT,
|
||||||
PROP_CONNECT_TIMEOUT,
|
PROP_READ_TIMEOUT,
|
||||||
PROP_READ_TIMEOUT,
|
PROP_DATE_HEADER,
|
||||||
PROP_DATE_HEADER,
|
PROP_FOLLOW_REDIRECTS,
|
||||||
PROP_FOLLOW_REDIRECTS,
|
PROP_ATTRIBUTES_TO_SEND
|
||||||
PROP_ATTRIBUTES_TO_SEND
|
));
|
||||||
));
|
|
||||||
|
|
||||||
// property to allow the hostname verifier to be overridden
|
// property to allow the hostname verifier to be overridden
|
||||||
// this is a "hidden" property - it's configured using a dynamic user property
|
// this is a "hidden" property - it's configured using a dynamic user property
|
||||||
|
@ -559,8 +558,7 @@ public final class InvokeHTTP extends AbstractProcessor {
|
||||||
private Map<String, String> convertAttributesFromHeaders() throws IOException {
|
private Map<String, String> convertAttributesFromHeaders() throws IOException {
|
||||||
// create a new hashmap to store the values from the connection
|
// create a new hashmap to store the values from the connection
|
||||||
Map<String, String> map = new HashMap<>();
|
Map<String, String> map = new HashMap<>();
|
||||||
for (Map.Entry<String, List<String>> entry : conn.getHeaderFields().
|
for (Map.Entry<String, List<String>> entry : conn.getHeaderFields().entrySet()) {
|
||||||
entrySet()) {
|
|
||||||
String key = entry.getKey();
|
String key = entry.getKey();
|
||||||
if (key == null) {
|
if (key == null) {
|
||||||
continue;
|
continue;
|
||||||
|
|
|
@ -175,7 +175,8 @@ public abstract class JmsConsumer extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static JmsProcessingSummary map2FlowFile(final ProcessContext context, final ProcessSession session, final Message message, final boolean addAttributes, ProcessorLog logger) throws Exception {
|
public static JmsProcessingSummary map2FlowFile(final ProcessContext context, final ProcessSession session, final Message message, final boolean addAttributes, ProcessorLog logger)
|
||||||
|
throws Exception {
|
||||||
|
|
||||||
// Currently not very useful, because always one Message == one FlowFile
|
// Currently not very useful, because always one Message == one FlowFile
|
||||||
final IntegerHolder msgsThisFlowFile = new IntegerHolder(1);
|
final IntegerHolder msgsThisFlowFile = new IntegerHolder(1);
|
||||||
|
@ -186,8 +187,7 @@ public abstract class JmsConsumer extends AbstractProcessor {
|
||||||
if (message instanceof MapMessage) {
|
if (message instanceof MapMessage) {
|
||||||
MapMessage mapMessage = (MapMessage) message;
|
MapMessage mapMessage = (MapMessage) message;
|
||||||
flowFile = session.putAllAttributes(flowFile, createMapMessageValues(mapMessage));
|
flowFile = session.putAllAttributes(flowFile, createMapMessageValues(mapMessage));
|
||||||
} // all other message types, write Message body to FlowFile content
|
} else { // all other message types, write Message body to FlowFile content
|
||||||
else {
|
|
||||||
flowFile = session.write(flowFile, new OutputStreamCallback() {
|
flowFile = session.write(flowFile, new OutputStreamCallback() {
|
||||||
@Override
|
@Override
|
||||||
public void process(final OutputStream rawOut) throws IOException {
|
public void process(final OutputStream rawOut) throws IOException {
|
||||||
|
|
|
@ -226,11 +226,11 @@ public class ListenUDP extends AbstractSessionFactoryProcessor {
|
||||||
} catch (SocketException e) {
|
} catch (SocketException e) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
public static final PropertyDescriptor NETWORK_INTF_NAME = new PropertyDescriptor.Builder().
|
public static final PropertyDescriptor NETWORK_INTF_NAME = new PropertyDescriptor.Builder()
|
||||||
name("Local Network Interface").
|
.name("Local Network Interface")
|
||||||
description("The name of a local network interface to be used to restrict listening for UDP Datagrams to a specific LAN."
|
.description("The name of a local network interface to be used to restrict listening for UDP Datagrams to a specific LAN."
|
||||||
+ "May be a system property or an environment variable.").
|
+ "May be a system property or an environment variable.")
|
||||||
addValidator(new Validator() {
|
.addValidator(new Validator() {
|
||||||
@Override
|
@Override
|
||||||
public ValidationResult validate(String subject, String input, ValidationContext context) {
|
public ValidationResult validate(String subject, String input, ValidationContext context) {
|
||||||
ValidationResult result = new ValidationResult.Builder()
|
ValidationResult result = new ValidationResult.Builder()
|
||||||
|
@ -257,7 +257,8 @@ public class ListenUDP extends AbstractSessionFactoryProcessor {
|
||||||
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
}).expressionLanguageSupported(true).build();
|
})
|
||||||
|
.expressionLanguageSupported(true).build();
|
||||||
|
|
||||||
static {
|
static {
|
||||||
List<PropertyDescriptor> props = new ArrayList<>();
|
List<PropertyDescriptor> props = new ArrayList<>();
|
||||||
|
@ -303,102 +304,100 @@ public class ListenUDP extends AbstractSessionFactoryProcessor {
|
||||||
/**
|
/**
|
||||||
* Create the ChannelListener and a thread that causes the Consumer to create flow files.
|
* Create the ChannelListener and a thread that causes the Consumer to create flow files.
|
||||||
*
|
*
|
||||||
* @param context
|
* @param context context
|
||||||
* @throws IOException
|
* @throws IOException ex
|
||||||
*/
|
*/
|
||||||
@OnScheduled
|
@OnScheduled
|
||||||
public void initializeChannelListenerAndConsumerProcessing(final ProcessContext context) throws IOException {
|
public void initializeChannelListenerAndConsumerProcessing(final ProcessContext context) throws IOException {
|
||||||
getChannelListener(context);
|
getChannelListener(context);
|
||||||
stopping.set(false);
|
stopping.set(false);
|
||||||
Future<Tuple<ProcessSession, List<FlowFile>>> consumerFuture = consumerExecutorService.
|
Future<Tuple<ProcessSession, List<FlowFile>>> consumerFuture = consumerExecutorService.submit(new Callable<Tuple<ProcessSession, List<FlowFile>>>() {
|
||||||
submit(new Callable<Tuple<ProcessSession, List<FlowFile>>>() {
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Tuple<ProcessSession, List<FlowFile>> call() {
|
public Tuple<ProcessSession, List<FlowFile>> call() {
|
||||||
final int maxFlowFilesPerSession = context.getProperty(FLOW_FILES_PER_SESSION).asInteger();
|
final int maxFlowFilesPerSession = context.getProperty(FLOW_FILES_PER_SESSION).asInteger();
|
||||||
final long channelReaderIntervalMSecs = context.getProperty(CHANNEL_READER_PERIOD).asTimePeriod(TimeUnit.MILLISECONDS);
|
final long channelReaderIntervalMSecs = context.getProperty(CHANNEL_READER_PERIOD).asTimePeriod(TimeUnit.MILLISECONDS);
|
||||||
// number of waits in 5 secs, or 1
|
// number of waits in 5 secs, or 1
|
||||||
final int maxWaits = (int) (channelReaderIntervalMSecs <= 1000 ? 5000 / channelReaderIntervalMSecs : 1);
|
final int maxWaits = (int) (channelReaderIntervalMSecs <= 1000 ? 5000 / channelReaderIntervalMSecs : 1);
|
||||||
final ProcessorLog logger = getLogger();
|
final ProcessorLog logger = getLogger();
|
||||||
int flowFileCount = maxFlowFilesPerSession;
|
int flowFileCount = maxFlowFilesPerSession;
|
||||||
ProcessSession session = null;
|
ProcessSession session = null;
|
||||||
int numWaits = 0;
|
int numWaits = 0;
|
||||||
while (!stopping.get()) {
|
while (!stopping.get()) {
|
||||||
UDPStreamConsumer consumer = consumerRef.get();
|
UDPStreamConsumer consumer = consumerRef.get();
|
||||||
if (consumer == null || sessionFactoryRef.get() == null) {
|
if (consumer == null || sessionFactoryRef.get() == null) {
|
||||||
try {
|
try {
|
||||||
Thread.sleep(100L);
|
Thread.sleep(100L);
|
||||||
} catch (InterruptedException swallow) {
|
} catch (InterruptedException swallow) {
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
try {
|
||||||
|
// first time through, flowFileCount is maxFlowFilesPerSession so that a session
|
||||||
|
// is created and the consumer is updated with it.
|
||||||
|
if (flowFileCount == maxFlowFilesPerSession || numWaits == maxWaits) {
|
||||||
|
logger.debug("Have waited {} times", new Object[]{numWaits});
|
||||||
|
numWaits = 0;
|
||||||
|
if (session != null) {
|
||||||
|
Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(session, new ArrayList<>(newFlowFiles));
|
||||||
|
newFlowFiles.clear();
|
||||||
|
flowFilesPerSessionQueue.add(flowFilesPerSession);
|
||||||
|
}
|
||||||
|
session = sessionFactoryRef.get().createSession();
|
||||||
|
consumer.setSession(session);
|
||||||
|
flowFileCount = 0;
|
||||||
|
}
|
||||||
|
// this will throttle the processing of the received datagrams. If there are no more
|
||||||
|
// buffers to read into because none have been returned to the pool via consumer.process(),
|
||||||
|
// then the desired back pressure on the channel is created.
|
||||||
|
if (context.getAvailableRelationships().size() > 0) {
|
||||||
|
consumer.process();
|
||||||
|
if (flowFileCount == newFlowFiles.size()) {
|
||||||
|
// no new datagrams received, need to throttle this thread back so it does
|
||||||
|
// not consume all cpu...but don't want to cause back pressure on the channel
|
||||||
|
// so the sleep time is same as the reader interval
|
||||||
|
// If have done this for approx. 5 secs, assume datagram sender is down. So, push
|
||||||
|
// out the remaining flow files (see numWaits == maxWaits above)
|
||||||
|
Thread.sleep(channelReaderIntervalMSecs);
|
||||||
|
if (flowFileCount > 0) {
|
||||||
|
numWaits++;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
flowFileCount = newFlowFiles.size();
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
try {
|
logger.debug("Creating back pressure...no available destinations");
|
||||||
// first time through, flowFileCount is maxFlowFilesPerSession so that a session
|
Thread.sleep(1000L);
|
||||||
// is created and the consumer is updated with it.
|
}
|
||||||
if (flowFileCount == maxFlowFilesPerSession || numWaits == maxWaits) {
|
} catch (final IOException ioe) {
|
||||||
logger.debug("Have waited {} times", new Object[]{numWaits});
|
logger.error("Unable to fully process consumer {}", new Object[]{consumer}, ioe);
|
||||||
numWaits = 0;
|
} catch (InterruptedException e) {
|
||||||
if (session != null) {
|
// don't care
|
||||||
Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(session, new ArrayList<>(newFlowFiles));
|
} finally {
|
||||||
newFlowFiles.clear();
|
if (consumer.isConsumerFinished()) {
|
||||||
flowFilesPerSessionQueue.
|
logger.info("Consumer {} was closed and is finished", new Object[]{consumer});
|
||||||
add(flowFilesPerSession);
|
consumerRef.set(null);
|
||||||
}
|
disconnect();
|
||||||
session = sessionFactoryRef.get().createSession();
|
if (!stopping.get()) {
|
||||||
consumer.setSession(session);
|
resetChannelListener.set(true);
|
||||||
flowFileCount = 0;
|
|
||||||
}
|
|
||||||
// this will throttle the processing of the received datagrams. If there are no more
|
|
||||||
// buffers to read into because none have been returned to the pool via consumer.process(),
|
|
||||||
// then the desired back pressure on the channel is created.
|
|
||||||
if (context.getAvailableRelationships().size() > 0) {
|
|
||||||
consumer.process();
|
|
||||||
if (flowFileCount == newFlowFiles.size()) {
|
|
||||||
// no new datagrams received, need to throttle this thread back so it does
|
|
||||||
// not consume all cpu...but don't want to cause back pressure on the channel
|
|
||||||
// so the sleep time is same as the reader interval
|
|
||||||
// If have done this for approx. 5 secs, assume datagram sender is down. So, push
|
|
||||||
// out the remaining flow files (see numWaits == maxWaits above)
|
|
||||||
Thread.sleep(channelReaderIntervalMSecs);
|
|
||||||
if (flowFileCount > 0) {
|
|
||||||
numWaits++;
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
flowFileCount = newFlowFiles.size();
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
logger.debug("Creating back pressure...no available destinations");
|
|
||||||
Thread.sleep(1000L);
|
|
||||||
}
|
|
||||||
} catch (final IOException ioe) {
|
|
||||||
logger.error("Unable to fully process consumer {}", new Object[]{consumer}, ioe);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
// don't care
|
|
||||||
} finally {
|
|
||||||
if (consumer.isConsumerFinished()) {
|
|
||||||
logger.info("Consumer {} was closed and is finished", new Object[]{consumer});
|
|
||||||
consumerRef.set(null);
|
|
||||||
disconnect();
|
|
||||||
if (!stopping.get()) {
|
|
||||||
resetChannelListener.set(true);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// when shutting down, need consumer to drain rest of cached buffers and clean up.
|
|
||||||
// prior to getting here, the channelListener was shutdown
|
|
||||||
UDPStreamConsumer consumer;
|
|
||||||
while ((consumer = consumerRef.get()) != null && !consumer.isConsumerFinished()) {
|
|
||||||
try {
|
|
||||||
consumer.process();
|
|
||||||
} catch (IOException swallow) {
|
|
||||||
// if this is blown...consumer.isConsumerFinished will be true
|
|
||||||
}
|
|
||||||
}
|
|
||||||
Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(session, new ArrayList<>(newFlowFiles));
|
|
||||||
return flowFilesPerSession;
|
|
||||||
}
|
}
|
||||||
});
|
}
|
||||||
|
// when shutting down, need consumer to drain rest of cached buffers and clean up.
|
||||||
|
// prior to getting here, the channelListener was shutdown
|
||||||
|
UDPStreamConsumer consumer;
|
||||||
|
while ((consumer = consumerRef.get()) != null && !consumer.isConsumerFinished()) {
|
||||||
|
try {
|
||||||
|
consumer.process();
|
||||||
|
} catch (IOException swallow) {
|
||||||
|
// if this is blown...consumer.isConsumerFinished will be true
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Tuple<ProcessSession, List<FlowFile>> flowFilesPerSession = new Tuple<ProcessSession, List<FlowFile>>(session, new ArrayList<>(newFlowFiles));
|
||||||
|
return flowFilesPerSession;
|
||||||
|
}
|
||||||
|
});
|
||||||
consumerFutureRef.set(consumerFuture);
|
consumerFutureRef.set(consumerFuture);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -434,8 +433,7 @@ public class ListenUDP extends AbstractSessionFactoryProcessor {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public StreamConsumer newInstance(final String streamId) {
|
public StreamConsumer newInstance(final String streamId) {
|
||||||
final UDPStreamConsumer consumer = new UDPStreamConsumer(streamId, newFlowFiles, flowFileSizeTrigger.
|
final UDPStreamConsumer consumer = new UDPStreamConsumer(streamId, newFlowFiles, flowFileSizeTrigger.intValue(), getLogger());
|
||||||
intValue(), getLogger());
|
|
||||||
consumerRef.set(consumer);
|
consumerRef.set(consumer);
|
||||||
return consumer;
|
return consumer;
|
||||||
}
|
}
|
||||||
|
|
|
@ -327,8 +327,7 @@ public class MergeContent extends BinFiles {
|
||||||
protected boolean processBin(final Bin unmodifiableBin, final List<FlowFileSessionWrapper> binCopy, final ProcessContext context,
|
protected boolean processBin(final Bin unmodifiableBin, final List<FlowFileSessionWrapper> binCopy, final ProcessContext context,
|
||||||
final ProcessSession session) throws ProcessException {
|
final ProcessSession session) throws ProcessException {
|
||||||
|
|
||||||
final String mergeFormat = context.getProperty(MERGE_FORMAT).
|
final String mergeFormat = context.getProperty(MERGE_FORMAT).getValue();
|
||||||
getValue();
|
|
||||||
MergeBin merger;
|
MergeBin merger;
|
||||||
switch (mergeFormat) {
|
switch (mergeFormat) {
|
||||||
case MERGE_FORMAT_TAR_VALUE:
|
case MERGE_FORMAT_TAR_VALUE:
|
||||||
|
@ -458,8 +457,7 @@ public class MergeContent extends BinFiles {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
return NUMBER_PATTERN.matcher(value).
|
return NUMBER_PATTERN.matcher(value).matches();
|
||||||
matches();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private class BinaryConcatenationMerge implements MergeBin {
|
private class BinaryConcatenationMerge implements MergeBin {
|
||||||
|
|
|
@ -418,9 +418,8 @@ public class PostHTTP extends AbstractProcessor {
|
||||||
try {
|
try {
|
||||||
new java.net.URL(url);
|
new java.net.URL(url);
|
||||||
} catch (final MalformedURLException e) {
|
} catch (final MalformedURLException e) {
|
||||||
logger.
|
logger.error("After substituting attribute values for {}, URL is {}; this is not a valid URL, so routing to failure",
|
||||||
error("After substituting attribute values for {}, URL is {}; this is not a valid URL, so routing to failure",
|
new Object[]{flowFile, url});
|
||||||
new Object[]{flowFile, url});
|
|
||||||
flowFile = session.penalize(flowFile);
|
flowFile = session.penalize(flowFile);
|
||||||
session.transfer(flowFile, REL_FAILURE);
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
continue;
|
continue;
|
||||||
|
@ -442,29 +441,28 @@ public class PostHTTP extends AbstractProcessor {
|
||||||
final HttpClientBuilder clientBuilder = HttpClientBuilder.create();
|
final HttpClientBuilder clientBuilder = HttpClientBuilder.create();
|
||||||
clientBuilder.setConnectionManager(conMan);
|
clientBuilder.setConnectionManager(conMan);
|
||||||
clientBuilder.setUserAgent(userAgent);
|
clientBuilder.setUserAgent(userAgent);
|
||||||
clientBuilder.
|
clientBuilder.addInterceptorFirst(new HttpResponseInterceptor() {
|
||||||
addInterceptorFirst(new HttpResponseInterceptor() {
|
@Override
|
||||||
@Override
|
public void process(final HttpResponse response, final HttpContext httpContext) throws HttpException, IOException {
|
||||||
public void process(final HttpResponse response, final HttpContext httpContext) throws HttpException, IOException {
|
HttpCoreContext coreContext = HttpCoreContext.adapt(httpContext);
|
||||||
HttpCoreContext coreContext = HttpCoreContext.adapt(httpContext);
|
ManagedHttpClientConnection conn = coreContext.getConnection(ManagedHttpClientConnection.class);
|
||||||
ManagedHttpClientConnection conn = coreContext.getConnection(ManagedHttpClientConnection.class);
|
if (!conn.isOpen()) {
|
||||||
if (!conn.isOpen()) {
|
return;
|
||||||
return;
|
}
|
||||||
}
|
|
||||||
|
|
||||||
SSLSession sslSession = conn.getSSLSession();
|
SSLSession sslSession = conn.getSSLSession();
|
||||||
|
|
||||||
if (sslSession != null) {
|
if (sslSession != null) {
|
||||||
final X509Certificate[] certChain = sslSession.getPeerCertificateChain();
|
final X509Certificate[] certChain = sslSession.getPeerCertificateChain();
|
||||||
if (certChain == null || certChain.length == 0) {
|
if (certChain == null || certChain.length == 0) {
|
||||||
throw new SSLPeerUnverifiedException("No certificates found");
|
throw new SSLPeerUnverifiedException("No certificates found");
|
||||||
}
|
|
||||||
|
|
||||||
final X509Certificate cert = certChain[0];
|
|
||||||
dnHolder.set(cert.getSubjectDN().getName().trim());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
});
|
|
||||||
|
final X509Certificate cert = certChain[0];
|
||||||
|
dnHolder.set(cert.getSubjectDN().getName().trim());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
clientBuilder.disableAutomaticRetries();
|
clientBuilder.disableAutomaticRetries();
|
||||||
clientBuilder.disableContentCompression();
|
clientBuilder.disableContentCompression();
|
||||||
|
@ -783,8 +781,7 @@ public class PostHTTP extends AbstractProcessor {
|
||||||
|
|
||||||
if (!isScheduled()) {
|
if (!isScheduled()) {
|
||||||
context.yield();
|
context.yield();
|
||||||
logger.
|
logger.warn("Failed to delete Hold that destination placed on {}; Processor has been stopped so routing FlowFile(s) to failure", new Object[]{flowFileDescription});
|
||||||
warn("Failed to delete Hold that destination placed on {}; Processor has been stopped so routing FlowFile(s) to failure", new Object[]{flowFileDescription});
|
|
||||||
for (FlowFile flowFile : toSend) {
|
for (FlowFile flowFile : toSend) {
|
||||||
flowFile = session.penalize(flowFile);
|
flowFile = session.penalize(flowFile);
|
||||||
session.transfer(flowFile, REL_FAILURE);
|
session.transfer(flowFile, REL_FAILURE);
|
||||||
|
|
|
@ -274,8 +274,7 @@ public class PutEmail extends AbstractProcessor {
|
||||||
final String bcc = context.getProperty(BCC).getValue();
|
final String bcc = context.getProperty(BCC).getValue();
|
||||||
|
|
||||||
if (to == null && cc == null && bcc == null) {
|
if (to == null && cc == null && bcc == null) {
|
||||||
errors.add(new ValidationResult.Builder().subject("To, CC, BCC").
|
errors.add(new ValidationResult.Builder().subject("To, CC, BCC").valid(false).explanation("Must specify at least one To/CC/BCC address").build());
|
||||||
valid(false).explanation("Must specify at least one To/CC/BCC address").build());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
return errors;
|
return errors;
|
||||||
|
@ -390,8 +389,7 @@ public class PutEmail extends AbstractProcessor {
|
||||||
|
|
||||||
final ProcessorLog logger = this.getLogger();
|
final ProcessorLog logger = this.getLogger();
|
||||||
|
|
||||||
for (Entry<String, PropertyDescriptor> entry : propertyToContext.
|
for (Entry<String, PropertyDescriptor> entry : propertyToContext.entrySet()) {
|
||||||
entrySet()) {
|
|
||||||
|
|
||||||
// Evaluate the property descriptor against the flow file
|
// Evaluate the property descriptor against the flow file
|
||||||
String flowFileValue = context.getProperty(entry.getValue()).evaluateAttributeExpressions(flowFile).getValue();
|
String flowFileValue = context.getProperty(entry.getValue()).evaluateAttributeExpressions(flowFile).getValue();
|
||||||
|
|
|
@ -57,10 +57,8 @@ import org.apache.nifi.processors.standard.util.FTPTransfer;
|
||||||
+ " you leave off the .")})
|
+ " you leave off the .")})
|
||||||
public class PutFTP extends PutFileTransfer<FTPTransfer> {
|
public class PutFTP extends PutFileTransfer<FTPTransfer> {
|
||||||
|
|
||||||
private static final Pattern PRE_SEND_CMD_PATTERN = Pattern.
|
private static final Pattern PRE_SEND_CMD_PATTERN = Pattern.compile("^pre\\.cmd\\.(\\d+)$");
|
||||||
compile("^pre\\.cmd\\.(\\d+)$");
|
private static final Pattern POST_SEND_CMD_PATTERN = Pattern.compile("^post\\.cmd\\.(\\d+)$");
|
||||||
private static final Pattern POST_SEND_CMD_PATTERN = Pattern.
|
|
||||||
compile("^post\\.cmd\\.(\\d+)$");
|
|
||||||
|
|
||||||
private final AtomicReference<List<PropertyDescriptor>> preSendDescriptorRef = new AtomicReference<>();
|
private final AtomicReference<List<PropertyDescriptor>> preSendDescriptorRef = new AtomicReference<>();
|
||||||
private final AtomicReference<List<PropertyDescriptor>> postSendDescriptorRef = new AtomicReference<>();
|
private final AtomicReference<List<PropertyDescriptor>> postSendDescriptorRef = new AtomicReference<>();
|
||||||
|
@ -109,8 +107,7 @@ public class PutFTP extends PutFileTransfer<FTPTransfer> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void afterPut(final FlowFile flowFile, final ProcessContext context, final FTPTransfer transfer) throws IOException {
|
protected void afterPut(final FlowFile flowFile, final ProcessContext context, final FTPTransfer transfer) throws IOException {
|
||||||
transfer.
|
transfer.sendCommands(getCommands(postSendDescriptorRef.get(), context, flowFile), flowFile);
|
||||||
sendCommands(getCommands(postSendDescriptorRef.get(), context, flowFile), flowFile);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -44,7 +44,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
/**
|
/**
|
||||||
* Base class for PutFTP & PutSFTP
|
* Base class for PutFTP & PutSFTP
|
||||||
*
|
*
|
||||||
* @param <T>
|
* @param <T> type of transfer
|
||||||
*/
|
*/
|
||||||
public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractProcessor {
|
public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractProcessor {
|
||||||
|
|
||||||
|
@ -181,7 +181,14 @@ public abstract class PutFileTransfer<T extends FileTransfer> extends AbstractPr
|
||||||
}
|
}
|
||||||
|
|
||||||
//Attempts to identify naming or content issues with files before they are transferred.
|
//Attempts to identify naming or content issues with files before they are transferred.
|
||||||
private ConflictResult identifyAndResolveConflictFile(final String conflictResolutionType, final T transfer, final String path, final FlowFile flowFile, final boolean rejectZeroByteFiles, final ProcessorLog logger) throws IOException {
|
private ConflictResult identifyAndResolveConflictFile(
|
||||||
|
final String conflictResolutionType,
|
||||||
|
final T transfer,
|
||||||
|
final String path,
|
||||||
|
final FlowFile flowFile,
|
||||||
|
final boolean rejectZeroByteFiles,
|
||||||
|
final ProcessorLog logger)
|
||||||
|
throws IOException {
|
||||||
Relationship destinationRelationship = REL_SUCCESS;
|
Relationship destinationRelationship = REL_SUCCESS;
|
||||||
String fileName = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
String fileName = flowFile.getAttribute(CoreAttributes.FILENAME.key());
|
||||||
boolean transferFile = true;
|
boolean transferFile = true;
|
||||||
|
|
|
@ -336,8 +336,7 @@ public class PutJMS extends AbstractProcessor {
|
||||||
final String key = entry.getKey();
|
final String key = entry.getKey();
|
||||||
final String value = entry.getValue();
|
final String value = entry.getValue();
|
||||||
|
|
||||||
if (key.toLowerCase().
|
if (key.toLowerCase().startsWith(ATTRIBUTE_PREFIX.toLowerCase()) && !key.toLowerCase().endsWith(ATTRIBUTE_TYPE_SUFFIX.toLowerCase())) {
|
||||||
startsWith(ATTRIBUTE_PREFIX.toLowerCase()) && !key.toLowerCase().endsWith(ATTRIBUTE_TYPE_SUFFIX.toLowerCase())) {
|
|
||||||
|
|
||||||
final String jmsPropName = key.substring(ATTRIBUTE_PREFIX.length());
|
final String jmsPropName = key.substring(ATTRIBUTE_PREFIX.length());
|
||||||
final String type = attributes.get(key + ATTRIBUTE_TYPE_SUFFIX);
|
final String type = attributes.get(key + ATTRIBUTE_TYPE_SUFFIX);
|
||||||
|
|
|
@ -76,12 +76,10 @@ public class PutSFTP extends PutFileTransfer<SFTPTransfer> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
|
protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) {
|
||||||
if (SFTPTransfer.DISABLE_DIRECTORY_LISTING.getName().
|
if (SFTPTransfer.DISABLE_DIRECTORY_LISTING.getName().equalsIgnoreCase(propertyDescriptorName)) {
|
||||||
equalsIgnoreCase(propertyDescriptorName)) {
|
|
||||||
return SFTPTransfer.DISABLE_DIRECTORY_LISTING;
|
return SFTPTransfer.DISABLE_DIRECTORY_LISTING;
|
||||||
}
|
}
|
||||||
return super.
|
return super.getSupportedDynamicPropertyDescriptor(propertyDescriptorName);
|
||||||
getSupportedDynamicPropertyDescriptor(propertyDescriptorName);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -49,7 +49,11 @@ import java.io.InputStreamReader;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.io.OutputStreamWriter;
|
import java.io.OutputStreamWriter;
|
||||||
import java.nio.charset.Charset;
|
import java.nio.charset.Charset;
|
||||||
import java.util.*;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.regex.Matcher;
|
import java.util.regex.Matcher;
|
||||||
import java.util.regex.Pattern;
|
import java.util.regex.Pattern;
|
||||||
|
@ -58,7 +62,8 @@ import java.util.regex.Pattern;
|
||||||
@SideEffectFree
|
@SideEffectFree
|
||||||
@SupportsBatching
|
@SupportsBatching
|
||||||
@Tags({"Text", "Regular Expression", "Update", "Change", "Replace", "Modify", "Regex"})
|
@Tags({"Text", "Regular Expression", "Update", "Change", "Replace", "Modify", "Regex"})
|
||||||
@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression against it and replacing the section of the content that matches the Regular Expression with some alternate value.")
|
@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression against it and replacing the section of "
|
||||||
|
+ "the content that matches the Regular Expression with some alternate value.")
|
||||||
public class ReplaceText extends AbstractProcessor {
|
public class ReplaceText extends AbstractProcessor {
|
||||||
|
|
||||||
//Constants
|
//Constants
|
||||||
|
@ -77,7 +82,8 @@ public class ReplaceText extends AbstractProcessor {
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor REPLACEMENT_VALUE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor REPLACEMENT_VALUE = new PropertyDescriptor.Builder()
|
||||||
.name("Replacement Value")
|
.name("Replacement Value")
|
||||||
.description("The value to replace the regular expression with. Back-references to Regular Expression capturing groups are supported, but back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value.")
|
.description("The value to replace the regular expression with. Back-references to Regular Expression capturing groups are supported, but "
|
||||||
|
+ "back-references that reference capturing groups that do not exist in the regular expression will be treated as literal value.")
|
||||||
.required(true)
|
.required(true)
|
||||||
.defaultValue("$1")
|
.defaultValue("$1")
|
||||||
.addValidator(Validator.VALID)
|
.addValidator(Validator.VALID)
|
||||||
|
@ -92,15 +98,20 @@ public class ReplaceText extends AbstractProcessor {
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||||
.name("Maximum Buffer Size")
|
.name("Maximum Buffer Size")
|
||||||
.description("Specifies the maximum amount of data to buffer (per file or per line, depending on the Evaluation Mode) in order to apply the regular expressions. If 'Entire Text' (in Evaluation Mode) is selected and the FlowFile is larger than this value, the FlowFile will be routed to 'failure'. "
|
.description("Specifies the maximum amount of data to buffer (per file or per line, depending on the Evaluation Mode) in order to "
|
||||||
+ "In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. This value is ignored and the buffer is not used if 'Regular Expression' is set to '.*'")
|
+ "apply the regular expressions. If 'Entire Text' (in Evaluation Mode) is selected and the FlowFile is larger than this value, "
|
||||||
|
+ "the FlowFile will be routed to 'failure'. "
|
||||||
|
+ "In 'Line-by-Line' Mode, if a single line is larger than this value, the FlowFile will be routed to 'failure'. A default value "
|
||||||
|
+ "of 1 MB is provided, primarily for 'Entire Text' mode. In 'Line-by-Line' Mode, a value such as 8 KB or 16 KB is suggested. "
|
||||||
|
+ "This value is ignored and the buffer is not used if 'Regular Expression' is set to '.*'")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||||
.defaultValue("1 MB")
|
.defaultValue("1 MB")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor EVALUATION_MODE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor EVALUATION_MODE = new PropertyDescriptor.Builder()
|
||||||
.name("Evaluation Mode")
|
.name("Evaluation Mode")
|
||||||
.description("Evaluate the 'Regular Expression' against each line (Line-by-Line) or buffer the entire file into memory (Entire Text) and then evaluate the 'Regular Expression'.")
|
.description("Evaluate the 'Regular Expression' against each line (Line-by-Line) or buffer the entire file into memory (Entire Text) and "
|
||||||
|
+ "then evaluate the 'Regular Expression'.")
|
||||||
.allowableValues(LINE_BY_LINE, ENTIRE_TEXT)
|
.allowableValues(LINE_BY_LINE, ENTIRE_TEXT)
|
||||||
.defaultValue(ENTIRE_TEXT)
|
.defaultValue(ENTIRE_TEXT)
|
||||||
.required(true)
|
.required(true)
|
||||||
|
@ -108,7 +119,8 @@ public class ReplaceText extends AbstractProcessor {
|
||||||
// Relationships
|
// Relationships
|
||||||
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
public static final Relationship REL_SUCCESS = new Relationship.Builder()
|
||||||
.name("success")
|
.name("success")
|
||||||
.description("FlowFiles that have been successfully updated are routed to this relationship, as well as FlowFiles whose content does not match the given Regular Expression")
|
.description("FlowFiles that have been successfully updated are routed to this relationship, as well as FlowFiles whose content does not "
|
||||||
|
+ "match the given Regular Expression")
|
||||||
.build();
|
.build();
|
||||||
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
public static final Relationship REL_FAILURE = new Relationship.Builder()
|
||||||
.name("failure")
|
.name("failure")
|
||||||
|
@ -205,7 +217,7 @@ public class ReplaceText extends AbstractProcessor {
|
||||||
final int originalBackRefIndex = Integer.parseInt(backRefNum);
|
final int originalBackRefIndex = Integer.parseInt(backRefNum);
|
||||||
int backRefIndex = originalBackRefIndex;
|
int backRefIndex = originalBackRefIndex;
|
||||||
|
|
||||||
// if we have a replacement value like $123, and we have less than 123 capturing groups, then
|
// if we have a replacement value like $123, and we have less than 123 capturing groups, then
|
||||||
// we want to truncate the 3 and use capturing group 12; if we have less than 12 capturing groups,
|
// we want to truncate the 3 and use capturing group 12; if we have less than 12 capturing groups,
|
||||||
// then we want to truncate the 2 and use capturing group 1; if we don't have a capturing group then
|
// then we want to truncate the 2 and use capturing group 1; if we don't have a capturing group then
|
||||||
// we want to truncate the 1 and get 0.
|
// we want to truncate the 1 and get 0.
|
||||||
|
|
|
@ -68,7 +68,8 @@ import org.apache.commons.lang3.StringUtils;
|
||||||
@SideEffectFree
|
@SideEffectFree
|
||||||
@SupportsBatching
|
@SupportsBatching
|
||||||
@Tags({"Text", "Regular Expression", "Update", "Change", "Replace", "Modify", "Regex", "Mapping"})
|
@Tags({"Text", "Regular Expression", "Update", "Change", "Replace", "Modify", "Regex", "Mapping"})
|
||||||
@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression against it and replacing the section of the content that matches the Regular Expression with some alternate value provided in a mapping file.")
|
@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression against it and replacing the section of the content that "
|
||||||
|
+ "matches the Regular Expression with some alternate value provided in a mapping file.")
|
||||||
public class ReplaceTextWithMapping extends AbstractProcessor {
|
public class ReplaceTextWithMapping extends AbstractProcessor {
|
||||||
|
|
||||||
public static final PropertyDescriptor REGEX = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor REGEX = new PropertyDescriptor.Builder()
|
||||||
|
@ -109,7 +110,8 @@ public class ReplaceTextWithMapping extends AbstractProcessor {
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor MAX_BUFFER_SIZE = new PropertyDescriptor.Builder()
|
||||||
.name("Maximum Buffer Size")
|
.name("Maximum Buffer Size")
|
||||||
.description("Specifies the maximum amount of data to buffer (per file) in order to apply the regular expressions. If a FlowFile is larger than this value, the FlowFile will be routed to 'failure'")
|
.description("Specifies the maximum amount of data to buffer (per file) in order to apply the regular expressions. If a FlowFile is larger "
|
||||||
|
+ "than this value, the FlowFile will be routed to 'failure'")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
.addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
|
||||||
.defaultValue("1 MB")
|
.defaultValue("1 MB")
|
||||||
|
@ -270,13 +272,6 @@ public class ReplaceTextWithMapping extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Loads a file containing mappings.
|
|
||||||
*
|
|
||||||
* @param is
|
|
||||||
* @return
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
protected Map<String, String> loadMappingFile(InputStream is) throws IOException {
|
protected Map<String, String> loadMappingFile(InputStream is) throws IOException {
|
||||||
Map<String, String> mapping = new HashMap<>();
|
Map<String, String> mapping = new HashMap<>();
|
||||||
BufferedReader reader = new BufferedReader(new InputStreamReader(is));
|
BufferedReader reader = new BufferedReader(new InputStreamReader(is));
|
||||||
|
|
|
@ -77,7 +77,8 @@ public class RouteOnAttribute extends AbstractProcessor {
|
||||||
"A copy of the FlowFile will be routed to each relationship whose corresponding expression evaluates to 'true'");
|
"A copy of the FlowFile will be routed to each relationship whose corresponding expression evaluates to 'true'");
|
||||||
public static final AllowableValue ROUTE_ALL_MATCH = new AllowableValue(routeAllMatchValue, "Route to 'matched' if all match",
|
public static final AllowableValue ROUTE_ALL_MATCH = new AllowableValue(routeAllMatchValue, "Route to 'matched' if all match",
|
||||||
"Requires that all user-defined expressions evaluate to 'true' for the FlowFile to be considered a match");
|
"Requires that all user-defined expressions evaluate to 'true' for the FlowFile to be considered a match");
|
||||||
public static final AllowableValue ROUTE_ANY_MATCHES = new AllowableValue(routeAnyMatches, // keep the word 'match' instead of 'matched' to maintain backward compatibility (there was a typo originally)
|
// keep the word 'match' instead of 'matched' to maintain backward compatibility (there was a typo originally)
|
||||||
|
public static final AllowableValue ROUTE_ANY_MATCHES = new AllowableValue(routeAnyMatches,
|
||||||
"Route to 'matched' if any matches",
|
"Route to 'matched' if any matches",
|
||||||
"Requires that at least one user-defined expression evaluate to 'true' for hte FlowFile to be considered a match");
|
"Requires that at least one user-defined expression evaluate to 'true' for hte FlowFile to be considered a match");
|
||||||
|
|
||||||
|
@ -243,8 +244,7 @@ public class RouteOnAttribute extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
//now transfer the original flow file
|
//now transfer the original flow file
|
||||||
logger.
|
logger.info("Routing {} to {}", new Object[]{flowFile, firstRelationship});
|
||||||
info("Routing {} to {}", new Object[]{flowFile, firstRelationship});
|
|
||||||
session.getProvenanceReporter().route(flowFile, firstRelationship);
|
session.getProvenanceReporter().route(flowFile, firstRelationship);
|
||||||
flowFile = session.putAttribute(flowFile, ROUTE_ATTRIBUTE_KEY, firstRelationship.getName());
|
flowFile = session.putAttribute(flowFile, ROUTE_ATTRIBUTE_KEY, firstRelationship.getName());
|
||||||
session.transfer(flowFile, firstRelationship);
|
session.transfer(flowFile, firstRelationship);
|
||||||
|
|
|
@ -86,7 +86,10 @@ public class ScanAttribute extends AbstractProcessor {
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor DICTIONARY_FILTER = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor DICTIONARY_FILTER = new PropertyDescriptor.Builder()
|
||||||
.name("Dictionary Filter Pattern")
|
.name("Dictionary Filter Pattern")
|
||||||
.description("A Regular Expression that will be applied to each line in the dictionary file. If the regular expression does not match the line, the line will not be included in the list of terms to search for. If a Matching Group is specified, only the portion of the term that matches that Matching Group will be used instead of the entire term. If not specified, all terms in the dictionary will be used and each term will consist of the text of the entire line in the file")
|
.description("A Regular Expression that will be applied to each line in the dictionary file. If the regular expression does not "
|
||||||
|
+ "match the line, the line will not be included in the list of terms to search for. If a Matching Group is specified, only the "
|
||||||
|
+ "portion of the term that matches that Matching Group will be used instead of the entire term. If not specified, all terms in "
|
||||||
|
+ "the dictionary will be used and each term will consist of the text of the entire line in the file")
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(StandardValidators.createRegexValidator(0, 1, false))
|
.addValidator(StandardValidators.createRegexValidator(0, 1, false))
|
||||||
.defaultValue(null)
|
.defaultValue(null)
|
||||||
|
|
|
@ -225,8 +225,7 @@ public class SplitContent extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
|
|
||||||
bytesRead++;
|
bytesRead++;
|
||||||
boolean matched = buffer.
|
boolean matched = buffer.addAndCompare((byte) (nextByte & 0xFF));
|
||||||
addAndCompare((byte) (nextByte & 0xFF));
|
|
||||||
if (matched) {
|
if (matched) {
|
||||||
long splitLength;
|
long splitLength;
|
||||||
|
|
||||||
|
@ -255,8 +254,7 @@ public class SplitContent extends AbstractProcessor {
|
||||||
FlowFile clone = session.clone(flowFile);
|
FlowFile clone = session.clone(flowFile);
|
||||||
session.transfer(flowFile, REL_ORIGINAL);
|
session.transfer(flowFile, REL_ORIGINAL);
|
||||||
session.transfer(clone, REL_SPLITS);
|
session.transfer(clone, REL_SPLITS);
|
||||||
logger.
|
logger.info("Found no match for {}; transferring original 'original' and transferring clone {} to 'splits'", new Object[]{flowFile, clone});
|
||||||
info("Found no match for {}; transferring original 'original' and transferring clone {} to 'splits'", new Object[]{flowFile, clone});
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -303,8 +301,7 @@ public class SplitContent extends AbstractProcessor {
|
||||||
* @param splits splits
|
* @param splits splits
|
||||||
*/
|
*/
|
||||||
private void finishFragmentAttributes(final ProcessSession session, final FlowFile source, final List<FlowFile> splits) {
|
private void finishFragmentAttributes(final ProcessSession session, final FlowFile source, final List<FlowFile> splits) {
|
||||||
final String originalFilename = source.
|
final String originalFilename = source.getAttribute(CoreAttributes.FILENAME.key());
|
||||||
getAttribute(CoreAttributes.FILENAME.key());
|
|
||||||
|
|
||||||
final String fragmentId = UUID.randomUUID().toString();
|
final String fragmentId = UUID.randomUUID().toString();
|
||||||
final ArrayList<FlowFile> newList = new ArrayList<>(splits);
|
final ArrayList<FlowFile> newList = new ArrayList<>(splits);
|
||||||
|
|
|
@ -140,15 +140,6 @@ public class SplitText extends AbstractProcessor {
|
||||||
return properties;
|
return properties;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Reads up to the given maximum number of lines, copying them to out
|
|
||||||
*
|
|
||||||
* @param in
|
|
||||||
* @param maxNumLines
|
|
||||||
* @param out
|
|
||||||
* @return the number of lines actually copied
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines) throws IOException {
|
private int readLines(final InputStream in, final int maxNumLines, final OutputStream out, final boolean keepAllNewLines) throws IOException {
|
||||||
int numLines = 0;
|
int numLines = 0;
|
||||||
for (int i = 0; i < maxNumLines; i++) {
|
for (int i = 0; i < maxNumLines; i++) {
|
||||||
|
@ -279,7 +270,7 @@ public class SplitText extends AbstractProcessor {
|
||||||
if (linesCopied.get() > 0) {
|
if (linesCopied.get() > 0) {
|
||||||
splits.add(splitFile);
|
splits.add(splitFile);
|
||||||
} else {
|
} else {
|
||||||
// if the number of content lines is a multiple of the SPLIT_LINE_COUNT,
|
// if the number of content lines is a multiple of the SPLIT_LINE_COUNT,
|
||||||
// the last flow file will contain just a header; don't forward that one
|
// the last flow file will contain just a header; don't forward that one
|
||||||
session.remove(splitFile);
|
session.remove(splitFile);
|
||||||
}
|
}
|
||||||
|
@ -341,13 +332,6 @@ public class SplitText extends AbstractProcessor {
|
||||||
session.transfer(splits, REL_SPLITS);
|
session.transfer(splits, REL_SPLITS);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Apply split index, count and other attributes.
|
|
||||||
*
|
|
||||||
* @param session
|
|
||||||
* @param source
|
|
||||||
* @param unpacked
|
|
||||||
*/
|
|
||||||
private void finishFragmentAttributes(final ProcessSession session, final FlowFile source, final List<FlowFile> splits) {
|
private void finishFragmentAttributes(final ProcessSession session, final FlowFile source, final List<FlowFile> splits) {
|
||||||
final String originalFilename = source.getAttribute(CoreAttributes.FILENAME.key());
|
final String originalFilename = source.getAttribute(CoreAttributes.FILENAME.key());
|
||||||
|
|
||||||
|
|
|
@ -68,7 +68,8 @@ public class SplitXml extends AbstractProcessor {
|
||||||
|
|
||||||
public static final PropertyDescriptor SPLIT_DEPTH = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor SPLIT_DEPTH = new PropertyDescriptor.Builder()
|
||||||
.name("Split Depth")
|
.name("Split Depth")
|
||||||
.description("Indicates the XML-nesting depth to start splitting XML fragments. A depth of 1 means split the root's children, whereas a depth of 2 means split the root's children's children and so forth.")
|
.description("Indicates the XML-nesting depth to start splitting XML fragments. A depth of 1 means split the root's children, whereas a depth of"
|
||||||
|
+ " 2 means split the root's children's children and so forth.")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||||
.defaultValue("1")
|
.defaultValue("1")
|
||||||
|
|
|
@ -132,35 +132,33 @@ public class TransformXml extends AbstractProcessor {
|
||||||
final StopWatch stopWatch = new StopWatch(true);
|
final StopWatch stopWatch = new StopWatch(true);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
FlowFile transformed = session.
|
FlowFile transformed = session.write(original, new StreamCallback() {
|
||||||
write(original, new StreamCallback() {
|
@Override
|
||||||
@Override
|
public void process(final InputStream rawIn, final OutputStream out) throws IOException {
|
||||||
public void process(final InputStream rawIn, final OutputStream out) throws IOException {
|
try (final InputStream in = new BufferedInputStream(rawIn)) {
|
||||||
try (final InputStream in = new BufferedInputStream(rawIn)) {
|
|
||||||
|
|
||||||
File stylesheet = new File(context.getProperty(XSLT_FILE_NAME).getValue());
|
File stylesheet = new File(context.getProperty(XSLT_FILE_NAME).getValue());
|
||||||
StreamSource styleSource = new StreamSource(stylesheet);
|
StreamSource styleSource = new StreamSource(stylesheet);
|
||||||
TransformerFactory tfactory = new net.sf.saxon.TransformerFactoryImpl();
|
TransformerFactory tfactory = new net.sf.saxon.TransformerFactoryImpl();
|
||||||
Transformer transformer = tfactory.newTransformer(styleSource);
|
Transformer transformer = tfactory.newTransformer(styleSource);
|
||||||
|
|
||||||
// pass all dynamic properties to the transformer
|
// pass all dynamic properties to the transformer
|
||||||
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().
|
for (final Map.Entry<PropertyDescriptor, String> entry : context.getProperties().entrySet()) {
|
||||||
entrySet()) {
|
if (entry.getKey().isDynamic()) {
|
||||||
if (entry.getKey().isDynamic()) {
|
String value = context.newPropertyValue(entry.getValue()).evaluateAttributeExpressions(original).getValue();
|
||||||
String value = context.newPropertyValue(entry.getValue()).evaluateAttributeExpressions(original).getValue();
|
transformer.setParameter(entry.getKey().getName(), value);
|
||||||
transformer.setParameter(entry.getKey().getName(), value);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// use a StreamSource with Saxon
|
|
||||||
StreamSource source = new StreamSource(in);
|
|
||||||
StreamResult result = new StreamResult(out);
|
|
||||||
transformer.transform(source, result);
|
|
||||||
} catch (final Exception e) {
|
|
||||||
throw new IOException(e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
|
||||||
|
// use a StreamSource with Saxon
|
||||||
|
StreamSource source = new StreamSource(in);
|
||||||
|
StreamResult result = new StreamResult(out);
|
||||||
|
transformer.transform(source, result);
|
||||||
|
} catch (final Exception e) {
|
||||||
|
throw new IOException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
session.transfer(transformed, REL_SUCCESS);
|
session.transfer(transformed, REL_SUCCESS);
|
||||||
session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
session.getProvenanceReporter().modifyContent(transformed, stopWatch.getElapsed(TimeUnit.MILLISECONDS));
|
||||||
logger.info("Transformed {}", new Object[]{original});
|
logger.info("Transformed {}", new Object[]{original});
|
||||||
|
|
|
@ -68,14 +68,24 @@ import org.apache.nifi.util.ObjectHolder;
|
||||||
@SideEffectFree
|
@SideEffectFree
|
||||||
@SupportsBatching
|
@SupportsBatching
|
||||||
@Tags({"Unpack", "un-merge", "tar", "zip", "archive", "flowfile-stream", "flowfile-stream-v3"})
|
@Tags({"Unpack", "un-merge", "tar", "zip", "archive", "flowfile-stream", "flowfile-stream-v3"})
|
||||||
@CapabilityDescription("Unpacks the content of FlowFiles that have been packaged with one of several different Packaging Formats, emitting one to many FlowFiles for each input FlowFile")
|
@CapabilityDescription("Unpacks the content of FlowFiles that have been packaged with one of several different Packaging Formats, emitting one to many "
|
||||||
@ReadsAttribute(attribute = "mime.type", description = "If the <Packaging Format> property is set to use mime.type attribute, this attribute is used to determine the FlowFile's MIME Type. In this case, if the attribute is set to application/tar, the TAR Packaging Format will be used. If the attribute is set to application/zip, the ZIP Packaging Format will be used. If the attribute is set to application/flowfile-v3 or application/flowfile-v2 or application/flowfile-v1, the appropriate FlowFile Packaging Format will be used. If this attribute is missing, the FlowFile will be routed to 'failure'. Otherwise, if the attribute's value is not one of those mentioned above, the FlowFile will be routed to 'success' without being unpacked")
|
+ "FlowFiles for each input FlowFile")
|
||||||
|
@ReadsAttribute(attribute = "mime.type", description = "If the <Packaging Format> property is set to use mime.type attribute, this attribute is used "
|
||||||
|
+ "to determine the FlowFile's MIME Type. In this case, if the attribute is set to application/tar, the TAR Packaging Format will be used. If "
|
||||||
|
+ "the attribute is set to application/zip, the ZIP Packaging Format will be used. If the attribute is set to application/flowfile-v3 or "
|
||||||
|
+ "application/flowfile-v2 or application/flowfile-v1, the appropriate FlowFile Packaging Format will be used. If this attribute is missing, "
|
||||||
|
+ "the FlowFile will be routed to 'failure'. Otherwise, if the attribute's value is not one of those mentioned above, the FlowFile will be "
|
||||||
|
+ "routed to 'success' without being unpacked")
|
||||||
@WritesAttributes({
|
@WritesAttributes({
|
||||||
@WritesAttribute(attribute = "mime.type", description = "If the FlowFile is successfully unpacked, its MIME Type is no longer known, so the mime.type attribute is set to application/octet-stream."),
|
@WritesAttribute(attribute = "mime.type", description = "If the FlowFile is successfully unpacked, its MIME Type is no longer known, so the mime.type "
|
||||||
@WritesAttribute(attribute = "fragment.identifier", description = "All unpacked FlowFiles produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute"),
|
+ "attribute is set to application/octet-stream."),
|
||||||
@WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the unpacked FlowFiles that were created from a single parent FlowFile"),
|
@WritesAttribute(attribute = "fragment.identifier", description = "All unpacked FlowFiles produced from the same parent FlowFile will have the same randomly generated "
|
||||||
|
+ "UUID added for this attribute"),
|
||||||
|
@WritesAttribute(attribute = "fragment.index", description = "A one-up number that indicates the ordering of the unpacked FlowFiles that were created from a single "
|
||||||
|
+ "parent FlowFile"),
|
||||||
@WritesAttribute(attribute = "fragment.count", description = "The number of unpacked FlowFiles generated from the parent FlowFile"),
|
@WritesAttribute(attribute = "fragment.count", description = "The number of unpacked FlowFiles generated from the parent FlowFile"),
|
||||||
@WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile. Extensions of .tar, .zip or .pkg are removed because the MergeContent processor automatically adds those extensions if it is used to rebuild the original FlowFile")})
|
@WritesAttribute(attribute = "segment.original.filename ", description = "The filename of the parent FlowFile. Extensions of .tar, .zip or .pkg are removed because "
|
||||||
|
+ "the MergeContent processor automatically adds those extensions if it is used to rebuild the original FlowFile")})
|
||||||
@SeeAlso(MergeContent.class)
|
@SeeAlso(MergeContent.class)
|
||||||
public class UnpackContent extends AbstractProcessor {
|
public class UnpackContent extends AbstractProcessor {
|
||||||
|
|
||||||
|
@ -380,8 +390,7 @@ public class UnpackContent extends AbstractProcessor {
|
||||||
mapAttributes(attributes, "content-encoding", CoreAttributes.MIME_TYPE.key());
|
mapAttributes(attributes, "content-encoding", CoreAttributes.MIME_TYPE.key());
|
||||||
mapAttributes(attributes, "content-type", CoreAttributes.MIME_TYPE.key());
|
mapAttributes(attributes, "content-type", CoreAttributes.MIME_TYPE.key());
|
||||||
|
|
||||||
if (!attributes.
|
if (!attributes.containsKey(CoreAttributes.MIME_TYPE.key())) {
|
||||||
containsKey(CoreAttributes.MIME_TYPE.key())) {
|
|
||||||
attributes.put(CoreAttributes.MIME_TYPE.key(), OCTET_STREAM);
|
attributes.put(CoreAttributes.MIME_TYPE.key(), OCTET_STREAM);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -396,26 +405,12 @@ public class UnpackContent extends AbstractProcessor {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Maps attributes from legacy nifi to the new naming scheme
|
|
||||||
*
|
|
||||||
* @param attributes
|
|
||||||
* @param oldKey
|
|
||||||
* @param newKey
|
|
||||||
*/
|
|
||||||
private static void mapAttributes(final Map<String, String> attributes, final String oldKey, final String newKey) {
|
private static void mapAttributes(final Map<String, String> attributes, final String oldKey, final String newKey) {
|
||||||
if (!attributes.containsKey(newKey) && attributes.containsKey(oldKey)) {
|
if (!attributes.containsKey(newKey) && attributes.containsKey(oldKey)) {
|
||||||
attributes.put(newKey, attributes.get(oldKey));
|
attributes.put(newKey, attributes.get(oldKey));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* If the unpacked flowfiles contain fragment index attributes, then we need to apply fragment count and other attributes for completeness.
|
|
||||||
*
|
|
||||||
* @param session
|
|
||||||
* @param source
|
|
||||||
* @param unpacked
|
|
||||||
*/
|
|
||||||
private void finishFragmentAttributes(final ProcessSession session, final FlowFile source, final List<FlowFile> unpacked) {
|
private void finishFragmentAttributes(final ProcessSession session, final FlowFile source, final List<FlowFile> unpacked) {
|
||||||
// first pass verifies all FlowFiles have the FRAGMENT_INDEX attribute and gets the total number of fragments
|
// first pass verifies all FlowFiles have the FRAGMENT_INDEX attribute and gets the total number of fragments
|
||||||
int fragmentCount = 0;
|
int fragmentCount = 0;
|
||||||
|
|
|
@ -58,21 +58,21 @@ import org.xml.sax.SAXException;
|
||||||
@CapabilityDescription("Validates the contents of FlowFiles against a user-specified XML Schema file")
|
@CapabilityDescription("Validates the contents of FlowFiles against a user-specified XML Schema file")
|
||||||
public class ValidateXml extends AbstractProcessor {
|
public class ValidateXml extends AbstractProcessor {
|
||||||
|
|
||||||
public static final PropertyDescriptor SCHEMA_FILE = new PropertyDescriptor.Builder().
|
public static final PropertyDescriptor SCHEMA_FILE = new PropertyDescriptor.Builder()
|
||||||
name("Schema File").
|
.name("Schema File")
|
||||||
description("The path to the Schema file that is to be used for validation").
|
.description("The path to the Schema file that is to be used for validation")
|
||||||
required(true).
|
.required(true)
|
||||||
addValidator(StandardValidators.FILE_EXISTS_VALIDATOR).
|
.addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
|
||||||
build();
|
.build();
|
||||||
|
|
||||||
public static final Relationship REL_VALID = new Relationship.Builder().
|
public static final Relationship REL_VALID = new Relationship.Builder()
|
||||||
name("valid").
|
.name("valid")
|
||||||
description("FlowFiles that are successfully validated against the schema are routed to this relationship").
|
.description("FlowFiles that are successfully validated against the schema are routed to this relationship")
|
||||||
build();
|
.build();
|
||||||
public static final Relationship REL_INVALID = new Relationship.Builder().
|
public static final Relationship REL_INVALID = new Relationship.Builder()
|
||||||
name("invalid").
|
.name("invalid")
|
||||||
description("FlowFiles that are not valid according to the specified schema are routed to this relationship").
|
.description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
|
||||||
build();
|
.build();
|
||||||
|
|
||||||
private static final String SCHEMA_LANGUAGE = "http://www.w3.org/2001/XMLSchema";
|
private static final String SCHEMA_LANGUAGE = "http://www.w3.org/2001/XMLSchema";
|
||||||
|
|
||||||
|
@ -105,10 +105,8 @@ public class ValidateXml extends AbstractProcessor {
|
||||||
@OnScheduled
|
@OnScheduled
|
||||||
public void parseSchema(final ProcessContext context) throws IOException, SAXException {
|
public void parseSchema(final ProcessContext context) throws IOException, SAXException {
|
||||||
try {
|
try {
|
||||||
final File file = new File(context.getProperty(SCHEMA_FILE).
|
final File file = new File(context.getProperty(SCHEMA_FILE).getValue());
|
||||||
getValue());
|
final SchemaFactory schemaFactory = SchemaFactory.newInstance(SCHEMA_LANGUAGE);
|
||||||
final SchemaFactory schemaFactory = SchemaFactory.
|
|
||||||
newInstance(SCHEMA_LANGUAGE);
|
|
||||||
final Schema schema = schemaFactory.newSchema(file);
|
final Schema schema = schemaFactory.newSchema(file);
|
||||||
this.schemaRef.set(schema);
|
this.schemaRef.set(schema);
|
||||||
} catch (final SAXException e) {
|
} catch (final SAXException e) {
|
||||||
|
@ -136,23 +134,18 @@ public class ValidateXml extends AbstractProcessor {
|
||||||
validator.validate(new StreamSource(in));
|
validator.validate(new StreamSource(in));
|
||||||
} catch (final IllegalArgumentException | SAXException e) {
|
} catch (final IllegalArgumentException | SAXException e) {
|
||||||
valid.set(false);
|
valid.set(false);
|
||||||
logger.
|
logger.debug("Failed to validate {} against schema due to {}", new Object[]{flowFile, e});
|
||||||
debug("Failed to validate {} against schema due to {}", new Object[]{flowFile, e});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
if (valid.get()) {
|
if (valid.get()) {
|
||||||
logger.
|
logger.info("Successfully validated {} against schema; routing to 'valid'", new Object[]{flowFile});
|
||||||
info("Successfully validated {} against schema; routing to 'valid'", new Object[]{flowFile});
|
session.getProvenanceReporter().route(flowFile, REL_VALID);
|
||||||
session.getProvenanceReporter().
|
|
||||||
route(flowFile, REL_VALID);
|
|
||||||
session.transfer(flowFile, REL_VALID);
|
session.transfer(flowFile, REL_VALID);
|
||||||
} else {
|
} else {
|
||||||
logger.
|
logger.info("Failed to validate {} against schema; routing to 'invalid'", new Object[]{flowFile});
|
||||||
info("Failed to validate {} against schema; routing to 'invalid'", new Object[]{flowFile});
|
session.getProvenanceReporter().route(flowFile, REL_INVALID);
|
||||||
session.getProvenanceReporter().
|
|
||||||
route(flowFile, REL_INVALID);
|
|
||||||
session.transfer(flowFile, REL_INVALID);
|
session.transfer(flowFile, REL_INVALID);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -50,11 +50,6 @@ public class ContentAcknowledgmentServlet extends HttpServlet {
|
||||||
private ProcessorLog logger;
|
private ProcessorLog logger;
|
||||||
private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
|
private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* @param config
|
|
||||||
* @throws ServletException
|
|
||||||
*/
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Override
|
@Override
|
||||||
public void init(final ServletConfig config) throws ServletException {
|
public void init(final ServletConfig config) throws ServletException {
|
||||||
|
|
|
@ -94,11 +94,6 @@ public class ListenHTTPServlet extends HttpServlet {
|
||||||
private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
|
private ConcurrentMap<String, FlowFileEntryTimeWrapper> flowFileMap;
|
||||||
private StreamThrottler streamThrottler;
|
private StreamThrottler streamThrottler;
|
||||||
|
|
||||||
/**
|
|
||||||
*
|
|
||||||
* @param config
|
|
||||||
* @throws ServletException
|
|
||||||
*/
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Override
|
@Override
|
||||||
public void init(final ServletConfig config) throws ServletException {
|
public void init(final ServletConfig config) throws ServletException {
|
||||||
|
|
|
@ -41,11 +41,11 @@ public class Bin {
|
||||||
/**
|
/**
|
||||||
* Constructs a new bin
|
* Constructs a new bin
|
||||||
*
|
*
|
||||||
* @param minSizeBytes
|
* @param minSizeBytes min bytes
|
||||||
* @param maxSizeBytes
|
* @param maxSizeBytes max bytes
|
||||||
* @param minEntries
|
* @param minEntries min entries
|
||||||
* @param maxEntries
|
* @param maxEntries max entries
|
||||||
* @param fileCountAttribute
|
* @param fileCountAttribute num files
|
||||||
* @throws IllegalArgumentException if the min is not less than or equal to the max.
|
* @throws IllegalArgumentException if the min is not less than or equal to the max.
|
||||||
*/
|
*/
|
||||||
public Bin(final long minSizeBytes, final long maxSizeBytes, final int minEntries, final int maxEntries, final String fileCountAttribute) {
|
public Bin(final long minSizeBytes, final long maxSizeBytes, final int minEntries, final int maxEntries, final String fileCountAttribute) {
|
||||||
|
@ -75,7 +75,7 @@ public class Bin {
|
||||||
/**
|
/**
|
||||||
* Indicates enough size exists to meet the minimum requirements
|
* Indicates enough size exists to meet the minimum requirements
|
||||||
*
|
*
|
||||||
* @return
|
* @return true if full enough
|
||||||
*/
|
*/
|
||||||
public boolean isFullEnough() {
|
public boolean isFullEnough() {
|
||||||
return isFull() || (size >= minimumSizeBytes && (binContents.size() >= minimumEntries));
|
return isFull() || (size >= minimumSizeBytes && (binContents.size() >= minimumEntries));
|
||||||
|
@ -84,8 +84,8 @@ public class Bin {
|
||||||
/**
|
/**
|
||||||
* Determines if this bin is older than the time specified.
|
* Determines if this bin is older than the time specified.
|
||||||
*
|
*
|
||||||
* @param duration
|
* @param duration duration
|
||||||
* @param unit
|
* @param unit unit
|
||||||
* @return true if this bin is older than the length of time given; false otherwise
|
* @return true if this bin is older than the length of time given; false otherwise
|
||||||
*/
|
*/
|
||||||
public boolean isOlderThan(final int duration, final TimeUnit unit) {
|
public boolean isOlderThan(final int duration, final TimeUnit unit) {
|
||||||
|
@ -96,8 +96,8 @@ public class Bin {
|
||||||
/**
|
/**
|
||||||
* Determines if this bin is older than the specified bin
|
* Determines if this bin is older than the specified bin
|
||||||
*
|
*
|
||||||
* @param other
|
* @param other other bin
|
||||||
* @return
|
* @return true if this is older than given bin
|
||||||
*/
|
*/
|
||||||
public boolean isOlderThan(final Bin other) {
|
public boolean isOlderThan(final Bin other) {
|
||||||
return creationMomentEpochNs < other.creationMomentEpochNs;
|
return creationMomentEpochNs < other.creationMomentEpochNs;
|
||||||
|
@ -106,7 +106,7 @@ public class Bin {
|
||||||
/**
|
/**
|
||||||
* If this bin has enough room for the size of the given flow file then it is added otherwise it is not
|
* If this bin has enough room for the size of the given flow file then it is added otherwise it is not
|
||||||
*
|
*
|
||||||
* @param flowFile
|
* @param flowFile flowfile to offer
|
||||||
* @param session the ProcessSession to which the FlowFile belongs
|
* @param session the ProcessSession to which the FlowFile belongs
|
||||||
* @return true if added; false otherwise
|
* @return true if added; false otherwise
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -152,7 +152,7 @@ public class BinManager {
|
||||||
* <p/>
|
* <p/>
|
||||||
* @param relaxFullnessConstraint if false will require bins to be full before considered ready; if true bins only have to meet their minimum size criteria or be 'old' and then they'll be
|
* @param relaxFullnessConstraint if false will require bins to be full before considered ready; if true bins only have to meet their minimum size criteria or be 'old' and then they'll be
|
||||||
* considered ready
|
* considered ready
|
||||||
* @return
|
* @return bins that are considered full
|
||||||
*/
|
*/
|
||||||
public Collection<Bin> removeReadyBins(boolean relaxFullnessConstraint) {
|
public Collection<Bin> removeReadyBins(boolean relaxFullnessConstraint) {
|
||||||
final Map<String, List<Bin>> newGroupMap = new HashMap<>();
|
final Map<String, List<Bin>> newGroupMap = new HashMap<>();
|
||||||
|
|
|
@ -42,12 +42,6 @@ public class DocumentReaderCallback implements InputStreamCallback {
|
||||||
this.isNamespaceAware = isNamespaceAware;
|
this.isNamespaceAware = isNamespaceAware;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Loads the Document from the specified stream.
|
|
||||||
*
|
|
||||||
* @param stream
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
@Override
|
@Override
|
||||||
public void process(final InputStream stream) throws IOException {
|
public void process(final InputStream stream) throws IOException {
|
||||||
try {
|
try {
|
||||||
|
@ -63,9 +57,7 @@ public class DocumentReaderCallback implements InputStreamCallback {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the document.
|
* @return the document
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
*/
|
||||||
public Document getDocument() {
|
public Document getDocument() {
|
||||||
return document;
|
return document;
|
||||||
|
|
|
@ -305,8 +305,7 @@ public class FTPTransfer implements FileTransfer {
|
||||||
final FTPFile[] files = client.listFiles(path);
|
final FTPFile[] files = client.listFiles(path);
|
||||||
FTPFile matchingFile = null;
|
FTPFile matchingFile = null;
|
||||||
for (final FTPFile file : files) {
|
for (final FTPFile file : files) {
|
||||||
if (file.getName().
|
if (file.getName().equalsIgnoreCase(remoteFileName)) {
|
||||||
equalsIgnoreCase(remoteFileName)) {
|
|
||||||
matchingFile = file;
|
matchingFile = file;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
|
@ -75,7 +75,7 @@ public class FTPUtils {
|
||||||
* value of zero means do not timeout. Users should probably set a value here unless using very reliable communications links or else risk indefinite hangs that require a restart.</li>
|
* value of zero means do not timeout. Users should probably set a value here unless using very reliable communications links or else risk indefinite hangs that require a restart.</li>
|
||||||
* </ul>
|
* </ul>
|
||||||
*
|
*
|
||||||
* @param conf
|
* @param conf conf
|
||||||
* @param monitor if provided will be used to monitor FTP commands processed but may be null
|
* @param monitor if provided will be used to monitor FTP commands processed but may be null
|
||||||
* @return FTPClient connected to FTP server as configured
|
* @return FTPClient connected to FTP server as configured
|
||||||
* @throws NullPointerException if either argument is null
|
* @throws NullPointerException if either argument is null
|
||||||
|
|
|
@ -136,7 +136,10 @@ public interface FileTransfer extends Closeable {
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor REMOTE_POLL_BATCH_SIZE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor REMOTE_POLL_BATCH_SIZE = new PropertyDescriptor.Builder()
|
||||||
.name("Remote Poll Batch Size")
|
.name("Remote Poll Batch Size")
|
||||||
.description("The value specifies how many file paths to find in a given directory on the remote system when doing a file listing. This value in general should not need to be modified but when polling against a remote system with a tremendous number of files this value can be critical. Setting this value too high can result very poor performance and setting it too low can cause the flow to be slower than normal.")
|
.description("The value specifies how many file paths to find in a given directory on the remote system when doing a file listing. This value "
|
||||||
|
+ "in general should not need to be modified but when polling against a remote system with a tremendous number of files this value can "
|
||||||
|
+ "be critical. Setting this value too high can result very poor performance and setting it too low can cause the flow to be slower "
|
||||||
|
+ "than normal.")
|
||||||
.defaultValue("5000")
|
.defaultValue("5000")
|
||||||
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR)
|
||||||
.required(true)
|
.required(true)
|
||||||
|
@ -194,41 +197,53 @@ public interface FileTransfer extends Closeable {
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor DOT_RENAME = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor DOT_RENAME = new PropertyDescriptor.Builder()
|
||||||
.name("Dot Rename")
|
.name("Dot Rename")
|
||||||
.description("If true, then the filename of the sent file is prepended with a \".\" and then renamed back to the original once the file is completely sent. Otherwise, there is no rename. This property is ignored if the Temporary Filename property is set.")
|
.description("If true, then the filename of the sent file is prepended with a \".\" and then renamed back to the "
|
||||||
|
+ "original once the file is completely sent. Otherwise, there is no rename. This property is ignored if the "
|
||||||
|
+ "Temporary Filename property is set.")
|
||||||
.allowableValues("true", "false")
|
.allowableValues("true", "false")
|
||||||
.defaultValue("true")
|
.defaultValue("true")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor TEMP_FILENAME = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor TEMP_FILENAME = new PropertyDescriptor.Builder()
|
||||||
.name("Temporary Filename")
|
.name("Temporary Filename")
|
||||||
.description("If set, the filename of the sent file will be equal to the value specified during the transfer and after successful completion will be renamed to the original filename. If this value is set, the Dot Rename property is ignored.")
|
.description("If set, the filename of the sent file will be equal to the value specified during the transfer and after successful "
|
||||||
|
+ "completion will be renamed to the original filename. If this value is set, the Dot Rename property is ignored.")
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.expressionLanguageSupported(true)
|
.expressionLanguageSupported(true)
|
||||||
.required(false)
|
.required(false)
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor LAST_MODIFIED_TIME = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor LAST_MODIFIED_TIME = new PropertyDescriptor.Builder()
|
||||||
.name("Last Modified Time")
|
.name("Last Modified Time")
|
||||||
.description("The lastModifiedTime to assign to the file after transferring it. If not set, the lastModifiedTime will not be changed. Format must be yyyy-MM-dd'T'HH:mm:ssZ. You may also use expression language such as ${file.lastModifiedTime}. If the value is invalid, the processor will not be invalid but will fail to change lastModifiedTime of the file.")
|
.description("The lastModifiedTime to assign to the file after transferring it. If not set, the lastModifiedTime will not be changed. "
|
||||||
|
+ "Format must be yyyy-MM-dd'T'HH:mm:ssZ. You may also use expression language such as ${file.lastModifiedTime}. If the value "
|
||||||
|
+ "is invalid, the processor will not be invalid but will fail to change lastModifiedTime of the file.")
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.expressionLanguageSupported(true)
|
.expressionLanguageSupported(true)
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor PERMISSIONS = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor PERMISSIONS = new PropertyDescriptor.Builder()
|
||||||
.name("Permissions")
|
.name("Permissions")
|
||||||
.description("The permissions to assign to the file after transferring it. Format must be either UNIX rwxrwxrwx with a - in place of denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). If not set, the permissions will not be changed. You may also use expression language such as ${file.permissions}. If the value is invalid, the processor will not be invalid but will fail to change permissions of the file.")
|
.description("The permissions to assign to the file after transferring it. Format must be either UNIX rwxrwxrwx with a - in place of "
|
||||||
|
+ "denied permissions (e.g. rw-r--r--) or an octal number (e.g. 644). If not set, the permissions will not be changed. You may "
|
||||||
|
+ "also use expression language such as ${file.permissions}. If the value is invalid, the processor will not be invalid but will "
|
||||||
|
+ "fail to change permissions of the file.")
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.expressionLanguageSupported(true)
|
.expressionLanguageSupported(true)
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor REMOTE_OWNER = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor REMOTE_OWNER = new PropertyDescriptor.Builder()
|
||||||
.name("Remote Owner")
|
.name("Remote Owner")
|
||||||
.description("Integer value representing the User ID to set on the file after transferring it. If not set, the owner will not be set. You may also use expression language such as ${file.owner}. If the value is invalid, the processor will not be invalid but will fail to change the owner of the file.")
|
.description("Integer value representing the User ID to set on the file after transferring it. If not set, the owner will not be set. "
|
||||||
|
+ "You may also use expression language such as ${file.owner}. If the value is invalid, the processor will not be invalid but "
|
||||||
|
+ "will fail to change the owner of the file.")
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.expressionLanguageSupported(true)
|
.expressionLanguageSupported(true)
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor REMOTE_GROUP = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor REMOTE_GROUP = new PropertyDescriptor.Builder()
|
||||||
.name("Remote Group")
|
.name("Remote Group")
|
||||||
.description("Integer value representing the Group ID to set on the file after transferring it. If not set, the group will not be set. You may also use expression language such as ${file.group}. If the value is invalid, the processor will not be invalid but will fail to change the group of the file.")
|
.description("Integer value representing the Group ID to set on the file after transferring it. If not set, the group will not be set. "
|
||||||
|
+ "You may also use expression language such as ${file.group}. If the value is invalid, the processor will not be invalid but "
|
||||||
|
+ "will fail to change the group of the file.")
|
||||||
.required(false)
|
.required(false)
|
||||||
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
.addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
|
||||||
.expressionLanguageSupported(true)
|
.expressionLanguageSupported(true)
|
||||||
|
|
|
@ -149,8 +149,7 @@ public class SFTPTransfer implements FileTransfer {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
final boolean ignoreDottedFiles = ctx.
|
final boolean ignoreDottedFiles = ctx.getProperty(FileTransfer.IGNORE_DOTTED_FILES).asBoolean();
|
||||||
getProperty(FileTransfer.IGNORE_DOTTED_FILES).asBoolean();
|
|
||||||
final boolean recurse = ctx.getProperty(FileTransfer.RECURSIVE_SEARCH).asBoolean();
|
final boolean recurse = ctx.getProperty(FileTransfer.RECURSIVE_SEARCH).asBoolean();
|
||||||
final String fileFilterRegex = ctx.getProperty(FileTransfer.FILE_FILTER_REGEX).getValue();
|
final String fileFilterRegex = ctx.getProperty(FileTransfer.FILE_FILTER_REGEX).getValue();
|
||||||
final Pattern pattern = (fileFilterRegex == null) ? null : Pattern.compile(fileFilterRegex);
|
final Pattern pattern = (fileFilterRegex == null) ? null : Pattern.compile(fileFilterRegex);
|
||||||
|
@ -234,8 +233,7 @@ public class SFTPTransfer implements FileTransfer {
|
||||||
try {
|
try {
|
||||||
getListing(newFullForwardPath, depth + 1, maxResults, listing);
|
getListing(newFullForwardPath, depth + 1, maxResults, listing);
|
||||||
} catch (final IOException e) {
|
} catch (final IOException e) {
|
||||||
logger.
|
logger.error("Unable to get listing from " + newFullForwardPath + "; skipping this subdirectory");
|
||||||
error("Unable to get listing from " + newFullForwardPath + "; skipping this subdirectory");
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -310,8 +308,7 @@ public class SFTPTransfer implements FileTransfer {
|
||||||
channel.mkdir(remoteDirectory);
|
channel.mkdir(remoteDirectory);
|
||||||
} catch (SftpException e) {
|
} catch (SftpException e) {
|
||||||
if (e.id != ChannelSftp.SSH_FX_FAILURE) {
|
if (e.id != ChannelSftp.SSH_FX_FAILURE) {
|
||||||
throw new IOException("Could not blindly create remote directory due to " + e.
|
throw new IOException("Could not blindly create remote directory due to " + e.getMessage(), e);
|
||||||
getMessage(), e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return;
|
return;
|
||||||
|
|
|
@ -71,9 +71,7 @@ public class XmlSplitterSaxParser extends DefaultHandler {
|
||||||
@Override
|
@Override
|
||||||
public void endElement(final String uri, final String localName, final String qName) throws SAXException {
|
public void endElement(final String uri, final String localName, final String qName) throws SAXException {
|
||||||
// Add the element end tag.
|
// Add the element end tag.
|
||||||
sb.append("</").
|
sb.append("</").append(qName).append(">");
|
||||||
append(qName).
|
|
||||||
append(">");
|
|
||||||
|
|
||||||
// We have finished processing this element. Decrement the depth.
|
// We have finished processing this element. Decrement the depth.
|
||||||
int newDepth = depth.decrementAndGet();
|
int newDepth = depth.decrementAndGet();
|
||||||
|
@ -104,12 +102,7 @@ public class XmlSplitterSaxParser extends DefaultHandler {
|
||||||
for (int i = 0; i < attCount; i++) {
|
for (int i = 0; i < attCount; i++) {
|
||||||
String attName = atts.getQName(i);
|
String attName = atts.getQName(i);
|
||||||
String attValue = atts.getValue(i);
|
String attValue = atts.getValue(i);
|
||||||
sb.append(" ").
|
sb.append(" ").append(attName).append("=").append("\"").append(attValue).append("\"");
|
||||||
append(attName).
|
|
||||||
append("=").
|
|
||||||
append("\"").
|
|
||||||
append(attValue).
|
|
||||||
append("\"");
|
|
||||||
}
|
}
|
||||||
|
|
||||||
sb.append(">");
|
sb.append(">");
|
||||||
|
|
|
@ -29,8 +29,7 @@ public class TestDistributeLoad {
|
||||||
public static void before() {
|
public static void before() {
|
||||||
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
|
System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info");
|
||||||
System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
|
System.setProperty("org.slf4j.simpleLogger.showDateTime", "true");
|
||||||
System.
|
System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DistributeLoad", "debug");
|
||||||
setProperty("org.slf4j.simpleLogger.log.nifi.processors.standard.DistributeLoad", "debug");
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -60,8 +60,8 @@ public class TestHandleHttpRequest {
|
||||||
public void run() {
|
public void run() {
|
||||||
try {
|
try {
|
||||||
final int port = ((HandleHttpRequest) runner.getProcessor()).getPort();
|
final int port = ((HandleHttpRequest) runner.getProcessor()).getPort();
|
||||||
final HttpURLConnection connection = (HttpURLConnection) new URL("http://localhost:" + port + "/my/path?query=true&value1=value1&value2=&value3&value4=apple=orange").
|
final HttpURLConnection connection = (HttpURLConnection) new URL("http://localhost:"
|
||||||
openConnection();
|
+ port + "/my/path?query=true&value1=value1&value2=&value3&value4=apple=orange").openConnection();
|
||||||
connection.setDoOutput(false);
|
connection.setDoOutput(false);
|
||||||
connection.setRequestMethod("GET");
|
connection.setRequestMethod("GET");
|
||||||
connection.setRequestProperty("header1", "value1");
|
connection.setRequestProperty("header1", "value1");
|
||||||
|
|
|
@ -27,20 +27,20 @@ import javax.net.ssl.SSLContext;
|
||||||
public interface CommsSession extends Closeable {
|
public interface CommsSession extends Closeable {
|
||||||
|
|
||||||
void setTimeout(final long value, final TimeUnit timeUnit);
|
void setTimeout(final long value, final TimeUnit timeUnit);
|
||||||
|
|
||||||
InputStream getInputStream() throws IOException;
|
InputStream getInputStream() throws IOException;
|
||||||
|
|
||||||
OutputStream getOutputStream() throws IOException;
|
OutputStream getOutputStream() throws IOException;
|
||||||
|
|
||||||
boolean isClosed();
|
boolean isClosed();
|
||||||
|
|
||||||
void interrupt();
|
void interrupt();
|
||||||
|
|
||||||
String getHostname();
|
String getHostname();
|
||||||
|
|
||||||
int getPort();
|
int getPort();
|
||||||
|
|
||||||
long getTimeout(TimeUnit timeUnit);
|
long getTimeout(TimeUnit timeUnit);
|
||||||
|
|
||||||
SSLContext getSSLContext();
|
SSLContext getSSLContext();
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,7 +42,7 @@ import org.apache.nifi.stream.io.DataOutputStream;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
@SeeAlso(classNames={"org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"})
|
@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"})
|
||||||
@CapabilityDescription("Provides the ability to communicate with a DistributedMapCacheServer. This can be used in order to share a Map "
|
@CapabilityDescription("Provides the ability to communicate with a DistributedMapCacheServer. This can be used in order to share a Map "
|
||||||
+ "between nodes in a NiFi cluster")
|
+ "between nodes in a NiFi cluster")
|
||||||
public class DistributedMapCacheClientService extends AbstractControllerService implements DistributedMapCacheClient {
|
public class DistributedMapCacheClientService extends AbstractControllerService implements DistributedMapCacheClient {
|
||||||
|
@ -65,14 +65,14 @@ public class DistributedMapCacheClientService extends AbstractControllerService
|
||||||
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
||||||
.name("SSL Context Service")
|
.name("SSL Context Service")
|
||||||
.description("If specified, indicates the SSL Context Service that is used to communicate with the "
|
.description("If specified, indicates the SSL Context Service that is used to communicate with the "
|
||||||
+ "remote server. If not specified, communications will not be encrypted")
|
+ "remote server. If not specified, communications will not be encrypted")
|
||||||
.required(false)
|
.required(false)
|
||||||
.identifiesControllerService(SSLContextService.class)
|
.identifiesControllerService(SSLContextService.class)
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder()
|
||||||
.name("Communications Timeout")
|
.name("Communications Timeout")
|
||||||
.description("Specifies how long to wait when communicating with the remote server before determining that "
|
.description("Specifies how long to wait when communicating with the remote server before determining that "
|
||||||
+ "there is a communications failure if data cannot be sent or received")
|
+ "there is a communications failure if data cannot be sent or received")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||||
.defaultValue("30 secs")
|
.defaultValue("30 secs")
|
||||||
|
@ -299,6 +299,7 @@ public class DistributedMapCacheClientService extends AbstractControllerService
|
||||||
}
|
}
|
||||||
|
|
||||||
private static interface CommsAction<T> {
|
private static interface CommsAction<T> {
|
||||||
|
|
||||||
T execute(CommsSession commsSession) throws IOException;
|
T execute(CommsSession commsSession) throws IOException;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -42,7 +42,7 @@ import org.apache.nifi.stream.io.DataOutputStream;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
@SeeAlso(classNames={"org.apache.nifi.distributed.cache.server.DistributedSetCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"})
|
@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.server.DistributedSetCacheServer", "org.apache.nifi.ssl.StandardSSLContextService"})
|
||||||
@CapabilityDescription("Provides the ability to communicate with a DistributedSetCacheServer. This can be used in order to share a Set "
|
@CapabilityDescription("Provides the ability to communicate with a DistributedSetCacheServer. This can be used in order to share a Set "
|
||||||
+ "between nodes in a NiFi cluster")
|
+ "between nodes in a NiFi cluster")
|
||||||
public class DistributedSetCacheClientService extends AbstractControllerService implements DistributedSetCacheClient {
|
public class DistributedSetCacheClientService extends AbstractControllerService implements DistributedSetCacheClient {
|
||||||
|
@ -65,14 +65,14 @@ public class DistributedSetCacheClientService extends AbstractControllerService
|
||||||
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
||||||
.name("SSL Context Service")
|
.name("SSL Context Service")
|
||||||
.description("If specified, indicates the SSL Context Service that is used to communicate with the "
|
.description("If specified, indicates the SSL Context Service that is used to communicate with the "
|
||||||
+ "remote server. If not specified, communications will not be encrypted")
|
+ "remote server. If not specified, communications will not be encrypted")
|
||||||
.required(false)
|
.required(false)
|
||||||
.identifiesControllerService(SSLContextService.class)
|
.identifiesControllerService(SSLContextService.class)
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder()
|
||||||
.name("Communications Timeout")
|
.name("Communications Timeout")
|
||||||
.description("Specifices how long to wait when communicating with the remote server before determining "
|
.description("Specifices how long to wait when communicating with the remote server before determining "
|
||||||
+ "that there is a communications failure if data cannot be sent or received")
|
+ "that there is a communications failure if data cannot be sent or received")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||||
.defaultValue("30 secs")
|
.defaultValue("30 secs")
|
||||||
|
|
|
@ -30,36 +30,37 @@ import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelInputStream;
|
||||||
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelOutputStream;
|
import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelOutputStream;
|
||||||
|
|
||||||
public class SSLCommsSession implements CommsSession {
|
public class SSLCommsSession implements CommsSession {
|
||||||
|
|
||||||
private final SSLSocketChannel sslSocketChannel;
|
private final SSLSocketChannel sslSocketChannel;
|
||||||
private final SSLContext sslContext;
|
private final SSLContext sslContext;
|
||||||
private final String hostname;
|
private final String hostname;
|
||||||
private final int port;
|
private final int port;
|
||||||
|
|
||||||
private final SSLSocketChannelInputStream in;
|
private final SSLSocketChannelInputStream in;
|
||||||
private final BufferedInputStream bufferedIn;
|
private final BufferedInputStream bufferedIn;
|
||||||
|
|
||||||
private final SSLSocketChannelOutputStream out;
|
private final SSLSocketChannelOutputStream out;
|
||||||
private final BufferedOutputStream bufferedOut;
|
private final BufferedOutputStream bufferedOut;
|
||||||
|
|
||||||
public SSLCommsSession(final SSLContext sslContext, final String hostname, final int port) throws IOException {
|
public SSLCommsSession(final SSLContext sslContext, final String hostname, final int port) throws IOException {
|
||||||
sslSocketChannel = new SSLSocketChannel(sslContext, hostname, port, true);
|
sslSocketChannel = new SSLSocketChannel(sslContext, hostname, port, true);
|
||||||
|
|
||||||
in = new SSLSocketChannelInputStream(sslSocketChannel);
|
in = new SSLSocketChannelInputStream(sslSocketChannel);
|
||||||
bufferedIn = new BufferedInputStream(in);
|
bufferedIn = new BufferedInputStream(in);
|
||||||
|
|
||||||
out = new SSLSocketChannelOutputStream(sslSocketChannel);
|
out = new SSLSocketChannelOutputStream(sslSocketChannel);
|
||||||
bufferedOut = new BufferedOutputStream(out);
|
bufferedOut = new BufferedOutputStream(out);
|
||||||
|
|
||||||
this.sslContext = sslContext;
|
this.sslContext = sslContext;
|
||||||
this.hostname = hostname;
|
this.hostname = hostname;
|
||||||
this.port = port;
|
this.port = port;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void interrupt() {
|
public void interrupt() {
|
||||||
sslSocketChannel.interrupt();
|
sslSocketChannel.interrupt();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException {
|
public void close() throws IOException {
|
||||||
sslSocketChannel.close();
|
sslSocketChannel.close();
|
||||||
|
@ -84,23 +85,25 @@ public class SSLCommsSession implements CommsSession {
|
||||||
public boolean isClosed() {
|
public boolean isClosed() {
|
||||||
return sslSocketChannel.isClosed();
|
return sslSocketChannel.isClosed();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getHostname() {
|
public String getHostname() {
|
||||||
return hostname;
|
return hostname;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int getPort() {
|
public int getPort() {
|
||||||
return port;
|
return port;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SSLContext getSSLContext() {
|
public SSLContext getSSLContext() {
|
||||||
return sslContext;
|
return sslContext;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getTimeout(final TimeUnit timeUnit) {
|
public long getTimeout(final TimeUnit timeUnit) {
|
||||||
return timeUnit.convert(sslSocketChannel.getTimeout(), TimeUnit.MILLISECONDS);
|
return timeUnit.convert(sslSocketChannel.getTimeout(), TimeUnit.MILLISECONDS);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.nifi.remote.io.socket.SocketChannelInputStream;
|
||||||
import org.apache.nifi.remote.io.socket.SocketChannelOutputStream;
|
import org.apache.nifi.remote.io.socket.SocketChannelOutputStream;
|
||||||
|
|
||||||
public class StandardCommsSession implements CommsSession {
|
public class StandardCommsSession implements CommsSession {
|
||||||
|
|
||||||
private final SocketChannel socketChannel;
|
private final SocketChannel socketChannel;
|
||||||
private final String hostname;
|
private final String hostname;
|
||||||
private final int port;
|
private final int port;
|
||||||
|
|
|
@ -1,35 +1,35 @@
|
||||||
<!DOCTYPE html>
|
<!DOCTYPE html>
|
||||||
<html lang="en">
|
<html lang="en">
|
||||||
<!--
|
<!--
|
||||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
contributor license agreements. See the NOTICE file distributed with
|
contributor license agreements. See the NOTICE file distributed with
|
||||||
this work for additional information regarding copyright ownership.
|
this work for additional information regarding copyright ownership.
|
||||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
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 not use this file except in compliance with
|
||||||
the License. You may obtain a copy of the License at
|
the License. You may obtain a copy of the License at
|
||||||
http://www.apache.org/licenses/LICENSE-2.0
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
Unless required by applicable law or agreed to in writing, software
|
Unless required by applicable law or agreed to in writing, software
|
||||||
distributed under the License is distributed on an "AS IS" BASIS,
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
See the License for the specific language governing permissions and
|
See the License for the specific language governing permissions and
|
||||||
limitations under the License.
|
limitations under the License.
|
||||||
-->
|
-->
|
||||||
<head>
|
<head>
|
||||||
<meta charset="utf-8" />
|
<meta charset="utf-8" />
|
||||||
<title>Distributed Map Cache Client Service</title>
|
<title>Distributed Map Cache Client Service</title>
|
||||||
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
|
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
|
||||||
</head>
|
</head>
|
||||||
|
|
||||||
<body>
|
<body>
|
||||||
<p>
|
<p>
|
||||||
Below is an example of how to create a client connection to your distributed map cache server.
|
Below is an example of how to create a client connection to your distributed map cache server.
|
||||||
Note that the identifier in this example is <code>cache-client</code>. If you are using this template
|
Note that the identifier in this example is <code>cache-client</code>. If you are using this template
|
||||||
to create your own MapCacheClient service, replace the values in this template with values that are
|
to create your own MapCacheClient service, replace the values in this template with values that are
|
||||||
suitable for your system. Possible options for <code>Server Hostname</code>, <code>Server Port</code>,
|
suitable for your system. Possible options for <code>Server Hostname</code>, <code>Server Port</code>,
|
||||||
<code>Communications Timeout</code>, and <span style="font-style: italic;">SSL Context Service</span>.
|
<code>Communications Timeout</code>, and <span style="font-style: italic;">SSL Context Service</span>.
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
<pre>
|
<pre>
|
||||||
<?xml version="1.0" encoding="UTF-8" ?>
|
<?xml version="1.0" encoding="UTF-8" ?>
|
||||||
<services>
|
<services>
|
||||||
<service>
|
<service>
|
||||||
|
@ -40,6 +40,6 @@
|
||||||
<property name="Communications Timeout">30 secs</property>
|
<property name="Communications Timeout">30 secs</property>
|
||||||
</service>
|
</service>
|
||||||
</services>
|
</services>
|
||||||
</pre>
|
</pre>
|
||||||
</body>
|
</body>
|
||||||
</html>
|
</html>
|
||||||
|
|
|
@ -51,7 +51,8 @@ public abstract class AbstractCacheServer implements CacheServer {
|
||||||
private final int port;
|
private final int port;
|
||||||
private final SSLContext sslContext;
|
private final SSLContext sslContext;
|
||||||
protected volatile boolean stopped = false;
|
protected volatile boolean stopped = false;
|
||||||
private final Set<Thread> processInputThreads = new CopyOnWriteArraySet<>();;
|
private final Set<Thread> processInputThreads = new CopyOnWriteArraySet<>();
|
||||||
|
;
|
||||||
|
|
||||||
private volatile ServerSocketChannel serverSocketChannel;
|
private volatile ServerSocketChannel serverSocketChannel;
|
||||||
|
|
||||||
|
@ -75,7 +76,7 @@ public abstract class AbstractCacheServer implements CacheServer {
|
||||||
final SocketChannel socketChannel;
|
final SocketChannel socketChannel;
|
||||||
try {
|
try {
|
||||||
socketChannel = serverSocketChannel.accept();
|
socketChannel = serverSocketChannel.accept();
|
||||||
logger.debug("Connected to {}", new Object[] { socketChannel });
|
logger.debug("Connected to {}", new Object[]{socketChannel});
|
||||||
} catch (final IOException e) {
|
} catch (final IOException e) {
|
||||||
if (!stopped) {
|
if (!stopped) {
|
||||||
logger.error("{} unable to accept connection from remote peer due to {}", this, e.toString());
|
logger.error("{} unable to accept connection from remote peer due to {}", this, e.toString());
|
||||||
|
@ -104,7 +105,7 @@ public abstract class AbstractCacheServer implements CacheServer {
|
||||||
rawOutputStream = new SSLSocketChannelOutputStream(sslSocketChannel);
|
rawOutputStream = new SSLSocketChannelOutputStream(sslSocketChannel);
|
||||||
}
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
logger.error("Cannot create input and/or output streams for {}", new Object[] { identifier }, e);
|
logger.error("Cannot create input and/or output streams for {}", new Object[]{identifier}, e);
|
||||||
if (logger.isDebugEnabled()) {
|
if (logger.isDebugEnabled()) {
|
||||||
logger.error("", e);
|
logger.error("", e);
|
||||||
}
|
}
|
||||||
|
@ -112,7 +113,7 @@ public abstract class AbstractCacheServer implements CacheServer {
|
||||||
socketChannel.close();
|
socketChannel.close();
|
||||||
} catch (IOException swallow) {
|
} catch (IOException swallow) {
|
||||||
}
|
}
|
||||||
|
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
try (final InputStream in = new BufferedInputStream(rawInputStream);
|
try (final InputStream in = new BufferedInputStream(rawInputStream);
|
||||||
|
@ -127,12 +128,12 @@ public abstract class AbstractCacheServer implements CacheServer {
|
||||||
continueComms = listen(in, out, versionNegotiator.getVersion());
|
continueComms = listen(in, out, versionNegotiator.getVersion());
|
||||||
}
|
}
|
||||||
// client has issued 'close'
|
// client has issued 'close'
|
||||||
logger.debug("Client issued close on {}", new Object[] { socketChannel });
|
logger.debug("Client issued close on {}", new Object[]{socketChannel});
|
||||||
} catch (final SocketTimeoutException e) {
|
} catch (final SocketTimeoutException e) {
|
||||||
logger.debug("30 sec timeout reached", e);
|
logger.debug("30 sec timeout reached", e);
|
||||||
} catch (final IOException | HandshakeException e) {
|
} catch (final IOException | HandshakeException e) {
|
||||||
if (!stopped) {
|
if (!stopped) {
|
||||||
logger.error("{} unable to communicate with remote peer {} due to {}", new Object[] { this, peer, e.toString() });
|
logger.error("{} unable to communicate with remote peer {} due to {}", new Object[]{this, peer, e.toString()});
|
||||||
if (logger.isDebugEnabled()) {
|
if (logger.isDebugEnabled()) {
|
||||||
logger.error("", e);
|
logger.error("", e);
|
||||||
}
|
}
|
||||||
|
@ -161,7 +162,7 @@ public abstract class AbstractCacheServer implements CacheServer {
|
||||||
@Override
|
@Override
|
||||||
public void stop() throws IOException {
|
public void stop() throws IOException {
|
||||||
stopped = true;
|
stopped = true;
|
||||||
logger.info("Stopping CacheServer {}", new Object[] { this.identifier });
|
logger.info("Stopping CacheServer {}", new Object[]{this.identifier});
|
||||||
|
|
||||||
if (serverSocketChannel != null && serverSocketChannel.isOpen()) {
|
if (serverSocketChannel != null && serverSocketChannel.isOpen()) {
|
||||||
serverSocketChannel.close();
|
serverSocketChannel.close();
|
||||||
|
@ -188,12 +189,12 @@ public abstract class AbstractCacheServer implements CacheServer {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Listens for incoming data and communicates with remote peer
|
* Listens for incoming data and communicates with remote peer
|
||||||
*
|
*
|
||||||
* @param in
|
* @param in in
|
||||||
* @param out
|
* @param out out
|
||||||
* @param version
|
* @param version version
|
||||||
* @return <code>true</code> if communications should continue, <code>false</code> otherwise
|
* @return <code>true</code> if communications should continue, <code>false</code> otherwise
|
||||||
* @throws IOException
|
* @throws IOException ex
|
||||||
*/
|
*/
|
||||||
protected abstract boolean listen(InputStream in, OutputStream out, int version) throws IOException;
|
protected abstract boolean listen(InputStream in, OutputStream out, int version) throws IOException;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,26 +22,26 @@ import java.util.concurrent.atomic.AtomicLong;
|
||||||
public class CacheRecord {
|
public class CacheRecord {
|
||||||
|
|
||||||
private static final AtomicLong idGenerator = new AtomicLong(0L);
|
private static final AtomicLong idGenerator = new AtomicLong(0L);
|
||||||
|
|
||||||
private final long id;
|
private final long id;
|
||||||
private final long entryDate;
|
private final long entryDate;
|
||||||
private volatile long lastHitDate;
|
private volatile long lastHitDate;
|
||||||
private final AtomicInteger hitCount = new AtomicInteger(0);
|
private final AtomicInteger hitCount = new AtomicInteger(0);
|
||||||
|
|
||||||
public CacheRecord() {
|
public CacheRecord() {
|
||||||
entryDate = System.currentTimeMillis();
|
entryDate = System.currentTimeMillis();
|
||||||
lastHitDate = entryDate;
|
lastHitDate = entryDate;
|
||||||
id = idGenerator.getAndIncrement();
|
id = idGenerator.getAndIncrement();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getEntryDate() {
|
public long getEntryDate() {
|
||||||
return entryDate;
|
return entryDate;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getLastHitDate() {
|
public long getLastHitDate() {
|
||||||
return lastHitDate;
|
return lastHitDate;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getHitCount() {
|
public int getHitCount() {
|
||||||
return hitCount.get();
|
return hitCount.get();
|
||||||
}
|
}
|
||||||
|
@ -50,7 +50,7 @@ public class CacheRecord {
|
||||||
hitCount.getAndIncrement();
|
hitCount.getAndIncrement();
|
||||||
lastHitDate = System.currentTimeMillis();
|
lastHitDate = System.currentTimeMillis();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getId() {
|
public long getId() {
|
||||||
return id;
|
return id;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
||||||
public interface CacheServer {
|
public interface CacheServer {
|
||||||
|
|
||||||
void start() throws IOException;
|
void start() throws IOException;
|
||||||
|
|
||||||
void stop() throws IOException;
|
void stop() throws IOException;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
||||||
import org.apache.nifi.ssl.SSLContextService;
|
import org.apache.nifi.ssl.SSLContextService;
|
||||||
|
|
||||||
public abstract class DistributedCacheServer extends AbstractControllerService {
|
public abstract class DistributedCacheServer extends AbstractControllerService {
|
||||||
|
|
||||||
public static final String EVICTION_STRATEGY_LFU = "Least Frequently Used";
|
public static final String EVICTION_STRATEGY_LFU = "Least Frequently Used";
|
||||||
public static final String EVICTION_STRATEGY_LRU = "Least Recently Used";
|
public static final String EVICTION_STRATEGY_LRU = "Least Recently Used";
|
||||||
public static final String EVICTION_STRATEGY_FIFO = "First In, First Out";
|
public static final String EVICTION_STRATEGY_FIFO = "First In, First Out";
|
||||||
|
@ -43,7 +44,7 @@ public abstract class DistributedCacheServer extends AbstractControllerService {
|
||||||
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
|
||||||
.name("SSL Context Service")
|
.name("SSL Context Service")
|
||||||
.description("If specified, this service will be used to create an SSL Context that will be used "
|
.description("If specified, this service will be used to create an SSL Context that will be used "
|
||||||
+ "to secure communications; if not specified, communications will not be secure")
|
+ "to secure communications; if not specified, communications will not be secure")
|
||||||
.required(false)
|
.required(false)
|
||||||
.identifiesControllerService(SSLContextService.class)
|
.identifiesControllerService(SSLContextService.class)
|
||||||
.build();
|
.build();
|
||||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.nifi.annotation.documentation.Tags;
|
||||||
import org.apache.nifi.controller.ConfigurationContext;
|
import org.apache.nifi.controller.ConfigurationContext;
|
||||||
import org.apache.nifi.ssl.SSLContextService;
|
import org.apache.nifi.ssl.SSLContextService;
|
||||||
import org.apache.nifi.ssl.SSLContextService.ClientAuth;
|
import org.apache.nifi.ssl.SSLContextService.ClientAuth;
|
||||||
|
|
||||||
@Tags({"distributed", "set", "distinct", "cache", "server"})
|
@Tags({"distributed", "set", "distinct", "cache", "server"})
|
||||||
@CapabilityDescription("Provides a set (collection of unique values) cache that can be accessed over a socket. "
|
@CapabilityDescription("Provides a set (collection of unique values) cache that can be accessed over a socket. "
|
||||||
+ "Interaction with this service is typically accomplished via a DistributedSetCacheClient service.")
|
+ "Interaction with this service is typically accomplished via a DistributedSetCacheClient service.")
|
||||||
|
@ -37,14 +38,14 @@ public class DistributedSetCacheServer extends DistributedCacheServer {
|
||||||
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
||||||
final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger();
|
final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger();
|
||||||
final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue();
|
final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue();
|
||||||
|
|
||||||
final SSLContext sslContext;
|
final SSLContext sslContext;
|
||||||
if ( sslContextService == null ) {
|
if (sslContextService == null) {
|
||||||
sslContext = null;
|
sslContext = null;
|
||||||
} else {
|
} else {
|
||||||
sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED);
|
sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED);
|
||||||
}
|
}
|
||||||
|
|
||||||
final EvictionPolicy evictionPolicy;
|
final EvictionPolicy evictionPolicy;
|
||||||
switch (evictionPolicyName) {
|
switch (evictionPolicyName) {
|
||||||
case EVICTION_STRATEGY_FIFO:
|
case EVICTION_STRATEGY_FIFO:
|
||||||
|
@ -59,14 +60,14 @@ public class DistributedSetCacheServer extends DistributedCacheServer {
|
||||||
default:
|
default:
|
||||||
throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName);
|
throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final File persistenceDir = persistencePath == null ? null : new File(persistencePath);
|
final File persistenceDir = persistencePath == null ? null : new File(persistencePath);
|
||||||
|
|
||||||
return new SetCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir);
|
return new SetCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir);
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,37 +19,40 @@ package org.apache.nifi.distributed.cache.server;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
|
||||||
public enum EvictionPolicy {
|
public enum EvictionPolicy {
|
||||||
|
|
||||||
LFU(new LFUComparator()),
|
LFU(new LFUComparator()),
|
||||||
LRU(new LRUComparator()),
|
LRU(new LRUComparator()),
|
||||||
FIFO(new FIFOComparator());
|
FIFO(new FIFOComparator());
|
||||||
|
|
||||||
private final Comparator<CacheRecord> comparator;
|
private final Comparator<CacheRecord> comparator;
|
||||||
|
|
||||||
private EvictionPolicy(final Comparator<CacheRecord> comparator) {
|
private EvictionPolicy(final Comparator<CacheRecord> comparator) {
|
||||||
this.comparator = comparator;
|
this.comparator = comparator;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Comparator<CacheRecord> getComparator() {
|
public Comparator<CacheRecord> getComparator() {
|
||||||
return comparator;
|
return comparator;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class LFUComparator implements Comparator<CacheRecord> {
|
public static class LFUComparator implements Comparator<CacheRecord> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compare(final CacheRecord o1, final CacheRecord o2) {
|
public int compare(final CacheRecord o1, final CacheRecord o2) {
|
||||||
if ( o1.equals(o2) ) {
|
if (o1.equals(o2)) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
final int hitCountComparison = Integer.compare(o1.getHitCount(), o2.getHitCount());
|
final int hitCountComparison = Integer.compare(o1.getHitCount(), o2.getHitCount());
|
||||||
final int entryDateComparison = (hitCountComparison == 0) ? Long.compare(o1.getEntryDate(), o2.getEntryDate()) : hitCountComparison;
|
final int entryDateComparison = (hitCountComparison == 0) ? Long.compare(o1.getEntryDate(), o2.getEntryDate()) : hitCountComparison;
|
||||||
return (entryDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : entryDateComparison);
|
return (entryDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : entryDateComparison);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class LRUComparator implements Comparator<CacheRecord> {
|
public static class LRUComparator implements Comparator<CacheRecord> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compare(final CacheRecord o1, final CacheRecord o2) {
|
public int compare(final CacheRecord o1, final CacheRecord o2) {
|
||||||
if ( o1.equals(o2) ) {
|
if (o1.equals(o2)) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -57,11 +60,12 @@ public enum EvictionPolicy {
|
||||||
return (lastHitDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : lastHitDateComparison);
|
return (lastHitDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : lastHitDateComparison);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class FIFOComparator implements Comparator<CacheRecord> {
|
public static class FIFOComparator implements Comparator<CacheRecord> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int compare(final CacheRecord o1, final CacheRecord o2) {
|
public int compare(final CacheRecord o1, final CacheRecord o2) {
|
||||||
if ( o1.equals(o2) ) {
|
if (o1.equals(o2)) {
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -67,17 +67,17 @@ public class SetCacheServer extends AbstractCacheServer {
|
||||||
|
|
||||||
final SetCacheResult response;
|
final SetCacheResult response;
|
||||||
switch (action) {
|
switch (action) {
|
||||||
case "addIfAbsent":
|
case "addIfAbsent":
|
||||||
response = cache.addIfAbsent(valueBuffer);
|
response = cache.addIfAbsent(valueBuffer);
|
||||||
break;
|
break;
|
||||||
case "contains":
|
case "contains":
|
||||||
response = cache.contains(valueBuffer);
|
response = cache.contains(valueBuffer);
|
||||||
break;
|
break;
|
||||||
case "remove":
|
case "remove":
|
||||||
response = cache.remove(valueBuffer);
|
response = cache.remove(valueBuffer);
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
throw new IOException("IllegalRequest");
|
throw new IOException("IllegalRequest");
|
||||||
}
|
}
|
||||||
|
|
||||||
dos.writeBoolean(response.getResult());
|
dos.writeBoolean(response.getResult());
|
||||||
|
@ -97,8 +97,9 @@ public class SetCacheServer extends AbstractCacheServer {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void finalize() throws Throwable {
|
protected void finalize() throws Throwable {
|
||||||
if (!stopped)
|
if (!stopped) {
|
||||||
stop();
|
stop();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.nifi.ssl.SSLContextService.ClientAuth;
|
||||||
@Tags({"distributed", "cluster", "map", "cache", "server", "key/value"})
|
@Tags({"distributed", "cluster", "map", "cache", "server", "key/value"})
|
||||||
@CapabilityDescription("Provides a map (key/value) cache that can be accessed over a socket. Interaction with this service"
|
@CapabilityDescription("Provides a map (key/value) cache that can be accessed over a socket. Interaction with this service"
|
||||||
+ " is typically accomplished via a DistributedMapCacheClient service.")
|
+ " is typically accomplished via a DistributedMapCacheClient service.")
|
||||||
@SeeAlso(classNames={"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.ssl.StandardSSLContextService"})
|
@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.ssl.StandardSSLContextService"})
|
||||||
public class DistributedMapCacheServer extends DistributedCacheServer {
|
public class DistributedMapCacheServer extends DistributedCacheServer {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -43,14 +43,14 @@ public class DistributedMapCacheServer extends DistributedCacheServer {
|
||||||
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
|
||||||
final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger();
|
final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger();
|
||||||
final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue();
|
final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue();
|
||||||
|
|
||||||
final SSLContext sslContext;
|
final SSLContext sslContext;
|
||||||
if ( sslContextService == null ) {
|
if (sslContextService == null) {
|
||||||
sslContext = null;
|
sslContext = null;
|
||||||
} else {
|
} else {
|
||||||
sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED);
|
sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED);
|
||||||
}
|
}
|
||||||
|
|
||||||
final EvictionPolicy evictionPolicy;
|
final EvictionPolicy evictionPolicy;
|
||||||
switch (evictionPolicyName) {
|
switch (evictionPolicyName) {
|
||||||
case EVICTION_STRATEGY_FIFO:
|
case EVICTION_STRATEGY_FIFO:
|
||||||
|
@ -65,10 +65,10 @@ public class DistributedMapCacheServer extends DistributedCacheServer {
|
||||||
default:
|
default:
|
||||||
throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName);
|
throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final File persistenceDir = persistencePath == null ? null : new File(persistencePath);
|
final File persistenceDir = persistencePath == null ? null : new File(persistencePath);
|
||||||
|
|
||||||
return new MapCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir);
|
return new MapCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir);
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
|
|
|
@ -22,8 +22,12 @@ import java.nio.ByteBuffer;
|
||||||
public interface MapCache {
|
public interface MapCache {
|
||||||
|
|
||||||
MapPutResult putIfAbsent(ByteBuffer key, ByteBuffer value) throws IOException;
|
MapPutResult putIfAbsent(ByteBuffer key, ByteBuffer value) throws IOException;
|
||||||
|
|
||||||
boolean containsKey(ByteBuffer key) throws IOException;
|
boolean containsKey(ByteBuffer key) throws IOException;
|
||||||
|
|
||||||
ByteBuffer get(ByteBuffer key) throws IOException;
|
ByteBuffer get(ByteBuffer key) throws IOException;
|
||||||
|
|
||||||
ByteBuffer remove(ByteBuffer key) throws IOException;
|
ByteBuffer remove(ByteBuffer key) throws IOException;
|
||||||
|
|
||||||
void shutdown() throws IOException;
|
void shutdown() throws IOException;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,38 +21,39 @@ import java.nio.ByteBuffer;
|
||||||
import org.apache.nifi.distributed.cache.server.CacheRecord;
|
import org.apache.nifi.distributed.cache.server.CacheRecord;
|
||||||
|
|
||||||
public class MapCacheRecord extends CacheRecord {
|
public class MapCacheRecord extends CacheRecord {
|
||||||
|
|
||||||
private final ByteBuffer key;
|
private final ByteBuffer key;
|
||||||
private final ByteBuffer value;
|
private final ByteBuffer value;
|
||||||
|
|
||||||
public MapCacheRecord(final ByteBuffer key, final ByteBuffer value) {
|
public MapCacheRecord(final ByteBuffer key, final ByteBuffer value) {
|
||||||
this.key = key;
|
this.key = key;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getKey() {
|
public ByteBuffer getKey() {
|
||||||
return key;
|
return key;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getValue() {
|
public ByteBuffer getValue() {
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return 2938476 + key.hashCode() * value.hashCode();
|
return 2938476 + key.hashCode() * value.hashCode();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(final Object obj) {
|
public boolean equals(final Object obj) {
|
||||||
if ( obj == this ) {
|
if (obj == this) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
if ( obj instanceof MapCacheRecord ) {
|
if (obj instanceof MapCacheRecord) {
|
||||||
final MapCacheRecord that = ((MapCacheRecord) obj);
|
final MapCacheRecord that = ((MapCacheRecord) obj);
|
||||||
return key.equals(that.key) && value.equals(that.value);
|
return key.equals(that.key) && value.equals(that.value);
|
||||||
}
|
}
|
||||||
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -55,63 +55,63 @@ public class MapCacheServer extends AbstractCacheServer {
|
||||||
final String action = dis.readUTF();
|
final String action = dis.readUTF();
|
||||||
try {
|
try {
|
||||||
switch (action) {
|
switch (action) {
|
||||||
case "close": {
|
case "close": {
|
||||||
return false;
|
return false;
|
||||||
}
|
|
||||||
case "putIfAbsent": {
|
|
||||||
final byte[] key = readValue(dis);
|
|
||||||
final byte[] value = readValue(dis);
|
|
||||||
final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
|
|
||||||
dos.writeBoolean(putResult.isSuccessful());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case "containsKey": {
|
|
||||||
final byte[] key = readValue(dis);
|
|
||||||
final boolean contains = cache.containsKey(ByteBuffer.wrap(key));
|
|
||||||
dos.writeBoolean(contains);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case "getAndPutIfAbsent": {
|
|
||||||
final byte[] key = readValue(dis);
|
|
||||||
final byte[] value = readValue(dis);
|
|
||||||
|
|
||||||
final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
|
|
||||||
if (putResult.isSuccessful()) {
|
|
||||||
// Put was successful. There was no old value to get.
|
|
||||||
dos.writeInt(0);
|
|
||||||
} else {
|
|
||||||
// we didn't put. Write back the previous value
|
|
||||||
final byte[] byteArray = putResult.getExistingValue().array();
|
|
||||||
dos.writeInt(byteArray.length);
|
|
||||||
dos.write(byteArray);
|
|
||||||
}
|
}
|
||||||
|
case "putIfAbsent": {
|
||||||
break;
|
final byte[] key = readValue(dis);
|
||||||
}
|
final byte[] value = readValue(dis);
|
||||||
case "get": {
|
final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
|
||||||
final byte[] key = readValue(dis);
|
dos.writeBoolean(putResult.isSuccessful());
|
||||||
final ByteBuffer existingValue = cache.get(ByteBuffer.wrap(key));
|
break;
|
||||||
if (existingValue == null) {
|
|
||||||
// there was no existing value; we did a "put".
|
|
||||||
dos.writeInt(0);
|
|
||||||
} else {
|
|
||||||
// a value already existed. we did not update the map
|
|
||||||
final byte[] byteArray = existingValue.array();
|
|
||||||
dos.writeInt(byteArray.length);
|
|
||||||
dos.write(byteArray);
|
|
||||||
}
|
}
|
||||||
|
case "containsKey": {
|
||||||
|
final byte[] key = readValue(dis);
|
||||||
|
final boolean contains = cache.containsKey(ByteBuffer.wrap(key));
|
||||||
|
dos.writeBoolean(contains);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case "getAndPutIfAbsent": {
|
||||||
|
final byte[] key = readValue(dis);
|
||||||
|
final byte[] value = readValue(dis);
|
||||||
|
|
||||||
break;
|
final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value));
|
||||||
}
|
if (putResult.isSuccessful()) {
|
||||||
case "remove": {
|
// Put was successful. There was no old value to get.
|
||||||
final byte[] key = readValue(dis);
|
dos.writeInt(0);
|
||||||
final boolean removed = cache.remove(ByteBuffer.wrap(key)) != null;
|
} else {
|
||||||
dos.writeBoolean(removed);
|
// we didn't put. Write back the previous value
|
||||||
break;
|
final byte[] byteArray = putResult.getExistingValue().array();
|
||||||
}
|
dos.writeInt(byteArray.length);
|
||||||
default: {
|
dos.write(byteArray);
|
||||||
throw new IOException("Illegal Request");
|
}
|
||||||
}
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case "get": {
|
||||||
|
final byte[] key = readValue(dis);
|
||||||
|
final ByteBuffer existingValue = cache.get(ByteBuffer.wrap(key));
|
||||||
|
if (existingValue == null) {
|
||||||
|
// there was no existing value; we did a "put".
|
||||||
|
dos.writeInt(0);
|
||||||
|
} else {
|
||||||
|
// a value already existed. we did not update the map
|
||||||
|
final byte[] byteArray = existingValue.array();
|
||||||
|
dos.writeInt(byteArray.length);
|
||||||
|
dos.write(byteArray);
|
||||||
|
}
|
||||||
|
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case "remove": {
|
||||||
|
final byte[] key = readValue(dis);
|
||||||
|
final boolean removed = cache.remove(ByteBuffer.wrap(key)) != null;
|
||||||
|
dos.writeBoolean(removed);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
default: {
|
||||||
|
throw new IOException("Illegal Request");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
dos.flush();
|
dos.flush();
|
||||||
|
@ -131,8 +131,9 @@ public class MapCacheServer extends AbstractCacheServer {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void finalize() throws Throwable {
|
protected void finalize() throws Throwable {
|
||||||
if (!stopped)
|
if (!stopped) {
|
||||||
stop();
|
stop();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private byte[] readValue(final DataInputStream dis) throws IOException {
|
private byte[] readValue(final DataInputStream dis) throws IOException {
|
||||||
|
|
|
@ -19,11 +19,12 @@ package org.apache.nifi.distributed.cache.server.map;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
public class MapPutResult {
|
public class MapPutResult {
|
||||||
|
|
||||||
private final boolean successful;
|
private final boolean successful;
|
||||||
private final ByteBuffer key, value;
|
private final ByteBuffer key, value;
|
||||||
private final ByteBuffer existingValue;
|
private final ByteBuffer existingValue;
|
||||||
private final ByteBuffer evictedKey, evictedValue;
|
private final ByteBuffer evictedKey, evictedValue;
|
||||||
|
|
||||||
public MapPutResult(final boolean successful, final ByteBuffer key, final ByteBuffer value, final ByteBuffer existingValue, final ByteBuffer evictedKey, final ByteBuffer evictedValue) {
|
public MapPutResult(final boolean successful, final ByteBuffer key, final ByteBuffer value, final ByteBuffer existingValue, final ByteBuffer evictedKey, final ByteBuffer evictedValue) {
|
||||||
this.successful = successful;
|
this.successful = successful;
|
||||||
this.key = key;
|
this.key = key;
|
||||||
|
@ -44,7 +45,7 @@ public class MapPutResult {
|
||||||
public ByteBuffer getValue() {
|
public ByteBuffer getValue() {
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getExistingValue() {
|
public ByteBuffer getExistingValue() {
|
||||||
return existingValue;
|
return existingValue;
|
||||||
}
|
}
|
||||||
|
|
|
@ -38,9 +38,9 @@ public class PersistentMapCache implements MapCache {
|
||||||
|
|
||||||
private final MapCache wrapped;
|
private final MapCache wrapped;
|
||||||
private final WriteAheadRepository<MapWaliRecord> wali;
|
private final WriteAheadRepository<MapWaliRecord> wali;
|
||||||
|
|
||||||
private final AtomicLong modifications = new AtomicLong(0L);
|
private final AtomicLong modifications = new AtomicLong(0L);
|
||||||
|
|
||||||
public PersistentMapCache(final String serviceIdentifier, final File persistencePath, final MapCache cacheToWrap) throws IOException {
|
public PersistentMapCache(final String serviceIdentifier, final File persistencePath, final MapCache cacheToWrap) throws IOException {
|
||||||
wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null);
|
wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null);
|
||||||
wrapped = cacheToWrap;
|
wrapped = cacheToWrap;
|
||||||
|
@ -48,8 +48,8 @@ public class PersistentMapCache implements MapCache {
|
||||||
|
|
||||||
synchronized void restore() throws IOException {
|
synchronized void restore() throws IOException {
|
||||||
final Collection<MapWaliRecord> recovered = wali.recoverRecords();
|
final Collection<MapWaliRecord> recovered = wali.recoverRecords();
|
||||||
for ( final MapWaliRecord record : recovered ) {
|
for (final MapWaliRecord record : recovered) {
|
||||||
if ( record.getUpdateType() == UpdateType.CREATE ) {
|
if (record.getUpdateType() == UpdateType.CREATE) {
|
||||||
wrapped.putIfAbsent(record.getKey(), record.getValue());
|
wrapped.putIfAbsent(record.getKey(), record.getValue());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -58,24 +58,24 @@ public class PersistentMapCache implements MapCache {
|
||||||
@Override
|
@Override
|
||||||
public MapPutResult putIfAbsent(final ByteBuffer key, final ByteBuffer value) throws IOException {
|
public MapPutResult putIfAbsent(final ByteBuffer key, final ByteBuffer value) throws IOException {
|
||||||
final MapPutResult putResult = wrapped.putIfAbsent(key, value);
|
final MapPutResult putResult = wrapped.putIfAbsent(key, value);
|
||||||
if ( putResult.isSuccessful() ) {
|
if (putResult.isSuccessful()) {
|
||||||
// The put was successful.
|
// The put was successful.
|
||||||
final MapWaliRecord record = new MapWaliRecord(UpdateType.CREATE, key, value);
|
final MapWaliRecord record = new MapWaliRecord(UpdateType.CREATE, key, value);
|
||||||
final List<MapWaliRecord> records = new ArrayList<>();
|
final List<MapWaliRecord> records = new ArrayList<>();
|
||||||
records.add(record);
|
records.add(record);
|
||||||
|
|
||||||
if ( putResult.getEvictedKey() != null ) {
|
if (putResult.getEvictedKey() != null) {
|
||||||
records.add(new MapWaliRecord(UpdateType.DELETE, putResult.getEvictedKey(), putResult.getEvictedValue()));
|
records.add(new MapWaliRecord(UpdateType.DELETE, putResult.getEvictedKey(), putResult.getEvictedValue()));
|
||||||
}
|
}
|
||||||
|
|
||||||
wali.update(Collections.singletonList(record), false);
|
wali.update(Collections.singletonList(record), false);
|
||||||
|
|
||||||
final long modCount = modifications.getAndIncrement();
|
final long modCount = modifications.getAndIncrement();
|
||||||
if ( modCount > 0 && modCount % 100000 == 0 ) {
|
if (modCount > 0 && modCount % 100000 == 0) {
|
||||||
wali.checkpoint();
|
wali.checkpoint();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return putResult;
|
return putResult;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -92,65 +92,64 @@ public class PersistentMapCache implements MapCache {
|
||||||
@Override
|
@Override
|
||||||
public ByteBuffer remove(ByteBuffer key) throws IOException {
|
public ByteBuffer remove(ByteBuffer key) throws IOException {
|
||||||
final ByteBuffer removeResult = wrapped.remove(key);
|
final ByteBuffer removeResult = wrapped.remove(key);
|
||||||
if ( removeResult != null ) {
|
if (removeResult != null) {
|
||||||
final MapWaliRecord record = new MapWaliRecord(UpdateType.DELETE, key, removeResult);
|
final MapWaliRecord record = new MapWaliRecord(UpdateType.DELETE, key, removeResult);
|
||||||
final List<MapWaliRecord> records = new ArrayList<>(1);
|
final List<MapWaliRecord> records = new ArrayList<>(1);
|
||||||
records.add(record);
|
records.add(record);
|
||||||
wali.update(records, false);
|
wali.update(records, false);
|
||||||
|
|
||||||
final long modCount = modifications.getAndIncrement();
|
final long modCount = modifications.getAndIncrement();
|
||||||
if ( modCount > 0 && modCount % 1000 == 0 ) {
|
if (modCount > 0 && modCount % 1000 == 0) {
|
||||||
wali.checkpoint();
|
wali.checkpoint();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return removeResult;
|
return removeResult;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void shutdown() throws IOException {
|
public void shutdown() throws IOException {
|
||||||
wali.shutdown();
|
wali.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private static class MapWaliRecord {
|
private static class MapWaliRecord {
|
||||||
|
|
||||||
private final UpdateType updateType;
|
private final UpdateType updateType;
|
||||||
private final ByteBuffer key;
|
private final ByteBuffer key;
|
||||||
private final ByteBuffer value;
|
private final ByteBuffer value;
|
||||||
|
|
||||||
public MapWaliRecord(final UpdateType updateType, final ByteBuffer key, final ByteBuffer value) {
|
public MapWaliRecord(final UpdateType updateType, final ByteBuffer key, final ByteBuffer value) {
|
||||||
this.updateType = updateType;
|
this.updateType = updateType;
|
||||||
this.key = key;
|
this.key = key;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
public UpdateType getUpdateType() {
|
public UpdateType getUpdateType() {
|
||||||
return updateType;
|
return updateType;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getKey() {
|
public ByteBuffer getKey() {
|
||||||
return key;
|
return key;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getValue() {
|
public ByteBuffer getValue() {
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class Serde implements SerDe<MapWaliRecord> {
|
private static class Serde implements SerDe<MapWaliRecord> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeEdit(MapWaliRecord previousRecordState, MapWaliRecord newRecordState, java.io.DataOutputStream out) throws IOException {
|
public void serializeEdit(MapWaliRecord previousRecordState, MapWaliRecord newRecordState, java.io.DataOutputStream out) throws IOException {
|
||||||
final UpdateType updateType = newRecordState.getUpdateType();
|
final UpdateType updateType = newRecordState.getUpdateType();
|
||||||
if ( updateType == UpdateType.DELETE ) {
|
if (updateType == UpdateType.DELETE) {
|
||||||
out.write(0);
|
out.write(0);
|
||||||
} else {
|
} else {
|
||||||
out.write(1);
|
out.write(1);
|
||||||
}
|
}
|
||||||
|
|
||||||
final byte[] key = newRecordState.getKey().array();
|
final byte[] key = newRecordState.getKey().array();
|
||||||
final byte[] value = newRecordState.getValue().array();
|
final byte[] value = newRecordState.getValue().array();
|
||||||
|
|
||||||
out.writeInt(key.length);
|
out.writeInt(key.length);
|
||||||
out.write(key);
|
out.write(key);
|
||||||
out.writeInt(value.length);
|
out.writeInt(value.length);
|
||||||
|
@ -165,12 +164,12 @@ public class PersistentMapCache implements MapCache {
|
||||||
@Override
|
@Override
|
||||||
public MapWaliRecord deserializeEdit(final DataInputStream in, final Map<Object, MapWaliRecord> currentRecordStates, final int version) throws IOException {
|
public MapWaliRecord deserializeEdit(final DataInputStream in, final Map<Object, MapWaliRecord> currentRecordStates, final int version) throws IOException {
|
||||||
final int updateTypeValue = in.read();
|
final int updateTypeValue = in.read();
|
||||||
if ( updateTypeValue < 0 ) {
|
if (updateTypeValue < 0) {
|
||||||
throw new EOFException();
|
throw new EOFException();
|
||||||
}
|
}
|
||||||
|
|
||||||
final UpdateType updateType = (updateTypeValue == 0 ? UpdateType.DELETE : UpdateType.CREATE);
|
final UpdateType updateType = (updateTypeValue == 0 ? UpdateType.DELETE : UpdateType.CREATE);
|
||||||
|
|
||||||
final int keySize = in.readInt();
|
final int keySize = in.readInt();
|
||||||
final byte[] key = new byte[keySize];
|
final byte[] key = new byte[keySize];
|
||||||
in.readFully(key);
|
in.readFully(key);
|
||||||
|
@ -207,4 +206,4 @@ public class PersistentMapCache implements MapCache {
|
||||||
return 1;
|
return 1;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,46 +33,47 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
public class SimpleMapCache implements MapCache {
|
public class SimpleMapCache implements MapCache {
|
||||||
|
|
||||||
private static final Logger logger = LoggerFactory.getLogger(SimpleMapCache.class);
|
private static final Logger logger = LoggerFactory.getLogger(SimpleMapCache.class);
|
||||||
|
|
||||||
private final Map<ByteBuffer, MapCacheRecord> cache = new HashMap<>();
|
private final Map<ByteBuffer, MapCacheRecord> cache = new HashMap<>();
|
||||||
private final SortedMap<MapCacheRecord, ByteBuffer> inverseCacheMap;
|
private final SortedMap<MapCacheRecord, ByteBuffer> inverseCacheMap;
|
||||||
|
|
||||||
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
|
private final ReadWriteLock rwLock = new ReentrantReadWriteLock();
|
||||||
private final Lock readLock = rwLock.readLock();
|
private final Lock readLock = rwLock.readLock();
|
||||||
private final Lock writeLock = rwLock.writeLock();
|
private final Lock writeLock = rwLock.writeLock();
|
||||||
|
|
||||||
private final String serviceIdentifier;
|
private final String serviceIdentifier;
|
||||||
|
|
||||||
private final int maxSize;
|
private final int maxSize;
|
||||||
|
|
||||||
public SimpleMapCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) {
|
public SimpleMapCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) {
|
||||||
// need to change to ConcurrentMap as this is modified when only the readLock is held
|
// need to change to ConcurrentMap as this is modified when only the readLock is held
|
||||||
inverseCacheMap = new ConcurrentSkipListMap<>(evictionPolicy.getComparator());
|
inverseCacheMap = new ConcurrentSkipListMap<>(evictionPolicy.getComparator());
|
||||||
this.serviceIdentifier = serviceIdentifier;
|
this.serviceIdentifier = serviceIdentifier;
|
||||||
this.maxSize = maxSize;
|
this.maxSize = maxSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "SimpleSetCache[service id=" + serviceIdentifier + "]";
|
return "SimpleSetCache[service id=" + serviceIdentifier + "]";
|
||||||
}
|
}
|
||||||
|
|
||||||
// don't need synchronized because this method is only called when the writeLock is held, and all
|
// don't need synchronized because this method is only called when the writeLock is held, and all
|
||||||
// public methods obtain either the read or write lock
|
// public methods obtain either the read or write lock
|
||||||
private MapCacheRecord evict() {
|
private MapCacheRecord evict() {
|
||||||
if ( cache.size() < maxSize ) {
|
if (cache.size() < maxSize) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
final MapCacheRecord recordToEvict = inverseCacheMap.firstKey();
|
final MapCacheRecord recordToEvict = inverseCacheMap.firstKey();
|
||||||
final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict);
|
final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict);
|
||||||
cache.remove(valueToEvict);
|
cache.remove(valueToEvict);
|
||||||
|
|
||||||
if ( logger.isDebugEnabled() ) {
|
if (logger.isDebugEnabled()) {
|
||||||
logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8));
|
logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8));
|
||||||
}
|
}
|
||||||
|
|
||||||
return recordToEvict;
|
return recordToEvict;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -81,44 +82,44 @@ public class SimpleMapCache implements MapCache {
|
||||||
writeLock.lock();
|
writeLock.lock();
|
||||||
try {
|
try {
|
||||||
final MapCacheRecord record = cache.get(key);
|
final MapCacheRecord record = cache.get(key);
|
||||||
if ( record == null ) {
|
if (record == null) {
|
||||||
// Record is null. We will add.
|
// Record is null. We will add.
|
||||||
final MapCacheRecord evicted = evict();
|
final MapCacheRecord evicted = evict();
|
||||||
final MapCacheRecord newRecord = new MapCacheRecord(key, value);
|
final MapCacheRecord newRecord = new MapCacheRecord(key, value);
|
||||||
cache.put(key, newRecord);
|
cache.put(key, newRecord);
|
||||||
inverseCacheMap.put(newRecord, key);
|
inverseCacheMap.put(newRecord, key);
|
||||||
|
|
||||||
if ( evicted == null ) {
|
if (evicted == null) {
|
||||||
return new MapPutResult(true, key, value, null, null, null);
|
return new MapPutResult(true, key, value, null, null, null);
|
||||||
} else {
|
} else {
|
||||||
return new MapPutResult(true, key, value, null, evicted.getKey(), evicted.getValue());
|
return new MapPutResult(true, key, value, null, evicted.getKey(), evicted.getValue());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Record is not null. Increment hit count and return result indicating that record was not added.
|
// Record is not null. Increment hit count and return result indicating that record was not added.
|
||||||
inverseCacheMap.remove(record);
|
inverseCacheMap.remove(record);
|
||||||
record.hit();
|
record.hit();
|
||||||
inverseCacheMap.put(record, key);
|
inverseCacheMap.put(record, key);
|
||||||
|
|
||||||
return new MapPutResult(false, key, value, record.getValue(), null, null);
|
return new MapPutResult(false, key, value, record.getValue(), null, null);
|
||||||
} finally {
|
} finally {
|
||||||
writeLock.unlock();
|
writeLock.unlock();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean containsKey(final ByteBuffer key) {
|
public boolean containsKey(final ByteBuffer key) {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
try {
|
try {
|
||||||
final MapCacheRecord record = cache.get(key);
|
final MapCacheRecord record = cache.get(key);
|
||||||
if ( record == null ) {
|
if (record == null) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
inverseCacheMap.remove(record);
|
inverseCacheMap.remove(record);
|
||||||
record.hit();
|
record.hit();
|
||||||
inverseCacheMap.put(record, key);
|
inverseCacheMap.put(record, key);
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
} finally {
|
} finally {
|
||||||
readLock.unlock();
|
readLock.unlock();
|
||||||
|
@ -130,14 +131,14 @@ public class SimpleMapCache implements MapCache {
|
||||||
readLock.lock();
|
readLock.lock();
|
||||||
try {
|
try {
|
||||||
final MapCacheRecord record = cache.get(key);
|
final MapCacheRecord record = cache.get(key);
|
||||||
if ( record == null ) {
|
if (record == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
inverseCacheMap.remove(record);
|
inverseCacheMap.remove(record);
|
||||||
record.hit();
|
record.hit();
|
||||||
inverseCacheMap.put(record, key);
|
inverseCacheMap.put(record, key);
|
||||||
|
|
||||||
return record.getValue();
|
return record.getValue();
|
||||||
} finally {
|
} finally {
|
||||||
readLock.unlock();
|
readLock.unlock();
|
||||||
|
|
|
@ -38,34 +38,34 @@ public class PersistentSetCache implements SetCache {
|
||||||
|
|
||||||
private final SetCache wrapped;
|
private final SetCache wrapped;
|
||||||
private final WriteAheadRepository<SetRecord> wali;
|
private final WriteAheadRepository<SetRecord> wali;
|
||||||
|
|
||||||
private final AtomicLong modifications = new AtomicLong(0L);
|
private final AtomicLong modifications = new AtomicLong(0L);
|
||||||
|
|
||||||
public PersistentSetCache(final String serviceIdentifier, final File persistencePath, final SetCache cacheToWrap) throws IOException {
|
public PersistentSetCache(final String serviceIdentifier, final File persistencePath, final SetCache cacheToWrap) throws IOException {
|
||||||
wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null);
|
wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null);
|
||||||
wrapped = cacheToWrap;
|
wrapped = cacheToWrap;
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void restore() throws IOException {
|
public synchronized void restore() throws IOException {
|
||||||
final Collection<SetRecord> recovered = wali.recoverRecords();
|
final Collection<SetRecord> recovered = wali.recoverRecords();
|
||||||
for ( final SetRecord record : recovered ) {
|
for (final SetRecord record : recovered) {
|
||||||
if ( record.getUpdateType() == UpdateType.CREATE ) {
|
if (record.getUpdateType() == UpdateType.CREATE) {
|
||||||
addIfAbsent(record.getBuffer());
|
addIfAbsent(record.getBuffer());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized SetCacheResult remove(final ByteBuffer value) throws IOException {
|
public synchronized SetCacheResult remove(final ByteBuffer value) throws IOException {
|
||||||
final SetCacheResult removeResult = wrapped.remove(value);
|
final SetCacheResult removeResult = wrapped.remove(value);
|
||||||
if ( removeResult.getResult() ) {
|
if (removeResult.getResult()) {
|
||||||
final SetRecord record = new SetRecord(UpdateType.DELETE, value);
|
final SetRecord record = new SetRecord(UpdateType.DELETE, value);
|
||||||
final List<SetRecord> records = new ArrayList<>();
|
final List<SetRecord> records = new ArrayList<>();
|
||||||
records.add(record);
|
records.add(record);
|
||||||
wali.update(records, false);
|
wali.update(records, false);
|
||||||
|
|
||||||
final long modCount = modifications.getAndIncrement();
|
final long modCount = modifications.getAndIncrement();
|
||||||
if ( modCount > 0 && modCount % 1000 == 0 ) {
|
if (modCount > 0 && modCount % 1000 == 0) {
|
||||||
wali.checkpoint();
|
wali.checkpoint();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -76,24 +76,24 @@ public class PersistentSetCache implements SetCache {
|
||||||
@Override
|
@Override
|
||||||
public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) throws IOException {
|
public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) throws IOException {
|
||||||
final SetCacheResult addResult = wrapped.addIfAbsent(value);
|
final SetCacheResult addResult = wrapped.addIfAbsent(value);
|
||||||
if ( addResult.getResult() ) {
|
if (addResult.getResult()) {
|
||||||
final SetRecord record = new SetRecord(UpdateType.CREATE, value);
|
final SetRecord record = new SetRecord(UpdateType.CREATE, value);
|
||||||
final List<SetRecord> records = new ArrayList<>();
|
final List<SetRecord> records = new ArrayList<>();
|
||||||
records.add(record);
|
records.add(record);
|
||||||
|
|
||||||
final SetCacheRecord evictedRecord = addResult.getEvictedRecord();
|
final SetCacheRecord evictedRecord = addResult.getEvictedRecord();
|
||||||
if ( evictedRecord != null ) {
|
if (evictedRecord != null) {
|
||||||
records.add(new SetRecord(UpdateType.DELETE, evictedRecord.getValue()));
|
records.add(new SetRecord(UpdateType.DELETE, evictedRecord.getValue()));
|
||||||
}
|
}
|
||||||
|
|
||||||
wali.update(records, false);
|
wali.update(records, false);
|
||||||
|
|
||||||
final long modCount = modifications.getAndIncrement();
|
final long modCount = modifications.getAndIncrement();
|
||||||
if ( modCount > 0 && modCount % 1000 == 0 ) {
|
if (modCount > 0 && modCount % 1000 == 0) {
|
||||||
wali.checkpoint();
|
wali.checkpoint();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return addResult;
|
return addResult;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -101,45 +101,46 @@ public class PersistentSetCache implements SetCache {
|
||||||
public synchronized SetCacheResult contains(final ByteBuffer value) throws IOException {
|
public synchronized SetCacheResult contains(final ByteBuffer value) throws IOException {
|
||||||
return wrapped.contains(value);
|
return wrapped.contains(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void shutdown() throws IOException {
|
public void shutdown() throws IOException {
|
||||||
wali.shutdown();
|
wali.shutdown();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class SetRecord {
|
private static class SetRecord {
|
||||||
|
|
||||||
private final UpdateType updateType;
|
private final UpdateType updateType;
|
||||||
private final ByteBuffer value;
|
private final ByteBuffer value;
|
||||||
|
|
||||||
public SetRecord(final UpdateType updateType, final ByteBuffer value) {
|
public SetRecord(final UpdateType updateType, final ByteBuffer value) {
|
||||||
this.updateType = updateType;
|
this.updateType = updateType;
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
public UpdateType getUpdateType() {
|
public UpdateType getUpdateType() {
|
||||||
return updateType;
|
return updateType;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getBuffer() {
|
public ByteBuffer getBuffer() {
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
public byte[] getData() {
|
public byte[] getData() {
|
||||||
return value.array();
|
return value.array();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class Serde implements SerDe<SetRecord> {
|
private static class Serde implements SerDe<SetRecord> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serializeEdit(final SetRecord previousRecordState, final SetRecord newRecordState, final DataOutputStream out) throws IOException {
|
public void serializeEdit(final SetRecord previousRecordState, final SetRecord newRecordState, final DataOutputStream out) throws IOException {
|
||||||
final UpdateType updateType = newRecordState.getUpdateType();
|
final UpdateType updateType = newRecordState.getUpdateType();
|
||||||
if ( updateType == UpdateType.DELETE ) {
|
if (updateType == UpdateType.DELETE) {
|
||||||
out.write(0);
|
out.write(0);
|
||||||
} else {
|
} else {
|
||||||
out.write(1);
|
out.write(1);
|
||||||
}
|
}
|
||||||
|
|
||||||
final byte[] data = newRecordState.getData();
|
final byte[] data = newRecordState.getData();
|
||||||
out.writeInt(data.length);
|
out.writeInt(data.length);
|
||||||
out.write(newRecordState.getData());
|
out.write(newRecordState.getData());
|
||||||
|
@ -153,16 +154,16 @@ public class PersistentSetCache implements SetCache {
|
||||||
@Override
|
@Override
|
||||||
public SetRecord deserializeEdit(final DataInputStream in, final Map<Object, SetRecord> currentRecordStates, final int version) throws IOException {
|
public SetRecord deserializeEdit(final DataInputStream in, final Map<Object, SetRecord> currentRecordStates, final int version) throws IOException {
|
||||||
final int value = in.read();
|
final int value = in.read();
|
||||||
if ( value < 0 ) {
|
if (value < 0) {
|
||||||
throw new EOFException();
|
throw new EOFException();
|
||||||
}
|
}
|
||||||
|
|
||||||
final UpdateType updateType = (value == 0 ? UpdateType.DELETE : UpdateType.CREATE);
|
final UpdateType updateType = (value == 0 ? UpdateType.DELETE : UpdateType.CREATE);
|
||||||
|
|
||||||
final int size = in.readInt();
|
final int size = in.readInt();
|
||||||
final byte[] data = new byte[size];
|
final byte[] data = new byte[size];
|
||||||
in.readFully(data);
|
in.readFully(data);
|
||||||
|
|
||||||
return new SetRecord(updateType, ByteBuffer.wrap(data));
|
return new SetRecord(updateType, ByteBuffer.wrap(data));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -22,8 +22,11 @@ import java.nio.ByteBuffer;
|
||||||
public interface SetCache {
|
public interface SetCache {
|
||||||
|
|
||||||
SetCacheResult remove(ByteBuffer value) throws IOException;
|
SetCacheResult remove(ByteBuffer value) throws IOException;
|
||||||
|
|
||||||
SetCacheResult addIfAbsent(ByteBuffer value) throws IOException;
|
SetCacheResult addIfAbsent(ByteBuffer value) throws IOException;
|
||||||
|
|
||||||
SetCacheResult contains(ByteBuffer value) throws IOException;
|
SetCacheResult contains(ByteBuffer value) throws IOException;
|
||||||
|
|
||||||
void shutdown() throws IOException;
|
void shutdown() throws IOException;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,33 +21,34 @@ import java.nio.ByteBuffer;
|
||||||
import org.apache.nifi.distributed.cache.server.CacheRecord;
|
import org.apache.nifi.distributed.cache.server.CacheRecord;
|
||||||
|
|
||||||
public class SetCacheRecord extends CacheRecord {
|
public class SetCacheRecord extends CacheRecord {
|
||||||
|
|
||||||
private final ByteBuffer value;
|
private final ByteBuffer value;
|
||||||
|
|
||||||
public SetCacheRecord(final ByteBuffer value) {
|
public SetCacheRecord(final ByteBuffer value) {
|
||||||
this.value = value;
|
this.value = value;
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer getValue() {
|
public ByteBuffer getValue() {
|
||||||
return value;
|
return value;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode() {
|
public int hashCode() {
|
||||||
return value.hashCode();
|
return value.hashCode();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(final Object obj) {
|
public boolean equals(final Object obj) {
|
||||||
if ( this == obj ) {
|
if (this == obj) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (obj instanceof SetCacheRecord) {
|
if (obj instanceof SetCacheRecord) {
|
||||||
return value.equals(((SetCacheRecord) obj).value);
|
return value.equals(((SetCacheRecord) obj).value);
|
||||||
}
|
}
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "SetCacheRecord[value=" + new String(value.array()) + ", hitCount=" + getHitCount() + "]";
|
return "SetCacheRecord[value=" + new String(value.array()) + ", hitCount=" + getHitCount() + "]";
|
||||||
|
|
|
@ -16,27 +16,26 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.nifi.distributed.cache.server.set;
|
package org.apache.nifi.distributed.cache.server.set;
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
public class SetCacheResult {
|
public class SetCacheResult {
|
||||||
|
|
||||||
private final boolean result;
|
private final boolean result;
|
||||||
private final SetCacheRecord stats;
|
private final SetCacheRecord stats;
|
||||||
private final SetCacheRecord evictedRecord;
|
private final SetCacheRecord evictedRecord;
|
||||||
|
|
||||||
public SetCacheResult(final boolean result, final SetCacheRecord stats, final SetCacheRecord evictedRecord) {
|
public SetCacheResult(final boolean result, final SetCacheRecord stats, final SetCacheRecord evictedRecord) {
|
||||||
this.result = result;
|
this.result = result;
|
||||||
this.stats = stats;
|
this.stats = stats;
|
||||||
this.evictedRecord = evictedRecord;
|
this.evictedRecord = evictedRecord;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean getResult() {
|
public boolean getResult() {
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
public SetCacheRecord getRecord() {
|
public SetCacheRecord getRecord() {
|
||||||
return stats;
|
return stats;
|
||||||
}
|
}
|
||||||
|
|
||||||
public SetCacheRecord getEvictedRecord() {
|
public SetCacheRecord getEvictedRecord() {
|
||||||
return evictedRecord;
|
return evictedRecord;
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,41 +30,42 @@ import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
public class SimpleSetCache implements SetCache {
|
public class SimpleSetCache implements SetCache {
|
||||||
|
|
||||||
private static final Logger logger = LoggerFactory.getLogger(SimpleSetCache.class);
|
private static final Logger logger = LoggerFactory.getLogger(SimpleSetCache.class);
|
||||||
|
|
||||||
private final Map<ByteBuffer, SetCacheRecord> cache = new HashMap<>();
|
private final Map<ByteBuffer, SetCacheRecord> cache = new HashMap<>();
|
||||||
private final SortedMap<SetCacheRecord, ByteBuffer> inverseCacheMap;
|
private final SortedMap<SetCacheRecord, ByteBuffer> inverseCacheMap;
|
||||||
|
|
||||||
private final String serviceIdentifier;
|
private final String serviceIdentifier;
|
||||||
|
|
||||||
private final int maxSize;
|
private final int maxSize;
|
||||||
|
|
||||||
public SimpleSetCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) {
|
public SimpleSetCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) {
|
||||||
inverseCacheMap = new TreeMap<>(evictionPolicy.getComparator());
|
inverseCacheMap = new TreeMap<>(evictionPolicy.getComparator());
|
||||||
this.serviceIdentifier = serviceIdentifier;
|
this.serviceIdentifier = serviceIdentifier;
|
||||||
this.maxSize = maxSize;
|
this.maxSize = maxSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized SetCacheRecord evict() {
|
private synchronized SetCacheRecord evict() {
|
||||||
if ( cache.size() < maxSize ) {
|
if (cache.size() < maxSize) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
final SetCacheRecord recordToEvict = inverseCacheMap.firstKey();
|
final SetCacheRecord recordToEvict = inverseCacheMap.firstKey();
|
||||||
final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict);
|
final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict);
|
||||||
cache.remove(valueToEvict);
|
cache.remove(valueToEvict);
|
||||||
|
|
||||||
if ( logger.isDebugEnabled() ) {
|
if (logger.isDebugEnabled()) {
|
||||||
logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8));
|
logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8));
|
||||||
}
|
}
|
||||||
|
|
||||||
return recordToEvict;
|
return recordToEvict;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) {
|
public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) {
|
||||||
final SetCacheRecord record = cache.get(value);
|
final SetCacheRecord record = cache.get(value);
|
||||||
if ( record == null ) {
|
if (record == null) {
|
||||||
final SetCacheRecord evicted = evict();
|
final SetCacheRecord evicted = evict();
|
||||||
final SetCacheRecord newRecord = new SetCacheRecord(value);
|
final SetCacheRecord newRecord = new SetCacheRecord(value);
|
||||||
cache.put(value, newRecord);
|
cache.put(value, newRecord);
|
||||||
|
@ -75,42 +76,42 @@ public class SimpleSetCache implements SetCache {
|
||||||
inverseCacheMap.remove(record);
|
inverseCacheMap.remove(record);
|
||||||
record.hit();
|
record.hit();
|
||||||
inverseCacheMap.put(record, value);
|
inverseCacheMap.put(record, value);
|
||||||
|
|
||||||
return new SetCacheResult(false, record, null);
|
return new SetCacheResult(false, record, null);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized SetCacheResult contains(final ByteBuffer value) {
|
public synchronized SetCacheResult contains(final ByteBuffer value) {
|
||||||
final SetCacheRecord record = cache.get(value);
|
final SetCacheRecord record = cache.get(value);
|
||||||
if ( record == null ) {
|
if (record == null) {
|
||||||
return new SetCacheResult(false, null, null);
|
return new SetCacheResult(false, null, null);
|
||||||
} else {
|
} else {
|
||||||
// We have to remove the record and add it again in order to cause the Map to stay sorted
|
// We have to remove the record and add it again in order to cause the Map to stay sorted
|
||||||
inverseCacheMap.remove(record);
|
inverseCacheMap.remove(record);
|
||||||
record.hit();
|
record.hit();
|
||||||
inverseCacheMap.put(record, value);
|
inverseCacheMap.put(record, value);
|
||||||
|
|
||||||
return new SetCacheResult(true, record, null);
|
return new SetCacheResult(true, record, null);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized SetCacheResult remove(final ByteBuffer value) {
|
public synchronized SetCacheResult remove(final ByteBuffer value) {
|
||||||
final SetCacheRecord record = cache.remove(value);
|
final SetCacheRecord record = cache.remove(value);
|
||||||
if ( record == null ) {
|
if (record == null) {
|
||||||
return new SetCacheResult(false, null, null);
|
return new SetCacheResult(false, null, null);
|
||||||
} else {
|
} else {
|
||||||
inverseCacheMap.remove(record);
|
inverseCacheMap.remove(record);
|
||||||
return new SetCacheResult(true, record, null);
|
return new SetCacheResult(true, record, null);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "SimpleSetCache[service id=" + serviceIdentifier + "]";
|
return "SimpleSetCache[service id=" + serviceIdentifier + "]";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void shutdown() throws IOException {
|
public void shutdown() throws IOException {
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,36 +1,36 @@
|
||||||
<!DOCTYPE html>
|
<!DOCTYPE html>
|
||||||
<html lang="en">
|
<html lang="en">
|
||||||
<!--
|
<!--
|
||||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
contributor license agreements. See the NOTICE file distributed with
|
contributor license agreements. See the NOTICE file distributed with
|
||||||
this work for additional information regarding copyright ownership.
|
this work for additional information regarding copyright ownership.
|
||||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
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 not use this file except in compliance with
|
||||||
the License. You may obtain a copy of the License at
|
the License. You may obtain a copy of the License at
|
||||||
http://www.apache.org/licenses/LICENSE-2.0
|
http://www.apache.org/licenses/LICENSE-2.0
|
||||||
Unless required by applicable law or agreed to in writing, software
|
Unless required by applicable law or agreed to in writing, software
|
||||||
distributed under the License is distributed on an "AS IS" BASIS,
|
distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
See the License for the specific language governing permissions and
|
See the License for the specific language governing permissions and
|
||||||
limitations under the License.
|
limitations under the License.
|
||||||
-->
|
-->
|
||||||
<head>
|
<head>
|
||||||
<meta charset="utf-8" />
|
<meta charset="utf-8" />
|
||||||
<title>Distributed Map Cache Client Service</title>
|
<title>Distributed Map Cache Client Service</title>
|
||||||
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
|
<link rel="stylesheet" href="../../css/component-usage.css" type="text/css" />
|
||||||
</head>
|
</head>
|
||||||
|
|
||||||
<body>
|
<body>
|
||||||
<p>
|
<p>
|
||||||
Below is an example of how to create a distributed map cache server for clients to connect to.
|
Below is an example of how to create a distributed map cache server for clients to connect to.
|
||||||
Note that the identifier in this example is <code>cache-server</code>. If you are using this template
|
Note that the identifier in this example is <code>cache-server</code>. If you are using this template
|
||||||
to create your own DistributedMapCache server, replace the values in this template with values that are
|
to create your own DistributedMapCache server, replace the values in this template with values that are
|
||||||
suitable for your system. Possible options for <code>Port</code>, <code>Maximum Cache Entries</code>,
|
suitable for your system. Possible options for <code>Port</code>, <code>Maximum Cache Entries</code>,
|
||||||
<code>Eviction Strategy</code>, <span style="font-style: italic;">SSL Context Service</span>, and
|
<code>Eviction Strategy</code>, <span style="font-style: italic;">SSL Context Service</span>, and
|
||||||
<span style="font-style: italic;">Persistence Directory</span>
|
<span style="font-style: italic;">Persistence Directory</span>
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
<pre>
|
<pre>
|
||||||
<?xml version="1.0" encoding="UTF-8" ?>
|
<?xml version="1.0" encoding="UTF-8" ?>
|
||||||
<services>
|
<services>
|
||||||
<service>
|
<service>
|
||||||
|
@ -41,6 +41,6 @@
|
||||||
<property name="Eviction Strategy">Least Recently Used</property>
|
<property name="Eviction Strategy">Least Recently Used</property>
|
||||||
</service>
|
</service>
|
||||||
</services>
|
</services>
|
||||||
</pre>
|
</pre>
|
||||||
</body>
|
</body>
|
||||||
</html>
|
</html>
|
||||||
|
|
|
@ -152,7 +152,7 @@ public class TestServerAndClient {
|
||||||
newServer.shutdownServer();
|
newServer.shutdownServer();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
||||||
@Test
|
@Test
|
||||||
public void testPersistentSetServerAndClientWithLFUEvictions() throws InitializationException, IOException {
|
public void testPersistentSetServerAndClientWithLFUEvictions() throws InitializationException, IOException {
|
||||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||||
|
@ -215,7 +215,7 @@ public class TestServerAndClient {
|
||||||
newServer.shutdownServer();
|
newServer.shutdownServer();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
@Ignore("Test fails when in a maven parallel build due to address/port already taken - need to vary these so tests can run in parallel")
|
||||||
@Test
|
@Test
|
||||||
public void testPersistentSetServerAndClientWithFIFOEvictions() throws InitializationException, IOException {
|
public void testPersistentSetServerAndClientWithFIFOEvictions() throws InitializationException, IOException {
|
||||||
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName());
|
||||||
|
@ -374,8 +374,7 @@ public class TestServerAndClient {
|
||||||
public void testClientTermination() throws InitializationException, IOException, InterruptedException {
|
public void testClientTermination() throws InitializationException, IOException, InterruptedException {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug
|
* This bypasses the test for build environments in OS X running Java 1.8 due to a JVM bug See: https://issues.apache.org/jira/browse/NIFI-437
|
||||||
* See: https://issues.apache.org/jira/browse/NIFI-437
|
|
||||||
*/
|
*/
|
||||||
Assume.assumeFalse("testClientTermination is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
Assume.assumeFalse("testClientTermination is skipped due to build environment being OS X with JDK 1.8. See https://issues.apache.org/jira/browse/NIFI-437",
|
||||||
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
SystemUtils.IS_OS_MAC && SystemUtils.IS_JAVA_1_8);
|
||||||
|
@ -509,6 +508,7 @@ public class TestServerAndClient {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class StringSerializer implements Serializer<String> {
|
private static class StringSerializer implements Serializer<String> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serialize(final String value, final OutputStream output) throws SerializationException, IOException {
|
public void serialize(final String value, final OutputStream output) throws SerializationException, IOException {
|
||||||
output.write(value.getBytes(StandardCharsets.UTF_8));
|
output.write(value.getBytes(StandardCharsets.UTF_8));
|
||||||
|
@ -516,6 +516,7 @@ public class TestServerAndClient {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class StringDeserializer implements Deserializer<String> {
|
private static class StringDeserializer implements Deserializer<String> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String deserialize(final byte[] input) throws DeserializationException, IOException {
|
public String deserialize(final byte[] input) throws DeserializationException, IOException {
|
||||||
return (input.length == 0) ? null : new String(input, StandardCharsets.UTF_8);
|
return (input.length == 0) ? null : new String(input, StandardCharsets.UTF_8);
|
||||||
|
|
|
@ -14,24 +14,24 @@
|
||||||
limitations under the License.
|
limitations under the License.
|
||||||
-->
|
-->
|
||||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<modelVersion>4.0.0</modelVersion>
|
||||||
<parent>
|
<parent>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-standard-services</artifactId>
|
<artifactId>nifi-standard-services</artifactId>
|
||||||
<version>0.1.0-incubating-SNAPSHOT</version>
|
<version>0.1.0-incubating-SNAPSHOT</version>
|
||||||
</parent>
|
</parent>
|
||||||
|
|
||||||
<artifactId>nifi-http-context-map-api</artifactId>
|
<artifactId>nifi-http-context-map-api</artifactId>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-api</artifactId>
|
<artifactId>nifi-api</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>javax.servlet</groupId>
|
<groupId>javax.servlet</groupId>
|
||||||
<artifactId>javax.servlet-api</artifactId>
|
<artifactId>javax.servlet-api</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|
|
@ -22,51 +22,48 @@ import javax.servlet.http.HttpServletResponse;
|
||||||
|
|
||||||
import org.apache.nifi.controller.ControllerService;
|
import org.apache.nifi.controller.ControllerService;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p>
|
||||||
* An interface that provides the capability of receiving an HTTP servlet request in one component
|
* An interface that provides the capability of receiving an HTTP servlet request in one component and responding to that request in another component.
|
||||||
* and responding to that request in another component.
|
|
||||||
* </p>
|
* </p>
|
||||||
*
|
*
|
||||||
* <p>
|
* <p>
|
||||||
* The intended flow is for the component receiving the HTTP request to register the request, response,
|
* The intended flow is for the component receiving the HTTP request to register the request, response, and AsyncContext with a particular identifier via the
|
||||||
* and AsyncContext with a particular identifier via the
|
* {@link #register(String, HttpServletRequest, HttpServletResponse, AsyncContext)} method. Another component is then able to obtain the response by providing that identifier to the
|
||||||
* {@link #register(String, HttpServletRequest, HttpServletResponse, AsyncContext)}
|
* {@link #getResponse(String)} method. After writing to the HttpServletResponse, the transaction is to then be completed via the {@link #complete(String)} method.
|
||||||
* method. Another component is then able to obtain the response
|
|
||||||
* by providing that identifier to the {@link #getResponse(String)} method. After writing to the
|
|
||||||
* HttpServletResponse, the transaction is to then be completed via the {@link #complete(String)} method.
|
|
||||||
* </p>
|
* </p>
|
||||||
*/
|
*/
|
||||||
public interface HttpContextMap extends ControllerService {
|
public interface HttpContextMap extends ControllerService {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Registers an HttpServletRequest, HttpServletResponse, and the AsyncContext for a given identifier
|
* Registers an HttpServletRequest, HttpServletResponse, and the AsyncContext for a given identifier
|
||||||
*
|
*
|
||||||
* @param identifier
|
* @param identifier identifier
|
||||||
* @param request
|
* @param request request
|
||||||
* @param response
|
* @param response response
|
||||||
* @param context
|
* @param context context
|
||||||
*
|
*
|
||||||
* @return true if register is successful, false if the context map is too full because too many requests have already been received and not processed
|
* @return true if register is successful, false if the context map is too full because too many requests have already been received and not processed
|
||||||
*
|
*
|
||||||
* @throws IllegalStateException if the identifier is already registered
|
* @throws IllegalStateException if the identifier is already registered
|
||||||
*/
|
*/
|
||||||
boolean register(String identifier, HttpServletRequest request, HttpServletResponse response, AsyncContext context);
|
boolean register(String identifier, HttpServletRequest request, HttpServletResponse response, AsyncContext context);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieves the HttpServletResponse for the given identifier, if it exists
|
* Retrieves the HttpServletResponse for the given identifier, if it exists
|
||||||
* @param identifier
|
*
|
||||||
|
* @param identifier identifier
|
||||||
* @return the HttpServletResponse for the given identifier, or {@code null} if it does not exist
|
* @return the HttpServletResponse for the given identifier, or {@code null} if it does not exist
|
||||||
*/
|
*/
|
||||||
HttpServletResponse getResponse(String identifier);
|
HttpServletResponse getResponse(String identifier);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Marks the HTTP request/response for the given identifier as complete
|
* Marks the HTTP request/response for the given identifier as complete
|
||||||
* @param identifier
|
*
|
||||||
*
|
* @param identifier identifier
|
||||||
|
*
|
||||||
* @throws IllegalStateException if the identifier is not registered to a valid AsyncContext
|
* @throws IllegalStateException if the identifier is not registered to a valid AsyncContext
|
||||||
*/
|
*/
|
||||||
void complete(String identifier);
|
void complete(String identifier);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,21 +24,21 @@
|
||||||
<packaging>jar</packaging>
|
<packaging>jar</packaging>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-api</artifactId>
|
<artifactId>nifi-api</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-processor-utils</artifactId>
|
<artifactId>nifi-processor-utils</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.nifi</groupId>
|
<groupId>org.apache.nifi</groupId>
|
||||||
<artifactId>nifi-http-context-map-api</artifactId>
|
<artifactId>nifi-http-context-map-api</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>javax.servlet</groupId>
|
<groupId>javax.servlet</groupId>
|
||||||
<artifactId>javax.servlet-api</artifactId>
|
<artifactId>javax.servlet-api</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
</dependencies>
|
</dependencies>
|
||||||
</project>
|
</project>
|
||||||
|
|
|
@ -42,34 +42,35 @@ import org.apache.nifi.controller.ConfigurationContext;
|
||||||
import org.apache.nifi.processor.util.StandardValidators;
|
import org.apache.nifi.processor.util.StandardValidators;
|
||||||
|
|
||||||
@Tags({"http", "request", "response"})
|
@Tags({"http", "request", "response"})
|
||||||
@SeeAlso(classNames={
|
@SeeAlso(classNames = {
|
||||||
"org.apache.nifi.processors.standard.HandleHttpRequest",
|
"org.apache.nifi.processors.standard.HandleHttpRequest",
|
||||||
"org.apache.nifi.processors.standard.HandleHttpResponse"})
|
"org.apache.nifi.processors.standard.HandleHttpResponse"})
|
||||||
@CapabilityDescription("Provides the ability to store and retrieve HTTP requests and responses external to a Processor, so that "
|
@CapabilityDescription("Provides the ability to store and retrieve HTTP requests and responses external to a Processor, so that "
|
||||||
+ "multiple Processors can interact with the same HTTP request.")
|
+ "multiple Processors can interact with the same HTTP request.")
|
||||||
public class StandardHttpContextMap extends AbstractControllerService implements HttpContextMap {
|
public class StandardHttpContextMap extends AbstractControllerService implements HttpContextMap {
|
||||||
|
|
||||||
public static final PropertyDescriptor MAX_OUTSTANDING_REQUESTS = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor MAX_OUTSTANDING_REQUESTS = new PropertyDescriptor.Builder()
|
||||||
.name("Maximum Outstanding Requests")
|
.name("Maximum Outstanding Requests")
|
||||||
.description("The maximum number of HTTP requests that can be outstanding at any one time. Any attempt to register an additional HTTP Request will cause an error")
|
.description("The maximum number of HTTP requests that can be outstanding at any one time. Any attempt to register an additional HTTP Request will cause an error")
|
||||||
.required(true)
|
.required(true)
|
||||||
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
.addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
|
||||||
.defaultValue("5000")
|
.defaultValue("5000")
|
||||||
.build();
|
.build();
|
||||||
public static final PropertyDescriptor REQUEST_EXPIRATION = new PropertyDescriptor.Builder()
|
public static final PropertyDescriptor REQUEST_EXPIRATION = new PropertyDescriptor.Builder()
|
||||||
.name("Request Expiration")
|
.name("Request Expiration")
|
||||||
.description("Specifies how long an HTTP Request should be left unanswered before being evicted from the cache and being responded to with a Service Unavailable status code")
|
.description("Specifies how long an HTTP Request should be left unanswered before being evicted from the cache and being responded to with a Service Unavailable status code")
|
||||||
.required(true)
|
.required(true)
|
||||||
.expressionLanguageSupported(false)
|
.expressionLanguageSupported(false)
|
||||||
.defaultValue("1 min")
|
.defaultValue("1 min")
|
||||||
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
.addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
private final ConcurrentMap<String, Wrapper> wrapperMap = new ConcurrentHashMap<>();
|
private final ConcurrentMap<String, Wrapper> wrapperMap = new ConcurrentHashMap<>();
|
||||||
|
|
||||||
private volatile int maxSize = 5000;
|
private volatile int maxSize = 5000;
|
||||||
private volatile long maxRequestNanos;
|
private volatile long maxRequestNanos;
|
||||||
private volatile ScheduledExecutorService executor;
|
private volatile ScheduledExecutorService executor;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
|
||||||
final List<PropertyDescriptor> properties = new ArrayList<>(2);
|
final List<PropertyDescriptor> properties = new ArrayList<>(2);
|
||||||
|
@ -77,67 +78,68 @@ public class StandardHttpContextMap extends AbstractControllerService implements
|
||||||
properties.add(REQUEST_EXPIRATION);
|
properties.add(REQUEST_EXPIRATION);
|
||||||
return properties;
|
return properties;
|
||||||
}
|
}
|
||||||
|
|
||||||
@OnEnabled
|
@OnEnabled
|
||||||
public void onConfigured(final ConfigurationContext context) {
|
public void onConfigured(final ConfigurationContext context) {
|
||||||
maxSize = context.getProperty(MAX_OUTSTANDING_REQUESTS).asInteger();
|
maxSize = context.getProperty(MAX_OUTSTANDING_REQUESTS).asInteger();
|
||||||
executor = Executors.newSingleThreadScheduledExecutor();
|
executor = Executors.newSingleThreadScheduledExecutor();
|
||||||
|
|
||||||
maxRequestNanos = context.getProperty(REQUEST_EXPIRATION).asTimePeriod(TimeUnit.NANOSECONDS);
|
maxRequestNanos = context.getProperty(REQUEST_EXPIRATION).asTimePeriod(TimeUnit.NANOSECONDS);
|
||||||
final long scheduleNanos = maxRequestNanos / 2;
|
final long scheduleNanos = maxRequestNanos / 2;
|
||||||
executor.scheduleWithFixedDelay(new CleanupExpiredRequests(), scheduleNanos, scheduleNanos, TimeUnit.NANOSECONDS);
|
executor.scheduleWithFixedDelay(new CleanupExpiredRequests(), scheduleNanos, scheduleNanos, TimeUnit.NANOSECONDS);
|
||||||
}
|
}
|
||||||
|
|
||||||
@OnDisabled
|
@OnDisabled
|
||||||
public void cleanup() {
|
public void cleanup() {
|
||||||
if ( executor != null ) {
|
if (executor != null) {
|
||||||
executor.shutdown();
|
executor.shutdown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean register(final String identifier, final HttpServletRequest request, final HttpServletResponse response, final AsyncContext context) {
|
public boolean register(final String identifier, final HttpServletRequest request, final HttpServletResponse response, final AsyncContext context) {
|
||||||
// fail if there are too many already. Maybe add a configuration property for how many
|
// fail if there are too many already. Maybe add a configuration property for how many
|
||||||
// outstanding, with a default of say 5000
|
// outstanding, with a default of say 5000
|
||||||
if ( wrapperMap.size() >= maxSize ) {
|
if (wrapperMap.size() >= maxSize) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
final Wrapper wrapper = new Wrapper(request, response, context);
|
final Wrapper wrapper = new Wrapper(request, response, context);
|
||||||
final Wrapper existing = wrapperMap.putIfAbsent(identifier, wrapper);
|
final Wrapper existing = wrapperMap.putIfAbsent(identifier, wrapper);
|
||||||
if ( existing != null ) {
|
if (existing != null) {
|
||||||
throw new IllegalStateException("HTTP Request already registered with identifier " + identifier);
|
throw new IllegalStateException("HTTP Request already registered with identifier " + identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public HttpServletResponse getResponse(final String identifier) {
|
public HttpServletResponse getResponse(final String identifier) {
|
||||||
final Wrapper wrapper = wrapperMap.get(identifier);
|
final Wrapper wrapper = wrapperMap.get(identifier);
|
||||||
if ( wrapper == null ) {
|
if (wrapper == null) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
return wrapper.getResponse();
|
return wrapper.getResponse();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void complete(final String identifier) {
|
public void complete(final String identifier) {
|
||||||
final Wrapper wrapper = wrapperMap.remove(identifier);
|
final Wrapper wrapper = wrapperMap.remove(identifier);
|
||||||
if ( wrapper == null ) {
|
if (wrapper == null) {
|
||||||
throw new IllegalStateException("No HTTP Request registered with identifier " + identifier);
|
throw new IllegalStateException("No HTTP Request registered with identifier " + identifier);
|
||||||
}
|
}
|
||||||
|
|
||||||
wrapper.getAsync().complete();
|
wrapper.getAsync().complete();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class Wrapper {
|
private static class Wrapper {
|
||||||
|
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
private final HttpServletRequest request;
|
private final HttpServletRequest request;
|
||||||
private final HttpServletResponse response;
|
private final HttpServletResponse response;
|
||||||
private final AsyncContext async;
|
private final AsyncContext async;
|
||||||
private final long nanoTimeAdded = System.nanoTime();
|
private final long nanoTimeAdded = System.nanoTime();
|
||||||
|
|
||||||
public Wrapper(final HttpServletRequest request, final HttpServletResponse response, final AsyncContext async) {
|
public Wrapper(final HttpServletRequest request, final HttpServletResponse response, final AsyncContext async) {
|
||||||
this.request = request;
|
this.request = request;
|
||||||
this.response = response;
|
this.response = response;
|
||||||
|
@ -151,24 +153,25 @@ public class StandardHttpContextMap extends AbstractControllerService implements
|
||||||
public AsyncContext getAsync() {
|
public AsyncContext getAsync() {
|
||||||
return async;
|
return async;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getNanoTimeAdded() {
|
public long getNanoTimeAdded() {
|
||||||
return nanoTimeAdded;
|
return nanoTimeAdded;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private class CleanupExpiredRequests implements Runnable {
|
private class CleanupExpiredRequests implements Runnable {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
final long now = System.nanoTime();
|
final long now = System.nanoTime();
|
||||||
final long threshold = now - maxRequestNanos;
|
final long threshold = now - maxRequestNanos;
|
||||||
|
|
||||||
final Iterator<Map.Entry<String, Wrapper>> itr = wrapperMap.entrySet().iterator();
|
final Iterator<Map.Entry<String, Wrapper>> itr = wrapperMap.entrySet().iterator();
|
||||||
while ( itr.hasNext() ) {
|
while (itr.hasNext()) {
|
||||||
final Map.Entry<String, Wrapper> entry = itr.next();
|
final Map.Entry<String, Wrapper> entry = itr.next();
|
||||||
if ( entry.getValue().getNanoTimeAdded() < threshold ) {
|
if (entry.getValue().getNanoTimeAdded() < threshold) {
|
||||||
itr.remove();
|
itr.remove();
|
||||||
|
|
||||||
// send SERVICE_UNAVAILABLE
|
// send SERVICE_UNAVAILABLE
|
||||||
try {
|
try {
|
||||||
final AsyncContext async = entry.getValue().getAsync();
|
final AsyncContext async = entry.getValue().getAsync();
|
||||||
|
|
|
@ -22,15 +22,15 @@
|
||||||
</head>
|
</head>
|
||||||
|
|
||||||
<body>
|
<body>
|
||||||
<h2>Description:</h2>
|
<h2>Description:</h2>
|
||||||
<p>
|
<p>
|
||||||
This is the standard implementation of the SSL Context Map. This service is used to provide
|
This is the standard implementation of the SSL Context Map. This service is used to provide
|
||||||
coordination between
|
coordination between
|
||||||
<a href="../org.apache.nifi.processors.standard.HandleHttpRequest/index.html">HandleHttpRequest</a>
|
<a href="../org.apache.nifi.processors.standard.HandleHttpRequest/index.html">HandleHttpRequest</a>
|
||||||
and
|
and
|
||||||
<a href="../org.apache.nifi.processors.standard.HandleHttpResponse/index.html">HandleHttpResponse</a>
|
<a href="../org.apache.nifi.processors.standard.HandleHttpResponse/index.html">HandleHttpResponse</a>
|
||||||
Processors.
|
Processors.
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
<!-- Service Documentation ================================================== -->
|
<!-- Service Documentation ================================================== -->
|
||||||
<h2>Configuring the HTTP Context Map:</h2>
|
<h2>Configuring the HTTP Context Map:</h2>
|
||||||
|
@ -40,9 +40,9 @@
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
<p>
|
<p>
|
||||||
This controller service exposes a single property named <code>Maximum Outstanding Requests</code>.
|
This controller service exposes a single property named <code>Maximum Outstanding Requests</code>.
|
||||||
This property determines the maximum number of HTTP requests that can be outstanding at any one time.
|
This property determines the maximum number of HTTP requests that can be outstanding at any one time.
|
||||||
Any attempt to register an additional HTTP Request will cause an error. The default value is 5000.
|
Any attempt to register an additional HTTP Request will cause an error. The default value is 5000.
|
||||||
Below is an example of the template for a StandardHttpContextMap controller service.
|
Below is an example of the template for a StandardHttpContextMap controller service.
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
|
@ -56,12 +56,12 @@
|
||||||
</service>
|
</service>
|
||||||
</services>
|
</services>
|
||||||
</pre>
|
</pre>
|
||||||
|
|
||||||
<p>
|
<p>
|
||||||
<strong>See Also:</strong><br />
|
<strong>See Also:</strong><br />
|
||||||
<a href="../org.apache.nifi.processors.standard.HandleHttpRequest/index.html">HandleHttpRequest</a><br />
|
<a href="../org.apache.nifi.processors.standard.HandleHttpRequest/index.html">HandleHttpRequest</a><br />
|
||||||
<a href="../org.apache.nifi.processors.standard.HandleHttpResponse/index.html">HandleHttpResponse</a><br />
|
<a href="../org.apache.nifi.processors.standard.HandleHttpResponse/index.html">HandleHttpResponse</a><br />
|
||||||
</p>
|
</p>
|
||||||
|
|
||||||
</body>
|
</body>
|
||||||
</html>
|
</html>
|
||||||
|
|
|
@ -205,7 +205,7 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
||||||
}
|
}
|
||||||
return results;
|
return results;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void verifySslConfig(final ValidationContext validationContext) throws ProcessException {
|
private void verifySslConfig(final ValidationContext validationContext) throws ProcessException {
|
||||||
try {
|
try {
|
||||||
final String keystoreFile = validationContext.getProperty(KEYSTORE).getValue();
|
final String keystoreFile = validationContext.getProperty(KEYSTORE).getValue();
|
||||||
|
@ -237,7 +237,6 @@ public class StandardSSLContextService extends AbstractControllerService impleme
|
||||||
throw new ProcessException(e);
|
throw new ProcessException(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public SSLContext createSSLContext(final ClientAuth clientAuth) throws ProcessException {
|
public SSLContext createSSLContext(final ClientAuth clientAuth) throws ProcessException {
|
||||||
|
|
|
@ -73,7 +73,7 @@ public class SSLContextServiceTest {
|
||||||
properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "wrongpassword");
|
properties.put(StandardSSLContextService.TRUSTSTORE_PASSWORD.getName(), "wrongpassword");
|
||||||
properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
|
properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
|
||||||
runner.addControllerService("test-bad4", service, properties);
|
runner.addControllerService("test-bad4", service, properties);
|
||||||
|
|
||||||
runner.assertNotValid(service);
|
runner.assertNotValid(service);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -126,7 +126,7 @@ public class SSLContextServiceTest {
|
||||||
properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
|
properties.put(StandardSSLContextService.TRUSTSTORE_TYPE.getName(), "JKS");
|
||||||
runner.addControllerService("test-good2", service, properties);
|
runner.addControllerService("test-good2", service, properties);
|
||||||
runner.enableControllerService(service);
|
runner.enableControllerService(service);
|
||||||
|
|
||||||
runner.setProperty("SSL Context Svc ID", "test-good2");
|
runner.setProperty("SSL Context Svc ID", "test-good2");
|
||||||
runner.assertValid();
|
runner.assertValid();
|
||||||
Assert.assertNotNull(service);
|
Assert.assertNotNull(service);
|
||||||
|
|
Loading…
Reference in New Issue