You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by mattyb149 <gi...@git.apache.org> on 2016/11/29 16:51:49 UTC

[GitHub] nifi pull request #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

GitHub user mattyb149 opened a pull request:

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

    NIFI-3095: Add EL support to Elasticsearch 2.x (and HTTP) processors

    For feature parity with the ES 5.0 processors, I added EL support to all appropriate properties. Also I moved the relationship and property descriptor creation to static blocks (as is the conventional wisdom). Unit tests were added to exercise the EL logic, and I changed some Java 7 constructs to Java 8 constructs (lambdas, e.g.)


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

    $ git pull https://github.com/mattyb149/nifi NIFI-3095

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

    https://github.com/apache/nifi/pull/1276.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 #1276
    
----
commit 3a849a0419671f81594f8a2c21d4b67f0ec357bf
Author: Matt Burgess <ma...@apache.org>
Date:   2016-11-29T16:29:47Z

    NIFI-3095: Add EL support to Elasticsearch 2.x (and HTTP) processors

----


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and HTTP) ...

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

    https://github.com/apache/nifi/pull/1276
  
    URL Connections aren't explicitly closed in these ElasticSearch processors.  It's outside the scope of this PR, since this issue existed prior to enabling EL on these properties. The responses from the connections pull all the bytes at once (and are implicitly closed), so it's technically OK at this point, but I think explicit calls to close the connections should be added for cleanliness.


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90534894
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -28,14 +29,20 @@
     
     import java.util.Collection;
     import java.util.HashSet;
    -import java.util.Map;
     import java.util.Set;
     
     /**
      * A base class for all Elasticsearch processors
      */
     public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
     
    +    protected static final Validator NON_EMPTY_EL_VALIDATOR = (subject, value, context) -> {
    --- End diff --
    
    Opened [NIFI-3139](https://issues.apache.org/jira/browse/NIFI-3139) to move the validators to StandardValidators


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90130637
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -74,8 +83,9 @@
             Set<ValidationResult> results = new HashSet<>();
     
             // Ensure that if username or password is set, then the other is too
    -        Map<PropertyDescriptor, String> propertyMap = validationContext.getProperties();
    -        if (StringUtils.isEmpty(propertyMap.get(USERNAME)) != StringUtils.isEmpty(propertyMap.get(PASSWORD))) {
    +        String userName = validationContext.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
    +        String password = validationContext.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
    --- End diff --
    
    I don't see EL enabled in the PASSWORD PropertyDescriptor, but evaluateAttributeExpressions() is being called on that property.  Is it intended that the PASSWORD property should allow EL?


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90133302
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -28,14 +29,20 @@
     
     import java.util.Collection;
     import java.util.HashSet;
    -import java.util.Map;
     import java.util.Set;
     
     /**
      * A base class for all Elasticsearch processors
      */
     public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
     
    +    protected static final Validator NON_EMPTY_EL_VALIDATOR = (subject, value, context) -> {
    --- End diff --
    
    Assuming that this validator would provide value in other processors, it might make sense to move this into StandardValidators?


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90141491
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -74,8 +83,9 @@
             Set<ValidationResult> results = new HashSet<>();
     
             // Ensure that if username or password is set, then the other is too
    -        Map<PropertyDescriptor, String> propertyMap = validationContext.getProperties();
    -        if (StringUtils.isEmpty(propertyMap.get(USERNAME)) != StringUtils.isEmpty(propertyMap.get(PASSWORD))) {
    +        String userName = validationContext.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
    +        String password = validationContext.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
    --- End diff --
    
    Oops, yes!


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90289603
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -28,14 +29,20 @@
     
     import java.util.Collection;
     import java.util.HashSet;
    -import java.util.Map;
     import java.util.Set;
     
     /**
      * A base class for all Elasticsearch processors
      */
     public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
     
    +    protected static final Validator NON_EMPTY_EL_VALIDATOR = (subject, value, context) -> {
    --- End diff --
    
    "HOSTNAME_PORT_VALIDATOR" may be good to move to Standard validators too, if you two think it could provide value outside the ES processors.


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90293399
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/test/java/org/apache/nifi/processors/elasticsearch/TestFetchElasticsearchHttp.java ---
    @@ -65,6 +65,35 @@ public void teardown() {
         public void testFetchElasticsearchOnTrigger() throws IOException {
             runner = TestRunners.newTestRunner(new FetchElasticsearchHttpTestProcessor(true)); // all docs are found
             runner.setValidateExpressionUsage(true);
    +        runner.setProperty(AbstractElasticsearchHttpProcessor.ES_URL, "${es.url}");
    +
    +        runner.setProperty(FetchElasticsearchHttp.INDEX, "doc");
    +        runner.assertNotValid();
    +        runner.setProperty(FetchElasticsearchHttp.TYPE, "status");
    +        runner.assertNotValid();
    +        runner.setProperty(FetchElasticsearchHttp.DOC_ID, "${doc_id}");
    +        runner.assertValid();
    +        runner.setProperty(AbstractElasticsearchHttpProcessor.CONNECT_TIMEOUT, "${connect.timeout}");
    +        runner.assertValid();
    +
    +        runner.setVariable("es.url", "http://127.0.0.1:9200");
    +        runner.setVariable("connect.timeout", "5s");
    +
    +        runner.enqueue(docExample, new HashMap<String, String>() {{
    +            put("doc_id", "28039652140");
    +        }});
    +        runner.run(1, true, true);
    +
    +        runner.assertAllFlowFilesTransferred(FetchElasticsearchHttp.REL_SUCCESS, 1);
    +        final MockFlowFile out = runner.getFlowFilesForRelationship(FetchElasticsearchHttp.REL_SUCCESS).get(0);
    +        assertNotNull(out);
    +        out.assertAttributeEquals("doc_id", "28039652140");
    +    }
    +
    +    @Test
    +    public void testFetchElasticsearchOnTriggerEL() throws IOException {
    --- End diff --
    
    Looks like the test names got mixed up 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 pull request #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90141413
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -74,8 +83,9 @@
             Set<ValidationResult> results = new HashSet<>();
     
             // Ensure that if username or password is set, then the other is too
    -        Map<PropertyDescriptor, String> propertyMap = validationContext.getProperties();
    -        if (StringUtils.isEmpty(propertyMap.get(USERNAME)) != StringUtils.isEmpty(propertyMap.get(PASSWORD))) {
    +        String userName = validationContext.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
    +        String password = validationContext.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
    --- End diff --
    
    On line 87, shouldn't the call to evaluationAttributeExpressions should be removed?


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90139067
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -74,8 +83,9 @@
             Set<ValidationResult> results = new HashSet<>();
     
             // Ensure that if username or password is set, then the other is too
    -        Map<PropertyDescriptor, String> propertyMap = validationContext.getProperties();
    -        if (StringUtils.isEmpty(propertyMap.get(USERNAME)) != StringUtils.isEmpty(propertyMap.get(PASSWORD))) {
    +        String userName = validationContext.getProperty(USERNAME).evaluateAttributeExpressions().getValue();
    +        String password = validationContext.getProperty(PASSWORD).evaluateAttributeExpressions().getValue();
    --- End diff --
    
    Sensitive properties don't support EL yet, I think that's going into NiFi 1.2.0. I'm guessing all/most/some of the sensitive properties will be updated as part of that (I think it's [NIFI-1465](https://issues.apache.org/jira/browse/NIFI-1465))


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90139216
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -28,14 +29,20 @@
     
     import java.util.Collection;
     import java.util.HashSet;
    -import java.util.Map;
     import java.util.Set;
     
     /**
      * A base class for all Elasticsearch processors
      */
     public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
     
    +    protected static final Validator NON_EMPTY_EL_VALIDATOR = (subject, value, context) -> {
    --- End diff --
    
    Yes will do. The Elasticsearch 5 bundle has this too, should I move them as part of a new Jira, or allow this PR to touch that bundle for this purpose?


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and HTTP) ...

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

    https://github.com/apache/nifi/pull/1276
  
    In another new JIRA, some test cleanup could be done to move the tests that are ignored due to their integration nature into their respective integration test classes, or new integration test classes made for ones that don't currently exist.


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90142182
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/AbstractElasticsearchProcessor.java ---
    @@ -28,14 +29,20 @@
     
     import java.util.Collection;
     import java.util.HashSet;
    -import java.util.Map;
     import java.util.Set;
     
     /**
      * A base class for all Elasticsearch processors
      */
     public abstract class AbstractElasticsearchProcessor extends AbstractProcessor {
     
    +    protected static final Validator NON_EMPTY_EL_VALIDATOR = (subject, value, context) -> {
    --- End diff --
    
    I think a new JIRA for that would be best to not have scope-creep on this PR.


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and HTTP) ...

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

    https://github.com/apache/nifi/pull/1276
  
    +1
    
    Visually verified code and did a contrib check build. In a standalone instance tested each processor with EL hitting secured and unsecured versions of ES 2.X. All of them (except for Fetch due to NIFI-3140) worked as expected. Thank you @mattyb149 I will squash and merge.


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

    https://github.com/apache/nifi/pull/1276#discussion_r90292509
  
    --- Diff: nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttp.java ---
    @@ -192,17 +201,17 @@ public void setup(ProcessContext context) {
     
         @Override
         public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    -        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final int batchSize = context.getProperty(BATCH_SIZE).evaluateAttributeExpressions().asInteger();
     
             final List<FlowFile> flowFiles = session.get(batchSize);
             if (flowFiles.isEmpty()) {
                 return;
             }
     
             final String id_attribute = context.getProperty(ID_ATTRIBUTE).getValue();
    -        final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
    +        final Charset charset = Charset.forName(context.getProperty(CHARSET).evaluateAttributeExpressions().getValue());
    --- End diff --
    
    Charset should be evaluated for each FlowFile like PutElasticsearch does


---
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 #1276: NIFI-3095: Add EL support to Elasticsearch 2.x (and...

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

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


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