You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2015/10/21 16:45:33 UTC

[1/4] nifi git commit: NIFI-988: PutDisributedMapCache processor implementation

Repository: nifi
Updated Branches:
  refs/heads/master e68fdca51 -> da28b81ee


NIFI-988: PutDisributedMapCache processor implementation


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/6b1328f3
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/6b1328f3
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/6b1328f3

Branch: refs/heads/master
Commit: 6b1328f3f181a27a5856d26983ed3329ee317522
Parents: 14eaeeb
Author: Joe <jo...@impresstv.com>
Authored: Wed Sep 23 13:16:02 2015 +0200
Committer: Joe <jo...@impresstv.com>
Committed: Wed Sep 23 14:31:13 2015 +0200

----------------------------------------------------------------------
 .../standard/PutDistributedMapCache.java        | 244 +++++++++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 2 files changed, 245 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/6b1328f3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java
new file mode 100644
index 0000000..8e50c9f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDistributedMapCache.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.standard;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.distributed.cache.client.exception.DeserializationException;
+import org.apache.nifi.distributed.cache.client.exception.SerializationException;
+import org.apache.nifi.expression.AttributeExpression.ResultType;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ProcessorLog;
+import org.apache.nifi.processor.*;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.charset.StandardCharsets;
+import java.util.*;
+
+@EventDriven
+@SupportsBatching
+@Tags({"map", "cache", "put", "distributed"})
+@CapabilityDescription("Gets the content of a FlowFile and puts it to a distributed map cache, using a cache key " +
+        "computed from FlowFile attributes. If the cache already contains the entry and the cache update strategy is " +
+        "'keep original' the entry is not replaced.'")
+@WritesAttribute(attribute = "cached", description = "All FlowFiles will have an attribute 'cached'. The value of this " +
+        "attribute is true, is the FlowFile is cached, otherwise false.")
+@SeeAlso(classNames = {"org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService", "org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer"})
+public class PutDistributedMapCache extends AbstractProcessor {
+
+    public static final String CACHED_ATTRIBUTE_NAME = "cached";
+
+    // Identifies the distributed map cache client
+    public static final PropertyDescriptor DISTRIBUTED_CACHE_SERVICE = new PropertyDescriptor.Builder()
+            .name("Distributed Cache Service")
+            .description("The Controller Service that is used to cache flow files")
+            .required(true)
+            .identifiesControllerService(DistributedMapCacheClient.class)
+            .build();
+
+    // Selects the FlowFile attribute, whose value is used as cache key
+    public static final PropertyDescriptor CACHE_ENTRY_IDENTIFIER = new PropertyDescriptor.Builder()
+            .name("Cache Entry Identifier")
+            .description("A FlowFile attribute, or the results of an Attribute Expression Language statement, which will " +
+                    "be evaluated against a FlowFile in order to determine the cache key")
+            .required(true)
+            .addValidator(StandardValidators.createAttributeExpressionLanguageValidator(ResultType.STRING, true))
+            .expressionLanguageSupported(true)
+            .build();
+
+    public static final AllowableValue CACHE_UPDATE_REPLACE = new AllowableValue("replace", "Replace if present",
+            "Adds the specified entry to the cache, replacing any value that is currently set.");
+
+    public static final AllowableValue CACHE_UPDATE_KEEP_ORIGINAL = new AllowableValue("keeporiginal", "Keep original",
+            "Adds the specified entry to the cache, if the key does not exist.");
+
+    public static final PropertyDescriptor CACHE_UPDATE_STRATEGY = new PropertyDescriptor.Builder()
+            .name("Cache update strategy")
+            .description("Determines how the cache is updated if the cache already contains the entry")
+            .required(true)
+            .allowableValues(CACHE_UPDATE_REPLACE, CACHE_UPDATE_KEEP_ORIGINAL)
+            .defaultValue(CACHE_UPDATE_REPLACE.getValue())
+            .build();
+
+    public static final PropertyDescriptor CACHE_ENTRY_MAX_BYTES = new PropertyDescriptor.Builder()
+            .name("Max cache entry size")
+            .description("The maximum amount of data to put into cache")
+            .required(false)
+            .addValidator(StandardValidators.DATA_SIZE_VALIDATOR)
+            .defaultValue("1 MB")
+            .expressionLanguageSupported(false)
+            .build();
+
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Any FlowFile that is successfully inserted into cache will be routed to this relationship")
+            .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("Any FlowFile that cannot be inserted into the cache will be routed to this relationship")
+            .build();
+    private final Set<Relationship> relationships;
+
+    private final Serializer<String> keySerializer = new StringSerializer();
+    private final Serializer<byte[]> valueSerializer = new CacheValueSerializer();
+    private final Deserializer<byte[]> valueDeserializer = new CacheValueDeserializer();
+
+    public PutDistributedMapCache() {
+        final Set<Relationship> rels = new HashSet<>();
+        rels.add(REL_SUCCESS);
+        rels.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(rels);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        final List<PropertyDescriptor> descriptors = new ArrayList<>();
+        descriptors.add(CACHE_ENTRY_IDENTIFIER);
+        descriptors.add(DISTRIBUTED_CACHE_SERVICE);
+        descriptors.add(CACHE_UPDATE_STRATEGY);
+        descriptors.add(CACHE_ENTRY_MAX_BYTES);
+        return descriptors;
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ProcessorLog logger = getLogger();
+
+        // cache key is computed from attribute 'CACHE_ENTRY_IDENTIFIER' with expression language support
+        final String cacheKey = context.getProperty(CACHE_ENTRY_IDENTIFIER).evaluateAttributeExpressions(flowFile).getValue();
+
+        // if the computed value is null, or empty, we transfer the flow file to failure relationship
+        if (StringUtils.isBlank(cacheKey)) {
+            logger.error("FlowFile {} has no attribute for given Cache Entry Identifier", new Object[] {flowFile});
+            flowFile = session.penalize(flowFile);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        // the cache client used to interact with the distributed cache
+        final DistributedMapCacheClient cache = context.getProperty(DISTRIBUTED_CACHE_SERVICE).asControllerService(DistributedMapCacheClient.class);
+
+        try {
+
+            final long maxCacheEntrySize = context.getProperty(CACHE_ENTRY_MAX_BYTES).asDataSize(DataUnit.B).longValue();
+            long flowFileSize = flowFile.getSize();
+
+            // too big flow file
+            if (flowFileSize > maxCacheEntrySize) {
+                logger.warn("Flow file {} size {} exceeds the max cache entry size ({} B).", new Object[] {flowFile, flowFileSize, maxCacheEntrySize});
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+            }
+
+            if (flowFileSize == 0) {
+                logger.warn("Flow file {} is empty, there is nothing to cache.", new Object[] {flowFile});
+                session.transfer(flowFile, REL_FAILURE);
+                return;
+
+            }
+
+            // get flow file content
+            final ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+            session.exportTo(flowFile, byteStream);
+            byte[] cacheValue = byteStream.toByteArray();
+            final String updateStrategy = context.getProperty(CACHE_UPDATE_STRATEGY).getValue();
+            boolean cached = false;
+
+            if (updateStrategy.equals(CACHE_UPDATE_REPLACE.getValue())) {
+                cache.put(cacheKey, cacheValue, keySerializer, valueSerializer);
+                cached = true;
+            } else if (updateStrategy.equals(CACHE_UPDATE_KEEP_ORIGINAL.getValue())) {
+                final byte[] oldValue = cache.getAndPutIfAbsent(cacheKey, cacheValue, keySerializer, valueSerializer, valueDeserializer);
+                if (oldValue == null) {
+                    cached = true;
+                }
+            }
+
+            // set 'cached' attribute
+            flowFile = session.putAttribute(flowFile, CACHED_ATTRIBUTE_NAME, String.valueOf(cached));
+
+            if (cached) {
+                session.transfer(flowFile, REL_SUCCESS);
+            } else {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+
+        } catch (final IOException e) {
+            flowFile = session.penalize(flowFile);
+            session.transfer(flowFile, REL_FAILURE);
+            logger.error("Unable to communicate with cache when processing {} due to {}", new Object[]{flowFile, e});
+        }
+    }
+
+    public static class CacheValueSerializer implements Serializer<byte[]> {
+
+        @Override
+        public void serialize(final byte[] bytes, final OutputStream out) throws SerializationException, IOException {
+            out.write(bytes);
+        }
+    }
+
+    public static class CacheValueDeserializer implements Deserializer<byte[]> {
+
+        @Override
+        public byte[] deserialize(final byte[] input) throws DeserializationException, IOException {
+            if (input == null || input.length == 0) {
+                return null;
+            }
+            return input;
+        }
+    }
+
+    /**
+     * Simple string serializer, used for serializing the cache key
+     */
+    public static class StringSerializer implements Serializer<String> {
+
+        @Override
+        public void serialize(final String value, final OutputStream out) throws SerializationException, IOException {
+            out.write(value.getBytes(StandardCharsets.UTF_8));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi/blob/6b1328f3/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index 0ce1456..ff39ad3 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -49,6 +49,7 @@ org.apache.nifi.processors.standard.ModifyBytes
 org.apache.nifi.processors.standard.MonitorActivity
 org.apache.nifi.processors.standard.PostHTTP
 org.apache.nifi.processors.standard.PutEmail
+org.apache.nifi.processors.standard.PutDistributedMapCache
 org.apache.nifi.processors.standard.PutFile
 org.apache.nifi.processors.standard.PutFTP
 org.apache.nifi.processors.standard.PutJMS


[3/4] nifi git commit: Merge branch 'NIFI-988' of https://github.com/ImpressTV/nifi into NIFI-988

Posted by ma...@apache.org.
Merge branch 'NIFI-988' of https://github.com/ImpressTV/nifi into NIFI-988


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/97441ea0
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/97441ea0
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/97441ea0

Branch: refs/heads/master
Commit: 97441ea0c23ebdc7c51939ff28ba270012c0fe55
Parents: 22924c6 ee7d89c
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Oct 21 10:18:05 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Oct 21 10:18:05 2015 -0400

----------------------------------------------------------------------
 .../standard/PutDistributedMapCache.java        | 244 ++++++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../standard/TestPutDistributedMapCache.java    | 280 +++++++++++++++++++
 3 files changed, 525 insertions(+)
----------------------------------------------------------------------



[4/4] nifi git commit: Merge branch 'NIFI-988'

Posted by ma...@apache.org.
Merge branch 'NIFI-988'


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/da28b81e
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/da28b81e
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/da28b81e

Branch: refs/heads/master
Commit: da28b81eece1277c7500f28708155239ae317e57
Parents: e68fdca 97441ea
Author: Mark Payne <ma...@hotmail.com>
Authored: Wed Oct 21 10:44:06 2015 -0400
Committer: Mark Payne <ma...@hotmail.com>
Committed: Wed Oct 21 10:44:06 2015 -0400

----------------------------------------------------------------------
 .../standard/PutDistributedMapCache.java        | 244 ++++++++++++++++
 .../org.apache.nifi.processor.Processor         |   1 +
 .../standard/TestPutDistributedMapCache.java    | 280 +++++++++++++++++++
 3 files changed, 525 insertions(+)
----------------------------------------------------------------------



[2/4] nifi git commit: NIFI-988: Test cases for PutDistributedMapCache

Posted by ma...@apache.org.
NIFI-988: Test cases for PutDistributedMapCache


Project: http://git-wip-us.apache.org/repos/asf/nifi/repo
Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/ee7d89cb
Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/ee7d89cb
Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/ee7d89cb

Branch: refs/heads/master
Commit: ee7d89cb01d4661cfff2c4f0d093e38758680a56
Parents: 6b1328f
Author: Joe <jo...@impresstv.com>
Authored: Wed Sep 23 14:32:37 2015 +0200
Committer: Joe <jo...@impresstv.com>
Committed: Wed Sep 23 14:32:37 2015 +0200

----------------------------------------------------------------------
 .../standard/TestPutDistributedMapCache.java    | 280 +++++++++++++++++++
 1 file changed, 280 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi/blob/ee7d89cb/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutDistributedMapCache.java
----------------------------------------------------------------------
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutDistributedMapCache.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutDistributedMapCache.java
new file mode 100644
index 0000000..8347e7f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestPutDistributedMapCache.java
@@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.standard;
+
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestPutDistributedMapCache {
+
+    private TestRunner runner;
+    private MockCacheClient service;
+    private PutDistributedMapCache processor;
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(PutDistributedMapCache.class);
+
+        service = new MockCacheClient();
+        runner.addControllerService("service", service);
+        runner.enableControllerService(service);
+        runner.setProperty(PutDistributedMapCache.DISTRIBUTED_CACHE_SERVICE, "service");
+    }
+
+    @Test
+    public void testNoCacheKey() throws InitializationException {
+
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_IDENTIFIER, "${caheKeyAttribute}");
+        runner.enqueue(new byte[]{});
+
+        runner.run();
+
+        // no cache key attribute
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_FAILURE, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_FAILURE, 1);
+        runner.clearTransferState();
+    }
+
+    @Test
+    public void testSingleFlowFile() throws InitializationException, IOException {
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_IDENTIFIER, "${caheKeyAttribute}");
+
+        final Map<String, String> props = new HashMap<>();
+        props.put("caheKeyAttribute", "1");
+
+        String flowFileContent = "content";
+        runner.enqueue(flowFileContent.getBytes("UTF-8"), props);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_SUCCESS, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_SUCCESS, 1);
+        byte[] value = service.get("1", new PutDistributedMapCache.StringSerializer(), new PutDistributedMapCache.CacheValueDeserializer());
+        assertEquals(flowFileContent, new String(value, "UTF-8"));
+
+        final MockFlowFile outputFlowFile = runner.getFlowFilesForRelationship(PutDistributedMapCache.REL_SUCCESS).get(0);
+        outputFlowFile.assertAttributeEquals("cached", "true");
+        outputFlowFile.assertContentEquals(flowFileContent);
+        runner.clearTransferState();
+
+    }
+
+    @Test
+    public void testNothingToCache() throws InitializationException, IOException {
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_IDENTIFIER, "${caheKeyAttribute}");
+
+        final Map<String, String> props = new HashMap<>();
+        props.put("caheKeyAttribute", "2");
+
+        // flow file without content
+        runner.enqueue(new byte[]{}, props);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_FAILURE, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_FAILURE, 1);
+    }
+
+    @Test
+    public void testMaxCacheEntrySize() throws InitializationException, IOException {
+
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_IDENTIFIER, "${uuid}");
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_MAX_BYTES, "10 B");
+
+        // max length is 10 bytes, flow file content is 20 bytes
+        String flowFileContent = "contentwhichistoobig";
+        runner.enqueue(flowFileContent.getBytes("UTF-8"));
+
+        runner.run();
+
+        // no cache key attribute
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_FAILURE, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_FAILURE, 1);
+
+        final MockFlowFile outputFlowFile = runner.getFlowFilesForRelationship(PutDistributedMapCache.REL_FAILURE).get(0);
+        outputFlowFile.assertAttributeNotExists("cached");
+        outputFlowFile.assertContentEquals(flowFileContent);
+
+
+        runner.clearTransferState();
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_MAX_BYTES, "1 MB");
+    }
+
+    @Test
+    public void testCacheStrategyReplace() throws InitializationException, IOException {
+
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_IDENTIFIER, "${caheKeyAttribute}");
+        runner.setProperty(PutDistributedMapCache.CACHE_UPDATE_STRATEGY, PutDistributedMapCache.CACHE_UPDATE_REPLACE.getValue());
+
+        final Map<String, String> props = new HashMap<>();
+        props.put("caheKeyAttribute", "replaceme");
+
+        String original = "original";
+        runner.enqueue(original.getBytes("UTF-8"), props);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_SUCCESS, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_SUCCESS, 1);
+
+        MockFlowFile outputFlowFile = runner.getFlowFilesForRelationship(PutDistributedMapCache.REL_SUCCESS).get(0);
+        outputFlowFile.assertAttributeEquals("cached", "true");
+        outputFlowFile.assertContentEquals(original);
+
+        runner.clearTransferState();
+        byte[] value = service.get("replaceme", new PutDistributedMapCache.StringSerializer(), new PutDistributedMapCache.CacheValueDeserializer());
+        assertEquals(original, new String(value, "UTF-8"));
+
+        String replaced = "replaced";
+        runner.enqueue(replaced.getBytes("UTF-8"), props);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_SUCCESS, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_SUCCESS, 1);
+
+        outputFlowFile = runner.getFlowFilesForRelationship(PutDistributedMapCache.REL_SUCCESS).get(0);
+        outputFlowFile.assertAttributeEquals("cached", "true");
+        outputFlowFile.assertContentEquals(replaced);
+
+        runner.clearTransferState();
+
+        //we expect that the cache entry is replaced
+        value = service.get("replaceme", new PutDistributedMapCache.StringSerializer(), new PutDistributedMapCache.CacheValueDeserializer());
+        assertEquals(replaced, new String(value, "UTF-8"));
+    }
+
+    @Test
+    public void testCacheStrategyKeepOriginal() throws InitializationException, IOException {
+
+        runner.setProperty(PutDistributedMapCache.CACHE_ENTRY_IDENTIFIER, "${caheKeyAttribute}");
+        runner.setProperty(PutDistributedMapCache.CACHE_UPDATE_STRATEGY, PutDistributedMapCache.CACHE_UPDATE_KEEP_ORIGINAL.getValue());
+
+        final Map<String, String> props = new HashMap<>();
+        props.put("caheKeyAttribute", "replaceme");
+
+        String original = "original";
+        runner.enqueue(original.getBytes("UTF-8"), props);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_SUCCESS, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_SUCCESS, 1);
+
+        MockFlowFile outputFlowFile = runner.getFlowFilesForRelationship(PutDistributedMapCache.REL_SUCCESS).get(0);
+        outputFlowFile.assertAttributeEquals("cached", "true");
+        outputFlowFile.assertContentEquals(original);
+
+        runner.clearTransferState();
+        byte[] value = service.get("replaceme", new PutDistributedMapCache.StringSerializer(), new PutDistributedMapCache.CacheValueDeserializer());
+        assertEquals(original, new String(value, "UTF-8"));
+
+        String replaced = "replaced";
+        runner.enqueue(replaced.getBytes("UTF-8"), props);
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(PutDistributedMapCache.REL_FAILURE, 1);
+        runner.assertTransferCount(PutDistributedMapCache.REL_FAILURE, 1);
+
+        outputFlowFile = runner.getFlowFilesForRelationship(PutDistributedMapCache.REL_FAILURE).get(0);
+        outputFlowFile.assertAttributeEquals("cached", "false");
+        outputFlowFile.assertContentEquals(replaced);
+
+        runner.clearTransferState();
+
+        //we expect that the cache entry is NOT replaced
+        value = service.get("replaceme", new PutDistributedMapCache.StringSerializer(), new PutDistributedMapCache.CacheValueDeserializer());
+        assertEquals(original, new String(value, "UTF-8"));
+    }
+
+    private class MockCacheClient extends AbstractControllerService implements DistributedMapCacheClient {
+        private final ConcurrentMap<Object, Object> values = new ConcurrentHashMap<>();
+        private boolean failOnCalls = false;
+
+        private void verifyNotFail() throws IOException {
+            if ( failOnCalls ) {
+                throw new IOException("Could not call to remote service because Unit Test marked service unavailable");
+            }
+        }
+
+        @Override
+        public <K, V> boolean putIfAbsent(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
+            verifyNotFail();
+            final Object retValue = values.putIfAbsent(key, value);
+            return (retValue == null);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V getAndPutIfAbsent(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer,
+                                          final Deserializer<V> valueDeserializer) throws IOException {
+            verifyNotFail();
+            return (V) values.putIfAbsent(key, value);
+        }
+
+        @Override
+        public <K> boolean containsKey(final K key, final Serializer<K> keySerializer) throws IOException {
+            verifyNotFail();
+            return values.containsKey(key);
+        }
+
+        @Override
+        public <K, V> void put(final K key, final V value, final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws IOException {
+            verifyNotFail();
+            values.put(key, value);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public <K, V> V get(final K key, final Serializer<K> keySerializer, final Deserializer<V> valueDeserializer) throws IOException {
+            verifyNotFail();
+            return (V) values.get(key);
+        }
+
+        @Override
+        public void close() throws IOException {
+        }
+
+        @Override
+        public <K> boolean remove(final K key, final Serializer<K> serializer) throws IOException {
+            verifyNotFail();
+            values.remove(key);
+            return true;
+        }
+    }
+
+
+}
\ No newline at end of file