You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/11/23 20:45:11 UTC

[GitHub] [nifi] mattyb149 commented on a change in pull request #4635: NIFI-7821 Added Cassandra-based DMC.

mattyb149 commented on a change in pull request #4635:
URL: https://github.com/apache/nifi/pull/4635#discussion_r528962136



##########
File path: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-distributedmapcache-service/src/main/java/org/apache/nifi/controller/cassandra/CassandraDistributedMapCache.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.controller.cassandra;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+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.cassandra.CassandraSessionProviderService;
+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.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.controller.cassandra.QueryUtils.createDeleteStatement;
+import static org.apache.nifi.controller.cassandra.QueryUtils.createExistsQuery;
+import static org.apache.nifi.controller.cassandra.QueryUtils.createFetchQuery;
+import static org.apache.nifi.controller.cassandra.QueryUtils.createInsertStatement;
+
+@Tags({"map", "cache", "distributed", "cassandra"})
+@CapabilityDescription("Provides a DistributedMapCache client that is based on Apache Cassandra.")
+public class CassandraDistributedMapCache extends AbstractControllerService implements DistributedMapCacheClient {
+    public static final PropertyDescriptor SESSION_PROVIDER = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-session-provider")
+            .displayName("Session Provider")
+            .description("The client service that will configure the cassandra client connection.")
+            .required(true)
+            .identifiesControllerService(CassandraSessionProviderService.class)
+            .build();
+
+    public static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-table-name")
+            .displayName("Table Name")
+            .description("The name of the table where the cache will be stored.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEY_FIELD_NAME = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-key-field-name")
+            .displayName("Key Field Name")
+            .description("The name of the field that acts as the unique key. (The CQL type should be \"blob\")")

Review comment:
       Is there a necessary restriction on the CQL type to blob? I would think TEXT, VARCHAR, or other string-based fields would be acceptable? Or is that so we can use a single serializer/deserializer?

##########
File path: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-distributedmapcache-service/src/main/java/org/apache/nifi/controller/cassandra/CassandraDistributedMapCache.java
##########
@@ -0,0 +1,241 @@
+/*
+ * 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.controller.cassandra;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+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.cassandra.CassandraSessionProviderService;
+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.Deserializer;
+import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
+import org.apache.nifi.distributed.cache.client.Serializer;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.controller.cassandra.QueryUtils.createDeleteStatement;
+import static org.apache.nifi.controller.cassandra.QueryUtils.createExistsQuery;
+import static org.apache.nifi.controller.cassandra.QueryUtils.createFetchQuery;
+import static org.apache.nifi.controller.cassandra.QueryUtils.createInsertStatement;
+
+@Tags({"map", "cache", "distributed", "cassandra"})
+@CapabilityDescription("Provides a DistributedMapCache client that is based on Apache Cassandra.")
+public class CassandraDistributedMapCache extends AbstractControllerService implements DistributedMapCacheClient {
+    public static final PropertyDescriptor SESSION_PROVIDER = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-session-provider")
+            .displayName("Session Provider")
+            .description("The client service that will configure the cassandra client connection.")
+            .required(true)
+            .identifiesControllerService(CassandraSessionProviderService.class)
+            .build();
+
+    public static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-table-name")
+            .displayName("Table Name")
+            .description("The name of the table where the cache will be stored.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor KEY_FIELD_NAME = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-key-field-name")
+            .displayName("Key Field Name")
+            .description("The name of the field that acts as the unique key. (The CQL type should be \"blob\")")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor VALUE_FIELD_NAME = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-value-field-name")
+            .displayName("Value Field Name")
+            .description("The name of the field that will store the value. (The CQL type should be \"blob\")")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_EL_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor TTL = new PropertyDescriptor.Builder()
+            .name("cassandra-dmc-ttl")
+            .displayName("TTL")
+            .description("If configured, this will set a TTL (Time to Live) for each row inserted into the table so that " +
+                    "old cache items expire after a certain period of time.")
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .required(false)
+            .build();
+
+    public static final List<PropertyDescriptor> DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
+        SESSION_PROVIDER, TABLE_NAME, KEY_FIELD_NAME, VALUE_FIELD_NAME, TTL
+    ));
+
+    private CassandraSessionProviderService sessionProviderService;
+    private String tableName;
+    private String keyField;
+    private String valueField;
+    private Long ttl;
+
+    private Session session;
+    private PreparedStatement deleteStatement;
+    private PreparedStatement existsStatement;
+    private PreparedStatement fetchStatement;
+    private PreparedStatement insertStatement;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return DESCRIPTORS;
+    }
+
+    @OnEnabled
+    public void onEnabled(ConfigurationContext context) {
+        sessionProviderService = context.getProperty(SESSION_PROVIDER).asControllerService(CassandraSessionProviderService.class);
+        tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions().getValue();
+        keyField = context.getProperty(KEY_FIELD_NAME).evaluateAttributeExpressions().getValue();
+        valueField = context.getProperty(VALUE_FIELD_NAME).evaluateAttributeExpressions().getValue();
+        if (context.getProperty(TTL).isSet()) {
+            ttl = context.getProperty(TTL).evaluateAttributeExpressions().asTimePeriod(TimeUnit.SECONDS);
+        }
+
+        session = sessionProviderService.getCassandraSession();
+
+        deleteStatement = session.prepare(createDeleteStatement(keyField, tableName));
+        existsStatement = session.prepare(createExistsQuery(keyField, tableName));
+        fetchStatement = session.prepare(createFetchQuery(keyField, valueField, tableName));
+        insertStatement = session.prepare(createInsertStatement(keyField, valueField, tableName, ttl));
+    }
+
+    @OnDisabled
+    public void onDisabled() {
+        session = null;
+        deleteStatement = null;
+        existsStatement = null;
+        fetchStatement = null;
+        insertStatement = null;
+    }
+
+    @Override
+    public <K, V> boolean putIfAbsent(K k, V v, Serializer<K> keySerializer, Serializer<V> valueSerializer) throws IOException {
+        if (containsKey(k, keySerializer)) {
+            return false;
+        } else {
+            put(k, v, keySerializer, valueSerializer);
+            return true;
+        }
+    }
+
+    @Override
+    public <K, V> V getAndPutIfAbsent(K k, V v, Serializer<K> keySerializer, Serializer<V> valueSerializer, Deserializer<V> deserializer) throws IOException {
+        V got = get(k, keySerializer, deserializer);
+        boolean wasPresent = putIfAbsent(k, v, keySerializer, valueSerializer);

Review comment:
       Shouldn't this be the inverse? `putIfAbsent` will return false if the value was present in the cache, so `wasPresent` should be true. The HBase versions do:
   
   ```
   final boolean wasAbsent = putIfAbsent(key, value, keySerializer, valueSerializer);
   
   if (! wasAbsent) return got;
   else return null;
   ```
   Would be good to add a unit test around this as well.




----------------------------------------------------------------
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:
users@infra.apache.org