simonbence commented on a change in pull request #4510:
URL: https://github.com/apache/nifi/pull/4510#discussion_r490974101



##########
File path: 
nifi-nar-bundles/nifi-hazelcast-bundle/nifi-hazelcast-services/src/main/java/org/apache/nifi/hazelcast/services/cacheclient/HazelcastMapCacheClient.java
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.hazelcast.services.cacheclient;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.distributed.cache.client.AtomicCacheEntry;
+import 
org.apache.nifi.distributed.cache.client.AtomicDistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Deserializer;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.hazelcast.services.cache.HazelcastCache;
+import org.apache.nifi.hazelcast.services.cachemanager.HazelcastCacheManager;
+import org.apache.nifi.hazelcast.services.util.LongUtil;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+
+/**
+ * An implementation of DistributedMapCacheClient that uses Hazelcast as the 
backing cache.
+ *
+ * Note: By design, the client should not directly depend on Hazelcast 
specific classes due to ease version and implementation changes.
+ */
+@Tags({ "hazelcast", "cache", "map"})
+@CapabilityDescription("An implementation of DistributedMapCacheClient that 
uses Hazelcast as the backing cache. This service relies on " +
+        "an abstracted repository manages the actual Hazelcast calls, provided 
by HazelcastConnectionService.")
+public class HazelcastMapCacheClient extends AbstractControllerService 
implements AtomicDistributedMapCacheClient<Long> {
+
+    public static final PropertyDescriptor HAZELCAST_CACHE_MANAGER = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-cache-manager")
+            .displayName("Hazelcast Cache Manager")
+            .description("A Hazelcast Cache Manager which manages connections 
to Hazelcast and providing cache instances")
+            .identifiesControllerService(HazelcastCacheManager.class)
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor HAZELCAST_CACHE_NAME = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-cache-name")
+            .displayName("Hazelcast Cache Name")
+            .description("The name of a given repository. Within a Hazelcast 
cluster, multiple unrelated caches might be used." +
+                    "Clients using the same cache name will depend on the same 
data structure.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HAZELCAST_ENTRY_TTL = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-entry-ttl")
+            .displayName("Hazelcast entry TTL")
+            .description("Indicates how long the written entries should exist 
in Hazelcast. Setting it to '0 secs' means that the data" +
+                    "will exists until it's deletion or until the Hazelcast 
server is shut down.")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 secs") // Note: in case of Hazelcast IMap, 
negative value would mean "map default" which might be overridden from a 
different client.
+            .build();
+
+    private static final long STARTING_VERSION = 1;
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS;
+
+    static {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(HAZELCAST_CACHE_MANAGER);
+        properties.add(HAZELCAST_CACHE_NAME);
+        properties.add(HAZELCAST_ENTRY_TTL);
+        PROPERTY_DESCRIPTORS = Collections.unmodifiableList(properties);
+    }
+
+    private volatile HazelcastCache cache = null;
+
+    @OnEnabled
+    public void onEnabled(final ConfigurationContext context) {
+        final HazelcastCacheManager hazelcastCacheManager = 
context.getProperty(HAZELCAST_CACHE_MANAGER).asControllerService(HazelcastCacheManager.class);
+        cache = hazelcastCacheManager.getCache(
+                context.getProperty(HAZELCAST_CACHE_NAME).getValue(),
+                
context.getProperty(HAZELCAST_ENTRY_TTL).asTimePeriod(TimeUnit.MILLISECONDS));
+        getLogger().debug("Enable Hazelcast cache client for cache " + 
cache.name());
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        // The cache state will be preserved until the Service is not stopped!
+        getLogger().debug("Disable Hazelcast cache client for cache " + 
cache.name());
+        cache = null;
+    }
+
+    @Override
+    public <K, V> AtomicCacheEntry<K, V, Long> fetch(final K key, final 
Serializer<K> keySerializer, final Deserializer<V> valueDeserializer) throws 
IOException {
+        final byte[] result = cache.get(getCacheEntryKey(key, keySerializer));
+        return (result == null) ? null : new AtomicCacheEntry<>(key, 
parsePayload(valueDeserializer, result), parseVersion(result));
+    }
+
+    @Override
+    public <K, V> boolean replace(final AtomicCacheEntry<K, V, Long> entry, 
final Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws 
IOException {
+        if (entry.getKey() == null) {
+            return false;
+        }
+
+        final String key = getCacheEntryKey(entry.getKey(), keySerializer);
+
+        try(final HazelcastCache.HazelcastCacheEntryLock lock = 
cache.acquireLock(key)) {
+            final byte[] oldValue = cache.get(key);
+
+            if (oldValue == null && (!entry.getRevision().isPresent() || 
entry.getRevision().get() < STARTING_VERSION)) {
+                cache.put(key, serialize(entry.getValue(), valueSerializer, 
STARTING_VERSION));
+                getLogger().debug("Entry with key " + key + " was added during 
replace");
+                return true;
+            } else if (oldValue != null && entry.getRevision().get() == 
parseVersion(oldValue)) {
+                cache.put(key, serialize(entry.getValue(), valueSerializer, 
entry.getRevision().get() + 1));
+                getLogger().debug("Entry with key " + key + " was updated 
during replace, with revision " + entry.getRevision().get() + 1);

Review comment:
       I was thinking for a while, how to approach this. I looked through the 
implementations and the implementation of MockCacheClient (came in together 
with the interface) looked to show the original intent best. Also, increasing 
revision during change seemed like more natural and the interface did not 
promised to keep it. Based on the description, the documentation focuses on the 
comparison before replacement. Currently fetch and replace are used in the Wait 
and Notify processors, and based on my testing, those work with this approach 
correctly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to