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/10/27 14:26:49 UTC

[GitHub] [nifi] KalmanJantner opened a new pull request, #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

KalmanJantner opened a new pull request, #6589:
URL: https://github.com/apache/nifi/pull/6589

   …anscribe
   
   <!-- 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. -->
   
   # Summary
   
   [NIFI-00000](https://issues.apache.org/jira/browse/NIFI-00000)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


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


[GitHub] [nifi] exceptionfactory commented on pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on PR #6589:
URL: https://github.com/apache/nifi/pull/6589#issuecomment-1339483465

   @KalmanJantner, I will take a closer look at the latest changes soon. Please avoid introducing merge commits, instead, please rebase and force-push changes to align with the main branch. Thanks!


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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1072008691


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.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.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
+            .name("textract-type")
+            .displayName("Textract Type")
+            .required(true)
+            .description("Supported values: \"Document Analysis\", \"Document Text Detection\", \"Expense Analysis\"")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(DOCUMENT_ANALYSIS)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES =
+            Collections.unmodifiableList(Stream.concat(PROPERTIES.stream(), Stream.of(TEXTRACT_TYPE)).collect(Collectors.toList()));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String textractType = context.getProperty(TEXTRACT_TYPE).evaluateAttributeExpressions(flowFile).getValue();
+
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);
+        JobStatus jobStatus = getTaskStatus(textractType, getClient(), awsTaskId);

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.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.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
+            .name("textract-type")
+            .displayName("Textract Type")
+            .required(true)
+            .description("Supported values: \"Document Analysis\", \"Document Text Detection\", \"Expense Analysis\"")

Review Comment:
   Thx, fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.aws.ml;
+
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    private final static ObjectMapper MAPPER = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null && !context.getProperty(JSON_PAYLOAD).isSet()) {
+            return;
+        }
+        final RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context, flowFile);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+            session.transfer(childFlowFile, REL_SUCCESS);
+        } catch (Exception e) {
+            if (context.getProperty(JSON_PAYLOAD).isSet()) {

Review Comment:
   Thx, fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.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.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
+            .name("textract-type")
+            .displayName("Textract Type")
+            .required(true)
+            .description("Supported values: \"Document Analysis\", \"Document Text Detection\", \"Expense Analysis\"")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(DOCUMENT_ANALYSIS)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES =
+            Collections.unmodifiableList(Stream.concat(PROPERTIES.stream(), Stream.of(TEXTRACT_TYPE)).collect(Collectors.toList()));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String textractType = context.getProperty(TEXTRACT_TYPE).evaluateAttributeExpressions(flowFile).getValue();
+
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);

Review Comment:
   I've added awsTaskId property. Thx for the comment



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.aws.ml;
+
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    private final static ObjectMapper MAPPER = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null && !context.getProperty(JSON_PAYLOAD).isSet()) {
+            return;
+        }
+        final RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context, flowFile);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+            session.transfer(childFlowFile, REL_SUCCESS);
+        } catch (Exception e) {
+            if (context.getProperty(JSON_PAYLOAD).isSet()) {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        if (flowFile != null) {
+            session.transfer(flowFile, REL_ORIGINAL);
+        }
+
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response, flowFile));
+        session.putAttribute(flowFile, MIME_TYPE.key(), "application/json");

Review Comment:
   thx, fixed



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1086202483


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.aws.ml;
+
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    private final static ObjectMapper MAPPER = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null && !context.getProperty(JSON_PAYLOAD).isSet()) {
+            return;
+        }
+        final RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context, flowFile);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+            session.transfer(childFlowFile, REL_SUCCESS);
+        } catch (Exception e) {
+            if (flowFile != null) {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        if (flowFile != null) {
+            session.transfer(flowFile, REL_ORIGINAL);
+        }
+
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        flowFile = session.putAttribute(flowFile, TASK_ID.getName(), getAwsTaskId(context, response, flowFile));
+        flowFile = session.putAttribute(flowFile, MIME_TYPE.key(), "application/json");
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response, flowFile));

Review Comment:
   ```suggestion
           final String awsTaskId = getAwsTaskId(context, response, flowFile);
           flowFile = session.putAttribute(flowFile, TASK_ID.getName(), awsTaskId);
           flowFile = session.putAttribute(flowFile, MIME_TYPE.key(), "application/json");
           getLogger().debug("AWS ML Task [{}] started", awsTaskId);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = "PollyS3OutputBucket", description = "The bucket name where polly output will be located."),
+        @WritesAttribute(attribute = "PollyS3OutputKey", description = "Object key of polly output."),
+        @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
+})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "filename";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(context, flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().warn("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        } else if (taskStatus == TaskStatus.Completed) {
+            String outputUri = speechSynthesisTask.getSynthesisTask().getOutputUri();
+
+            Matcher matcher = S3_PATH.matcher(outputUri);
+            if (matcher.find()) {
+                session.putAttribute(flowFile, AWS_S3_BUCKET, matcher.group(BUCKET));
+                session.putAttribute(flowFile, AWS_S3_KEY, matcher.group(KEY));
+            }
+            FlowFile childFlowFile = session.create(flowFile);
+            writeToFlowFile(session, childFlowFile, speechSynthesisTask);
+            childFlowFile = session.putAttribute(childFlowFile, AWS_TASK_OUTPUT_LOCATION, outputUri);
+            session.transfer(flowFile, REL_ORIGINAL);
+            session.transfer(childFlowFile, REL_SUCCESS);
+            getLogger().info("Amazon Polly reported that the task completed for {}", flowFile);

Review Comment:
   ```suggestion
               getLogger().info("Amazon Polly Task Completed {}", flowFile);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = "PollyS3OutputBucket", description = "The bucket name where polly output will be located."),
