Merge branch 'NIFI-988' of https://github.com/ImpressTV/nifi into NIFI-988

This commit is contained in:
Mark Payne 2015-10-21 10:18:05 -04:00
commit 97441ea0c2
3 changed files with 525 additions and 0 deletions

View File

@ -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));
}
}
}

View File

@ -49,6 +49,7 @@ org.apache.nifi.processors.standard.ModifyBytes
org.apache.nifi.processors.standard.MonitorActivity org.apache.nifi.processors.standard.MonitorActivity
org.apache.nifi.processors.standard.PostHTTP org.apache.nifi.processors.standard.PostHTTP
org.apache.nifi.processors.standard.PutEmail org.apache.nifi.processors.standard.PutEmail
org.apache.nifi.processors.standard.PutDistributedMapCache
org.apache.nifi.processors.standard.PutFile org.apache.nifi.processors.standard.PutFile
org.apache.nifi.processors.standard.PutFTP org.apache.nifi.processors.standard.PutFTP
org.apache.nifi.processors.standard.PutJMS org.apache.nifi.processors.standard.PutJMS

View File

@ -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;
}
}
}