You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by MikeThomsen <gi...@git.apache.org> on 2018/05/19 13:25:45 UTC

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

GitHub user MikeThomsen opened a pull request:

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

    NIFI-5214 Added REST LookupService

    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [ ] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    
    - [ ] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [ ] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [ ] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [ ] Have you written or updated unit tests to verify your changes?
    - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
    - [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
    - [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.


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

    $ git pull https://github.com/MikeThomsen/nifi NIFI-5214

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

    https://github.com/apache/nifi/pull/2723.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 #2723
    
----
commit 71e52dad00b7fcff091cda84831c953e01ac4af5
Author: Mike Thomsen <mi...@...>
Date:   2018-05-19T00:08:41Z

    NIFI-5214 Added REST LookupService

----


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193412512
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    --- End diff --
    
    MIME_TYPE is only used for put and post requests and should be optional. Also if we support 'get' is the default method, then this controller service does not have any required KEYS.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @markap14 I think your concerns are addressed now.
    
    @ijokarumawak I tried out the expression language APIs and found that I can't just submit a Map, so I'm going to hold off on providing a dynamic template URL for now. That could be be a good improvement to add after the expression language compiler api gets updated to support generic maps as well (if it ever happens).


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192486471
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy ---
    @@ -106,6 +106,37 @@ class RestLookupServiceIT {
             }
         }
     
    +    @Test
    +    void testHeaders() {
    +        runner.disableControllerService(lookupService)
    +        runner.setProperty(lookupService, "header.X-USER", "jane.doe")
    +        runner.setProperty(lookupService, "header.X-PASS", "testing7890")
    +        runner.enableControllerService(lookupService)
    +
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(SimpleJson.class, "/simple")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple",
    --- End diff --
    
    I like that. How about this breakdown:
    
    1. `endpoint` is a template string set on a property descriptor and we use an EL compiler to generate the endpoint on the fly with those lookup coordinates.
    2. Add `direct_endpoint` which is a treated as literal value to override that if present.
    3. If both are present, throw an exception.
    
    Thoughts?


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak the AWS gateway tests are broken because I moved test server. I'll do a patch for that.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192494945
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    --- End diff --
    
    We should avoid the creation of an anonymous inner class here and instead just create the MapRecord and then call put()


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192494134
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    --- End diff --
    
    Again, i think we should remove this 'header' prefix that is required.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193411030
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    --- End diff --
    
    `implements RecordLookupService` is better.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194604410
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    --- End diff --
    
    I want RestLookupService to have 'get' as default value. It will cover 80% of use-cases I assume. I will send a PR to your PR and if it looks fine, then I'd give my +1 so that you can merge.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak I can't believe I missed your comments. Will try to get to those today.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194039553
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    --- End diff --
    
    That's a fair point, and it needs validation on the verbs.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @MikeThomsen I squashed all commits into one, did manual test. LGTM, +1. merging to master! Thank you!


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556358
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    +                            put(last, value);
    +                        }});
    +                    }
    +
    +                    record = temp;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw new RuntimeException(ex);
    +        }
    +    }
    +
    +    private Request buildRequest(final String mimeType, final String method, final String body, final String endpoint) {
    +        final MediaType mt = MediaType.parse(mimeType);
    +        RequestBody requestBody = null;
    +        if (body != null) {
    +            requestBody = RequestBody.create(mt, body);
    +        }
    +        Request.Builder request = new Request.Builder()
    +                .url(endpoint);
    +        switch(method.toLowerCase()) {
    +            case "delete":
    +                request = body != null ? request.delete(requestBody) : request.delete();
    +                break;
    +            case "get":
    +                request = request.get();
    +                break;
    +            case "post":
    +                request = request.post(requestBody);
    +                break;
    +            case "put":
    +                request = request.put(requestBody);
    +                break;
    +        }
    +
    +        if (headers != null) {
    +            for (Map.Entry<String, String> header : headers.entrySet()) {
    +                request = request.addHeader(header.getKey(), header.getValue());
    +            }
    +        }
    +
    +        if (!basicUser.isEmpty() && !isDigest) {
    +            String credential = Credentials.basic(basicUser, basicPass);
    +            request = request.header("Authorization", credential);
    +        }
    +
    +        return request.build();
    +    }
    +
    +    private String basicUser;
    --- End diff --
    
    I think I fixed it.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r195273615
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static final List VALID_VERBS = Arrays.asList("delete", "get", "post", "put");
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            BASE_URL,
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        KEYS = Collections.emptySet();
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +
    +        final String url = context.getProperty(URL).getValue();
    +        compiledQuery = context.getProperty(BASE_URL).isSet()
    +                ? Query.prepare(context.getProperty(BASE_URL).evaluateAttributeExpressions().getValue() + url)
    +                : Query.prepare(url);
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    --- End diff --
    
    Done


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    Other than the issue w/ literals you cited, looks ready to merge IMO.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194821972
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static final List VALID_VERBS = Arrays.asList("delete", "get", "post", "put");
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            BASE_URL,
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        KEYS = Collections.emptySet();
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    --- End diff --
    
    If the user sets a RecordPath, enables the service, then disables the service, and removes the RecordPath, then this is going to keep evaluating the RecordPath. Believe we need an `} else { recordPath = null; }` clause in there


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak fixed AWS bundle and 2/3 Travis builds succeeded.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193032153
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -277,6 +280,20 @@ private void setProxy(OkHttpClient.Builder builder) {
             }
         }
     
    +    protected String determineEndpoint(Map<String, Object> coordinates) {
    +        if (coordinates.containsKey(ENDPOINT_KEY) && coordinates.containsKey(ENDPOINT_TEMPLATE_KEY)) {
    +            Map<String, String> converted = coordinates.entrySet().stream()
    +                .collect(Collectors.toMap(
    +                    e -> e.getKey(),
    +                    e -> e.getValue().toString()
    +                ));
    +            final PreparedQuery query = Query.prepare((String)coordinates.get(ENDPOINT_KEY));
    --- End diff --
    
    I think that would work.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193390265
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    --- End diff --
    
    This should support EL with Variable Registry.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556276
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    --- End diff --
    
    Done.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    InvokeHttp has a lot of options, so that might be worth splitting it into an initial merge and then an improvement that comes in 1.8.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @MikeThomsen I agree.  I haven't seen any conversation on it, but it seems more and more obvious, given the way things are going right?


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556241
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192496937
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    +                            put(last, value);
    +                        }});
    +                    }
    +
    +                    record = temp;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw new RuntimeException(ex);
    +        }
    +    }
    +
    +    private Request buildRequest(final String mimeType, final String method, final String body, final String endpoint) {
    +        final MediaType mt = MediaType.parse(mimeType);
    +        RequestBody requestBody = null;
    +        if (body != null) {
    +            requestBody = RequestBody.create(mt, body);
    +        }
    +        Request.Builder request = new Request.Builder()
    +                .url(endpoint);
    +        switch(method.toLowerCase()) {
    +            case "delete":
    +                request = body != null ? request.delete(requestBody) : request.delete();
    +                break;
    +            case "get":
    +                request = request.get();
    +                break;
    +            case "post":
    +                request = request.post(requestBody);
    +                break;
    +            case "put":
    +                request = request.put(requestBody);
    +                break;
    +        }
    +
    +        if (headers != null) {
    +            for (Map.Entry<String, String> header : headers.entrySet()) {
    +                request = request.addHeader(header.getKey(), header.getValue());
    +            }
    +        }
    +
    +        if (!basicUser.isEmpty() && !isDigest) {
    +            String credential = Credentials.basic(basicUser, basicPass);
    +            request = request.header("Authorization", credential);
    +        }
    +
    +        return request.build();
    +    }
    +
    +    private String basicUser;
    +    private String basicPass;
    +    private boolean isDigest;
    +
    +    private void setAuthenticator(OkHttpClient.Builder okHttpClientBuilder, ConfigurationContext context) {
    +        final String authUser = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_USERNAME).getValue());
    +        this.basicUser = authUser;
    +
    +
    +        isDigest = context.getProperty(PROP_DIGEST_AUTH).asBoolean();
    +        // If the username/password properties are set then check if digest auth is being used
    +        if (!authUser.isEmpty() && isDigest) {
    +            final String authPass = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_PASSWORD).getValue());
    +            this.basicPass = authPass;
    +
    +            /*
    +             * OkHttp doesn't have built-in Digest Auth Support. A ticket for adding it is here[1] but they authors decided instead to rely on a 3rd party lib.
    +             *
    +             * [1] https://github.com/square/okhttp/issues/205#issuecomment-154047052
    +             */
    +            final Map<String, CachingAuthenticator> authCache = new ConcurrentHashMap<>();
    +            com.burgstaller.okhttp.digest.Credentials credentials = new com.burgstaller.okhttp.digest.Credentials(authUser, authPass);
    +            final DigestAuthenticator digestAuthenticator = new DigestAuthenticator(credentials);
    +
    +            okHttpClientBuilder.interceptors().add(new AuthenticationCacheInterceptor(authCache));
    +            okHttpClientBuilder.authenticator(new CachingAuthenticatorDecorator(digestAuthenticator, authCache));
    +        }
    +    }
    +
    +    @Override
    +    public Class<?> getValueType() {
    +        return Record.class;
    +    }
    +
    +    @Override
    +    public Set<String> getRequiredKeys() {
    +        return new HashSet<String>(){{
    --- End diff --
    
    We should avoid the creation of anonymous inner classes when we can. We can also make this an UnmodifiableSet and just create a single instance of it as a member variable instead of constantly creating the Set


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194038840
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    --- End diff --
    
    Done.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @MikeThomsen 
    Here are the properties exposed for configuration of InvokeHttp:
    
    ```java
     public static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
                PROP_METHOD,
                PROP_URL,
                PROP_SSL_CONTEXT_SERVICE,
                PROP_CONNECT_TIMEOUT,
                PROP_READ_TIMEOUT,
                PROP_DATE_HEADER,
                PROP_FOLLOW_REDIRECTS,
                PROP_ATTRIBUTES_TO_SEND,
                PROP_BASIC_AUTH_USERNAME,
                PROP_BASIC_AUTH_PASSWORD,
                PROP_PROXY_HOST,
                PROP_PROXY_PORT,
                PROP_PROXY_TYPE,
                PROP_PROXY_USER,
                PROP_PROXY_PASSWORD,
                PROP_PUT_OUTPUT_IN_ATTRIBUTE,
                PROP_PUT_ATTRIBUTE_MAX_LENGTH,
                PROP_DIGEST_AUTH,
                PROP_OUTPUT_RESPONSE_REGARDLESS,
                PROP_TRUSTED_HOSTNAME,
                PROP_ADD_HEADERS_TO_REQUEST,
                PROP_CONTENT_TYPE,
                PROP_SEND_BODY,
                PROP_USE_CHUNKED_ENCODING,
                PROP_PENALIZE_NO_RETRY));
    ```
    
    Of these, I wonder if we should consider for the rest lookup
               PROP_CONNECT_TIMEOUT,
                PROP_READ_TIMEOUT,
                PROP_DATE_HEADER,
                PROP_FOLLOW_REDIRECTS,
                PROP_ATTRIBUTES_TO_SEND,
                PROP_TRUSTED_HOSTNAME,
                PROP_ADD_HEADERS_TO_REQUEST
    In whatever form makes sense for the lookup service.
    
    For example:  what if my lookup service is someone else's API and I need to send custom headers and api keys?
    



---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak Added a new commit that should get it to close out.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192610748
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy ---
    @@ -106,6 +106,37 @@ class RestLookupServiceIT {
             }
         }
     
    +    @Test
    +    void testHeaders() {
    +        runner.disableControllerService(lookupService)
    +        runner.setProperty(lookupService, "header.X-USER", "jane.doe")
    +        runner.setProperty(lookupService, "header.X-PASS", "testing7890")
    +        runner.enableControllerService(lookupService)
    +
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(SimpleJson.class, "/simple")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple",
    --- End diff --
    
    @MikeThomsen I've replied to your question on the dev ML. I think `endpoint` property can be used both cases where user want to use EL or just a literal endpoint. Because Query.prepare method can return the configured literal String if it does not contain any EL.
    
    ```
            final Map<String, String> map = Collections.singletonMap("name", "John Smith");
            PreparedQuery query = Query.prepare("${name}-${name:length()}");
            String result = query.evaluateExpressions(map, null);
            System.out.println(result);
    
            query = Query.prepare("name-name:length()");
            result = query.evaluateExpressions(map, null);
            System.out.println(result);
            final Map<String, String> map = Collections.singletonMap("name", "John Smith");
            PreparedQuery query = Query.prepare("${name}-${name:length()}");
            String result = query.evaluateExpressions(map, null);
            System.out.println(result);
    
            query = Query.prepare("name-name:length()");
            result = query.evaluateExpressions(map, null);
            System.out.println(result);
    ```
    
    The code prints:
    ```
    John Smith-10
    name-name:length()
    ```


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189580436
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,280 @@
    +/*
    + * 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.lookup;
    +
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    --- End diff --
    
    Probably


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556680
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194224970
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    --- End diff --
    
    I can see how that would be awkward. Do you want to handle that with a combination of property + lookup key (latter overriding the former) or defer to fixing `LookupRecord`?


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192492212
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    --- End diff --
    
    Should use a RecordPath validator, no?


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192493590
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    --- End diff --
    
    Can just use `Optional.ofNullable(record)` here


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak merged your PR. You can merge at any time.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193390225
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    --- End diff --
    
    This should support EL with Variable Registry.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192492300
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    --- End diff --
    
    No need for the explicit validator when identifying controller services.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556264
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    --- End diff --
    
    Done (minus docs).


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189751985
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.lookup;
    +
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +@Tags({ "rest", "lookup", "json", "xml" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return Optional.of(record);
    +        } catch (MalformedRecordException | SchemaNotFoundException | IOException e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger());
    --- End diff --
    
    Please use try-with-resource statement, similar to what AbstractRouteRecord does:
    ```
    try (final RecordReader reader = readerFactory.createRecordReader(originalAttributes, in, getLogger())) {
    ```


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194042057
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-utils/pom.xml ---
    @@ -0,0 +1,39 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!-- 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. -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <parent>
    +        <artifactId>nifi-standard-bundle</artifactId>
    +        <groupId>org.apache.nifi</groupId>
    +        <version>1.7.0-SNAPSHOT</version>
    +    </parent>
    +    <modelVersion>4.0.0</modelVersion>
    +    <artifactId>nifi-standard-web-utils</artifactId>
    --- End diff --
    
    Yeah. I can see this functionality being reused in plenty of places and don't think we should tie the transitive dependencies of the standard package's test scope into other packages as that could complicate things down the road. I'll change the module name to `nifi-standard-web-test-utils`. How about that?


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194038855
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194750294
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/SchemaUtil.groovy ---
    @@ -0,0 +1,74 @@
    +/*
    --- End diff --
    
    It's generally a best practice to put this sort of thing into .avsc files in src/test/resources


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193410520
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs/org.apache.nifi.lookup.RestLookupService/additionalDetails.html ---
    @@ -0,0 +1,54 @@
    +<!DOCTYPE html>
    +<html lang="en">
    +<!--
    +  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.
    +-->
    +<head>
    +    <meta charset="utf-8" />
    +    <title>RestLookupService</title>
    +    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
    +</head>
    +<body>
    +    <h2>General</h2>
    +    <p>This lookup service has the following required keys:</p>
    +    <ul>
    +        <li>mime.type</li>
    +        <li>request.method; valid values:
    +            <ul>
    +                <li>delete</li>
    +                <li>get</li>
    +                <li>post</li>
    +                <li>put</li>
    +            </ul>
    +        </li>
    +    </ul>
    +    <p>In addition to the required keys, a key "body" can be added which contains a string representing JSON, XML, etc. to be sent with any
    +    of those methods except for "get."</p>
    +    <p>The record reader is used to consume the response of the REST service call and turn it into one or more records. The record path property
    +    is provided to allow for a lookup path to either a nested record or a single point deep in the REST response. Note: a valid schema must be
    +    built that encapsulates the REST response accurately in order for this service to work.</p>
    +    <h2>Headers</h2>
    +    <p>Headers are supported using dynamic properties. Just add a dynamic property and the name will be the header name and the value will be the value for the header. Expression language
    +    powered by input from the variable registry is supported.</p>
    +    <h2>Dynamic URLs</h2>
    +    <p>The URL property supports expression language in a non-standard way: through the lookup key/value pairs configured on the processor. The configuration specified by the user will be passed
    --- End diff --
    
    I'd omit the expression, 'in a non-standard way', because I envision this pattern will be used more frequently and I'm planning to add copy FlowFile attribute to coordinates capability at LookupRecord processor


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189554878
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,280 @@
    +/*
    + * 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.lookup;
    +
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +@Tags({ "rest", "lookup", "json", "xml" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("rest-lookup-ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                    + "connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private SSLContextService sslContextService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    --- End diff --
    
    why are we getting the SSL_CONTEXT_SERVICE twice?


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194822881
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    --- End diff --
    
    This is not necessarily used to enrich records. It could be used by any number of processors that want to lookup some value. Or by LookupRecord to perform routing without doing any actual enrichment.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192488650
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.lookup;
    +
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +@Tags({ "rest", "lookup", "json", "xml" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return Optional.of(record);
    +        } catch (MalformedRecordException | SchemaNotFoundException | IOException e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger());
    +        Record record = reader.nextRecord();
    --- End diff --
    
    Ok, not sure how to trick it into doing null, but I put a check for that and made the optional be empty and added a new unit test to handle empty records (ex. badly defined schema has no fields from a rest response).


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556447
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    --- End diff --
    
    Great point. I think I'm going to have to make another property to handle this instead.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ottobackwards I think I might just go ahead and refactor that code now and get it done in this PR.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    This is cool, I will definitely make an aws web api version of this after it and my pr lands. 
     I think that any rest service needs to support the options that InvokeHttp supports.  Proxies etc.  This doesn't seem to do that.



---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193393615
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return Optional.ofNullable(record);
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected String determineEndpoint(Map<String, Object> coordinates) {
    +        Map<String, String> converted = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        return compiledQuery.evaluateExpressions(converted, null);
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        return new PropertyDescriptor.Builder()
    +            .name(propertyDescriptorName)
    +            .displayName(propertyDescriptorName)
    +            .addValidator(Validator.VALID)
    +            .dynamic(true)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        Map<String, Object> val = new HashMap<>();
    +                        val.put(recordPathName, value);
    +                        temp = new MapRecord(schema, val);
    +                    }
    +
    +                    record = temp;
    +                } else {
    +                    record = null;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw ex;
    +        }
    +    }
    +
    +    private Request buildRequest(final String mimeType, final String method, final String body, final String endpoint) {
    +        final MediaType mt = MediaType.parse(mimeType);
    +        RequestBody requestBody = null;
    +        if (body != null) {
    +            requestBody = RequestBody.create(mt, body);
    +        }
    +        Request.Builder request = new Request.Builder()
    +                .url(endpoint);
    +        switch(method.toLowerCase()) {
    +            case "delete":
    +                request = body != null ? request.delete(requestBody) : request.delete();
    +                break;
    +            case "get":
    +                request = request.get();
    +                break;
    +            case "post":
    +                request = request.post(requestBody);
    +                break;
    +            case "put":
    +                request = request.put(requestBody);
    +                break;
    +        }
    +
    +        if (headers != null) {
    +            for (Map.Entry<String, String> header : headers.entrySet()) {
    +                request = request.addHeader(header.getKey(), header.getValue());
    +            }
    +        }
    +
    +        if (!basicUser.isEmpty() && !isDigest) {
    +            String credential = Credentials.basic(basicUser, basicPass);
    +            request = request.header("Authorization", credential);
    +        }
    +
    +        return request.build();
    +    }
    +
    +    private void setAuthenticator(OkHttpClient.Builder okHttpClientBuilder, ConfigurationContext context) {
    +        final String authUser = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_USERNAME).getValue());
    +        this.basicUser = authUser;
    +
    +
    +        isDigest = context.getProperty(PROP_DIGEST_AUTH).asBoolean();
    +        // If the username/password properties are set then check if digest auth is being used
    +        if (!authUser.isEmpty() && isDigest) {
    +            final String authPass = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_PASSWORD).getValue());
    +            this.basicPass = authPass;
    --- End diff --
    
    I think `this.basicPass` should be set regardless of `isDigest`, similar to `this.basicUser`. Having it here will leave basicPass unset if isDigest is false. Is this correct?


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak Your first feedback points were merged in, let me know what you think of the last item and it should be easy to get done.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak I accidentally squashed the changes that @markap14 requested into your commit.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r195272389
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static final List VALID_VERBS = Arrays.asList("delete", "get", "post", "put");
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            BASE_URL,
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        KEYS = Collections.emptySet();
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    --- End diff --
    
    Done


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r195272720
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192555959
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    --- End diff --
    
    Ok. Didn't know it even existed :)


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192466939
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs.org.apache.nifi.lookup.RestLookupService/additionalDetails.html ---
    @@ -0,0 +1,42 @@
    +<!DOCTYPE html>
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189754939
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy ---
    @@ -106,6 +106,37 @@ class RestLookupServiceIT {
             }
         }
     
    +    @Test
    +    void testHeaders() {
    +        runner.disableControllerService(lookupService)
    +        runner.setProperty(lookupService, "header.X-USER", "jane.doe")
    +        runner.setProperty(lookupService, "header.X-PASS", "testing7890")
    +        runner.enableControllerService(lookupService)
    +
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(SimpleJson.class, "/simple")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple",
    --- End diff --
    
    In real use-cases, do we expect user to set 'server.url' at a LookupRecord processor's user defined property?
    
    As an alternative approach, I'd add `endpoint url` at RestLookupService to define a template string to resolve target endpoint, and let callers such as LookupRecord to pass required variables to complete the target endpoint.
    
    For example:
    - At RestLookupService
      - Endpoint URL: `https://some-api.example.com/buckets/${bucketId}/items/${itemId}`
    - At LookupRecord processor
      - bucketId as user-defined-property: ./bucketId (a record path to get a value)
      - itemId as user-defined-property: ./itemId (a record path to get a value)
    
    By doing so, it also supports passing the entire endpoint URL:
    - At RestLookupService
      - Endpoint URL: `${endpoint}`
    - At LookupRecord processor
      - endpoint as user-defined-property: a record path to construct an URL
    
    I'd expect RestLookupService to have more control on REST related configurations, and callers just pass variables to make actual requests. How do you think?


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192611657
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/rest/handlers/ComplexJson.groovy ---
    @@ -0,0 +1,45 @@
    +/*
    + * 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.lookup.rest.handlers
    +
    +import javax.servlet.http.HttpServlet
    +import javax.servlet.http.HttpServletRequest
    +import javax.servlet.http.HttpServletResponse
    +
    +import static groovy.json.JsonOutput.prettyPrint
    +import static groovy.json.JsonOutput.toJson
    +
    +class ComplexJson extends HttpServlet {
    +    @Override
    +    void doGet(HttpServletRequest request, HttpServletResponse response) throws IOException {
    +        response.contentType = "application/json"
    +        response.outputStream.write(prettyPrint(
    +            toJson([
    +                top: [
    +                    middle: [
    +                        inner: [
    +                            "username": "jane.doe",
    +                            "password": "testing7890",
    +                            "email": "jane.doe@company.com"
    --- End diff --
    
    The domain `company.com` actually exists. Please change it to `example.com`.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189750553
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs.org.apache.nifi.lookup.RestLookupService/additionalDetails.html ---
    @@ -0,0 +1,42 @@
    +<!DOCTYPE html>
    --- End diff --
    
    @MikeThomsen additionalDetails.html should be in a directory `src/main/resources/docs/qualifiedClassName/` not in `docs.qualifiedClassName`. Please check additional docs can be accessed from component's usage page from NiFi UI.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194039429
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192495332
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    --- End diff --
    
    I don't think this is actually what we want here. What if the path is something like `/myField[./second/third = 'hello']` - in that case, `last` will equal `third = 'hello']`


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192611673
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/test/groovy/org/apache/nifi/lookup/RestLookupServiceIT.groovy ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.lookup
    +
    +import org.apache.avro.Schema
    +import org.apache.nifi.avro.AvroTypeUtil
    +import org.apache.nifi.json.JsonTreeReader
    +import org.apache.nifi.lookup.rest.SchemaUtil
    +import org.apache.nifi.lookup.rest.handlers.BasicAuth
    +import org.apache.nifi.lookup.rest.handlers.ComplexJson
    +import org.apache.nifi.lookup.rest.handlers.NoRecord
    +import org.apache.nifi.lookup.rest.handlers.SimpleJson
    +import org.apache.nifi.lookup.rest.handlers.SimpleJsonArray
    +import org.apache.nifi.lookup.rest.handlers.VerbTest
    +import org.apache.nifi.schema.access.SchemaAccessUtils
    +import org.apache.nifi.serialization.record.MockSchemaRegistry
    +import org.apache.nifi.serialization.record.Record
    +import org.apache.nifi.serialization.record.RecordSchema
    +import org.apache.nifi.util.TestRunner
    +import org.apache.nifi.util.TestRunners
    +import org.apache.nifi.web.util.TestServer
    +import org.eclipse.jetty.servlet.ServletHandler
    +import org.junit.Assert
    +import org.junit.Before
    +import org.junit.Test
    +
    +import static groovy.json.JsonOutput.prettyPrint
    +import static groovy.json.JsonOutput.toJson
    +
    +class RestLookupServiceIT {
    +    static final JsonTreeReader reader
    +    static final MockSchemaRegistry registry = new MockSchemaRegistry()
    +    static final RecordSchema simpleSchema
    +    static final RecordSchema nestedSchema
    +
    +    TestRunner runner
    +    RestLookupService lookupService
    +
    +    static {
    +        simpleSchema = AvroTypeUtil.createSchema(new Schema.Parser().parse(SchemaUtil.SIMPLE))
    +        nestedSchema = AvroTypeUtil.createSchema(new Schema.Parser().parse(SchemaUtil.COMPLEX))
    +        registry.addSchema("simple", simpleSchema)
    +        registry.addSchema("complex", nestedSchema)
    +
    +        reader = new JsonTreeReader()
    +    }
    +
    +    @Before
    +    void setup() {
    +        lookupService = new RestLookupService()
    +
    +        runner = TestRunners.newTestRunner(TestRestLookupServiceProcessor.class)
    +        runner.addControllerService("jsonReader", reader)
    +        runner.addControllerService("registry", registry)
    +        runner.addControllerService("lookupService", lookupService)
    +        runner.setProperty(reader, SchemaAccessUtils.SCHEMA_REGISTRY, "registry")
    +        runner.setProperty(lookupService, SchemaAccessUtils.SCHEMA_REGISTRY, "registry")
    +        runner.setProperty(lookupService, RestLookupService.RECORD_READER, "jsonReader")
    +        runner.setProperty(TestRestLookupServiceProcessor.CLIENT_SERVICE, "lookupService")
    +        runner.enableControllerService(registry)
    +        runner.enableControllerService(reader)
    +        runner.enableControllerService(lookupService)
    +
    +        runner.assertValid()
    +    }
    +
    +    @Test
    +    void basicAuth() {
    +        runner.disableControllerService(lookupService)
    +        runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_USERNAME, "john.smith")
    +        runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_PASSWORD, "testing1234")
    +        runner.enableControllerService(lookupService)
    +
    +        TestServer server = new TestServer()
    +        server.addHandler(new BasicAuth())
    +        try {
    +            server.startServer()
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple",
    +                "mime.type": "application/json",
    +                "request.method": "get"
    +            ]
    +
    +            Optional<Record> response = lookupService.lookup(coordinates)
    +            Assert.assertTrue(response.isPresent())
    +            def record = response.get()
    +            Assert.assertEquals("john.smith", record.getAsString("username"))
    +            Assert.assertEquals("testing1234", record.getAsString("password"))
    +
    +            Throwable t
    +            try {
    +                runner.disableControllerService(lookupService)
    +                runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_USERNAME, "john.smith2")
    +                runner.setProperty(lookupService, RestLookupService.PROP_BASIC_AUTH_PASSWORD, ":wetadfasdfadf")
    +                runner.enableControllerService(lookupService)
    +
    +                lookupService.lookup(coordinates)
    +            } catch (Throwable lfe) {
    +                t = lfe
    +            }
    +
    +            Assert.assertNotNull(t)
    +            Assert.assertTrue(t.getClass().getCanonicalName(), t instanceof LookupFailureException)
    +        } finally {
    +            server.shutdownServer()
    +        }
    +    }
    +
    +    @Test
    +    void simpleJson() {
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(SimpleJson.class, "/simple")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple",
    +                "mime.type": "application/json",
    +                "request.method": "get"
    +            ]
    +
    +            Optional<Record> response = lookupService.lookup(coordinates)
    +            Assert.assertTrue(response.isPresent())
    +            def record = response.get()
    +            Assert.assertEquals("john.smith", record.getAsString("username"))
    +            Assert.assertEquals("testing1234", record.getAsString("password"))
    +        } finally {
    +            server.shutdownServer()
    +        }
    +    }
    +
    +    @Test
    +    void noRecord() {
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(NoRecord.class, "/simple")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple",
    +                "mime.type": "application/json",
    +                "request.method": "get"
    +            ]
    +
    +            Optional<Record> response = lookupService.lookup(coordinates)
    +            Assert.assertTrue(response.isPresent())
    +            def record = response.get()
    +            Assert.assertNull(record.getAsString("username"))
    +            Assert.assertNull(record.getAsString("password"))
    +        } finally {
    +            server.shutdownServer()
    +        }
    +    }
    +
    +    @Test
    +    void simpleJsonArray() {
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(SimpleJsonArray.class, "/simple_array")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple_array",
    +                "mime.type": "application/json",
    +                "request.method": "get"
    +            ]
    +
    +            Optional<Record> response = lookupService.lookup(coordinates)
    +            Assert.assertTrue(response.isPresent())
    +            def record = response.get()
    +            Assert.assertEquals("john.smith", record.getAsString("username"))
    +            Assert.assertEquals("testing1234", record.getAsString("password"))
    +        } finally {
    +            server.shutdownServer()
    +        }
    +    }
    +
    +    @Test
    +    void testHeaders() {
    +        runner.disableControllerService(lookupService)
    +        runner.setProperty(lookupService, "X-USER", "jane.doe")
    +        runner.setProperty(lookupService, "X-PASS", "testing7890")
    +        runner.enableControllerService(lookupService)
    +
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(SimpleJson.class, "/simple")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +
    +            def coordinates = [
    +                "schema.name": "simple",
    +                "endpoint": server.url + "/simple",
    +                "mime.type": "application/json",
    +                "request.method": "get"
    +            ]
    +
    +            Optional<Record> response = lookupService.lookup(coordinates)
    +            Assert.assertTrue(response.isPresent())
    +            def record = response.get()
    +            Assert.assertEquals("jane.doe", record.getAsString("username"))
    +            Assert.assertEquals("testing7890", record.getAsString("password"))
    +        } finally {
    +            server.shutdownServer()
    +        }
    +    }
    +
    +    @Test
    +    void complexJson() {
    +        runner.disableControllerService(lookupService)
    +        runner.setProperty(lookupService, RestLookupService.RECORD_PATH, "/top/middle/inner")
    +        runner.enableControllerService(lookupService)
    +
    +        TestServer server = new TestServer()
    +        ServletHandler handler = new ServletHandler()
    +        handler.addServletWithMapping(ComplexJson.class, "/complex")
    +        server.addHandler(handler)
    +        try {
    +            server.startServer()
    +            def coordinates = [
    +                "schema.name": "complex",
    +                "endpoint": server.url + "/complex",
    +                "mime.type": "application/json",
    +                "request.method": "get"
    +            ]
    +
    +            Optional<Record> response = lookupService.lookup(coordinates)
    +            Assert.assertTrue(response.isPresent())
    +            def record = response.get()
    +            Assert.assertEquals("jane.doe", record.getAsString("username"))
    +            Assert.assertEquals("testing7890", record.getAsString("password"))
    +            Assert.assertEquals("jane.doe@company.com", record.getAsString("email"))
    --- End diff --
    
    The domain `company.com` actually exists. Please change it to `example.com`.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194041176
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/resources/docs/org.apache.nifi.lookup.RestLookupService/additionalDetails.html ---
    @@ -0,0 +1,54 @@
    +<!DOCTYPE html>
    +<html lang="en">
    +<!--
    +  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.
    +-->
    +<head>
    +    <meta charset="utf-8" />
    +    <title>RestLookupService</title>
    +    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css" />
    +</head>
    +<body>
    +    <h2>General</h2>
    +    <p>This lookup service has the following required keys:</p>
    +    <ul>
    +        <li>mime.type</li>
    +        <li>request.method; valid values:
    +            <ul>
    +                <li>delete</li>
    +                <li>get</li>
    +                <li>post</li>
    +                <li>put</li>
    +            </ul>
    +        </li>
    +    </ul>
    +    <p>In addition to the required keys, a key "body" can be added which contains a string representing JSON, XML, etc. to be sent with any
    +    of those methods except for "get."</p>
    +    <p>The record reader is used to consume the response of the REST service call and turn it into one or more records. The record path property
    +    is provided to allow for a lookup path to either a nested record or a single point deep in the REST response. Note: a valid schema must be
    +    built that encapsulates the REST response accurately in order for this service to work.</p>
    +    <h2>Headers</h2>
    +    <p>Headers are supported using dynamic properties. Just add a dynamic property and the name will be the header name and the value will be the value for the header. Expression language
    +    powered by input from the variable registry is supported.</p>
    +    <h2>Dynamic URLs</h2>
    +    <p>The URL property supports expression language in a non-standard way: through the lookup key/value pairs configured on the processor. The configuration specified by the user will be passed
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556022
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556440
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    +                            put(last, value);
    +                        }});
    +                    }
    +
    +                    record = temp;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw new RuntimeException(ex);
    +        }
    +    }
    +
    +    private Request buildRequest(final String mimeType, final String method, final String body, final String endpoint) {
    +        final MediaType mt = MediaType.parse(mimeType);
    +        RequestBody requestBody = null;
    +        if (body != null) {
    +            requestBody = RequestBody.create(mt, body);
    +        }
    +        Request.Builder request = new Request.Builder()
    +                .url(endpoint);
    +        switch(method.toLowerCase()) {
    +            case "delete":
    +                request = body != null ? request.delete(requestBody) : request.delete();
    +                break;
    +            case "get":
    +                request = request.get();
    +                break;
    +            case "post":
    +                request = request.post(requestBody);
    +                break;
    +            case "put":
    +                request = request.put(requestBody);
    +                break;
    +        }
    +
    +        if (headers != null) {
    +            for (Map.Entry<String, String> header : headers.entrySet()) {
    +                request = request.addHeader(header.getKey(), header.getValue());
    +            }
    +        }
    +
    +        if (!basicUser.isEmpty() && !isDigest) {
    +            String credential = Credentials.basic(basicUser, basicPass);
    +            request = request.header("Authorization", credential);
    +        }
    +
    +        return request.build();
    +    }
    +
    +    private String basicUser;
    +    private String basicPass;
    +    private boolean isDigest;
    +
    +    private void setAuthenticator(OkHttpClient.Builder okHttpClientBuilder, ConfigurationContext context) {
    +        final String authUser = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_USERNAME).getValue());
    +        this.basicUser = authUser;
    +
    +
    +        isDigest = context.getProperty(PROP_DIGEST_AUTH).asBoolean();
    +        // If the username/password properties are set then check if digest auth is being used
    +        if (!authUser.isEmpty() && isDigest) {
    +            final String authPass = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_PASSWORD).getValue());
    +            this.basicPass = authPass;
    +
    +            /*
    +             * OkHttp doesn't have built-in Digest Auth Support. A ticket for adding it is here[1] but they authors decided instead to rely on a 3rd party lib.
    +             *
    +             * [1] https://github.com/square/okhttp/issues/205#issuecomment-154047052
    +             */
    +            final Map<String, CachingAuthenticator> authCache = new ConcurrentHashMap<>();
    +            com.burgstaller.okhttp.digest.Credentials credentials = new com.burgstaller.okhttp.digest.Credentials(authUser, authPass);
    +            final DigestAuthenticator digestAuthenticator = new DigestAuthenticator(credentials);
    +
    +            okHttpClientBuilder.interceptors().add(new AuthenticationCacheInterceptor(authCache));
    +            okHttpClientBuilder.authenticator(new CachingAuthenticatorDecorator(digestAuthenticator, authCache));
    +        }
    +    }
    +
    +    @Override
    +    public Class<?> getValueType() {
    +        return Record.class;
    +    }
    +
    +    @Override
    +    public Set<String> getRequiredKeys() {
    +        return new HashSet<String>(){{
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194930167
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    --- End diff --
    
    @MikeThomsen Sent a PR to your branch to fix this. Please merge.
    https://github.com/MikeThomsen/nifi/pull/2/commits


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @MikeThomsen that's ok, I think it's time to squash all commits into one, I'm fine to lose my commit histories. Now we have #2777 merged into master, so we can utilize it. Can you update RestLookupService to do that? Like I did it locally here. I am +1 once this gets done.
    https://github.com/ijokarumawak/nifi/commit/f97924e047a2a32a2a492d02a320497716c95a09



---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192495631
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    +                            put(last, value);
    +                        }});
    +                    }
    +
    +                    record = temp;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw new RuntimeException(ex);
    --- End diff --
    
    Would avoid wrapping checked exceptions that are known to be thrown by method (such as IOException) in a RuntimeException. Am guessing you can just use `throw ex` because all checked exceptions that can be caught are likely already declared for the method but not sure off the top of my head.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194824170
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static final List VALID_VERBS = Arrays.asList("delete", "get", "post", "put");
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            BASE_URL,
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        KEYS = Collections.emptySet();
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    --- End diff --
    
    headers, client, recordPath, readerFactory, and proxyConfigurationService are not accessed in a thread-safe way. Need to address thread safety there.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194041102
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    --- End diff --
    
    Done.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ottobackwards I grabbed a bunch of code from InvokeHttp and did a little copy pasta. Please take a look when you get a chance. I think moving the copy pasta into a shared utility package will need to be a separate ticket in order to keep the ticket scope down.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189580389
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,280 @@
    +/*
    + * 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.lookup;
    +
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +@Tags({ "rest", "lookup", "json", "xml" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("rest-lookup-ssl-context-service")
    +            .displayName("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                    + "connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private SSLContextService sslContextService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    --- End diff --
    
    Copy pasta


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556320
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    +                            put(last, value);
    +                        }});
    +                    }
    +
    +                    record = temp;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw new RuntimeException(ex);
    --- End diff --
    
    Done.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ijokarumawak I'll take a stab at adding the template url option.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192493332
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    --- End diff --
    
    If dynamic properties are used for specifying headers to use, why not just let the property name be the name of the header? I.e., why require a "header." prefix? We should also document how dynamic properties are used via the @DynamicProperties annotation.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556017
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189554397
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,280 @@
    +/*
    + * 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.lookup;
    +
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    --- End diff --
    
    Should this be tagged HTTP?


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    Don't I know it.  If you want to do this as a 'series' of prs, I think you should document that in jira with tasks representing each PR and note it clearly in the pr.
    
    That will set the right context for the reviewers.
    
    I chose with my InvokeAwsGatewayApi processor to make sure that I replicated the InvokeHttp test suite.  I would suggest that the end goal here would be to have and extensive set of tests adapted from that suite for the reader.
    



---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193398691
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    --- End diff --
    
    Using `getOrDefault(METHOD_KEY, "get")` would be more user friendly.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194823411
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    --- End diff --
    
    I don't understand why we have two different properties here for "URL" and "Base URL". Can you elaborate? If it's because we want to evaluate the Expression Language differently, then we can still do that by just using `context.getProperty(URL).evaluateExpressionLanguage(coordinates);` and that will evaluate against both the coordinates the Variable Registry...


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192494587
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    --- End diff --
    
    If this evaluates to `false` (i.e., the RecordPath doesn't match anything) then we return the entire Record. I don't think that's what we want here? Should we return `null` in that case? We should probably document the behavior, whatever it is.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194214335
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    --- End diff --
    
    @MikeThomsen RequiredKeys is not mandatory, it can be an empty Set. Also, this is a `Lookup` service. Get should be the default method like the famous cURL command.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194925388
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    --- End diff --
    
    @markap14 Thanks, that is correct! I added "Base URL" because I thought it is not possible to use Variable Registry, but that was because it uses PreparedQuery instead of PropertyValue. By using PropertyValue, it can use Variable Registry, too.
    
    @MikeThomsen I will send a PR again to fix this.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r195272595
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static final List VALID_VERBS = Arrays.asList("delete", "get", "post", "put");
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            BASE_URL,
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        KEYS = Collections.emptySet();
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    --- End diff --
    
    I put it in onDisabled


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

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


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556247
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556066
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192556019
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    --- End diff --
    
    Done.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194038826
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return Optional.ofNullable(record);
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected String determineEndpoint(Map<String, Object> coordinates) {
    +        Map<String, String> converted = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        return compiledQuery.evaluateExpressions(converted, null);
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        return new PropertyDescriptor.Builder()
    +            .name(propertyDescriptorName)
    +            .displayName(propertyDescriptorName)
    +            .addValidator(Validator.VALID)
    +            .dynamic(true)
    +            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +            .build();
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        Map<String, Object> val = new HashMap<>();
    +                        val.put(recordPathName, value);
    +                        temp = new MapRecord(schema, val);
    +                    }
    +
    +                    record = temp;
    +                } else {
    +                    record = null;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw ex;
    +        }
    +    }
    +
    +    private Request buildRequest(final String mimeType, final String method, final String body, final String endpoint) {
    +        final MediaType mt = MediaType.parse(mimeType);
    +        RequestBody requestBody = null;
    +        if (body != null) {
    +            requestBody = RequestBody.create(mt, body);
    +        }
    +        Request.Builder request = new Request.Builder()
    +                .url(endpoint);
    +        switch(method.toLowerCase()) {
    +            case "delete":
    +                request = body != null ? request.delete(requestBody) : request.delete();
    +                break;
    +            case "get":
    +                request = request.get();
    +                break;
    +            case "post":
    +                request = request.post(requestBody);
    +                break;
    +            case "put":
    +                request = request.put(requestBody);
    +                break;
    +        }
    +
    +        if (headers != null) {
    +            for (Map.Entry<String, String> header : headers.entrySet()) {
    +                request = request.addHeader(header.getKey(), header.getValue());
    +            }
    +        }
    +
    +        if (!basicUser.isEmpty() && !isDigest) {
    +            String credential = Credentials.basic(basicUser, basicPass);
    +            request = request.header("Authorization", credential);
    +        }
    +
    +        return request.build();
    +    }
    +
    +    private void setAuthenticator(OkHttpClient.Builder okHttpClientBuilder, ConfigurationContext context) {
    +        final String authUser = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_USERNAME).getValue());
    +        this.basicUser = authUser;
    +
    +
    +        isDigest = context.getProperty(PROP_DIGEST_AUTH).asBoolean();
    +        // If the username/password properties are set then check if digest auth is being used
    +        if (!authUser.isEmpty() && isDigest) {
    +            final String authPass = trimToEmpty(context.getProperty(PROP_BASIC_AUTH_PASSWORD).getValue());
    +            this.basicPass = authPass;
    --- End diff --
    
    Done.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ottobackwards @ijokarumawak Added support for the new ProxyConfigurationService. Would appreciate a review.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r189752084
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,276 @@
    +/*
    + * 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.lookup;
    +
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.stream.Collectors;
    +
    +@Tags({ "rest", "lookup", "json", "xml" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return Optional.of(record);
    +        } catch (MalformedRecordException | SchemaNotFoundException | IOException e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger());
    +        Record record = reader.nextRecord();
    --- End diff --
    
    We should handle the case where record is null.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192722942
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -277,6 +280,20 @@ private void setProxy(OkHttpClient.Builder builder) {
             }
         }
     
    --- End diff --
    
    I don't think so because the EL support is enabled on the fly by the user, and any user who is enterprising enough to do that should be responsible for validating it. There's no good fallback option short of having them specify yet another key, and IMO that's overkill.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    @ottobackwards I'll add user-defined headers and basic auth support, but that's it until there's a discussion around a HTTP Connection Service. I think that would be the place to go crazy on the options, not the lookup service.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192917579
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -277,6 +280,20 @@ private void setProxy(OkHttpClient.Builder builder) {
             }
         }
     
    +    protected String determineEndpoint(Map<String, Object> coordinates) {
    +        if (coordinates.containsKey(ENDPOINT_KEY) && coordinates.containsKey(ENDPOINT_TEMPLATE_KEY)) {
    +            Map<String, String> converted = coordinates.entrySet().stream()
    +                .collect(Collectors.toMap(
    +                    e -> e.getKey(),
    +                    e -> e.getValue().toString()
    +                ));
    +            final PreparedQuery query = Query.prepare((String)coordinates.get(ENDPOINT_KEY));
    --- End diff --
    
    @MikeThomsen Thanks for adding this! This gets closer to what I was imagined. Excuse me if I was not clear enough, but from a user's stand point, I'd like to configure this ENDPOINT_KEY value at the RestLookupService property rather than passing it via coordinates.
    
    Because, I think people would use one RestLookupService to lookup some endpoints (can be more than 1) of a particular Web service API. If they need to deal with different Web services, e.g. Twitter, Facebook and Salesforce ...etc they would use 3 RestLookupService, and each one can have the endpoint template defined.
    
    Further more, if we implemented like that, we can cache the PreparedQuery instance without parsing the EL every time. Also, no need to have `ENDPOINT_TEMPLATE_KEY` coordinate to tell if it contains EL or not because the PreparedQuery returns the original String if it does not contains EL.
    
    How do you think?


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192493834
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    --- End diff --
    
    Is generally considered 'bad practice' to log an Exception, then re-throw or wrap-and-throw. Would just throw new LookupFailureException(e)


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194039043
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    --- End diff --
    
    A value is required because it's a required key, so I don't think that's necessary.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192496067
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.getName().startsWith("header.")) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = (String)coordinates.get(ENDPOINT_KEY);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    +        final String body     = (String)coordinates.get(BODY_KEY);
    +
    +        if (StringUtils.isBlank(body) && (method.equals("post") || method.equals("put"))) {
    +            throw new LookupFailureException(
    +                String.format("Used HTTP verb %s without specifying the %s key to provide a payload.", method, BODY_KEY)
    +            );
    +        }
    +
    +        Request request = buildRequest(mimeType, method, body, endpoint);
    +        try {
    +            Response response = executeRequest(request);
    +            InputStream is = response.body().byteStream();
    +
    +            Record record = handleResponse(is, coordinates);
    +
    +            return record != null
    +                    ? Optional.of(record)
    +                    : Optional.empty();
    +        } catch (Exception e) {
    +            getLogger().error("Could not execute lookup.", e);
    +            throw new LookupFailureException(e);
    +        }
    +    }
    +
    +    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
    +        if (propertyDescriptorName.startsWith("header")) {
    +            String header = propertyDescriptorName.substring(propertyDescriptorName.indexOf(".") + 1);
    +            return new PropertyDescriptor.Builder()
    +                .name(propertyDescriptorName)
    +                .displayName(header)
    +                .addValidator(Validator.VALID)
    +                .dynamic(true)
    +                .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +                .build();
    +        }
    +
    +        return null;
    +    }
    +
    +    protected Response executeRequest(Request request) throws IOException {
    +        return client.newCall(request).execute();
    +    }
    +
    +    private Record handleResponse(InputStream is, Map<String, Object> coordinates) throws SchemaNotFoundException, MalformedRecordException, IOException {
    +        Map<String, String> variables = coordinates.entrySet().stream()
    +            .collect(Collectors.toMap(
    +                e -> e.getKey(),
    +                e -> e.getValue().toString()
    +            ));
    +        try (RecordReader reader = readerFactory.createRecordReader(variables, is, getLogger())) {
    +
    +            Record record = reader.nextRecord();
    +
    +            if (recordPath != null) {
    +                Optional<FieldValue> fv = recordPath.evaluate(record).getSelectedFields().findFirst();
    +                if (fv.isPresent()) {
    +                    FieldValue fieldValue = fv.get();
    +                    RecordSchema schema = new SimpleRecordSchema(Arrays.asList(fieldValue.getField()));
    +                    String[] parts = recordPath.getPath().split("/");
    +                    String last = parts[parts.length - 1];
    +
    +                    Record temp;
    +                    Object value = fieldValue.getValue();
    +                    if (value instanceof Record) {
    +                        temp = (Record) value;
    +                    } else if (value instanceof Map) {
    +                        temp = new MapRecord(schema, (Map<String, Object>) value);
    +                    } else {
    +                        temp = new MapRecord(schema, new HashMap<String, Object>() {{
    +                            put(last, value);
    +                        }});
    +                    }
    +
    +                    record = temp;
    +                }
    +            }
    +
    +            return record;
    +        } catch (Exception ex) {
    +            is.close();
    +            throw new RuntimeException(ex);
    +        }
    +    }
    +
    +    private Request buildRequest(final String mimeType, final String method, final String body, final String endpoint) {
    +        final MediaType mt = MediaType.parse(mimeType);
    +        RequestBody requestBody = null;
    +        if (body != null) {
    +            requestBody = RequestBody.create(mt, body);
    +        }
    +        Request.Builder request = new Request.Builder()
    +                .url(endpoint);
    +        switch(method.toLowerCase()) {
    +            case "delete":
    +                request = body != null ? request.delete(requestBody) : request.delete();
    +                break;
    +            case "get":
    +                request = request.get();
    +                break;
    +            case "post":
    +                request = request.post(requestBody);
    +                break;
    +            case "put":
    +                request = request.put(requestBody);
    +                break;
    +        }
    +
    +        if (headers != null) {
    +            for (Map.Entry<String, String> header : headers.entrySet()) {
    +                request = request.addHeader(header.getKey(), header.getValue());
    +            }
    +        }
    +
    +        if (!basicUser.isEmpty() && !isDigest) {
    +            String credential = Credentials.basic(basicUser, basicPass);
    +            request = request.header("Authorization", credential);
    +        }
    +
    +        return request.build();
    +    }
    +
    +    private String basicUser;
    --- End diff --
    
    We should avoid member variables throughout code and try to concentrate them at the top of the class. Additionally, these 3 member variables are not protected so the class is not thread-safe. Need to ensure that all ControllerServices are thread-safe.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r193387554
  
    --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-web-utils/pom.xml ---
    @@ -0,0 +1,39 @@
    +<?xml version="1.0" encoding="UTF-8"?>
    +<!-- 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. -->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +    <parent>
    +        <artifactId>nifi-standard-bundle</artifactId>
    +        <groupId>org.apache.nifi</groupId>
    +        <version>1.7.0-SNAPSHOT</version>
    +    </parent>
    +    <modelVersion>4.0.0</modelVersion>
    +    <artifactId>nifi-standard-web-utils</artifactId>
    --- End diff --
    
    I think extracting TestServer.java was not necessary, and I'd avoid doing this in this PR. Because:
    - It makes this PR to touch many files unnecessarily in terms of its objective.
    - Having test scope dependency to `nifi-standard-processors` should be fine to reuse TestServer.
    - Even if we do extract TestServer, the module name `nifi-standard-web-utils` will not be appropriate to contain Test utility class. It should be named more clearly to state that this module is for test. `nifi-mock` is a good example.
    
    Do you have a strong motivation to do this? 


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194824341
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,467 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import okhttp3.ResponseBody;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements RecordLookupService {
    +    static final PropertyDescriptor BASE_URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-base-url")
    +        .displayName("Base URL")
    +        .description("The base URL for the REST endpoint. Expression language is evaluated against variable registry." +
    +                " This property can be used to resolve environment specific part of the URL." +
    +                " The result string is prepended to the 'URL'." +
    +                " See 'Additional Details' to see an example.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .required(false)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static final List VALID_VERBS = Arrays.asList("delete", "get", "post", "put");
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            BASE_URL,
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        KEYS = Collections.emptySet();
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +
    +        final String url = context.getProperty(URL).getValue();
    +        compiledQuery = context.getProperty(BASE_URL).isSet()
    +                ? Query.prepare(context.getProperty(BASE_URL).evaluateAttributeExpressions().getValue() + url)
    +                : Query.prepare(url);
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    --- End diff --
    
    Probably best not to have 'get*' methods that are 'void' types


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192492813
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,390 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .addValidator(Validator.VALID)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String ENDPOINT_KEY = "endpoint";
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            RECORD_READER,
    +            RECORD_PATH,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        getHeaders(context);
    +    }
    +
    +    private Map<String, String> headers;
    --- End diff --
    
    Typical convention in NiFi is to add member variables at the top of the class before constructors.


---

[GitHub] nifi issue #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723
  
    I almost wonder if there should be an http rest connection service


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r194214189
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -0,0 +1,435 @@
    +/*
    + * 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.lookup;
    +
    +import com.burgstaller.okhttp.AuthenticationCacheInterceptor;
    +import com.burgstaller.okhttp.CachingAuthenticatorDecorator;
    +import com.burgstaller.okhttp.digest.CachingAuthenticator;
    +import com.burgstaller.okhttp.digest.DigestAuthenticator;
    +import okhttp3.Credentials;
    +import okhttp3.MediaType;
    +import okhttp3.OkHttpClient;
    +import okhttp3.Request;
    +import okhttp3.RequestBody;
    +import okhttp3.Response;
    +import org.apache.nifi.annotation.behavior.DynamicProperties;
    +import org.apache.nifi.annotation.behavior.DynamicProperty;
    +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.attribute.expression.language.PreparedQuery;
    +import org.apache.nifi.attribute.expression.language.Query;
    +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.processor.util.StandardValidators;
    +import org.apache.nifi.proxy.ProxyConfiguration;
    +import org.apache.nifi.proxy.ProxyConfigurationService;
    +import org.apache.nifi.proxy.ProxySpec;
    +import org.apache.nifi.record.path.FieldValue;
    +import org.apache.nifi.record.path.RecordPath;
    +import org.apache.nifi.record.path.validation.RecordPathValidator;
    +import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.nifi.serialization.MalformedRecordException;
    +import org.apache.nifi.serialization.RecordReader;
    +import org.apache.nifi.serialization.RecordReaderFactory;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.MapRecord;
    +import org.apache.nifi.serialization.record.Record;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.StringUtils;
    +
    +import javax.net.ssl.SSLContext;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.net.Proxy;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Optional;
    +import java.util.Set;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.regex.Pattern;
    +import java.util.stream.Collectors;
    +
    +import static org.apache.commons.lang3.StringUtils.trimToEmpty;
    +
    +@Tags({ "rest", "lookup", "json", "xml", "http" })
    +@CapabilityDescription("Use a REST service to enrich records.")
    +@DynamicProperties({
    +    @DynamicProperty(name = "*", value = "*", description = "All dynamic properties are added as HTTP headers with the name " +
    +            "as the header name and the value as the header value.")
    +})
    +public class RestLookupService extends AbstractControllerService implements LookupService<Record> {
    +    static final PropertyDescriptor URL = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-url")
    +        .displayName("URL")
    +        .description("The URL for the REST endpoint. Expression language is evaluated against the lookup key/value pairs, " +
    +                "not flowfile attributes or variable registry.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .required(true)
    +        .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-reader")
    +        .displayName("Record Reader")
    +        .description("The record reader to use for loading the payload and handling it as a record set.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.NONE)
    +        .identifiesControllerService(RecordReaderFactory.class)
    +        .required(true)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path")
    +        .displayName("Record Path")
    +        .description("An optional record path that can be used to define where in a record to get the real data to merge " +
    +                "into the record set to be enriched. See documentation for examples of when this might be useful.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(new RecordPathValidator())
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor RECORD_PATH_PROPERTY_NAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-record-path-name")
    +        .displayName("Record Path Property Name")
    +        .description("An optional name for the property loaded by the record path. This will be the key used for the value " +
    +                "when the loaded record is merged into record to be enriched. See docs for more details.")
    +        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
    +        .addValidator(Validator.VALID)
    +        .required(false)
    +        .build();
    +
    +    static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-ssl-context-service")
    +        .displayName("SSL Context Service")
    +        .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                + "connections.")
    +        .required(false)
    +        .identifiesControllerService(SSLContextService.class)
    +        .build();
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_USERNAME = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-username")
    +        .displayName("Basic Authentication Username")
    +        .description("The username to be used by the client to authenticate against the Remote URL.  Cannot include control characters (0-31), ':', or DEL (127).")
    +        .required(false)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +
    +    public static final PropertyDescriptor PROP_BASIC_AUTH_PASSWORD = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-basic-auth-password")
    +        .displayName("Basic Authentication Password")
    +        .description("The password to be used by the client to authenticate against the Remote URL.")
    +        .required(false)
    +        .sensitive(true)
    +        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
    +        .build();
    +    public static final PropertyDescriptor PROP_DIGEST_AUTH = new PropertyDescriptor.Builder()
    +        .name("rest-lookup-digest-auth")
    +        .displayName("Use Digest Authentication")
    +        .description("Whether to communicate with the website using Digest Authentication. 'Basic Authentication Username' and 'Basic Authentication Password' are used "
    +                + "for authentication.")
    +        .required(false)
    +        .defaultValue("false")
    +        .allowableValues("true", "false")
    +        .build();
    +
    +    private static final ProxySpec[] PROXY_SPECS = {ProxySpec.HTTP_AUTH, ProxySpec.SOCKS};
    +    public static final PropertyDescriptor PROXY_CONFIGURATION_SERVICE
    +            = ProxyConfiguration.createProxyConfigPropertyDescriptor(true, PROXY_SPECS);
    +
    +    static final String MIME_TYPE_KEY = "mime.type";
    +    static final String BODY_KEY = "request.body";
    +    static final String METHOD_KEY = "request.method";
    +
    +    static final List<PropertyDescriptor> DESCRIPTORS;
    +    static final Set<String> KEYS;
    +
    +    static {
    +        DESCRIPTORS = Collections.unmodifiableList(Arrays.asList(
    +            URL,
    +            RECORD_READER,
    +            RECORD_PATH,
    +            RECORD_PATH_PROPERTY_NAME,
    +            SSL_CONTEXT_SERVICE,
    +            PROXY_CONFIGURATION_SERVICE,
    +            PROP_BASIC_AUTH_USERNAME,
    +            PROP_BASIC_AUTH_PASSWORD,
    +            PROP_DIGEST_AUTH
    +        ));
    +        Set<String> _keys = new HashSet<>();
    +        _keys.add(MIME_TYPE_KEY);
    +        _keys.add(METHOD_KEY);
    +        KEYS = Collections.unmodifiableSet(_keys);
    +    }
    +
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return DESCRIPTORS;
    +    }
    +
    +    private ProxyConfigurationService proxyConfigurationService;
    +    private RecordReaderFactory readerFactory;
    +    private RecordPath recordPath;
    +    private OkHttpClient client;
    +    private Map<String, String> headers;
    +    private volatile PreparedQuery compiledQuery;
    +    private volatile String recordPathName;
    +    private volatile String basicUser;
    +    private volatile String basicPass;
    +    private volatile boolean isDigest;
    +
    +    @OnEnabled
    +    public void onEnabled(final ConfigurationContext context) {
    +        readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class);
    +        proxyConfigurationService = context.getProperty(PROXY_CONFIGURATION_SERVICE)
    +                .asControllerService(ProxyConfigurationService.class);
    +
    +        OkHttpClient.Builder builder = new OkHttpClient.Builder();
    +
    +        setAuthenticator(builder, context);
    +
    +        if (proxyConfigurationService != null) {
    +            setProxy(builder);
    +        }
    +
    +        final SSLContextService sslService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class);
    +        final SSLContext sslContext = sslService == null ? null : sslService.createSSLContext(SSLContextService.ClientAuth.WANT);
    +        if (sslService != null) {
    +            builder.sslSocketFactory(sslContext.getSocketFactory());
    +        }
    +
    +        client = builder.build();
    +
    +        String path = context.getProperty(RECORD_PATH).isSet() ? context.getProperty(RECORD_PATH).getValue() : null;
    +        if (!StringUtils.isBlank(path)) {
    +            recordPath = RecordPath.compile(path);
    +        }
    +
    +        recordPathName = context.getProperty(RECORD_PATH_PROPERTY_NAME).isSet()
    +            ? context.getProperty(RECORD_PATH_PROPERTY_NAME).evaluateAttributeExpressions().getValue()
    +            : null;
    +
    +        getHeaders(context);
    +
    +        compiledQuery = Query.prepare(context.getProperty(URL).getValue());
    +    }
    +
    +    @OnDisabled
    +    public void onDisabled() {
    +        this.compiledQuery = null;
    +    }
    +
    +    private void getHeaders(ConfigurationContext context) {
    +        headers = new HashMap<>();
    +        for (PropertyDescriptor descriptor : context.getProperties().keySet()) {
    +            if (descriptor.isDynamic()) {
    +                headers.put(
    +                    descriptor.getDisplayName(),
    +                    context.getProperty(descriptor).evaluateAttributeExpressions().getValue()
    +                );
    +            }
    +        }
    +    }
    +
    +    private void setProxy(OkHttpClient.Builder builder) {
    +        ProxyConfiguration config = proxyConfigurationService.getConfiguration();
    +        if (!config.getProxyType().equals(Proxy.Type.DIRECT)) {
    +            final Proxy proxy = config.createProxy();
    +            builder.proxy(proxy);
    +
    +            if (config.hasCredential()){
    +                builder.proxyAuthenticator((route, response) -> {
    +                    final String credential= Credentials.basic(config.getProxyUserName(), config.getProxyUserPassword());
    +                    return response.request().newBuilder()
    +                            .header("Proxy-Authorization", credential)
    +                            .build();
    +                });
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public Optional<Record> lookup(Map<String, Object> coordinates) throws LookupFailureException {
    +        final String endpoint = determineEndpoint(coordinates);
    +        final String mimeType = (String)coordinates.get(MIME_TYPE_KEY);
    +        final String method   = (String)coordinates.get(METHOD_KEY);
    --- End diff --
    
    When I used the lookup service from LookupRecord, I felt passing a literal value as RecordPath was a bit tricky. I had to use `concat('get')`.


---

[GitHub] nifi pull request #2723: NIFI-5214 Added REST LookupService

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

    https://github.com/apache/nifi/pull/2723#discussion_r192722395
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-lookup-services-bundle/nifi-lookup-services/src/main/java/org/apache/nifi/lookup/RestLookupService.java ---
    @@ -277,6 +280,20 @@ private void setProxy(OkHttpClient.Builder builder) {
             }
         }
     
    --- End diff --
    
    Should this handle AttributeExpressionLanguageParsingException?  Is there any validation that can be done?


---