You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/09/12 08:31:40 UTC

[GitHub] [nifi] ferencerdei commented on a diff in pull request #6376: NIFI-10455 Get workday report processor

ferencerdei commented on code in PR #6376:
URL: https://github.com/apache/nifi/pull/6376#discussion_r968131712


##########
nifi-assembly/pom.xml:
##########
@@ -227,6 +227,12 @@ language governing permissions and limitations under the License. -->
             <version>1.18.0-SNAPSHOT</version>
             <type>nar</type>
         </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-web-client-provider-service-nar</artifactId>
+            <version>1.18.0-SNAPSHOT</version>
+            <type>nar</type>
+        </dependency>

Review Comment:
   removed



##########
nifi-nar-bundles/nifi-workday-bundle/nifi-workday-processors/src/main/java/org/apache/nifi/processors/workday/GetWorkdayReport.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.workday;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.URL_VALIDATOR;
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collection;
+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.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+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.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.security.util.crypto.HashAlgorithm;
+import org.apache.nifi.security.util.crypto.HashService;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@Tags({"Workday", "report", "get"})
+@InputRequirement(Requirement.INPUT_ALLOWED)
+@CapabilityDescription("A processor which can interact with a configurable Workday Report. The processor can forward the content without modification, or you can transform it by"
+    + " providing the specific Record Reader and Record Writer services based on your needs. You can also hash, or remove fields using the input parameters and schemes in the Record writer. "
+    + "Supported Workday report formats are: csv, simplexml, json")
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@WritesAttributes({
+    @WritesAttribute(attribute = GetWorkdayReport.GET_WORKDAY_REPORT_JAVA_EXCEPTION_CLASS, description = "The Java exception class raised when the processor fails"),
+    @WritesAttribute(attribute = GetWorkdayReport.GET_WORKDAY_REPORT_JAVA_EXCEPTION_MESSAGE, description = "The Java exception message raised when the processor fails"),
+    @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Source / Record Writer"),
+    @WritesAttribute(attribute= GetWorkdayReport.RECORD_COUNT, description = "The number of records in an outgoing FlowFile. This is only populated on the 'success' relationship "
+        + "when Record Reader and Writer is set.")})
+public class GetWorkdayReport extends AbstractProcessor {
+
+    protected static final String STATUS_CODE = "getworkdayreport.status.code";
+    protected static final String REQUEST_URL = "getworkdayreport.request.url";
+    protected static final String REQUEST_DURATION = "getworkdayreport.request.duration";
+    protected static final String TRANSACTION_ID = "getworkdayreport.tx.id";
+    protected static final String GET_WORKDAY_REPORT_JAVA_EXCEPTION_CLASS = "getworkdayreport.java.exception.class";
+    protected static final String GET_WORKDAY_REPORT_JAVA_EXCEPTION_MESSAGE = "getworkdayreport.java.exception.message";
+    protected static final String RECORD_COUNT = "record.count";
+    protected static final String BASIC_PREFIX = "Basic ";
+    protected static final String COLUMNS_TO_HASH_SEPARATOR = ",";
+    protected static final String HEADER_AUTHORIZATION = "Authorization";
+    protected static final String HEADER_CONTENT_TYPE = "Content-Type";
+    protected static final String USERNAME_PASSWORD_SEPARATOR = ":";
+
+    protected static final PropertyDescriptor REPORT_URL = new PropertyDescriptor.Builder()
+        .name("Workday report URL")
+        .displayName("Workday report URL")

Review Comment:
   done



##########
nifi-nar-bundles/nifi-workday-bundle/nifi-workday-processors/src/main/java/org/apache/nifi/processors/workday/GetWorkdayReport.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.workday;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+import static org.apache.nifi.expression.ExpressionLanguageScope.NONE;
+import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.URL_VALIDATOR;
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collection;
+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.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.flowfile.attributes.CoreAttributes;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.AbstractProcessor;
+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.record.path.RecordPath;
+import org.apache.nifi.record.path.RecordPathResult;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.security.util.crypto.HashAlgorithm;
+import org.apache.nifi.security.util.crypto.HashService;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.RecordSetWriter;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.web.client.api.HttpResponseEntity;
+import org.apache.nifi.web.client.api.WebClientService;
+import org.apache.nifi.web.client.provider.api.WebClientServiceProvider;
+
+@Tags({"Workday", "report", "get"})
+@InputRequirement(Requirement.INPUT_ALLOWED)
+@CapabilityDescription("A processor which can interact with a configurable Workday Report. The processor can forward the content without modification, or you can transform it by"
+    + " providing the specific Record Reader and Record Writer services based on your needs. You can also hash, or remove fields using the input parameters and schemes in the Record writer. "
+    + "Supported Workday report formats are: csv, simplexml, json")
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@WritesAttributes({
+    @WritesAttribute(attribute = GetWorkdayReport.GET_WORKDAY_REPORT_JAVA_EXCEPTION_CLASS, description = "The Java exception class raised when the processor fails"),
+    @WritesAttribute(attribute = GetWorkdayReport.GET_WORKDAY_REPORT_JAVA_EXCEPTION_MESSAGE, description = "The Java exception message raised when the processor fails"),
+    @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Source / Record Writer"),
+    @WritesAttribute(attribute= GetWorkdayReport.RECORD_COUNT, description = "The number of records in an outgoing FlowFile. This is only populated on the 'success' relationship "
+        + "when Record Reader and Writer is set.")})
+public class GetWorkdayReport extends AbstractProcessor {
+
+    protected static final String STATUS_CODE = "getworkdayreport.status.code";
+    protected static final String REQUEST_URL = "getworkdayreport.request.url";
+    protected static final String REQUEST_DURATION = "getworkdayreport.request.duration";
+    protected static final String TRANSACTION_ID = "getworkdayreport.tx.id";
+    protected static final String GET_WORKDAY_REPORT_JAVA_EXCEPTION_CLASS = "getworkdayreport.java.exception.class";
+    protected static final String GET_WORKDAY_REPORT_JAVA_EXCEPTION_MESSAGE = "getworkdayreport.java.exception.message";
+    protected static final String RECORD_COUNT = "record.count";
+    protected static final String BASIC_PREFIX = "Basic ";
+    protected static final String COLUMNS_TO_HASH_SEPARATOR = ",";
+    protected static final String HEADER_AUTHORIZATION = "Authorization";
+    protected static final String HEADER_CONTENT_TYPE = "Content-Type";
+    protected static final String USERNAME_PASSWORD_SEPARATOR = ":";
+
+    protected static final PropertyDescriptor REPORT_URL = new PropertyDescriptor.Builder()
+        .name("Workday report URL")
+        .displayName("Workday report URL")
+        .description("HTTP remote URL of Workday report including a scheme of http or https, as well as a hostname or IP address with optional port and path elements.")
+        .required(true)
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .addValidator(URL_VALIDATOR)
+        .build();
+
+    protected static final PropertyDescriptor WORKDAY_USERNAME = new PropertyDescriptor.Builder()
+        .name("Workday Username")
+        .displayName("Workday Username")
+        .description("The username provided for authentication of Workday requests. Encoded using Base64 for HTTP Basic Authentication as described in RFC 7617.")
+        .required(true)
+        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x39\\x3b-\\x7e\\x80-\\xff]+$")))
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .build();
+
+    protected static final PropertyDescriptor WORKDAY_PASSWORD = new PropertyDescriptor.Builder()
+        .name("Workday Password")
+        .displayName("Workday Password")
+        .description("The password provided for authentication of Workday requests. Encoded using Base64 for HTTP Basic Authentication as described in RFC 7617.")
+        .required(true)
+        .sensitive(true)
+        .addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("^[\\x20-\\x7e\\x80-\\xff]+$")))
+        .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+        .build();
+
+    protected static final PropertyDescriptor WEB_CLIENT_SERVICE = new PropertyDescriptor.Builder()
+        .name("Standard Web Client Service")

Review Comment:
   done



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