From 6b1328f3f181a27a5856d26983ed3329ee317522 Mon Sep 17 00:00:00 2001 From: Joe Date: Wed, 23 Sep 2015 13:16:02 +0200 Subject: [PATCH 1/2] NIFI-988: PutDisributedMapCache processor implementation --- .../standard/PutDistributedMapCache.java | 244 ++++++++++++++++++ .../org.apache.nifi.processor.Processor | 1 + 2 files changed, 245 insertions(+) create mode 100644 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 0000000000..8e50c9f592 --- /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 relationships; + + private final Serializer keySerializer = new StringSerializer(); + private final Serializer valueSerializer = new CacheValueSerializer(); + private final Deserializer valueDeserializer = new CacheValueDeserializer(); + + public PutDistributedMapCache() { + final Set rels = new HashSet<>(); + rels.add(REL_SUCCESS); + rels.add(REL_FAILURE); + relationships = Collections.unmodifiableSet(rels); + } + + @Override + protected List getSupportedPropertyDescriptors() { + final List 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 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 { + + @Override + public void serialize(final byte[] bytes, final OutputStream out) throws SerializationException, IOException { + out.write(bytes); + } + } + + public static class CacheValueDeserializer implements Deserializer { + + @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 { + + @Override + public void serialize(final String value, final OutputStream out) throws SerializationException, IOException { + out.write(value.getBytes(StandardCharsets.UTF_8)); + } + } + +} 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 0ce1456009..ff39ad3460 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 From ee7d89cb01d4661cfff2c4f0d093e38758680a56 Mon Sep 17 00:00:00 2001 From: Joe Date: Wed, 23 Sep 2015 14:32:37 +0200 Subject: [PATCH 2/2] NIFI-988: Test cases for PutDistributedMapCache --- .../standard/TestPutDistributedMapCache.java | 280 ++++++++++++++++++ 1 file changed, 280 insertions(+) create mode 100644 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 0000000000..8347e7ffe1 --- /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 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 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 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 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 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 boolean putIfAbsent(final K key, final V value, final Serializer keySerializer, final Serializer valueSerializer) throws IOException { + verifyNotFail(); + final Object retValue = values.putIfAbsent(key, value); + return (retValue == null); + } + + @Override + @SuppressWarnings("unchecked") + public V getAndPutIfAbsent(final K key, final V value, final Serializer keySerializer, final Serializer valueSerializer, + final Deserializer valueDeserializer) throws IOException { + verifyNotFail(); + return (V) values.putIfAbsent(key, value); + } + + @Override + public boolean containsKey(final K key, final Serializer keySerializer) throws IOException { + verifyNotFail(); + return values.containsKey(key); + } + + @Override + public void put(final K key, final V value, final Serializer keySerializer, final Serializer valueSerializer) throws IOException { + verifyNotFail(); + values.put(key, value); + } + + @Override + @SuppressWarnings("unchecked") + public V get(final K key, final Serializer keySerializer, final Deserializer valueDeserializer) throws IOException { + verifyNotFail(); + return (V) values.get(key); + } + + @Override + public void close() throws IOException { + } + + @Override + public boolean remove(final K key, final Serializer serializer) throws IOException { + verifyNotFail(); + values.remove(key); + return true; + } + } + + +} \ No newline at end of file