mirror of https://github.com/apache/nifi.git
Merge branch 'NIFI-988' of https://github.com/ImpressTV/nifi into NIFI-988
This commit is contained in:
commit
97441ea0c2
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
}
|
Loading…
Reference in New Issue