You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "turcsanyip (via GitHub)" <gi...@apache.org> on 2023/10/06 08:04:59 UTC

Re: [PR] NIFI-11992: Processor and sink service for filing tickets in Zendesk [nifi]

turcsanyip commented on code in PR #7644:
URL: https://github.com/apache/nifi/pull/7644#discussion_r1342722329


##########
nifi-nar-bundles/nifi-zendesk-bundle/nifi-zendesk-common/src/main/java/org/apache/nifi/common/zendesk/ZendeskAuthenticationContext.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.common.zendesk;
+
+public class ZendeskAuthenticationContext {
+
+    private final String subDomain;

Review Comment:
   Minor:
   ```suggestion
       private final String subdomain;
   ```



##########
nifi-nar-bundles/nifi-zendesk-bundle/nifi-zendesk-processors/src/main/java/org/apache/nifi/processors/zendesk/PutZendeskTicket.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.zendesk;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.common.zendesk.validation.JsonPointerPropertyNameValidator;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+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.record.Record;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.common.zendesk.ZendeskProperties.APPLICATION_JSON;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.REL_FAILURE_NAME;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.WEB_CLIENT_SERVICE_PROVIDER;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_AUTHENTICATION_CREDENTIAL;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_AUTHENTICATION_TYPE;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_CREATE_TICKETS_RESOURCE;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_CREATE_TICKET_RESOURCE;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_SUBDOMAIN;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKET_COMMENT_BODY_NAME;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKET_PRIORITY_NAME;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKET_SUBJECT_NAME;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKET_TYPE_NAME;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_USER;
+import static org.apache.nifi.common.zendesk.util.ZendeskRecordPathUtils.addDynamicField;
+import static org.apache.nifi.common.zendesk.util.ZendeskRecordPathUtils.resolveFieldValue;
+import static org.apache.nifi.common.zendesk.util.ZendeskUtils.createRequestObject;
+import static org.apache.nifi.common.zendesk.util.ZendeskUtils.getDynamicProperties;
+import static org.apache.nifi.common.zendesk.util.ZendeskUtils.getResponseBody;
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+import static org.apache.nifi.processors.zendesk.AbstractZendesk.RECORD_COUNT_ATTRIBUTE_NAME;
+import static org.apache.nifi.processors.zendesk.PutZendeskTicket.ERROR_CODE_ATTRIBUTE_NAME;
+import static org.apache.nifi.processors.zendesk.PutZendeskTicket.ERROR_MESSAGE_ATTRIBUTE_NAME;
+import static org.apache.nifi.web.client.api.HttpResponseStatus.CREATED;
+import static org.apache.nifi.web.client.api.HttpResponseStatus.OK;
+
+@Tags({"zendesk, ticket"})
+@CapabilityDescription("Create Zendesk tickets using the Zendesk API.")
+@DynamicProperty(
+        name = "The path in the request object to add. The value needs be a valid JsonPointer.",
+        value = "The path in the incoming record to get the value from.",
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES,
+        description = "Additional property to be added to the Zendesk request object.")
+@WritesAttributes({
+        @WritesAttribute(attribute = RECORD_COUNT_ATTRIBUTE_NAME, description = "The number of records processed."),
+        @WritesAttribute(attribute = ERROR_CODE_ATTRIBUTE_NAME, description = "The error code of from the response."),
+        @WritesAttribute(attribute = ERROR_MESSAGE_ATTRIBUTE_NAME, description = "The error message of from the response.")})
+public class PutZendeskTicket extends AbstractZendesk {
+
+    static final String ZENDESK_RECORD_READER_NAME = "zendesk-record-reader";
+    static final String ERROR_CODE_ATTRIBUTE_NAME = "error.code";
+    static final String ERROR_MESSAGE_ATTRIBUTE_NAME = "error.message";
+
+    private static final ObjectMapper mapper = new ObjectMapper();
+
+    static final PropertyDescriptor RECORD_READER = new PropertyDescriptor.Builder()
+            .name(ZENDESK_RECORD_READER_NAME)
+            .displayName("Record Reader")
+            .description("Specifies the Controller Service to use for parsing incoming data and determining the data's schema.")
+            .identifiesControllerService(RecordReaderFactory.class)
+            .build();
+
+    public static final PropertyDescriptor ZENDESK_TICKET_COMMENT_BODY = new PropertyDescriptor.Builder()
+            .name(ZENDESK_TICKET_COMMENT_BODY_NAME)
+            .displayName("Comment Body")
+            .description("The content or the path to the comment body in the incoming record.")
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+            .required(true)
+            .dependsOn(RECORD_READER)
+            .build();

Review Comment:
   `Comment Body` and further properties below are common with `ZendeskRecordSink` so they can be defined in `ZendeskProperties` and then can be customized here like adding the `dependOn()` clause (which is specific to the processor).



##########
nifi-nar-bundles/nifi-zendesk-bundle/nifi-zendesk-common/src/main/java/org/apache/nifi/common/zendesk/util/ZendeskRecordPathUtils.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.common.zendesk.util;
+
+import com.fasterxml.jackson.core.JsonPointer;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.databind.node.TextNode;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.type.ChoiceDataType;
+
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKETS_ROOT_NODE;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKET_ROOT_NODE;
+
+public final class ZendeskRecordPathUtils {
+
+    private static final String NULL_VALUE = "null";
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+    private static final Pattern RECORD_PATH_PATTERN = Pattern.compile("%\\{(.*?)\\}");

Review Comment:
   The pattern does not the require leading `/` in the record path. Is it a valid record path which does not start with `/`?



##########
nifi-nar-bundles/nifi-zendesk-bundle/nifi-zendesk-common/src/main/java/org/apache/nifi/common/zendesk/util/ZendeskRecordPathUtils.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.common.zendesk.util;
+
+import com.fasterxml.jackson.core.JsonPointer;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.databind.node.TextNode;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.record.path.FieldValue;
+import org.apache.nifi.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.type.ChoiceDataType;
+
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import static java.util.stream.Collectors.toList;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKETS_ROOT_NODE;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_TICKET_ROOT_NODE;
+
+public final class ZendeskRecordPathUtils {

Review Comment:
   This util class covers 2 main functionalities in my understanding:
   
   1. resolve a record path (coming from a property value)
   2. add a value (e.g. the resolved record field) to a Json object (using a json pointer to locate the extension point)
   
   I would separate these two areas and promote # 1 to a common module (# 2 may worth being extracted to a common too).
   
   Suggested module for the record path util: `nifi-extension-utils/nifi-record-path-property` 
   Suggested class name: `RecordPathPropertyUtil`
   
   The resolve method may accept `PropertyValue` parameter because it is not a generic record path evaluator but related to component properties.



##########
nifi-nar-bundles/nifi-zendesk-bundle/nifi-zendesk-common/src/main/java/org/apache/nifi/common/zendesk/ZendeskClient.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.common.zendesk;
+
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.OptionalLong;
+
+import static java.lang.String.format;
+import static java.util.Base64.getEncoder;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.HTTPS;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_HOST_TEMPLATE;
+
+public class ZendeskClient {
+
+    private static final String AUTHORIZATION_HEADER_NAME = "Authorization";
+    private static final String CONTENT_TYPE_HEADER_NAME = "Content-Type";
+    private static final String BASIC_AUTH_PREFIX = "Basic ";
+
+    private final WebClientServiceProvider webClientServiceProvider;
+    private final ZendeskAuthenticationContext authenticationContext;
+
+    public ZendeskClient(WebClientServiceProvider webClientServiceProvider, ZendeskAuthenticationContext authenticationContext) {
+        this.webClientServiceProvider = webClientServiceProvider;
+        this.authenticationContext = authenticationContext;
+    }
+
+    /**
+     * Sends the ticket creation POST request to the Zendesk API.
+     *
+     * @param uri         target uri
+     * @param inputStream body of the request
+     * @return response from the Zendesk API
+     * @throws IOException error while performing the POST request
+     */
+    public HttpResponseEntity performPostRequest(URI uri, InputStream inputStream) throws IOException {
+        return webClientServiceProvider.getWebClientService()
+                .post()
+                .uri(uri)
+                .header(CONTENT_TYPE_HEADER_NAME, ZendeskProperties.APPLICATION_JSON)
+                .header(AUTHORIZATION_HEADER_NAME, basicAuthHeaderValue())
+                .body(inputStream, OptionalLong.of(inputStream.available()))
+                .retrieve();
+    }
+
+    /**
+     * Sends a query GET request to the Zendesk API.
+     *
+     * @param uri target uri
+     * @return result of query
+     */
+    public HttpResponseEntity performQuery(URI uri) {

Review Comment:
   It is a generic GET request method. I would suggest the following renaming:
   ```suggestion
       /**
        * Sends a GET request to the Zendesk API.
        *
        * @param uri target uri
        * @return response from the Zendesk API
        */
       public HttpResponseEntity performGetRequest(URI uri) {
   ```



##########
nifi-nar-bundles/nifi-zendesk-bundle/nifi-zendesk-common/src/main/java/org/apache/nifi/common/zendesk/ZendeskClient.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.common.zendesk;
+
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.HttpUriBuilder;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.OptionalLong;
+
+import static java.lang.String.format;
+import static java.util.Base64.getEncoder;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.HTTPS;
+import static org.apache.nifi.common.zendesk.ZendeskProperties.ZENDESK_HOST_TEMPLATE;
+
+public class ZendeskClient {
+
+    private static final String AUTHORIZATION_HEADER_NAME = "Authorization";
+    private static final String CONTENT_TYPE_HEADER_NAME = "Content-Type";
+    private static final String BASIC_AUTH_PREFIX = "Basic ";
+
+    private final WebClientServiceProvider webClientServiceProvider;
+    private final ZendeskAuthenticationContext authenticationContext;
+
+    public ZendeskClient(WebClientServiceProvider webClientServiceProvider, ZendeskAuthenticationContext authenticationContext) {
+        this.webClientServiceProvider = webClientServiceProvider;
+        this.authenticationContext = authenticationContext;
+    }
+
+    /**
+     * Sends the ticket creation POST request to the Zendesk API.
+     *
+     * @param uri         target uri
+     * @param inputStream body of the request
+     * @return response from the Zendesk API
+     * @throws IOException error while performing the POST request
+     */
+    public HttpResponseEntity performPostRequest(URI uri, InputStream inputStream) throws IOException {

Review Comment:
   As the method name shows, it is a generic POST method and could be used for other requests, not only ticket creation.
   ```suggestion
       /**
        * Sends a POST request to the Zendesk API.
        *
        * @param uri         target uri
        * @param inputStream body of the request
        * @return response from the Zendesk API
        * @throws IOException error while performing the POST request
        */
       public HttpResponseEntity performPostRequest(URI uri, InputStream inputStream) throws IOException {
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org