+        @WritesAttribute(attribute = "PollyS3OutputKey", description = "Object key of polly output."),
+        @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
+})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "filename";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(context, flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().warn("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        } else if (taskStatus == TaskStatus.Completed) {
+            String outputUri = speechSynthesisTask.getSynthesisTask().getOutputUri();
+
+            Matcher matcher = S3_PATH.matcher(outputUri);
+            if (matcher.find()) {
+                session.putAttribute(flowFile, AWS_S3_BUCKET, matcher.group(BUCKET));
+                session.putAttribute(flowFile, AWS_S3_KEY, matcher.group(KEY));
+            }
+            FlowFile childFlowFile = session.create(flowFile);
+            writeToFlowFile(session, childFlowFile, speechSynthesisTask);
+            childFlowFile = session.putAttribute(childFlowFile, AWS_TASK_OUTPUT_LOCATION, outputUri);
+            session.transfer(flowFile, REL_ORIGINAL);
+            session.transfer(childFlowFile, REL_SUCCESS);
+            getLogger().info("Amazon Polly reported that the task completed for {}", flowFile);
+        } else if (taskStatus == TaskStatus.Failed) {
+            final String failureReason =  speechSynthesisTask.getSynthesisTask().getTaskStatusReason();
+            flowFile = session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Amazon Polly reported that the task failed for {}: {}", flowFile, failureReason);

Review Comment:
   ```suggestion
               getLogger().error("Amazon Polly Task Failed {} Reason [{}]", flowFile, failureReason);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractType.DOCUMENT_ANALYSIS;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.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.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
+            .name("textract-type")
+            .displayName("Textract Type")
+            .required(true)
+            .description("Supported values: \"Document Analysis\", \"Document Text Detection\", \"Expense Analysis\"")
+            .allowableValues(TextractType.TEXTRACT_TYPES)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(DOCUMENT_ANALYSIS.getType())
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES =
+            Collections.unmodifiableList(Stream.concat(PROPERTIES.stream(), Stream.of(TEXTRACT_TYPE)).collect(Collectors.toList()));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String textractType = context.getProperty(TEXTRACT_TYPE).evaluateAttributeExpressions(flowFile).getValue();
+
+        String awsTaskId = context.getProperty(TASK_ID).evaluateAttributeExpressions(flowFile).getValue();
+        try {
+            JobStatus jobStatus = getTaskStatus(TextractType.fromString(textractType), getClient(), awsTaskId);
+            if (JobStatus.SUCCEEDED == jobStatus) {
+                Object task = getTask(TextractType.fromString(textractType), getClient(), awsTaskId);
+                writeToFlowFile(session, flowFile, task);
+                session.transfer(flowFile, REL_SUCCESS);
+            } else if (JobStatus.IN_PROGRESS == jobStatus) {
+                session.transfer(flowFile, REL_RUNNING);
+            } else if (JobStatus.PARTIAL_SUCCESS == jobStatus) {
+                session.transfer(flowFile, REL_THROTTLED);
+            } else if (JobStatus.FAILED == jobStatus) {
+                session.transfer(flowFile, REL_FAILURE);
+                getLogger().error("Amazon Textract reported that the task failed for awsTaskId: {}", awsTaskId);

Review Comment:
   ```suggestion
                   getLogger().error("Amazon Textract Task [{}] Failed", awsTaskId);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Transcribe"})
+@CapabilityDescription("Retrieves the current status of an AWS Transcribe job.")
+@SeeAlso({StartAwsTranscribeJob.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
+})
+public class GetAwsTranscribeJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTranscribeClient> {
+    @Override
+    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        try {
+            GetTranscriptionJobResult job = getJob(context, flowFile);
+            TranscriptionJobStatus jobStatus = TranscriptionJobStatus.fromValue(job.getTranscriptionJob().getTranscriptionJobStatus());
+
+            if (TranscriptionJobStatus.COMPLETED == jobStatus) {
+                writeToFlowFile(session, flowFile, job);
+                session.putAttribute(flowFile, AWS_TASK_OUTPUT_LOCATION, job.getTranscriptionJob().getTranscript().getTranscriptFileUri());
+                session.transfer(flowFile, REL_SUCCESS);
+            } else if (TranscriptionJobStatus.IN_PROGRESS == jobStatus) {
+                session.transfer(flowFile, REL_RUNNING);
+            } else if (TranscriptionJobStatus.FAILED == jobStatus) {
+                final String failureReason = job.getTranscriptionJob().getFailureReason();
+                session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);
+                session.transfer(flowFile, REL_FAILURE);
+                getLogger().error("Transcribe Task Failed for {}: {}", flowFile, failureReason);

Review Comment:
   ```suggestion
                   getLogger().error("Transcribe Task Failed {} Reason [{}]", flowFile, failureReason);
   ```



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


[GitHub] [nifi] KalmanJantner commented on pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on PR #6589:
URL: https://github.com/apache/nifi/pull/6589#issuecomment-1339536388

   > @KalmanJantner, I will take a closer look at the latest changes soon. Please avoid introducing merge commits, instead, please rebase and force-push changes to align with the main branch. Thanks!
   
   Thanks @exceptionfactory, I have removed the merge commit.


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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1006993638


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));

Review Comment:
   The individual Proxy properties are not necessary since all of that can be configured using the Proxy Configuration Service.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);

Review Comment:
   In general, error messages should not end with a `.` character. It would also be helpful to rephrase the message along the following lines:
   ```suggestion
               getLogger().error("Sending AWS ML Request failed", e);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);
+            return;
+        }
+
+        try {
+            writeToFlowFile(session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during writing aws response to flow file.", e);
+            return;
+        }
+
+        try {
+            postProcessFlowFile(context, session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during AWS ML post processing.", e);
+            return;
+        }
+
+        session.transfer(flowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        REQUEST request;
+        try {
+            request = mapper.readValue(getPayload(session, context, flowFile),
+                    getAwsRequestClass(context));
+        } catch (JsonProcessingException e) {
+            getLogger().error("Exception was thrown during AWS ML request creation.", e);
+            throw e;

Review Comment:
   In general, it is not necessary to log an error and then throw the exception. If it is necessary to provide more context, it is better to throw new exception with the specific error message.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsResponseMetadataDeserializer.java:
##########
@@ -0,0 +1,19 @@
+package org.apache.nifi.processors.aws.ml;

Review Comment:
   This class is missing a license header.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-textract</artifactId>
+            <version>1.12.328</version>
+        </dependency>

Review Comment:
   Is there a reason for not using version 2 of the AWS SDK with the group ID of `software.amazon.awssdk`?



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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1011237210


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-textract</artifactId>
+            <version>1.12.328</version>
+        </dependency>

Review Comment:
   Good point thank you. I checked it and the problem is around the super classes. I should refactor `AwsCredentialsProvider` related classes and we need to introduce version 2 compatible version of `AbstractAWSProcessor`, `AbstractAWSCredentialsProviderProcessor`, `AWSCredentialsProviderControllerService` etc. Should I go ahead and do this refactor as part of this PR?



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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1052517590


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(

Review Comment:
   Thx, fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        final RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.aws.ml.translate;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import com.amazonaws.services.translate.model.OutputDataConfig;
+import com.amazonaws.services.translate.model.TextTranslationJobProperties;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String CONTENT_STRING = "content";
+    private static final String AWS_CREDENTIALS_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocation";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranslateClient mockTranslateClient = null;
+    @Mock
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.aws.ml.translate;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import com.amazonaws.services.translate.model.OutputDataConfig;
+import com.amazonaws.services.translate.model.TextTranslationJobProperties;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String CONTENT_STRING = "content";
+    private static final String AWS_CREDENTIALS_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocation";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranslateClient mockTranslateClient = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = "PollyS3OutputBucket", description = "The bucket name where polly output will be located."),
+        @WritesAttribute(attribute = "PollyS3OutputKey", description = "Object key of polly output."),
+        @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
+})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";

Review Comment:
   Thanks, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.aws.ml.translate;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import com.amazonaws.services.translate.model.OutputDataConfig;
+import com.amazonaws.services.translate.model.TextTranslationJobProperties;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String CONTENT_STRING = "content";
+    private static final String AWS_CREDENTIALS_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocation";
+    private TestRunner runner = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";

Review Comment:
   thx fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.textract.GetAwsTextractJobStatus.DOCUMENT_ANALYSIS;
+import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TYPE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTextractClient mockTextractClient = null;
+    @Mock
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.textract.GetAwsTextractJobStatus.DOCUMENT_ANALYSIS;
+import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TYPE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTextractClient mockTextractClient = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")

Review Comment:
   Fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.Transcript;
+import com.amazonaws.services.transcribe.model.TranscriptionJob;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranscribeJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String AWS_CREDENTIAL_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocationPath";
+    private static final String REASON_OF_FAILURE = "reasonOfFailure";
+    private static final String CONTENT_STRING = "content";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranscribeClient mockTranscribeClient = null;
+    @Mock
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_RUNNING,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Client creation not supported");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule awsResponseModule = new SimpleModule();
+        awsResponseModule.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule sdkHttpModule = new SimpleModule();
+        awsResponseModule.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(awsResponseModule);
+        mapper.registerModule(sdkHttpModule);

Review Comment:
   Fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.textract.GetAwsTextractJobStatus.DOCUMENT_ANALYSIS;
+import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TYPE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")

Review Comment:
   Fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()

Review Comment:
   Thanks fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")

Review Comment:
   Fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")

Review Comment:
   Fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.Transcript;
+import com.amazonaws.services.transcribe.model.TranscriptionJob;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranscribeJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String AWS_CREDENTIAL_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocationPath";
+    private static final String REASON_OF_FAILURE = "reasonOfFailure";
+    private static final String CONTENT_STRING = "content";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranscribeClient mockTranscribeClient = null;

Review Comment:
   Thx, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES =
+        Collections.unmodifiableList(Stream.concat(PROPERTIES.stream(), Stream.of(TYPE)).collect(Collectors.toList()));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, AmazonWebServiceResult response) {
+        super.postProcessFlowFile(context, session, flowFile, response);
+        session.putAttribute(flowFile, TYPE.getName(), context.getProperty(TYPE.getName()).getValue());
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    protected AmazonWebServiceResult sendRequest(AmazonWebServiceRequest request, ProcessContext context) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();

Review Comment:
   Fixed



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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1031118221


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-textract</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   Fixed, thx.



##########
pom.xml:
##########
@@ -147,6 +147,7 @@
         <swagger.annotations.version>1.6.6</swagger.annotations.version>
         <h2.version>2.1.214</h2.version>
         <zookeeper.version>3.8.0</zookeeper.version>
+        <aws.ml.version>1.12.328</aws.ml.version>

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusGetter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusGetter<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_RUNNING,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Client creation not supported");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule awsResponseModule = new SimpleModule();
+        awsResponseModule.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule sdkHttpModule = new SimpleModule();
+        awsResponseModule.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(awsResponseModule);
+        mapper.registerModule(sdkHttpModule);
+    }
+
+
+    protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {
+        session.write(flowFile, out -> {
+            try (OutputStreamWriter outputStreamWriter = new OutputStreamWriter(out, StandardCharsets.UTF_8)) {
+                outputStreamWriter.write(mapper.writeValueAsString(response));
+                outputStreamWriter.flush();
+            }

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);
+
+    protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        FlowFile childFlowFile = session.create(flowFile);
+        session.write(childFlowFile, out -> {
+            try (OutputStreamWriter outputStreamWriter = new OutputStreamWriter(out, StandardCharsets.UTF_8)) {
+                outputStreamWriter.write(mapper.writeValueAsString(response));
+                outputStreamWriter.flush();
+            }

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusGetter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusGetter<T extends AmazonWebServiceClient>

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusGetter<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES = ImmutableList.<PropertyDescriptor>builder()
+            .addAll(PROPERTIES)
+            .add(TYPE)
+            .build();;
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String typeOfTextract = context.getProperty(TYPE).evaluateAttributeExpressions().getValue();
+
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);
+        JobStatus jobStatus = getTaskStatus(typeOfTextract, getClient(), awsTaskId);
+        if (JobStatus.SUCCEEDED == jobStatus) {
+            Object task = getTask(typeOfTextract, getClient(), awsTaskId);
+            writeToFlowFile(session, flowFile, task);
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+
+        if (JobStatus.IN_PROGRESS == jobStatus) {
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (JobStatus.PARTIAL_SUCCESS == jobStatus) {
+            session.transfer(flowFile, REL_THROTTLED);
+        }
+
+        if (JobStatus.FAILED == jobStatus) {

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableList;

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import com.google.common.collect.ImmutableList;

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;
+
+    @BeforeEach
+    public void setUp() throws InitializationException {
+        mockPollyClient = Mockito.mock(AmazonPollyClient.class);
+        mockAwsCredentialsProvider = Mockito.mock(AWSCredentialsProviderService.class);
+        when(mockAwsCredentialsProvider.getIdentifier()).thenReturn("awsCredetialProvider");
+        final GetAwsPollyJobStatus mockGetAwsPollyStatus = new GetAwsPollyJobStatus() {
+            protected AmazonPollyClient getClient() {
+                return mockPollyClient;
+            }
+
+            @Override
+            protected AmazonPollyClient createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+                return mockPollyClient;
+            }
+        };
+        runner = TestRunners.newTestRunner(mockGetAwsPollyStatus);
+        runner.addControllerService("awsCredetialProvider", mockAwsCredentialsProvider);
+        runner.enableControllerService(mockAwsCredentialsProvider);
+        runner.setProperty(AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredetialProvider");
+    }
+
+    @Test
+    public void testPollyTaskInProgress() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);
+        GetSpeechSynthesisTaskResult taskResult = new GetSpeechSynthesisTaskResult();
+        SynthesisTask task = new SynthesisTask().withTaskId(TEST_TASK_ID)
+                .withTaskStatus(TaskStatus.InProgress);
+        taskResult.setSynthesisTask(task);
+        when(mockPollyClient.getSpeechSynthesisTask(requestCaptor.capture())).thenReturn(taskResult);
+        runner.enqueue("content", Collections.singletonMap(AWS_TASK_ID_PROPERTY, TEST_TASK_ID));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(REL_RUNNING);
+        assertEquals(requestCaptor.getValue().getTaskId(), TEST_TASK_ID);
+    }
+
+    @Test
+    public void testPollyTaskCompleted() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);
+        GetSpeechSynthesisTaskResult taskResult = new GetSpeechSynthesisTaskResult();
+        SynthesisTask task = new SynthesisTask().withTaskId(TEST_TASK_ID)
+                .withTaskStatus(TaskStatus.Completed)
+                .withOutputUri("outputLocationPath");
+        taskResult.setSynthesisTask(task);
+        when(mockPollyClient.getSpeechSynthesisTask(requestCaptor.capture())).thenReturn(taskResult);
+        runner.enqueue("content", Collections.singletonMap(AWS_TASK_ID_PROPERTY, TEST_TASK_ID));
+        runner.run();
+
+        runner.assertTransferCount(REL_SUCCESS, 1);
+        runner.assertTransferCount(REL_ORIGINAL, 1);
+        runner.assertAllFlowFilesContainAttribute(REL_SUCCESS, AWS_TASK_OUTPUT_LOCATION);
+        assertEquals(requestCaptor.getValue().getTaskId(), TEST_TASK_ID);
+    }
+
+
+    @Test
+    public void testPollyTaskFailed() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);
+        GetSpeechSynthesisTaskResult taskResult = new GetSpeechSynthesisTaskResult();
+        SynthesisTask task = new SynthesisTask().withTaskId(TEST_TASK_ID)
+                .withTaskStatus(TaskStatus.Failed)
+                .withTaskStatusReason("reasonOfFailure");
+        taskResult.setSynthesisTask(task);
+        when(mockPollyClient.getSpeechSynthesisTask(requestCaptor.capture())).thenReturn(taskResult);
+        runner.enqueue("content", Collections.singletonMap(AWS_TASK_ID_PROPERTY, TEST_TASK_ID));

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Transcribe"})
+@CapabilityDescription("Retrieves the current status of an AWS Transcribe job.")
+@SeeAlso({StartAwsTranscribeJob.class})
+public class GetAwsTranscribeJobStatus extends AwsMachineLearningJobStatusGetter<AmazonTranscribeClient> {
+    @Override
+    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetTranscriptionJobResult job = getJob(flowFile);
+        TranscriptionJobStatus jobStatus = TranscriptionJobStatus.fromValue(job.getTranscriptionJob().getTranscriptionJobStatus());
+
+        if (TranscriptionJobStatus.COMPLETED == jobStatus) {
+            writeToFlowFile(session, flowFile, job);
+            session.putAttribute(flowFile, AWS_TASK_OUTPUT_LOCATION, job.getTranscriptionJob().getTranscript().getTranscriptFileUri());
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+
+        if (TranscriptionJobStatus.IN_PROGRESS == jobStatus) {

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);
+
+    protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        FlowFile childFlowFile = session.create(flowFile);
+        session.write(childFlowFile, out -> {

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusGetter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusGetter<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (taskStatus == TaskStatus.Completed) {
+            String outputUri = speechSynthesisTask.getSynthesisTask().getOutputUri();
+
+            Matcher matcher = S3_PATH.matcher(outputUri);
+            if (matcher.find()) {
+                session.putAttribute(flowFile, AWS_S3_BUCKET, matcher.group(BUCKET));
+                session.putAttribute(flowFile, AWS_S3_KEY, matcher.group(KEY));
+            }
+            FlowFile childFlowFile = session.create(flowFile);
+            writeToFlowFile(session, childFlowFile, speechSynthesisTask);
+            session.putAttribute(childFlowFile, AWS_TASK_OUTPUT_LOCATION, outputUri);

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (taskStatus == TaskStatus.Completed) {

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (taskStatus == TaskStatus.Completed) {
+            String outputUri = speechSynthesisTask.getSynthesisTask().getOutputUri();
+
+            Matcher matcher = S3_PATH.matcher(outputUri);
+            if (matcher.find()) {
+                session.putAttribute(flowFile, AWS_S3_BUCKET, matcher.group(BUCKET));
+                session.putAttribute(flowFile, AWS_S3_KEY, matcher.group(KEY));
+            }
+            FlowFile childFlowFile = session.create(flowFile);
+            writeToFlowFile(session, childFlowFile, speechSynthesisTask);
+            session.putAttribute(childFlowFile, AWS_TASK_OUTPUT_LOCATION, outputUri);
+            session.transfer(flowFile, REL_ORIGINAL);
+            session.transfer(childFlowFile, REL_SUCCESS);
+            getLogger().info("Amazon Polly reported that the task completed for {}", flowFile);
+            return;
+        }
+
+        if (taskStatus == TaskStatus.Failed) {
+            final String failureReason =  speechSynthesisTask.getSynthesisTask().getTaskStatusReason();
+            session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return new ImmutableList.Builder().add(TYPE).add(super.getSupportedPropertyDescriptors().toArray()).build();
+    }
+
+    @Override
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, AmazonWebServiceResult response) {
+        super.postProcessFlowFile(context, session, flowFile, response);
+        session.putAttribute(flowFile, TYPE.getName(), context.getProperty(TYPE.getName()).getValue());
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    protected AmazonWebServiceResult sendRequest(AmazonWebServiceRequest request, ProcessContext context) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();
+        AmazonWebServiceResult result;
+        switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS :
+                result = getClient().startDocumentAnalysis((StartDocumentAnalysisRequest) request);
+                break;
+            case DOCUMENT_TEXT_DETECTION:
+                result = getClient().startDocumentTextDetection((StartDocumentTextDetectionRequest) request);
+                break;
+            case EXPENSE_ANALYSIS :
+                result = getClient().startExpenseAnalysis((StartExpenseAnalysisRequest) request);
+                break;
+            default: throw new UnsupportedOperationException("Unsupported textract type.");
+        }
+        return result;
+    }
+
+    @Override
+    protected Class<? extends AmazonWebServiceRequest> getAwsRequestClass(ProcessContext context) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();
+        Class<? extends AmazonWebServiceRequest>  result;
+        switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS:
+                result = StartDocumentAnalysisRequest.class;
+                break;
+            case DOCUMENT_TEXT_DETECTION:
+                result = StartDocumentTextDetectionRequest.class;
+                break;
+            case EXPENSE_ANALYSIS:
+                result = StartExpenseAnalysisRequest.class;
+                break;
+            default: throw new UnsupportedOperationException("Unsupported textract type.");
+        }
+        return result;
+    }
+
+    @Override
+    protected String getAwsTaskId(ProcessContext context, AmazonWebServiceResult amazonWebServiceResult) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();
+        String  result;
+        switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS:
+                result = ((StartDocumentAnalysisResult) amazonWebServiceResult).getJobId();
+                break;
+            case DOCUMENT_TEXT_DETECTION:
+                result = ((StartDocumentTextDetectionResult) amazonWebServiceResult).getJobId();
+                break;
+            case EXPENSE_ANALYSIS:
+                result = ((StartExpenseAnalysisResult) amazonWebServiceResult).getJobId();
+                break;
+            default: throw new UnsupportedOperationException("Unsupported textract type.");

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatus.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.aws.ml.translate;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Translate"})
+@CapabilityDescription("Retrieves the current status of an AWS Translate job.")
+@SeeAlso({StartAwsTranslateJob.class})
+public class GetAwsTranslateJobStatus extends AwsMachineLearningJobStatusGetter<AmazonTranslateClient> {
+    @Override
+    protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranslateClient) AmazonTranslateClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);
+        DescribeTextTranslationJobResult describeTextTranslationJobResult = getStatusString(awsTaskId);
+        JobStatus status = JobStatus.fromValue(describeTextTranslationJobResult.getTextTranslationJobProperties().getJobStatus());
+
+        if (status == JobStatus.IN_PROGRESS || status == JobStatus.SUBMITTED) {
+            writeToFlowFile(session, flowFile, describeTextTranslationJobResult);
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (status == JobStatus.COMPLETED) {

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;
+
+    @BeforeEach
+    public void setUp() throws InitializationException {
+        mockPollyClient = Mockito.mock(AmazonPollyClient.class);
+        mockAwsCredentialsProvider = Mockito.mock(AWSCredentialsProviderService.class);
+        when(mockAwsCredentialsProvider.getIdentifier()).thenReturn("awsCredetialProvider");
+        final GetAwsPollyJobStatus mockGetAwsPollyStatus = new GetAwsPollyJobStatus() {
+            protected AmazonPollyClient getClient() {
+                return mockPollyClient;
+            }
+
+            @Override
+            protected AmazonPollyClient createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+                return mockPollyClient;
+            }
+        };
+        runner = TestRunners.newTestRunner(mockGetAwsPollyStatus);
+        runner.addControllerService("awsCredetialProvider", mockAwsCredentialsProvider);
+        runner.enableControllerService(mockAwsCredentialsProvider);
+        runner.setProperty(AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredetialProvider");
+    }
+
+    @Test
+    public void testPollyTaskInProgress() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);

Review Comment:
   Fixed, thx.



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1044467172


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = "PollyS3OutputBucket", description = "The bucket name where polly output will be located."),
+        @WritesAttribute(attribute = "PollyS3OutputKey", description = "Object key of polly output."),
+        @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
+})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";

Review Comment:
   That's a very good point @KalmanJantner. In that case, keeping the attributes sounds good. It might make sense to use the `filename` attribute for the key portion in that case, since that is the default input value for `FetchS3Object`.



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


[GitHub] [nifi] markap14 commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
markap14 commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1007004979


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLFetcherProcessor.java:
##########
@@ -0,0 +1,117 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLFetcherProcessor<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public final Relationship REL_PARTIAL_SUCCESS = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();

Review Comment:
   I'm not sure that "Partial Success" makes sense here. If something was partially successful it wouldn't really make sense to retry. Perhaps "throttled" would make sense for the Relationship name.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLFetcherProcessor.java:
##########
@@ -0,0 +1,117 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLFetcherProcessor<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public final Relationship REL_PARTIAL_SUCCESS = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule module = new SimpleModule();
+        module.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule module2 = new SimpleModule();
+        module.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(module);
+        mapper.registerModule(module2);
+    }
+
+
+    protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {
+        session.write(flowFile, out -> {
+            try (BufferedWriter bufferedWriter = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
+                bufferedWriter.write(mapper.writeValueAsString(response));
+                bufferedWriter.newLine();

Review Comment:
   Why add the newline?



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLFetcherProcessor.java:
##########
@@ -0,0 +1,117 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLFetcherProcessor<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public final Relationship REL_PARTIAL_SUCCESS = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule module = new SimpleModule();
+        module.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule module2 = new SimpleModule();

Review Comment:
   We should avoid naming conventions such as `module` and `module2` - perhaps this should be `awsResponseModule` and `sdkHttpModule` or something of that nature?



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyFetcher.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning. Checking Polly synthesis job's status.")

Review Comment:
   This should be more descriptive of what the processor is actually doing. "Retrieves the current status of an Amazon Polly job" perhaps?



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);
+            return;
+        }
+
+        try {
+            writeToFlowFile(session, flowFile, response);

Review Comment:
   We should not overwrite the contents of the FlowFile with the response. There will be use cases where we want the original content of the FlowFile. Instead, we should create a new child FlowFile and write the response to that. Then route the original one to an 'original' relationship. This way, the flow developer can access both the original content and the response.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyFetcher.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning. Checking Polly synthesis job's status.")
+public class PollyFetcher extends AwsMLFetcherProcessor<AmazonPollyClient> {
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard().withCredentials(credentialsProvider).build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask = getSynthesisTask(flowFile);

Review Comment:
   Given the above comments about the "Partially Successful" relationship, I would expect to have a relationship for exceeding request limits. We should also catch any other general exception that could be thrown from the client and route to 'failure'. 



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyProcessor.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.ml.AwsMlProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning.")
+@SeeAlso({PollyFetcher.class})

Review Comment:
   The fetcher should also have a `@SeeAlso` that mentions this processor as well.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyProcessor.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.ml.AwsMlProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning.")
+@SeeAlso({PollyFetcher.class})
+public class PollyProcessor extends AwsMlProcessor<AmazonPollyClient, StartSpeechSynthesisTaskRequest, StartSpeechSynthesisTaskResult> {

Review Comment:
   We never use the word 'Processor' in a processor's name. We need to follow the convention of <Verb> <Noun>. So probably should use `StartAwsPollyJob` or `StartPollySpeechSynthesis` or something of that nature.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")

Review Comment:
   Same comments as above on CapabilityDescription



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")
+@SeeAlso({TextractProcessor.class})
+public class TextractFetcher extends AwsMLFetcherProcessor<AmazonTextractClient> {

Review Comment:
   Same comments above regarding processor name



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);
+            return;
+        }
+
+        try {
+            writeToFlowFile(session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during writing aws response to flow file.", e);
+            return;
+        }
+
+        try {
+            postProcessFlowFile(context, session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during AWS ML post processing.", e);
+            return;
+        }

Review Comment:
   We can avoid separating into these different blocks. We will handle the processing in the same way, regardless of of where the Exception was thrown. The only difference here is the Exception text. But the differing text of the log message is not really meaningful to the end user, and for developers diagnosing the problem, this is clear via the stack trace.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);

Review Comment:
   We should verbiage such as "Exception was thrown" and instead indicate "Failed to send request to AWS Service"



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyFetcher.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning. Checking Polly synthesis job's status.")
+public class PollyFetcher extends AwsMLFetcherProcessor<AmazonPollyClient> {

Review Comment:
   Processor names should always be in the form of <Verb> <Noun>. So Perhaps `GetAwsPollyStatus`?



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyProcessor.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.ml.AwsMlProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning.")

Review Comment:
   The Capability Description should describe what this processor does, rather than describing what Amazon Polly does. This processor is not responsible for turning text into lifelike speech. Rather, it's responsible for starting an Amazon Polly job. Would probably make sense to also mention that this Processor starts the job, and it's expected to be followed up by the `GetPollyStatus` or whatever we call it, in order to make it clear how this should be used.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")
+@SeeAlso({TextractProcessor.class})
+public class TextractFetcher extends AwsMLFetcherProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    private static final Map<String, AwsTextractTaskAware> UTIL_MAPPING =
+            ImmutableMap.of(DOCUMENT_ANALYSIS, new TextractDocumentAnalysisResultUtil(),
+                    DOCUMENT_TEXT_DETECTION, new TextractDocumentTextDetectionUtil(),
+                    EXPENSE_ANALYSIS, new TextractDocumentExpenseAnalysisUtil());
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder().build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String typeOfTextract = flowFile.getAttribute(TYPE.getName());

Review Comment:
   This should not be an attribute. It should be a property of the processor. This property should support expression language. So if a user wants to use an attribute, they are welcome to. But there's no need to require a specific attribute name for this.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")
+@SeeAlso({TextractProcessor.class})
+public class TextractFetcher extends AwsMLFetcherProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    private static final Map<String, AwsTextractTaskAware> UTIL_MAPPING =
+            ImmutableMap.of(DOCUMENT_ANALYSIS, new TextractDocumentAnalysisResultUtil(),
+                    DOCUMENT_TEXT_DETECTION, new TextractDocumentTextDetectionUtil(),
+                    EXPENSE_ANALYSIS, new TextractDocumentExpenseAnalysisUtil());
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder().build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String typeOfTextract = flowFile.getAttribute(TYPE.getName());
+        AwsTextractTaskAware util = UTIL_MAPPING.get(typeOfTextract);

Review Comment:
   I think this approach of having an interface with 3 different implementations, and then using a mapping is overkill. It adds significant complexity when we could just do something here as simple as:
   ```
   final JobStatus jobStatus;
   switch (typeOfTextract) {
       case DOCUMENT_ANALYSIS:
           jobStatus = JobStatus.fromValue(client.getDocumentAnalysis(new GetDocumentAnalysisRequest().withJobId(awstaskId)).getJobStatus());
          break;
   }
   ```
   And the like. 



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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1011237210


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-textract</artifactId>
+            <version>1.12.328</version>
+        </dependency>

Review Comment:
   Good point thank you. I checked it and the problem is around the super classes. I should refactor `AWSCredentialsProvider` related classes and we need to introduce version 2 compatible version of `AbstractAWSProcessor`, `AbstractAWSCredentialsProviderProcessor`, `AWSCredentialsProviderControllerService` etc. Should I go ahead and do this refactor as part of this PR?



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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1014431551


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);

Review Comment:
   Thanks I've updated.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);
+            return;
+        }
+
+        try {
+            writeToFlowFile(session, flowFile, response);

Review Comment:
   Thanks, now we will keep the original flow file.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyFetcher.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning. Checking Polly synthesis job's status.")
+public class PollyFetcher extends AwsMLFetcherProcessor<AmazonPollyClient> {

Review Comment:
   Based on your suggestion I have renamed them.
   Now we have `AwsMLJobStatusGetter`, `AwsMLJobStatusGetter` and each service have  a `StartAws<serviceName>Job` and a `GetAws<serviceName>JobStatus`



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));

Review Comment:
   Thank you, I have removed them.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyProcessor.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.ml.AwsMlProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning.")
+@SeeAlso({PollyFetcher.class})

Review Comment:
   Sure, fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);
+            return;
+        }
+
+        try {
+            writeToFlowFile(session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during writing aws response to flow file.", e);
+            return;
+        }
+
+        try {
+            postProcessFlowFile(context, session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during AWS ML post processing.", e);
+            return;
+        }

Review Comment:
   Thanks, I have refactored this part.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")
+@SeeAlso({TextractProcessor.class})
+public class TextractFetcher extends AwsMLFetcherProcessor<AmazonTextractClient> {

Review Comment:
   Fixed, thx



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLFetcherProcessor.java:
##########
@@ -0,0 +1,117 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLFetcherProcessor<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public final Relationship REL_PARTIAL_SUCCESS = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule module = new SimpleModule();
+        module.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule module2 = new SimpleModule();

Review Comment:
   Thanks, I fixed that.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLFetcherProcessor.java:
##########
@@ -0,0 +1,117 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLFetcherProcessor<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public final Relationship REL_PARTIAL_SUCCESS = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule module = new SimpleModule();
+        module.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule module2 = new SimpleModule();
+        module.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(module);
+        mapper.registerModule(module2);
+    }
+
+
+    protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {
+        session.write(flowFile, out -> {
+            try (BufferedWriter bufferedWriter = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
+                bufferedWriter.write(mapper.writeValueAsString(response));
+                bufferedWriter.newLine();

Review Comment:
   Not needed, I removed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")
+@SeeAlso({TextractProcessor.class})
+public class TextractFetcher extends AwsMLFetcherProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    private static final Map<String, AwsTextractTaskAware> UTIL_MAPPING =
+            ImmutableMap.of(DOCUMENT_ANALYSIS, new TextractDocumentAnalysisResultUtil(),
+                    DOCUMENT_TEXT_DETECTION, new TextractDocumentTextDetectionUtil(),
+                    EXPENSE_ANALYSIS, new TextractDocumentExpenseAnalysisUtil());
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder().build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String typeOfTextract = flowFile.getAttribute(TYPE.getName());
+        AwsTextractTaskAware util = UTIL_MAPPING.get(typeOfTextract);

Review Comment:
   I have removed the mapping, thx



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyProcessor.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.ml.AwsMlProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning.")
+@SeeAlso({PollyFetcher.class})
+public class PollyProcessor extends AwsMlProcessor<AmazonPollyClient, StartSpeechSynthesisTaskRequest, StartSpeechSynthesisTaskResult> {

Review Comment:
   Fixed, thank you.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLFetcherProcessor.java:
##########
@@ -0,0 +1,117 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLFetcherProcessor<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public final Relationship REL_PARTIAL_SUCCESS = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();

Review Comment:
   Relation has been renamed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);

Review Comment:
   Thanks, it has been updated based on your suggestions.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyProcessor.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.StartSpeechSynthesisTaskResult;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.ml.AwsMlProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning.")

Review Comment:
   Thanks, I've updated them.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyFetcher.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning. Checking Polly synthesis job's status.")

Review Comment:
   I've updated the descriptions. thx



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlProcessor.java:
##########
@@ -0,0 +1,148 @@
+package org.apache.nifi.processors.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.commons.io.IOUtils;
+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.exception.ProcessException;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlProcessor<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult> extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during sending AWS ML request.", e);
+            return;
+        }
+
+        try {
+            writeToFlowFile(session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during writing aws response to flow file.", e);
+            return;
+        }
+
+        try {
+            postProcessFlowFile(context, session, flowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Exception was thrown during AWS ML post processing.", e);
+            return;
+        }
+
+        session.transfer(flowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        REQUEST request;
+        try {
+            request = mapper.readValue(getPayload(session, context, flowFile),
+                    getAwsRequestClass(context));
+        } catch (JsonProcessingException e) {
+            getLogger().error("Exception was thrown during AWS ML request creation.", e);
+            throw e;

Review Comment:
   Thank you, I've updated it.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/PollyFetcher.java:
##########
@@ -0,0 +1,63 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Turn text into lifelike speech using deep learning. Checking Polly synthesis job's status.")
+public class PollyFetcher extends AwsMLFetcherProcessor<AmazonPollyClient> {
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard().withCredentials(credentialsProvider).build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask = getSynthesisTask(flowFile);

Review Comment:
   According to the aws doc `ThrottlingException` will be thrown when we exceed the rate limit. So I added a separated catch for that and kept the existing general purpose catch.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")
+@SeeAlso({TextractProcessor.class})
+public class TextractFetcher extends AwsMLFetcherProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    private static final Map<String, AwsTextractTaskAware> UTIL_MAPPING =
+            ImmutableMap.of(DOCUMENT_ANALYSIS, new TextractDocumentAnalysisResultUtil(),
+                    DOCUMENT_TEXT_DETECTION, new TextractDocumentTextDetectionUtil(),
+                    EXPENSE_ANALYSIS, new TextractDocumentExpenseAnalysisUtil());
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder().build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String typeOfTextract = flowFile.getAttribute(TYPE.getName());

Review Comment:
   Thanks, fixed



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcher.java:
##########
@@ -0,0 +1,68 @@
+package org.apache.nifi.processors.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.ml.textract.TextractProcessor.TYPE;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLFetcherProcessor;
+import org.apache.nifi.processors.aws.ml.polly.PollyFetcher;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Automatically extract printed text, handwriting, and data from any document")

Review Comment:
   Yep, fixed thank you.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsResponseMetadataDeserializer.java:
##########
@@ -0,0 +1,19 @@
+package org.apache.nifi.processors.aws.ml;

Review Comment:
   I have fixed these, thank you.



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1030613669


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;

Review Comment:
   Recommend adding `final` keyword:
   ```suggestion
           final RESPONSE response;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);
