tpalfy commented on a change in pull request #4349:
URL: https://github.com/apache/nifi/pull/4349#discussion_r447104375



##########
File path: 
nifi-nar-bundles/nifi-hazelcast-bundle/nifi-hazelcast-services/src/main/java/org/apache/nifi/hazelcast/services/cachemanager/IMapBasedHazelcastCacheManager.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.cachemanager;
+
+import com.hazelcast.core.HazelcastException;
+import com.hazelcast.core.HazelcastInstance;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.hazelcast.services.cache.HazelcastCache;
+import org.apache.nifi.hazelcast.services.cache.IMapBasedHazelcastCache;
+import org.apache.nifi.reporting.InitializationException;
+
+abstract class IMapBasedHazelcastCacheManager extends 
AbstractControllerService implements HazelcastCacheManager {
+    public static final PropertyDescriptor HAZELCAST_INSTANCE_NAME = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-instance-name")
+            .displayName("Hazelcast Instance Name")
+            .description("Name of the embedded Hazelcast instance")
+            .required(true)
+            .addValidator(Validator.VALID)
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HAZELCAST_CLUSTER_NAME = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-cluster-name")
+            .displayName("Hazelcast Cluster Name")
+            .description("Name of the embedded Hazelcast instance's cluster")

Review comment:
       `embedded` in this abstract class?

##########
File path: 
nifi-nar-bundles/nifi-hazelcast-bundle/nifi-hazelcast-services/src/main/java/org/apache/nifi/hazelcast/services/cachemanager/IMapBasedHazelcastCacheManager.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.cachemanager;
+
+import com.hazelcast.core.HazelcastException;
+import com.hazelcast.core.HazelcastInstance;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.hazelcast.services.cache.HazelcastCache;
+import org.apache.nifi.hazelcast.services.cache.IMapBasedHazelcastCache;
+import org.apache.nifi.reporting.InitializationException;
+
+abstract class IMapBasedHazelcastCacheManager extends 
AbstractControllerService implements HazelcastCacheManager {
+    public static final PropertyDescriptor HAZELCAST_INSTANCE_NAME = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-instance-name")
+            .displayName("Hazelcast Instance Name")
+            .description("Name of the embedded Hazelcast instance")
+            .required(true)
+            .addValidator(Validator.VALID)
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    public static final PropertyDescriptor HAZELCAST_CLUSTER_NAME = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-cluster-name")
+            .displayName("Hazelcast Cluster Name")
+            .description("Name of the embedded Hazelcast instance's cluster")
+            .required(false)

Review comment:
       The default cluster name is "dev". Not sure if we want to leave it that 
way.

##########
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,253 @@
+/*
+ * 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")
+            .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 data should exist in 
Hazelcast. Setting '0 secs' would mean the data would exist forever")
+            .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 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 repository " + cache.name());
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        // The cache state will be preserved until the Service is not stopped!
+        getLogger().debug("Disable Hazelcast repository " + 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)) {

Review comment:
       When does `entry.getRevision().get() < STARTING_VERSION` resolve to 
`true`?

##########
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,253 @@
+/*
+ * 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")
+            .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 data should exist in 
Hazelcast. Setting '0 secs' would mean the data would exist forever")
+            .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 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 repository " + cache.name());
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        // The cache state will be preserved until the Service is not stopped!
+        getLogger().debug("Disable Hazelcast repository " + 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);
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    @Override
+    public <K, V> boolean putIfAbsent(final K key, final V value, final 
Serializer<K> keySerializer, final Serializer<V> valueSerializer) throws 
IOException {
+        return cache.putIfAbsent(getCacheEntryKey(key, keySerializer), 
serialize(value, valueSerializer, STARTING_VERSION)) == null;
+    }
+
+    @Override
+    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 {
+        final byte[] result = cache.putIfAbsent(getCacheEntryKey(key, 
keySerializer), serialize(value, valueSerializer, STARTING_VERSION));
+        return (result == null) ? null : parsePayload(valueDeserializer, 
result);
+    }
+
+    @Override
+    public <K> boolean containsKey(final K key, final Serializer<K> 
keySerializer) throws IOException {
+        return cache.contains(getCacheEntryKey(key, keySerializer));
+    }
+
+    @Override
+    public <K, V> void put(final K key, final V value, final Serializer<K> 
keySerializer, final Serializer<V> valueSerializer) throws IOException {
+        cache.put(getCacheEntryKey(key, keySerializer), serialize(value, 
valueSerializer, STARTING_VERSION));
+    }
+
+    @Override
+    public <K, V> V get(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 : parsePayload(valueDeserializer, result);
+    }
+
+    @Override
+    public <K> boolean remove(final K key, final Serializer<K> keySerializer) 
throws IOException {
+        return cache.remove(getCacheEntryKey(key, keySerializer));
+    }
+
+    @Override
+    public long removeByPattern(final String regex) throws IOException {
+        return cache.removeAll(new RegexPredicate(regex));
+    }
+
+    private static class RegexPredicate implements Predicate<String>, 
Serializable {
+        private final Pattern pattern;
+
+        private RegexPredicate(final String regex) {
+            this.pattern = Pattern.compile(regex);
+        }
+
+        @Override
+        public boolean test(final String string) {
+            return pattern.matcher(string).matches();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        getLogger().debug("Closing HazelcastMapCacheClient");

Review comment:
       ```suggestion
           getLogger().debug("Closing " + this.getClass().getSimpleName());
   ```

##########
File path: 
nifi-nar-bundles/nifi-hazelcast-bundle/nifi-hazelcast-services/src/main/java/org/apache/nifi/hazelcast/services/cachemanager/IMapBasedHazelcastCacheManager.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.cachemanager;
+
+import com.hazelcast.core.HazelcastException;
+import com.hazelcast.core.HazelcastInstance;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.hazelcast.services.cache.HazelcastCache;
+import org.apache.nifi.hazelcast.services.cache.IMapBasedHazelcastCache;
+import org.apache.nifi.reporting.InitializationException;
+
+abstract class IMapBasedHazelcastCacheManager extends 
AbstractControllerService implements HazelcastCacheManager {
+    public static final PropertyDescriptor HAZELCAST_INSTANCE_NAME = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-instance-name")
+            .displayName("Hazelcast Instance Name")
+            .description("Name of the embedded Hazelcast instance")
+            .required(true)

Review comment:
       I think it shouldn't be necessary for the user to be bothered with this.
   We could default it to the uuid of the service for example.

##########
File path: 
nifi-nar-bundles/nifi-hazelcast-bundle/nifi-hazelcast-services/src/main/java/org/apache/nifi/hazelcast/services/cachemanager/EmbeddedHazelcastCacheManager.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.cachemanager;
+
+import com.hazelcast.config.Config;
+import com.hazelcast.config.NetworkConfig;
+import com.hazelcast.core.Hazelcast;
+import com.hazelcast.core.HazelcastInstance;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+@Tags({"hazelcast", "cache"})
+@CapabilityDescription("A service that provides connections to an embedded 
Hazelcast instance started by NiFi.")
+public class EmbeddedHazelcastCacheManager extends 
IMapBasedHazelcastCacheManager {
+
+    static final PropertyDescriptor HAZELCAST_PORT = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-port")
+            .displayName("Hazelcast Port")
+            .description("Port the Hazelcast uses as starting port. If not 
specified, the default value will be used, which is 5701")
+            .required(false)
+            .addValidator(StandardValidators.PORT_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor HAZELCAST_PORT_COUNT = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-port-count")
+            .displayName("Hazelcast Port Count")
+            // With the current version, the default is 100
+            .description("The maximum number of ports the Hazelcast uses. If 
not specified, the default will be used")
+            .required(false)
+            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    static final PropertyDescriptor HAZELCAST_PORT_AUTO_INCREMENT = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-port-auto-increment")
+            .displayName("Hazelcast Port Auto Increment")
+            .description("Might turn on Hazelcast's auto increment port 
feature.")
+            .required(false)
+            .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTY_DESCRIPTORS;
+
+    static {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(HAZELCAST_INSTANCE_NAME);
+        properties.add(HAZELCAST_CLUSTER_NAME);
+        properties.add(HAZELCAST_PORT);
+        properties.add(HAZELCAST_PORT_COUNT);
+        properties.add(HAZELCAST_PORT_AUTO_INCREMENT);
+        PROPERTY_DESCRIPTORS = Collections.unmodifiableList(properties);
+    }
+
+    @Override
+    protected HazelcastInstance getInstance(final ConfigurationContext 
context) {
+        final Config config = new 
Config(context.getProperty(HAZELCAST_INSTANCE_NAME).getValue());
+
+        if (context.getProperty(HAZELCAST_CLUSTER_NAME).isSet()) {
+            
config.setClusterName(context.getProperty(HAZELCAST_CLUSTER_NAME).getValue());
+        }
+
+        if (context.getProperty(HAZELCAST_PORT).isSet()) {
+            final NetworkConfig networkConfig = config.getNetworkConfig();
+            
networkConfig.setPort(context.getProperty(HAZELCAST_PORT).asInteger());
+
+            if (context.getProperty(HAZELCAST_PORT_COUNT).isSet()) {

Review comment:
       Shouldn't `port count` and `port increment` be allowed to be configured 
even when using the default port (i.e. 
`context.getProperty(HAZELCAST_PORT).isSet()` is `false`)?

##########
File path: 
nifi-nar-bundles/nifi-hazelcast-bundle/nifi-hazelcast-services/src/main/java/org/apache/nifi/hazelcast/services/cachemanager/IMapBasedHazelcastCacheManager.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.cachemanager;
+
+import com.hazelcast.core.HazelcastException;
+import com.hazelcast.core.HazelcastInstance;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.hazelcast.services.cache.HazelcastCache;
+import org.apache.nifi.hazelcast.services.cache.IMapBasedHazelcastCache;
+import org.apache.nifi.reporting.InitializationException;
+
+abstract class IMapBasedHazelcastCacheManager extends 
AbstractControllerService implements HazelcastCacheManager {
+    public static final PropertyDescriptor HAZELCAST_INSTANCE_NAME = new 
PropertyDescriptor.Builder()
+            .name("hazelcast-instance-name")
+            .displayName("Hazelcast Instance Name")
+            .description("Name of the embedded Hazelcast instance")

Review comment:
       `embedded` in this abstract class?

##########
File path: 
nifi-nar-bundles/nifi-hazelcast-bundle/nifi-hazelcast-services/src/main/java/org/apache/nifi/hazelcast/services/cache/IMapBasedHazelcastCache.java
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.cache;
+
+import com.hazelcast.map.IMap;
+import com.hazelcast.map.ReachedMaxSizeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+
+/**
+ * Implementation of {@link HazelcastCache} backed by Hazelcast's IMap data 
structure. It's purpose is to wrap Hazelcast implementation specific details in 
order to
+ * make it possible to easily change version or data structure.
+ */
+public class IMapBasedHazelcastCache implements HazelcastCache {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(IMapBasedHazelcastCache.class);
+
+    private final String name;
+    private final long ttlInMillis;
+    private final IMap<String, byte[]> repository;
+
+    /**
+     * @param name Name of the cache stored for identification.
+     * @param ttlInMillis The guaranteed lifetime of a cache entry in 
milliseconds.
+     * @param repository Reference to the actual storage. It should be the 
IMap with the same identifier as cache name.
+     */
+    public IMapBasedHazelcastCache(
+            final String name,
+            final long ttlInMillis,
+            final IMap<String, byte[]> repository) {
+        this.name = name;
+        this.ttlInMillis = ttlInMillis;
+        this.repository = repository;
+    }
+
+    @Override
+    public String name() {
+        return name;
+    }
+
+    @Override
+    public byte[] get(final String key) {
+        return repository.get(key);
+    }
+
+    @Override
+    public byte[] putIfAbsent(final String key, final byte[] value) {
+        return repository.putIfAbsent(key, value, ttlInMillis, 
TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public boolean put(final String key, final byte[] value) {
+        try {
+            repository.put(key, value, ttlInMillis, TimeUnit.MILLISECONDS);
+            return true;
+        } catch (final ReachedMaxSizeException e) {
+            LOGGER.error("Cache {} reached the maximum allowed size!", name);
+            return false;
+        }
+    }
+
+    @Override
+    public boolean contains(final String key) {
+        return repository.containsKey(key);
+    }
+
+    @Override
+    public boolean remove(final String key) {
+        return repository.remove(key) != null;
+    }
+
+    @Override
+    public int removeAll(final Predicate<String> keyMatcher) {
+        // Note: the Hazelcast IMap provides support for predicate based 
<code>removeAll</code> method, but it neither atomic or provides information 
about the number of deleted items.
+        final Set<String> keys = repository.keySet();
+        int result = 0;
+
+        for (final String key : keys) {
+            if (keyMatcher.test(key)) {
+                repository.remove(key);
+                result++;
+            }
+        }
+
+        return result;
+    }
+
+    @Override
+    public HazelcastCacheEntryLock acquireLock(final String key) {
+        if (key == null) {
+            throw new IllegalArgumentException("The key of acquired lock 
cannot be null!");
+        }
+
+        final IMapAdapterEntryLock lock = new IMapAdapterEntryLock(key);
+        lock.lock();
+        return lock;
+    }
+
+    private final class IMapAdapterEntryLock implements 
HazelcastCacheEntryLock {
+        private final String key;
+
+        private IMapAdapterEntryLock(final String key) {
+            this.key = key;
+        }
+
+        void lock() {
+            repository.lock(key);

Review comment:
       The lock behind the scenes is reentrant so extra care should be taken so 
that the number of `lock` and `unlock` calls are consistent. I'd move this 
`repository.lock(key)` into the constructor.




----------------------------------------------------------------
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