You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by mans2singh <gi...@git.apache.org> on 2016/07/26 00:12:42 UTC

[GitHub] nifi pull request #721: Nifi 2398 - Apache Ignite Get Processor

GitHub user mans2singh opened a pull request:

    https://github.com/apache/nifi/pull/721

    Nifi 2398 - Apache Ignite Get Processor

    Ignite get processor.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mans2singh/nifi Nifi-2398

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/721.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #721
    
----
commit b3a202919771b924d6771ad8564a1b36ec7dba65
Author: mans2singh <ma...@yahoo.com>
Date:   2016-07-24T23:18:55Z

    First commit for ignite cache get

commit 6e9c8fb35b08502dbcf711ffa98b7a08b2bd0551
Author: mans2singh <ma...@yahoo.com>
Date:   2016-07-26T00:06:29Z

    NIFI-2398 - Updated code and added test cases

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #721: Nifi 2398 - Apache Ignite Get Processor

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/721


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by mans2singh <gi...@git.apache.org>.
Github user mans2singh commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    @pvillard31 - 
    
    I've updated the code based on your comments.  Let me know if you have any other thoughts/recommendations.
    
    Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by mans2singh <gi...@git.apache.org>.
Github user mans2singh commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    @pvillard31 - I will work on the issues.  Thanks for your review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #721: Nifi 2398 - Apache Ignite Get Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/721#discussion_r79940293
  
    --- Diff: nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/GetIgniteCache.java ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.ignite.cache;
    +
    +import java.io.ByteArrayInputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +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.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +
    +/**
    + * Get cache processors which gets byte array for the key from Ignite cache and set the array
    + * as the FlowFile content.
    + */
    +@EventDriven
    +@SupportsBatching
    +@Tags({ "Ignite", "get", "read", "cache", "key" })
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Get the byte array from Ignite Cache and adds it as the content of a FlowFile." +
    +    "The processor uses the value of FlowFile attribute (Ignite cache entry key) as the cache key lookup. " +
    +    "If the entry corresponding to the key is not found in the cache an error message is associated with the FlowFile " +
    +    "Note - The Ignite Kernel periodically outputs node performance statistics to the logs. This message " +
    +    " can be turned off by setting the log level for logger 'org.apache.ignite' to WARN in the logback.xml configuration file.")
    +@WritesAttributes({
    +    @WritesAttribute(attribute = GetIgniteCache.IGNITE_GET_FAILED_REASON_ATTRIBUTE_KEY, description = "The reason for getting entry from cache"),
    +    @WritesAttribute(attribute = GetIgniteCache.IGNITE_GET_FAILED_MISSING_KEY_MESSAGE, description = "The FlowFile key attribute was missing.")
    +    })
    +@SeeAlso({PutIgniteCache.class})
    +public class GetIgniteCache extends AbstractIgniteCacheProcessor {
    +
    +    /** Flow file attribute keys and messages */
    +    public static final String IGNITE_GET_FAILED_REASON_ATTRIBUTE_KEY = "ignite.cache.get.failed.reason";
    +    public static final String IGNITE_GET_FAILED_MISSING_KEY_MESSAGE = "The FlowFile key attribute was missing";
    +    public static final String IGNITE_GET_FAILED_MISSING_ENTRY_MESSAGE = "The cache byte array entry was null or zero length";
    +    public static final String IGNITE_GET_FAILED_MESSAGE_PREFIX = "The cache request failed because of ";
    +
    +    static {
    +        descriptors = new ArrayList<>();
    --- End diff --
    
    You are sharing a static reference for descriptors in both processors, it results in a race condition and not all properties are present. Have a look to other processors extending an abstract class to have an example.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by mans2singh <gi...@git.apache.org>.
Github user mans2singh commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    Hi @pvillard31 - Can you please point me to which attribute you are referring to since the constant (IGNITE_GET_FAILED_MISSING_KEY_MESSAGE) is a message not a key.
    I will correct the 2nd issue that you mentioned.
    Thanks 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #721: Nifi 2398 - Apache Ignite Get Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/721#discussion_r79939789
  
    --- Diff: nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/GetIgniteCache.java ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.ignite.cache;
    +
    +import java.io.ByteArrayInputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +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.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +
    +/**
    + * Get cache processors which gets byte array for the key from Ignite cache and set the array
    + * as the FlowFile content.
    + */
    +@EventDriven
    +@SupportsBatching
    +@Tags({ "Ignite", "get", "read", "cache", "key" })
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Get the byte array from Ignite Cache and adds it as the content of a FlowFile." +
    +    "The processor uses the value of FlowFile attribute (Ignite cache entry key) as the cache key lookup. " +
    +    "If the entry corresponding to the key is not found in the cache an error message is associated with the FlowFile " +
    +    "Note - The Ignite Kernel periodically outputs node performance statistics to the logs. This message " +
    +    " can be turned off by setting the log level for logger 'org.apache.ignite' to WARN in the logback.xml configuration file.")
    +@WritesAttributes({
    +    @WritesAttribute(attribute = GetIgniteCache.IGNITE_GET_FAILED_REASON_ATTRIBUTE_KEY, description = "The reason for getting entry from cache"),
    +    @WritesAttribute(attribute = GetIgniteCache.IGNITE_GET_FAILED_MISSING_KEY_MESSAGE, description = "The FlowFile key attribute was missing.")
    +    })
    +@SeeAlso({PutIgniteCache.class})
    +public class GetIgniteCache extends AbstractIgniteCacheProcessor {
    +
    +    /** Flow file attribute keys and messages */
    +    public static final String IGNITE_GET_FAILED_REASON_ATTRIBUTE_KEY = "ignite.cache.get.failed.reason";
    +    public static final String IGNITE_GET_FAILED_MISSING_KEY_MESSAGE = "The FlowFile key attribute was missing";
    --- End diff --
    
    Shouldn't it be a key instead of a string message here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by mans2singh <gi...@git.apache.org>.
Github user mans2singh commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    @pvillard31 - 
    
    I've updated the code and checked it with your template.  I've also added more integration tests to check processors can be restarted multiple times.  Please let me know if you have any additional comments.
    
    Thanks again


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by mans2singh <gi...@git.apache.org>.
Github user mans2singh commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    Thanks @pvillard31 for your review and advice.  Mans 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    I have played with both Put and Get processors and it seems to work fine. However, when stopping/starting processors to change properties (to test batching and performances), I got in a situation with this exception:
    
    ````
    2016-10-03 23:32:42,633 WARN [Timer-Driven Process Thread-4] o.a.n.c.t.ContinuallyRunProcessorTask Administratively Yielding PutIgniteCache[id=4d80623c-0157-1000-0000-0000179dcc8f] due to uncaught Exception: java.lang.IllegalStateException: Data streamer has been closed.
    2016-10-03 23:32:42,634 WARN [Timer-Driven Process Thread-4] o.a.n.c.t.ContinuallyRunProcessorTask
    java.lang.IllegalStateException: Data streamer has been closed.
    	at org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl.enterBusy(DataStreamerImpl.java:355) ~[na:na]
    	at org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl.addData(DataStreamerImpl.java:489) ~[na:na]
    	at org.apache.nifi.processors.ignite.cache.PutIgniteCache.onTrigger(PutIgniteCache.java:274) ~[na:na]
    	at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27) ~[nifi-api-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
    	at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1060) ~[nifi-framework-core-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
    	at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:136) [nifi-framework-core-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
    	at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47) [nifi-framework-core-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
    	at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:123) [nifi-framework-core-1.0.0-SNAPSHOT.jar:1.0.0-SNAPSHOT]
    	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_77]
    	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_77]
    	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_77]
    	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_77]
    	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_77]
    	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_77]
    	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_77]
    ````
    
    It was not possible to get the flow back to OK unless restarting NiFi. I believe something is not closed correctly.
    
    To reproduce, take this template, run all processors, everything should be OK, stop GetIgnite processor and restart it, do the same with PutIgnite. You should get the error.
    Here is the template I use: https://gist.github.com/pvillard31/8a0c1d0344bf5bdfdf4cd460e936905c


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    @mans2singh you are using the constant defined L62 in the annotation L55 as the attribute name. I believe this is a mistake but correct me if I'm wrong.
    ````java
    @WritesAttribute(attribute = GetIgniteCache.IGNITE_GET_FAILED_MISSING_KEY_MESSAGE, ...
    ````


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #721: Nifi 2398 - Apache Ignite Get Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/721#discussion_r81645352
  
    --- Diff: nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/AbstractIgniteProcessor.java ---
    @@ -92,6 +94,14 @@ public void initializeIgnite(ProcessContext context) {
                 return;
             }
     
    +        List<Ignite> grids = Ignition.allGrids();
    +
    +        if ( grids.size() == 1 ) {
    +            getLogger().warn("Ignite grid already avaialble");
    --- End diff --
    
    I think this should not be a warning because if using Put and Get processors on the canvas we will get a warning bulletin although there is no issue. Maybe log as info and inform the user we use a shared grid? (also take care of the typo).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #721: Nifi 2398 - Apache Ignite Get Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on the issue:

    https://github.com/apache/nifi/pull/721
  
    LGTM +1
    Full build with contrib-check.
    Ran multiple workflows with various configurations with a local Ignite instance. Everything worked as expected, problems raised before have been solved. Thanks for your work @mans2singh !


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #721: Nifi 2398 - Apache Ignite Get Processor

Posted by pvillard31 <gi...@git.apache.org>.
Github user pvillard31 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/721#discussion_r79868592
  
    --- Diff: nifi-nar-bundles/nifi-ignite-bundle/nifi-ignite-processors/src/main/java/org/apache/nifi/processors/ignite/cache/GetIgniteCache.java ---
    @@ -0,0 +1,119 @@
    +/*
    + * 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.ignite.cache;
    +
    +import java.io.ByteArrayInputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.behavior.WritesAttributes;
    +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.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +
    +/**
    + * Get cache processors which gets byte array for the key from Ignite cache and set the array
    + * as the FlowFile content.
    + */
    +@EventDriven
    +@SupportsBatching
    +@Tags({ "Ignite", "get", "read", "cache", "key" })
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Get the byte array from Ignite Cache and adds it as the content of a FlowFile." +
    +    "The processor uses the value of FlowFile attribute (Ignite cache entry key) as the cache key lookup. " +
    +    "If the entry corresponding to the key is not found in the cache an error message is associated with the FlowFile " +
    +    "Note - The Ignite Kernel periodically outputs node performance statistics to the logs. This message " +
    +    " can be turned off by setting the log level for logger 'org.apache.ignite' to WARN in the logback.xml configuration file.")
    +@WritesAttributes({
    +    @WritesAttribute(attribute = GetIgniteCache.IGNITE_GET_FAILED_REASON_ATTRIBUTE_KEY, description = "The reason for getting entry from cache"),
    +    @WritesAttribute(attribute = GetIgniteCache.IGNITE_GET_FAILED_MISSING_KEY_MESSAGE, description = "The FlowFile key attribute was missing.")
    +    })
    +@SeeAlso({PutIgniteCache.class})
    +public class GetIgniteCache extends AbstractIgniteCacheProcessor {
    +
    +    /** Flow file attribute keys and messages */
    +    public static final String IGNITE_GET_FAILED_REASON_ATTRIBUTE_KEY = "ignite.cache.get.failed.reason";
    +    public static final String IGNITE_GET_FAILED_MISSING_KEY_MESSAGE = "The FlowFile key attribute was missing";
    --- End diff --
    
    Shouldn't it be a key and not a description (as you did with the other written attribute)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---