+
+    protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        FlowFile childFlowFile = session.create(flowFile);
+        session.write(childFlowFile, out -> {
+            try (OutputStreamWriter outputStreamWriter = new OutputStreamWriter(out, StandardCharsets.UTF_8)) {
+                outputStreamWriter.write(mapper.writeValueAsString(response));
+                outputStreamWriter.flush();
+            }

Review Comment:
   The ObjectMapper.writeValue() method can be used:
   ```suggestion
               mapper.writeValue(out, response));
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";

Review Comment:
   These values should be defined using the `WritesAttributes` documentation annotation on the Processor.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (taskStatus == TaskStatus.Completed) {
+            String outputUri = speechSynthesisTask.getSynthesisTask().getOutputUri();
+
+            Matcher matcher = S3_PATH.matcher(outputUri);
+            if (matcher.find()) {
+                session.putAttribute(flowFile, AWS_S3_BUCKET, matcher.group(BUCKET));
+                session.putAttribute(flowFile, AWS_S3_KEY, matcher.group(KEY));
+            }
+            FlowFile childFlowFile = session.create(flowFile);
+            writeToFlowFile(session, childFlowFile, speechSynthesisTask);
+            session.putAttribute(childFlowFile, AWS_TASK_OUTPUT_LOCATION, outputUri);
+            session.transfer(flowFile, REL_ORIGINAL);
+            session.transfer(childFlowFile, REL_SUCCESS);
+            getLogger().info("Amazon Polly reported that the task completed for {}", flowFile);
+            return;
+        }
+
+        if (taskStatus == TaskStatus.Failed) {
+            final String failureReason =  speechSynthesisTask.getSynthesisTask().getTaskStatusReason();
+            session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);

Review Comment:
   ```suggestion
               flowFile = session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {

Review Comment:
   ```suggestion
       public void onTrigger(final ProcessContext context, final ProcessSession session) {
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;
+
+    @BeforeEach
+    public void setUp() throws InitializationException {
+        mockPollyClient = Mockito.mock(AmazonPollyClient.class);
+        mockAwsCredentialsProvider = Mockito.mock(AWSCredentialsProviderService.class);
+        when(mockAwsCredentialsProvider.getIdentifier()).thenReturn("awsCredetialProvider");
+        final GetAwsPollyJobStatus mockGetAwsPollyStatus = new GetAwsPollyJobStatus() {
+            protected AmazonPollyClient getClient() {
+                return mockPollyClient;
+            }
+
+            @Override
+            protected AmazonPollyClient createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+                return mockPollyClient;
+            }
+        };
+        runner = TestRunners.newTestRunner(mockGetAwsPollyStatus);
+        runner.addControllerService("awsCredetialProvider", mockAwsCredentialsProvider);
+        runner.enableControllerService(mockAwsCredentialsProvider);
+        runner.setProperty(AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredetialProvider");
+    }
+
+    @Test
+    public void testPollyTaskInProgress() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);

Review Comment:
   The `ArgumentCaptor` can be defined with an `@Captor` annotation at the class level.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");

Review Comment:
   Recommend adjusting the message:
   ```suggestion
           throw new UnsupportedOperationException("createClient(ProcessContext, AWSCredentials, ClientConfiguration) is not supported");
   ```



##########
pom.xml:
##########
@@ -147,6 +147,7 @@
         <swagger.annotations.version>1.6.6</swagger.annotations.version>
         <h2.version>2.1.214</h2.version>
         <zookeeper.version>3.8.0</zookeeper.version>
+        <aws.ml.version>1.12.328</aws.ml.version>

Review Comment:
   This property is not necessary since the BOM dependency sets the version.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   This version number can be removed since it is already managed by the `aws-java-sdk-bom` in the root Maven configuration.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>${aws.ml.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-textract</artifactId>
+            <version>${aws.ml.version}</version>

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusGetter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusGetter<T extends AmazonWebServiceClient>

Review Comment:
   ```suggestion
   public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")

Review Comment:
   Recommend indicating that when this property is not provided, the JSON request will be read from the FlowFile. The AWS API reference is not very helpful without a link, and the details are covered in additionalDetails, so recommend removing that sentence.
   ```suggestion
               .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")

Review Comment:
   Recommend simplifying the property name:
   ```suggestion
               .name("aws-region")
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusGetter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusGetter<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_RUNNING,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Client creation not supported");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule awsResponseModule = new SimpleModule();
+        awsResponseModule.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule sdkHttpModule = new SimpleModule();
+        awsResponseModule.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(awsResponseModule);
+        mapper.registerModule(sdkHttpModule);
+    }
+
+
+    protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {
+        session.write(flowFile, out -> {
+            try (OutputStreamWriter outputStreamWriter = new OutputStreamWriter(out, StandardCharsets.UTF_8)) {
+                outputStreamWriter.write(mapper.writeValueAsString(response));
+                outputStreamWriter.flush();
+            }

Review Comment:
   ```suggestion
               mapper.writeValue(out, response);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);
+
+    protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        FlowFile childFlowFile = session.create(flowFile);
+        session.write(childFlowFile, out -> {

Review Comment:
   ```suggestion
           childFlowFile = session.write(childFlowFile, out -> {
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);

Review Comment:
   ```suggestion
               getLogger().warn("Failed to get Polly Job status", e);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);

Review Comment:
   Recommend moving these abstract methods to the end of the class.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusGetter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusGetter<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-ml-region")

Review Comment:
   ```suggestion
               .name("aws-region")
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (taskStatus == TaskStatus.Completed) {
+            String outputUri = speechSynthesisTask.getSynthesisTask().getOutputUri();
+
+            Matcher matcher = S3_PATH.matcher(outputUri);
+            if (matcher.find()) {
+                session.putAttribute(flowFile, AWS_S3_BUCKET, matcher.group(BUCKET));
+                session.putAttribute(flowFile, AWS_S3_KEY, matcher.group(KEY));
+            }
+            FlowFile childFlowFile = session.create(flowFile);
+            writeToFlowFile(session, childFlowFile, speechSynthesisTask);
+            session.putAttribute(childFlowFile, AWS_TASK_OUTPUT_LOCATION, outputUri);

Review Comment:
   ```suggestion
               childFlowFile = session.putAttribute(childFlowFile, AWS_TASK_OUTPUT_LOCATION, outputUri);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard()
+                .withCredentials(credentialsProvider)
+                .withRegion(context.getProperty(REGION).getValue())
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Request Rate Limit exceeded", e);
+            session.transfer(flowFile, REL_THROTTLED);
+            return;
+        } catch (Exception e) {
+            getLogger().info("Failed to get Polly Job status", e);
+            session.transfer(flowFile, REL_FAILURE);
+            return;
+        }
+
+        TaskStatus taskStatus = TaskStatus.fromValue(speechSynthesisTask.getSynthesisTask().getTaskStatus());
+
+        if (taskStatus == TaskStatus.InProgress || taskStatus == TaskStatus.Scheduled) {
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (taskStatus == TaskStatus.Completed) {

Review Comment:
   It looks like this should be defined as an `else if` with with previous conditional.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusGetter<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES = ImmutableList.<PropertyDescriptor>builder()
+            .addAll(PROPERTIES)
+            .add(TYPE)
+            .build();;
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String typeOfTextract = context.getProperty(TYPE).evaluateAttributeExpressions().getValue();
+
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);
+        JobStatus jobStatus = getTaskStatus(typeOfTextract, getClient(), awsTaskId);
+        if (JobStatus.SUCCEEDED == jobStatus) {
+            Object task = getTask(typeOfTextract, getClient(), awsTaskId);
+            writeToFlowFile(session, flowFile, task);
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+
+        if (JobStatus.IN_PROGRESS == jobStatus) {
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (JobStatus.PARTIAL_SUCCESS == jobStatus) {
+            session.transfer(flowFile, REL_THROTTLED);
+        }
+
+        if (JobStatus.FAILED == jobStatus) {

Review Comment:
   It looks like these conditionals shoudl be chained together with `else if`



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableList;

Review Comment:
   Google Guava classes should not be used, this can be replaced with `Collections.unmodifiableList()`



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import com.google.common.collect.ImmutableList;

Review Comment:
   Should be replaced with `Collections.unmodifiableList()`



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return new ImmutableList.Builder().add(TYPE).add(super.getSupportedPropertyDescriptors().toArray()).build();
+    }
+
+    @Override
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, AmazonWebServiceResult response) {
+        super.postProcessFlowFile(context, session, flowFile, response);
+        session.putAttribute(flowFile, TYPE.getName(), context.getProperty(TYPE.getName()).getValue());
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    protected AmazonWebServiceResult sendRequest(AmazonWebServiceRequest request, ProcessContext context) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();
+        AmazonWebServiceResult result;
+        switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS :
+                result = getClient().startDocumentAnalysis((StartDocumentAnalysisRequest) request);
+                break;
+            case DOCUMENT_TEXT_DETECTION:
+                result = getClient().startDocumentTextDetection((StartDocumentTextDetectionRequest) request);
+                break;
+            case EXPENSE_ANALYSIS :
+                result = getClient().startExpenseAnalysis((StartExpenseAnalysisRequest) request);
+                break;
+            default: throw new UnsupportedOperationException("Unsupported textract type.");
+        }
+        return result;
+    }
+
+    @Override
+    protected Class<? extends AmazonWebServiceRequest> getAwsRequestClass(ProcessContext context) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();
+        Class<? extends AmazonWebServiceRequest>  result;
+        switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS:
+                result = StartDocumentAnalysisRequest.class;
+                break;
+            case DOCUMENT_TEXT_DETECTION:
+                result = StartDocumentTextDetectionRequest.class;
+                break;
+            case EXPENSE_ANALYSIS:
+                result = StartExpenseAnalysisRequest.class;
+                break;
+            default: throw new UnsupportedOperationException("Unsupported textract type.");
+        }
+        return result;
+    }
+
+    @Override
+    protected String getAwsTaskId(ProcessContext context, AmazonWebServiceResult amazonWebServiceResult) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();
+        String  result;
+        switch (typeOfTextract) {
+            case DOCUMENT_ANALYSIS:
+                result = ((StartDocumentAnalysisResult) amazonWebServiceResult).getJobId();
+                break;
+            case DOCUMENT_TEXT_DETECTION:
+                result = ((StartDocumentTextDetectionResult) amazonWebServiceResult).getJobId();
+                break;
+            case EXPENSE_ANALYSIS:
+                result = ((StartExpenseAnalysisResult) amazonWebServiceResult).getJobId();
+                break;
+            default: throw new UnsupportedOperationException("Unsupported textract type.");

Review Comment:
   The `typeOfTextract` string should be included:
   ```suggestion
               default: throw new UnsupportedOperationException("Unsupported textract type: " + typeOfTextract);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;
+
+    @BeforeEach
+    public void setUp() throws InitializationException {
+        mockPollyClient = Mockito.mock(AmazonPollyClient.class);
+        mockAwsCredentialsProvider = Mockito.mock(AWSCredentialsProviderService.class);

Review Comment:
   These mocks can be configured using the `@Mock` annotation on the property



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatus.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.aws.ml.translate;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Translate"})
+@CapabilityDescription("Retrieves the current status of an AWS Translate job.")
+@SeeAlso({StartAwsTranslateJob.class})
+public class GetAwsTranslateJobStatus extends AwsMachineLearningJobStatusGetter<AmazonTranslateClient> {
+    @Override
+    protected AmazonTranslateClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranslateClient) AmazonTranslateClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);
+        DescribeTextTranslationJobResult describeTextTranslationJobResult = getStatusString(awsTaskId);
+        JobStatus status = JobStatus.fromValue(describeTextTranslationJobResult.getTextTranslationJobProperties().getJobStatus());
+
+        if (status == JobStatus.IN_PROGRESS || status == JobStatus.SUBMITTED) {
+            writeToFlowFile(session, flowFile, describeTextTranslationJobResult);
+            session.penalize(flowFile);
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (status == JobStatus.COMPLETED) {

Review Comment:
   ```suggestion
           } else if (status == JobStatus.COMPLETED) {
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;

Review Comment:
   ```suggestion
       @Mock
       private AmazonPollyClient mockPollyClient;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Transcribe"})
+@CapabilityDescription("Retrieves the current status of an AWS Transcribe job.")
+@SeeAlso({StartAwsTranscribeJob.class})
+public class GetAwsTranscribeJobStatus extends AwsMachineLearningJobStatusGetter<AmazonTranscribeClient> {
+    @Override
+    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetTranscriptionJobResult job = getJob(flowFile);
+        TranscriptionJobStatus jobStatus = TranscriptionJobStatus.fromValue(job.getTranscriptionJob().getTranscriptionJobStatus());
+
+        if (TranscriptionJobStatus.COMPLETED == jobStatus) {
+            writeToFlowFile(session, flowFile, job);
+            session.putAttribute(flowFile, AWS_TASK_OUTPUT_LOCATION, job.getTranscriptionJob().getTranscript().getTranscriptFileUri());
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+
+        if (TranscriptionJobStatus.IN_PROGRESS == jobStatus) {
+            session.transfer(flowFile, REL_RUNNING);
+        }
+
+        if (TranscriptionJobStatus.FAILED == jobStatus) {
+            final String failureReason = job.getTranscriptionJob().getFailureReason();
+            session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Transcribe Task Failed for {}: {}", flowFile, failureReason);
+            return;

Review Comment:
   This can be removed
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+

Review Comment:
   Using the Mockito Extension enables annotated mocks:
   ```suggestion
   @ExtendWith(MockitoExtension.class)
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Transcribe"})
+@CapabilityDescription("Retrieves the current status of an AWS Transcribe job.")
+@SeeAlso({StartAwsTranscribeJob.class})
+public class GetAwsTranscribeJobStatus extends AwsMachineLearningJobStatusGetter<AmazonTranscribeClient> {
+    @Override
+    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranscribeClient) AmazonTranscribeClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetTranscriptionJobResult job = getJob(flowFile);
+        TranscriptionJobStatus jobStatus = TranscriptionJobStatus.fromValue(job.getTranscriptionJob().getTranscriptionJobStatus());
+
+        if (TranscriptionJobStatus.COMPLETED == jobStatus) {
+            writeToFlowFile(session, flowFile, job);
+            session.putAttribute(flowFile, AWS_TASK_OUTPUT_LOCATION, job.getTranscriptionJob().getTranscript().getTranscriptFileUri());
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+
+        if (TranscriptionJobStatus.IN_PROGRESS == jobStatus) {

Review Comment:
   ```suggestion
           } else if (TranscriptionJobStatus.IN_PROGRESS == jobStatus) {
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;
+
+    @BeforeEach
+    public void setUp() throws InitializationException {
+        mockPollyClient = Mockito.mock(AmazonPollyClient.class);
+        mockAwsCredentialsProvider = Mockito.mock(AWSCredentialsProviderService.class);
+        when(mockAwsCredentialsProvider.getIdentifier()).thenReturn("awsCredetialProvider");
+        final GetAwsPollyJobStatus mockGetAwsPollyStatus = new GetAwsPollyJobStatus() {
+            protected AmazonPollyClient getClient() {
+                return mockPollyClient;
+            }
+
+            @Override
+            protected AmazonPollyClient createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+                return mockPollyClient;
+            }
+        };
+        runner = TestRunners.newTestRunner(mockGetAwsPollyStatus);
+        runner.addControllerService("awsCredetialProvider", mockAwsCredentialsProvider);
+        runner.enableControllerService(mockAwsCredentialsProvider);
+        runner.setProperty(AWS_CREDENTIALS_PROVIDER_SERVICE, "awsCredetialProvider");
+    }
+
+    @Test
+    public void testPollyTaskInProgress() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);
+        GetSpeechSynthesisTaskResult taskResult = new GetSpeechSynthesisTaskResult();
+        SynthesisTask task = new SynthesisTask().withTaskId(TEST_TASK_ID)
+                .withTaskStatus(TaskStatus.InProgress);
+        taskResult.setSynthesisTask(task);
+        when(mockPollyClient.getSpeechSynthesisTask(requestCaptor.capture())).thenReturn(taskResult);
+        runner.enqueue("content", Collections.singletonMap(AWS_TASK_ID_PROPERTY, TEST_TASK_ID));
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(REL_RUNNING);
+        assertEquals(requestCaptor.getValue().getTaskId(), TEST_TASK_ID);
+    }
+
+    @Test
+    public void testPollyTaskCompleted() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);
+        GetSpeechSynthesisTaskResult taskResult = new GetSpeechSynthesisTaskResult();
+        SynthesisTask task = new SynthesisTask().withTaskId(TEST_TASK_ID)
+                .withTaskStatus(TaskStatus.Completed)
+                .withOutputUri("outputLocationPath");
+        taskResult.setSynthesisTask(task);
+        when(mockPollyClient.getSpeechSynthesisTask(requestCaptor.capture())).thenReturn(taskResult);
+        runner.enqueue("content", Collections.singletonMap(AWS_TASK_ID_PROPERTY, TEST_TASK_ID));
+        runner.run();
+
+        runner.assertTransferCount(REL_SUCCESS, 1);
+        runner.assertTransferCount(REL_ORIGINAL, 1);
+        runner.assertAllFlowFilesContainAttribute(REL_SUCCESS, AWS_TASK_OUTPUT_LOCATION);
+        assertEquals(requestCaptor.getValue().getTaskId(), TEST_TASK_ID);
+    }
+
+
+    @Test
+    public void testPollyTaskFailed() {
+        ArgumentCaptor<GetSpeechSynthesisTaskRequest> requestCaptor = ArgumentCaptor.forClass(GetSpeechSynthesisTaskRequest.class);
+        GetSpeechSynthesisTaskResult taskResult = new GetSpeechSynthesisTaskResult();
+        SynthesisTask task = new SynthesisTask().withTaskId(TEST_TASK_ID)
+                .withTaskStatus(TaskStatus.Failed)
+                .withTaskStatusReason("reasonOfFailure");
+        taskResult.setSynthesisTask(task);
+        when(mockPollyClient.getSpeechSynthesisTask(requestCaptor.capture())).thenReturn(taskResult);
+        runner.enqueue("content", Collections.singletonMap(AWS_TASK_ID_PROPERTY, TEST_TASK_ID));

Review Comment:
   Recommend defining and reusing a static variable the placeholder content string.
   ```suggestion
           runner.enqueue(PLACEHOLDER_CONTENT, Collections.singletonMap(AWS_TASK_ID_PROPERTY, TEST_TASK_ID));
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.aws.ml.polly;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_FAILURE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_ORIGINAL;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusGetter.REL_SUCCESS;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.SynthesisTask;
+import com.amazonaws.services.polly.model.TaskStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+public class GetAwsPollyStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    private AmazonPollyClient mockPollyClient = null;
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   ```suggestion
       @Mock
       private AWSCredentialsProviderService mockAwsCredentialsProvider;
   ```



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


[GitHub] [nifi] exceptionfactory commented on pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on PR #6589:
URL: https://github.com/apache/nifi/pull/6589#issuecomment-1403090702

   @KalmanJantner Following additional testing, I realized the uncaught exception was the result of running with a previous build, the latest commit appears to have resolved that problem, and handles failures as expected through the failiure relationship.


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


[GitHub] [nifi] pvillard31 commented on pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on PR #6589:
URL: https://github.com/apache/nifi/pull/6589#issuecomment-1347061165

   Note - I do see the additionalDetails page on the pull request but it's not showing in the usage page of the processor. Maybe something wrong somewhere when building the documentation?


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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1023592279


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>1.12.328</version>

Review Comment:
   Fixed thanks.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {

Review Comment:
   Updated, thanks.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>

Review Comment:
   Updated thanks.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlJobStarter.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlJobStarter<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);
+
+    protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        FlowFile childFlowFile = session.create(flowFile);
+        session.write(childFlowFile, out -> {
+            try (BufferedWriter bufferedWriter = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
+                bufferedWriter.write(mapper.writeValueAsString(response));
+                bufferedWriter.newLine();
+                bufferedWriter.flush();
+            }

Review Comment:
   Fixed. thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Transcribe"})
+@CapabilityDescription("Retrieves the current status of an AWS Transcribe job.")
+@SeeAlso({StartAwsTranscribeJob.class})
+public class GetAwsTranscribeJobStatus extends AwsMLJobStatusGetter<AmazonTranscribeClient> {
+    @Override
+    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranscribeClient) AmazonTranscribeClient.builder().build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetTranscriptionJobResult job = getJob(flowFile);
+        TranscriptionJobStatus jobStatus = TranscriptionJobStatus.fromValue(job.getTranscriptionJob().getTranscriptionJobStatus());
+
+        if (TranscriptionJobStatus.COMPLETED == jobStatus) {
+            writeToFlowFile(session, flowFile, job);
+            session.putAttribute(flowFile, AWS_TASK_OUTPUT_LOCATION, job.getTranscriptionJob().getTranscript().getTranscriptFileUri());
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+
+        if (TranscriptionJobStatus.IN_PROGRESS == jobStatus) {
+            session.transfer(flowFile, REL_IN_PROGRESS);
+        }
+
+        if (TranscriptionJobStatus.FAILED == jobStatus) {
+            final String failureReason = job.getTranscriptionJob().getFailureReason();
+            session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Amazon transcribe reported that the task failed for {}: {}", flowFile, failureReason);

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/MockAwsCredentialsProvider.java:
##########
@@ -0,0 +1,24 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+
+public class MockAwsCredentialsProvider extends AbstractControllerService implements AWSCredentialsProviderService {

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,106 @@
+package org.apache.nifi.processors.aws.ml.polly;

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/SdkHttpMetadataDeserializer.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.deser.std.StdNodeBasedDeserializer;
+import java.io.IOException;
+
+public class SdkHttpMetadataDeserializer extends StdNodeBasedDeserializer<SdkHttpMetadata> {
+
+    protected SdkHttpMetadataDeserializer() {
+        super(SdkHttpMetadata.class);
+    }
+
+    @Override
+    public SdkHttpMetadata convert(JsonNode root, DeserializationContext ctxt) throws IOException {
+        return null;

Review Comment:
   Same motivation just like at `ResponseMetadata`. Workaround to avoid issues with deserialising class without default constructor with jackson.  



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));

Review Comment:
   I missed that, thank you.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")

Review Comment:
   Make sense, I picked `running`, thanks.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/StartAwsPollyJob.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcherTest.java:
##########
@@ -0,0 +1,98 @@
+package org.apache.nifi.processors.aws.ml.textract;

Review Comment:
   Fixed, thx



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,94 @@
+package org.apache.nifi.processors.aws.ml.polly;

Review Comment:
   Fixed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule awsResponseModule = new SimpleModule();
+        awsResponseModule.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule sdkHttpModule = new SimpleModule();
+        awsResponseModule.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(awsResponseModule);
+        mapper.registerModule(sdkHttpModule);
+    }
+
+
+    protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {
+        session.write(flowFile, out -> {
+            try (BufferedWriter bufferedWriter = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
+                bufferedWriter.write(mapper.writeValueAsString(response));
+                bufferedWriter.flush();
+            }

Review Comment:
   It turned out we can just simply leave out `BufferedWriter` and OutputStream works with `writeValueAsString` as well. I updated this part according to this.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsResponseMetadataDeserializer.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.ResponseMetadata;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.deser.std.StdNodeBasedDeserializer;
+import java.io.IOException;
+import java.util.Map;
+
+public class AwsResponseMetadataDeserializer extends StdNodeBasedDeserializer<ResponseMetadata> {
+    protected AwsResponseMetadataDeserializer() {
+        super(ResponseMetadata.class);
+    }
+
+    @Override
+    public ResponseMetadata convert(JsonNode root, DeserializationContext ctxt) throws IOException {
+        return new ResponseMetadata((Map<String, String>) null);

Review Comment:
   `ResponseMetadata` doesn't have default constructor but it is required for jackson deserialization. In addition we don't need anything from this metadata at the moment. So this deserialiser is just a workaround to be able to use jackson mappings.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,94 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMLJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard().withCredentials(credentialsProvider).build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Aws Client reached rate limit", e);

Review Comment:
   Fixed, thx.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")

Review Comment:
   Fixed.



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1011958621


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-textract</artifactId>
+            <version>1.12.328</version>
+        </dependency>

Review Comment:
   Thanks for the reply. It seems like it would be best to create a new Jira and pull request to create a new AWS Credentials Service based on version 2 of the SDK.



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1018397754


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>

Review Comment:
   Recommend spelling out the name:
   
   ```suggestion
   public abstract class AwsMachineLearningJobStatusGetter<SERVICE extends AmazonWebServiceClient>
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {

Review Comment:
   The general convention for type variables is a single letter, so recommend changing `SERVICE` to `T`:
   ```suggestion
           extends AbstractAWSCredentialsProviderProcessor<T>  {
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")

Review Comment:
   ```suggestion
               .name("throttled")
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));

Review Comment:
   These properties should be removed:
   ```suggestion
   ));
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsResponseMetadataDeserializer.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.ResponseMetadata;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.deser.std.StdNodeBasedDeserializer;
+import java.io.IOException;
+import java.util.Map;
+
+public class AwsResponseMetadataDeserializer extends StdNodeBasedDeserializer<ResponseMetadata> {
+    protected AwsResponseMetadataDeserializer() {
+        super(ResponseMetadata.class);
+    }
+
+    @Override
+    public ResponseMetadata convert(JsonNode root, DeserializationContext ctxt) throws IOException {
+        return new ResponseMetadata((Map<String, String>) null);

Review Comment:
   Is this supposed to convert anything from the `JsonNode`, or just return an empty response?



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,94 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+public class GetAwsPollyJobStatus extends AwsMLJobStatusGetter<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";
+
+    @Override
+    protected AmazonPollyClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonPollyClient) AmazonPollyClientBuilder.standard().withCredentials(credentialsProvider).build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetSpeechSynthesisTaskResult speechSynthesisTask;
+        try {
+            speechSynthesisTask = getSynthesisTask(flowFile);
+        } catch (ThrottlingException e) {
+            getLogger().info("Aws Client reached rate limit", e);

Review Comment:
   ```suggestion
               getLogger().info("Request Rate Limit exceeded", e);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/TextractFetcherTest.java:
##########
@@ -0,0 +1,98 @@
+package org.apache.nifi.processors.aws.ml.textract;

Review Comment:
   License header needed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyStatusTest.java:
##########
@@ -0,0 +1,106 @@
+package org.apache.nifi.processors.aws.ml.polly;

Review Comment:
   License header needed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>1.12.328</version>

Review Comment:
   These version references can be removed. The root Maven configuration includes the `aws-java-sdk-bom`, which controls the version for all AWS SDK Version 1 libraries.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");

Review Comment:
   Recommend rewording the error:
   ```suggestion
           throw new UnsupportedOperationException("Client creation not supported");
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")

Review Comment:
   What do you think about naming this relationship `processing` or `running`?



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/SdkHttpMetadataDeserializer.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.deser.std.StdNodeBasedDeserializer;
+import java.io.IOException;
+
+public class SdkHttpMetadataDeserializer extends StdNodeBasedDeserializer<SdkHttpMetadata> {
+
+    protected SdkHttpMetadataDeserializer() {
+        super(SdkHttpMetadata.class);
+    }
+
+    @Override
+    public SdkHttpMetadata convert(JsonNode root, DeserializationContext ctxt) throws IOException {
+        return null;

Review Comment:
   Is there a reason for returning null instead of something else? It seems like there should be a default converter for this approach.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,94 @@
+package org.apache.nifi.processors.aws.ml.polly;

Review Comment:
   This class needs a license header.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMLJobStatusGetter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.io.BufferedWriter;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+abstract public class AwsMLJobStatusGetter<SERVICE extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_IN_PROGRESS = new Relationship.Builder()
+            .name("in progress")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("partial success")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE,
+            PROXY_HOST,
+            PROXY_HOST_PORT,
+            PROXY_USERNAME,
+            PROXY_PASSWORD));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_IN_PROGRESS,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule awsResponseModule = new SimpleModule();
+        awsResponseModule.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule sdkHttpModule = new SimpleModule();
+        awsResponseModule.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(awsResponseModule);
+        mapper.registerModule(sdkHttpModule);
+    }
+
+
+    protected void writeToFlowFile(ProcessSession session, FlowFile flowFile, Object response) {
+        session.write(flowFile, out -> {
+            try (BufferedWriter bufferedWriter = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
+                bufferedWriter.write(mapper.writeValueAsString(response));
+                bufferedWriter.flush();
+            }

Review Comment:
   This should be changed to use `writeValueAsBytes()`, which avoids the need to wrap the OutputStream in a `BufferedWriter`. 



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMlJobStarter.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMlJobStarter<SERVICE extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<SERVICE> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON Payload that represent an AWS ML Request. See more details in AWS API documentation.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {
+        String payloadPropertyValue = context.getProperty(JSON_PAYLOAD).evaluateAttributeExpressions(flowFile).getValue();
+        if (payloadPropertyValue == null) {
+            payloadPropertyValue = readFlowFile(session, flowFile);
+        }
+        return payloadPropertyValue;
+    }
+
+    @Override
+    protected SERVICE createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Tried to create client in a deprecated way.");
+    }
+
+    abstract protected RESPONSE sendRequest(REQUEST request, ProcessContext context) throws JsonProcessingException;
+
+    abstract protected Class<? extends REQUEST> getAwsRequestClass(ProcessContext context);
+
+    abstract protected String getAwsTaskId(ProcessContext context, RESPONSE response);
+
+    protected FlowFile writeToFlowFile(ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        FlowFile childFlowFile = session.create(flowFile);
+        session.write(childFlowFile, out -> {
+            try (BufferedWriter bufferedWriter = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))) {
+                bufferedWriter.write(mapper.writeValueAsString(response));
+                bufferedWriter.newLine();
+                bufferedWriter.flush();
+            }

Review Comment:
   See note on using `writeValueAsBytes()` instead of the BufferedWriter and String approach.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatus.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMLJobStatusGetter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Transcribe"})
+@CapabilityDescription("Retrieves the current status of an AWS Transcribe job.")
+@SeeAlso({StartAwsTranscribeJob.class})
+public class GetAwsTranscribeJobStatus extends AwsMLJobStatusGetter<AmazonTranscribeClient> {
+    @Override
+    protected AmazonTranscribeClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTranscribeClient) AmazonTranscribeClient.builder().build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        GetTranscriptionJobResult job = getJob(flowFile);
+        TranscriptionJobStatus jobStatus = TranscriptionJobStatus.fromValue(job.getTranscriptionJob().getTranscriptionJobStatus());
+
+        if (TranscriptionJobStatus.COMPLETED == jobStatus) {
+            writeToFlowFile(session, flowFile, job);
+            session.putAttribute(flowFile, AWS_TASK_OUTPUT_LOCATION, job.getTranscriptionJob().getTranscript().getTranscriptFileUri());
+            session.transfer(flowFile, REL_SUCCESS);
+        }
+
+        if (TranscriptionJobStatus.IN_PROGRESS == jobStatus) {
+            session.transfer(flowFile, REL_IN_PROGRESS);
+        }
+
+        if (TranscriptionJobStatus.FAILED == jobStatus) {
+            final String failureReason = job.getTranscriptionJob().getFailureReason();
+            session.putAttribute(flowFile, FAILURE_REASON_ATTRIBUTE, failureReason);
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Amazon transcribe reported that the task failed for {}: {}", flowFile, failureReason);

Review Comment:
   ```suggestion
               getLogger().error("Transcribe Task Failed for {}: {}", flowFile, failureReason);
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/StartAwsPollyJob.java:
##########
@@ -0,0 +1,38 @@
+package org.apache.nifi.processors.aws.ml.polly;

Review Comment:
   License header needed.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/polly/MockAwsCredentialsProvider.java:
##########
@@ -0,0 +1,24 @@
+package org.apache.nifi.processors.aws.ml.polly;
+
+import com.amazonaws.auth.AWSCredentialsProvider;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+
+public class MockAwsCredentialsProvider extends AbstractControllerService implements AWSCredentialsProviderService {

Review Comment:
   This could be removed and replaced with a Mockito-managed mock instance of `AWSCredentialsProviderService`.



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1044077701


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";

Review Comment:
   `protected static` variables should be listed after `public static` variables.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(

Review Comment:
   This should be below `public static` variables.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";

Review Comment:
   This should be moved above `protected static` members.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")

Review Comment:
   ```suggestion
               .defaultValue(DOCUMENT_ANALYSIS)
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")

Review Comment:
   ```suggestion
               .displayName("Textract Type")
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.textract.GetAwsTextractJobStatus.DOCUMENT_ANALYSIS;
+import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TYPE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;

Review Comment:
   ```suggestion
       private TestRunner runner;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.aws.ml.translate;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import com.amazonaws.services.translate.model.OutputDataConfig;
+import com.amazonaws.services.translate.model.TextTranslationJobProperties;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String CONTENT_STRING = "content";
+    private static final String AWS_CREDENTIALS_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocation";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranslateClient mockTranslateClient = null;
+    @Mock
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   ```suggestion
       private AWSCredentialsProviderService mockAwsCredentialsProvider;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.aws.ml.translate;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import com.amazonaws.services.translate.model.OutputDataConfig;
+import com.amazonaws.services.translate.model.TextTranslationJobProperties;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String CONTENT_STRING = "content";
+    private static final String AWS_CREDENTIALS_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocation";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranslateClient mockTranslateClient = null;

Review Comment:
   ```suggestion
       private AmazonTranslateClient mockTranslateClient;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/translate/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.aws.ml.translate;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_OUTPUT_LOCATION;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.translate.AmazonTranslateClient;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobRequest;
+import com.amazonaws.services.translate.model.DescribeTextTranslationJobResult;
+import com.amazonaws.services.translate.model.JobStatus;
+import com.amazonaws.services.translate.model.OutputDataConfig;
+import com.amazonaws.services.translate.model.TextTranslationJobProperties;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String CONTENT_STRING = "content";
+    private static final String AWS_CREDENTIALS_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocation";
+    private TestRunner runner = null;

Review Comment:
   ```suggestion
       private TestRunner runner;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()

Review Comment:
   The `mapper` instance can be `static`:
   ```suggestion
       protected static final ObjectMapper mapper = JsonMapper.builder()
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .expressionLanguageSupported(FLOWFILE_ATTRIBUTES)
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")

Review Comment:
   ```suggestion
               .defaultValue(DOCUMENT_ANALYSIS)
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = "PollyS3OutputBucket", description = "The bucket name where polly output will be located."),
+        @WritesAttribute(attribute = "PollyS3OutputKey", description = "Object key of polly output."),
+        @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
+})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";

Review Comment:
   Is there a particular reason for writing out these attributes as opposed to just the `outputLocation` attribute? It seems unnecessary, and would remove the need for the regular expression matching, unless there is some added value in having these attributes.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    private final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        final RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+        } catch (Exception e) {
+            session.transfer(flowFile, REL_FAILURE);
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        session.transfer(flowFile, REL_ORIGINAL);
+        session.transfer(childFlowFile, REL_SUCCESS);
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response));
+        getLogger().debug("AWS ML task has been started with task id: {}", getAwsTaskId(context, response));
+    }
+
+    protected REQUEST buildRequest(ProcessSession session, ProcessContext context, FlowFile flowFile) throws JsonProcessingException {
+        return mapper.readValue(getPayload(session, context, flowFile), getAwsRequestClass(context));
+    }
+
+    private String getPayload(ProcessSession session, ProcessContext context, FlowFile flowFile) {

Review Comment:
   `private` methods should be listed after `protected` methods.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStatusProcessor.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.ResponseMetadata;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.http.SdkHttpMetadata;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStatusProcessor<T extends AmazonWebServiceClient>
+        extends AbstractAWSCredentialsProviderProcessor<T>  {
+    public static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    public static final String AWS_TASK_OUTPUT_LOCATION = "outputLocation";
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final Relationship REL_RUNNING = new Relationship.Builder()
+            .name("running")
+            .description("The job is currently still being processed")
+            .build();
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .name("success")
+            .description("Job successfully finished. FlowFile will be routed to this relation.")
+            .build();
+    public static final Relationship REL_THROTTLED = new Relationship.Builder()
+            .name("throttled")
+            .description("Retrieving results failed for some reason, but the issue is likely to resolve on its own, such as Provisioned Throughput Exceeded or a Throttling failure. " +
+                    "It is generally expected to retry this relationship.")
+            .build();
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+            .name("failure")
+            .description("The job failed, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    protected final ObjectMapper mapper = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE,
+            PROXY_CONFIGURATION_SERVICE));
+
+    public static final String FAILURE_REASON_ATTRIBUTE = "failure.reason";
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_RUNNING,
+            REL_THROTTLED,
+            REL_FAILURE
+    )));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected T createClient(ProcessContext context, AWSCredentials credentials, ClientConfiguration config) {
+        throw new UnsupportedOperationException("Client creation not supported");
+    }
+
+    @Override
+    protected void init(ProcessorInitializationContext context) {
+        SimpleModule awsResponseModule = new SimpleModule();
+        awsResponseModule.addDeserializer(ResponseMetadata.class, new AwsResponseMetadataDeserializer());
+        SimpleModule sdkHttpModule = new SimpleModule();
+        awsResponseModule.addDeserializer(SdkHttpMetadata.class, new SdkHttpMetadataDeserializer());
+        mapper.registerModule(awsResponseModule);
+        mapper.registerModule(sdkHttpModule);

Review Comment:
   Can this be moved to a static initializer block? The current approach calls this registration process every time the processor is initialized, which is not necessary.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")

Review Comment:
   ```suggestion
               .name("textract-type")
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")
+            .allowableValues(DOCUMENT_ANALYSIS, DOCUMENT_TEXT_DETECTION, EXPENSE_ANALYSIS)
+            .required(true)
+            .defaultValue("Document Analysis")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES =
+        Collections.unmodifiableList(Stream.concat(PROPERTIES.stream(), Stream.of(TYPE)).collect(Collectors.toList()));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, AmazonWebServiceResult response) {
+        super.postProcessFlowFile(context, session, flowFile, response);
+        session.putAttribute(flowFile, TYPE.getName(), context.getProperty(TYPE.getName()).getValue());
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    protected AmazonWebServiceResult sendRequest(AmazonWebServiceRequest request, ProcessContext context) {
+        String typeOfTextract = context.getProperty(TYPE.getName()).getValue();

Review Comment:
   ```suggestion
           String textractType = context.getProperty(TYPE.getName()).getValue();
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.expression.ExpressionLanguageScope.FLOWFILE_ATTRIBUTES;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()

Review Comment:
   ```suggestion
       public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/StartAwsTextractJob.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.StartDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.StartDocumentTextDetectionResult;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.StartExpenseAnalysisResult;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStarter;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Trigger a AWS Textract job. It should be followed by GetAwsTextractJobStatus processor in order to monitor job status.")
+@SeeAlso({GetAwsTextractJobStatus.class})
+public class StartAwsTextractJob extends AwsMachineLearningJobStarter<AmazonTextractClient, AmazonWebServiceRequest, AmazonWebServiceResult> {
+    private static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    private static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    private static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TYPE = new PropertyDescriptor.Builder()
+            .name("type-of-service")
+            .displayName("Type of textract")

Review Comment:
   ```suggestion
       public static final PropertyDescriptor TEXTRACT)TYPE = new PropertyDescriptor.Builder()
               .name("textract-type")
               .displayName("Textract Type")
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.textract.GetAwsTextractJobStatus.DOCUMENT_ANALYSIS;
+import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TYPE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTextractClient mockTextractClient = null;
+    @Mock
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   ```suggestion
       private AWSCredentialsProviderService mockAwsCredentialsProvider;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTranslateJobStatusTest.java:
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.aws.ml.textract;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.apache.nifi.processors.aws.ml.textract.GetAwsTextractJobStatus.DOCUMENT_ANALYSIS;
+import static org.apache.nifi.processors.aws.ml.textract.StartAwsTextractJob.TYPE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisResult;
+import com.amazonaws.services.textract.model.JobStatus;
+import com.google.common.collect.ImmutableMap;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranslateJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTextractClient mockTextractClient = null;

Review Comment:
   ```suggestion
       private AmazonTextractClient mockTextractClient;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.Transcript;
+import com.amazonaws.services.transcribe.model.TranscriptionJob;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranscribeJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String AWS_CREDENTIAL_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocationPath";
+    private static final String REASON_OF_FAILURE = "reasonOfFailure";
+    private static final String CONTENT_STRING = "content";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranscribeClient mockTranscribeClient = null;

Review Comment:
   ```suggestion
       private AmazonTranscribeClient mockTranscribeClient;
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/test/java/org/apache/nifi/processors/aws/ml/transcribe/GetAwsTranscribeJobStatusTest.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.transcribe;
+
+import static org.apache.nifi.processors.aws.AbstractAWSCredentialsProviderProcessor.AWS_CREDENTIALS_PROVIDER_SERVICE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_FAILURE;
+import static org.apache.nifi.processors.aws.AbstractAWSProcessor.REL_SUCCESS;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.AWS_TASK_ID_PROPERTY;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.FAILURE_REASON_ATTRIBUTE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.REL_RUNNING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.services.transcribe.AmazonTranscribeClient;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobRequest;
+import com.amazonaws.services.transcribe.model.GetTranscriptionJobResult;
+import com.amazonaws.services.transcribe.model.Transcript;
+import com.amazonaws.services.transcribe.model.TranscriptionJob;
+import com.amazonaws.services.transcribe.model.TranscriptionJobStatus;
+import java.util.Collections;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.aws.credentials.provider.service.AWSCredentialsProviderService;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class GetAwsTranscribeJobStatusTest {
+    private static final String TEST_TASK_ID = "testTaskId";
+    private static final String AWS_CREDENTIAL_PROVIDER_NAME = "awsCredetialProvider";
+    private static final String OUTPUT_LOCATION_PATH = "outputLocationPath";
+    private static final String REASON_OF_FAILURE = "reasonOfFailure";
+    private static final String CONTENT_STRING = "content";
+    private TestRunner runner = null;
+    @Mock
+    private AmazonTranscribeClient mockTranscribeClient = null;
+    @Mock
+    private AWSCredentialsProviderService mockAwsCredentialsProvider = null;

Review Comment:
   ```suggestion
       private AWSCredentialsProviderService mockAwsCredentialsProvider;
   ```



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


[GitHub] [nifi] KalmanJantner commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1044125278


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/polly/GetAwsPollyJobStatus.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.aws.ml.polly;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.polly.AmazonPollyClient;
+import com.amazonaws.services.polly.AmazonPollyClientBuilder;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskRequest;
+import com.amazonaws.services.polly.model.GetSpeechSynthesisTaskResult;
+import com.amazonaws.services.polly.model.TaskStatus;
+import com.amazonaws.services.textract.model.ThrottlingException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Polly"})
+@CapabilityDescription("Retrieves the current status of an AWS Polly job.")
+@SeeAlso({StartAwsPollyJob.class})
+@WritesAttributes({
+        @WritesAttribute(attribute = "PollyS3OutputBucket", description = "The bucket name where polly output will be located."),
+        @WritesAttribute(attribute = "PollyS3OutputKey", description = "Object key of polly output."),
+        @WritesAttribute(attribute = "outputLocation", description = "S3 path-style output location of the result.")
+})
+public class GetAwsPollyJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonPollyClient> {
+    private static final String BUCKET = "bucket";
+    private static final String KEY = "key";
+    private static final Pattern S3_PATH = Pattern.compile("https://s3.*amazonaws.com/(?<" + BUCKET + ">[^/]+)/(?<" + KEY + ">.*)");
+    private static final String AWS_S3_BUCKET = "PollyS3OutputBucket";
+    private static final String AWS_S3_KEY = "PollyS3OutputKey";

Review Comment:
   It would make the processor a bit more user friendly. In case of Polly the output is always one single file. The response would contains a path-style reference to the s3 object but (as far as I know) FetchS3Object does not support that format, only bucket + key combination. So with these 2 attributes FetchS3Object processor can be feed without additional processors. In case of other ML processor this path could be different e.g.: out put folder and not a single object or the result of the job directly returned in the json and not stored in s3. That is the reason why this was applied only in this processor. So this is not critical we can skip it if you do not see it useful. What do you think @exceptionfactory how to proceed?



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


[GitHub] [nifi] KalmanJantner commented on pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
KalmanJantner commented on PR #6589:
URL: https://github.com/apache/nifi/pull/6589#issuecomment-1358960221

   > Playing with this PR. Some remarks:
   > 
   > * set mime.type to json when we know that the generated flow file is going to contain a JSON payload (example: StartAwsTextractJob, GetAwsTextractJobStatus, etc).
   > * please set description on all properties and enum values where it makes sense. We should not assume the user knows the difference between Text Analysis and Document Text Detection for example (in StartAwsTextractJob).
   > * please add additionalDetails with examples of expected JSON payloads + links to AWS documentation if that makes sense (example: StartAwsTextractJob)
   > * the type of extract in GetAwsTextractJobStatus is a string and not an enum. If that's because we want to be able to reference a flow file attribute, then we should directly reference the flow file attribute that we created in  StartAwsTextractJob. In this case: `${type-of-service}`. However when trying to use EL, it fails because it's not one of the allowable values. Something to fix here.
   
   Thank you for the comments, I went through and fixed them.


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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1018396631


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/pom.xml:
##########
@@ -117,6 +117,26 @@
             <version>1.19.0-SNAPSHOT</version>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-translate</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-polly</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-transcribe</artifactId>
+            <version>1.12.328</version>
+        </dependency>
+        <dependency>
+            <groupId>com.amazonaws</groupId>
+            <artifactId>aws-java-sdk-textract</artifactId>
+            <version>1.12.328</version>
+        </dependency>

Review Comment:
   On further review, moving forward with the current implementation based on SDK version 1 works, and subsequent refactoring for SDK version 2 can be handled once a new AWS Credentials Service is ready.



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1064920895


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.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.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
+            .name("textract-type")
+            .displayName("Textract Type")
+            .required(true)
+            .description("Supported values: \"Document Analysis\", \"Document Text Detection\", \"Expense Analysis\"")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(DOCUMENT_ANALYSIS)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES =
+            Collections.unmodifiableList(Stream.concat(PROPERTIES.stream(), Stream.of(TEXTRACT_TYPE)).collect(Collectors.toList()));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String textractType = context.getProperty(TEXTRACT_TYPE).evaluateAttributeExpressions(flowFile).getValue();
+
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);

Review Comment:
   This approach has an implicit contract on input FlowFiles having the `awsTaskId` property. Although this aligns with the output of the start Processors, it would be better to make this explicit. All of the Get-Status Processors could have a `Task ID` property that supports Expression Language, and the default value would be `${awsTaskId}`, which would provide the same basic behavior, but support alternative configuration options.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.aws.ml;
+
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    private final static ObjectMapper MAPPER = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null && !context.getProperty(JSON_PAYLOAD).isSet()) {
+            return;
+        }
+        final RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context, flowFile);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+            session.transfer(childFlowFile, REL_SUCCESS);
+        } catch (Exception e) {
+            if (context.getProperty(JSON_PAYLOAD).isSet()) {
+                session.transfer(flowFile, REL_FAILURE);
+            }
+            getLogger().error("Sending AWS ML Request failed", e);
+            return;
+        }
+        if (flowFile != null) {
+            session.transfer(flowFile, REL_ORIGINAL);
+        }
+
+    }
+
+    protected void postProcessFlowFile(ProcessContext context, ProcessSession session, FlowFile flowFile, RESPONSE response) {
+        session.putAttribute(flowFile, AWS_TASK_ID_PROPERTY, getAwsTaskId(context, response, flowFile));
+        session.putAttribute(flowFile, MIME_TYPE.key(), "application/json");

Review Comment:
   These calls need to return an updated instance of `FlowFile` for subsequent processing.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.aws.ml;
+
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));
+    protected static final String AWS_TASK_ID_PROPERTY = "awsTaskId";
+    private final static ObjectMapper MAPPER = JsonMapper.builder()
+            .configure(MapperFeature.ACCEPT_CASE_INSENSITIVE_PROPERTIES, true)
+            .build();
+    private static final Set<Relationship> relationships = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(
+            REL_ORIGINAL,
+            REL_SUCCESS,
+            REL_FAILURE
+    )));
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) {
+        FlowFile flowFile = session.get();
+        if (flowFile == null && !context.getProperty(JSON_PAYLOAD).isSet()) {
+            return;
+        }
+        final RESPONSE response;
+        FlowFile childFlowFile;
+        try {
+            response = sendRequest(buildRequest(session, context, flowFile), context, flowFile);
+            childFlowFile = writeToFlowFile(session, flowFile, response);
+            postProcessFlowFile(context, session, childFlowFile, response);
+            session.transfer(childFlowFile, REL_SUCCESS);
+        } catch (Exception e) {
+            if (context.getProperty(JSON_PAYLOAD).isSet()) {

Review Comment:
   It looks like the logic should be reversed, or for clarity, this conditional should be changed to look for a defined `flowFile`:
   ```suggestion
               if (flowFile != null) {
   ```



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.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.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
+            .name("textract-type")
+            .displayName("Textract Type")
+            .required(true)
+            .description("Supported values: \"Document Analysis\", \"Document Text Detection\", \"Expense Analysis\"")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .defaultValue(DOCUMENT_ANALYSIS)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    private static final List<PropertyDescriptor> TEXTRACT_PROPERTIES =
+            Collections.unmodifiableList(Stream.concat(PROPERTIES.stream(), Stream.of(TEXTRACT_TYPE)).collect(Collectors.toList()));
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return TEXTRACT_PROPERTIES;
+    }
+
+    @Override
+    protected AmazonTextractClient createClient(ProcessContext context, AWSCredentialsProvider credentialsProvider, ClientConfiguration config) {
+        return (AmazonTextractClient) AmazonTextractClient.builder()
+                .withRegion(context.getProperty(REGION).getValue())
+                .withCredentials(credentialsProvider)
+                .build();
+    }
+
+    @Override
+    public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+        String textractType = context.getProperty(TEXTRACT_TYPE).evaluateAttributeExpressions(flowFile).getValue();
+
+        String awsTaskId = flowFile.getAttribute(AWS_TASK_ID_PROPERTY);
+        JobStatus jobStatus = getTaskStatus(textractType, getClient(), awsTaskId);

Review Comment:
   The `getTaskStatus` method can throw an exception that `onTrigger` does not catch, causing the input FlowFile to remaining in the input queue. The `GetAwsPollyJobStatus` Processor has a try-catch around that status call, so it seems like that approach should be implemented in this and other status retrieval Processors.



##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/textract/GetAwsTextractJobStatus.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.aws.ml.textract;
+
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentialsProvider;
+import com.amazonaws.services.textract.AmazonTextractClient;
+import com.amazonaws.services.textract.model.GetDocumentAnalysisRequest;
+import com.amazonaws.services.textract.model.GetDocumentTextDetectionRequest;
+import com.amazonaws.services.textract.model.GetExpenseAnalysisRequest;
+import com.amazonaws.services.textract.model.JobStatus;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.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.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor;
+
+@Tags({"Amazon", "AWS", "ML", "Machine Learning", "Textract"})
+@CapabilityDescription("Retrieves the current status of an AWS Textract job.")
+@SeeAlso({StartAwsTextractJob.class})
+public class GetAwsTextractJobStatus extends AwsMachineLearningJobStatusProcessor<AmazonTextractClient> {
+    public static final String DOCUMENT_ANALYSIS = "Document Analysis";
+    public static final String DOCUMENT_TEXT_DETECTION = "Document Text Detection";
+    public static final String EXPENSE_ANALYSIS = "Expense Analysis";
+    public static final PropertyDescriptor TEXTRACT_TYPE = new PropertyDescriptor.Builder()
+            .name("textract-type")
+            .displayName("Textract Type")
+            .required(true)
+            .description("Supported values: \"Document Analysis\", \"Document Text Detection\", \"Expense Analysis\"")

Review Comment:
   It seems like these supported values should be defined using the `allowableValues()` property descriptor setting.



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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #6589:
URL: https://github.com/apache/nifi/pull/6589#discussion_r1084247803


##########
nifi-nar-bundles/nifi-aws-bundle/nifi-aws-processors/src/main/java/org/apache/nifi/processors/aws/ml/AwsMachineLearningJobStarter.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.aws.ml;
+
+import static org.apache.nifi.flowfile.attributes.CoreAttributes.MIME_TYPE;
+import static org.apache.nifi.processors.aws.ml.AwsMachineLearningJobStatusProcessor.TASK_ID;
+
+import com.amazonaws.AmazonWebServiceClient;
+import com.amazonaws.AmazonWebServiceRequest;
+import com.amazonaws.AmazonWebServiceResult;
+import com.amazonaws.ClientConfiguration;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.regions.Regions;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.json.JsonMapper;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.io.IOUtils;
+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.processors.aws.AbstractAWSCredentialsProviderProcessor;
+
+public abstract class AwsMachineLearningJobStarter<T extends AmazonWebServiceClient, REQUEST extends AmazonWebServiceRequest, RESPONSE extends AmazonWebServiceResult>
+        extends AbstractAWSCredentialsProviderProcessor<T> {
+    public static final PropertyDescriptor JSON_PAYLOAD = new PropertyDescriptor.Builder()
+            .name("json-payload")
+            .displayName("JSON Payload")
+            .description("JSON request for AWS Machine Learning services. The Processor will use FlowFile content for the request when this property is not specified.")
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .required(false)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+    public static final PropertyDescriptor MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE =
+            new PropertyDescriptor.Builder().fromPropertyDescriptor(AWS_CREDENTIALS_PROVIDER_SERVICE)
+                    .required(true)
+                    .build();
+    public static final PropertyDescriptor REGION = new PropertyDescriptor.Builder()
+            .displayName("Region")
+            .name("aws-region")
+            .required(true)
+            .allowableValues(getAvailableRegions())
+            .defaultValue(createAllowableValue(Regions.DEFAULT_REGION).getValue())
+            .build();
+    public static final Relationship REL_ORIGINAL = new Relationship.Builder()
+            .name("original")
+            .description("Upon successful completion, the original FlowFile will be routed to this relationship.")
+            .autoTerminateDefault(true)
+            .build();
+    protected static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            JSON_PAYLOAD,
+            MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
+            REGION,
+            TIMEOUT,
+            SSL_CONTEXT_SERVICE,
+            ENDPOINT_OVERRIDE));

Review Comment:
   It is best practice to have all required properties before optional properties, so recommend adjusting the ordering:
   ```suggestion
               MANDATORY_AWS_CREDENTIALS_PROVIDER_SERVICE,
               REGION,
               TIMEOUT,
               JSON_PAYLOAD,
               SSL_CONTEXT_SERVICE,
               ENDPOINT_OVERRIDE));
   ```



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


[GitHub] [nifi] exceptionfactory closed pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #6589: NIFI-10710 implement processor for AWS Polly, Textract, Translate, Tr…
URL: https://github.com/apache/nifi/pull/6589


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