dan-s1 commented on code in PR #8501:
URL: https://github.com/apache/nifi/pull/8501#discussion_r1525017249


##########
nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/AbstractKuduProcessor.java:
##########
@@ -361,6 +363,27 @@ protected Collection<ValidationResult> 
customValidate(ValidationContext context)
         return results;
     }
 
+    protected List<RowError> flushKuduSession(final KuduSession kuduSession) 
throws KuduException {
+        final List<OperationResponse> responses = kuduSession.flush();
+        // RowErrors will only be present in the OperationResponses in this 
case if the flush mode
+        // selected is MANUAL_FLUSH. It will be empty otherwise.
+        return responses.stream()
+                .filter(OperationResponse::hasRowError)
+                .map(OperationResponse::getRowError)
+                .collect(Collectors.toList());
+    }
+
+    protected List<RowError> closeKuduSession(final KuduSession kuduSession) 
throws KuduException {
+        final List<OperationResponse> responses = kuduSession.close();
+        // RowErrors will only be present in the OperationResponses in this 
case if the flush mode
+        // selected is MANUAL_FLUSH, since the underlying implementation of 
kuduSession.close() returns
+        // the OperationResponses from a flush() call.
+        return responses.stream()
+                .filter(OperationResponse::hasRowError)
+                .map(OperationResponse::getRowError)
+                .collect(Collectors.toList());
+    }

Review Comment:
   The only difference in these methods is the source of the responses not how 
the errors are collected. Refactor how the errors are collected into a common 
method.
   ```suggestion
       protected List<RowError> flushKuduSession(final KuduSession kuduSession) 
throws KuduException {
           final List<OperationResponse> responses = kuduSession.flush();
           // RowErrors will only be present in the OperationResponses in this 
case if the flush mode
           // selected is MANUAL_FLUSH. It will be empty otherwise.
           return getRowErrors(responses);
       }
   
       protected List<RowError> closeKuduSession(final KuduSession kuduSession) 
throws KuduException {
           final List<OperationResponse> responses = kuduSession.close();
           // RowErrors will only be present in the OperationResponses in this 
case if the flush mode
           // selected is MANUAL_FLUSH, since the underlying implementation of 
kuduSession.close() returns
           // the OperationResponses from a flush() call.
           return getRowErrors(responses);
       }
       
       private List<RowError> getRowErrors(List<OperationResponse> responses) {
             return responses.stream()
                   .filter(OperationResponse::hasRowError)
                   .map(OperationResponse::getRowError)
                   .collect(Collectors.toList());
       }
   ```



##########
nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/StandardPutKuduResult.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.kudu;
+
+import org.apache.kudu.client.Operation;
+import org.apache.kudu.client.RowError;
+import org.apache.nifi.flowfile.FlowFile;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class StandardPutKuduResult extends PutKuduResult {
+    private final Map<Operation, FlowFile> operationFlowFileMap;
+    private final List<RowError> pendingRowErrors;
+    private final Map<FlowFile, List<RowError>> flowFileRowErrorsMap;
+
+    public StandardPutKuduResult() {
+        super();
+        this.operationFlowFileMap = new HashMap<>();
+        this.pendingRowErrors = new ArrayList<>();
+        this.flowFileRowErrorsMap = new HashMap<>();
+    }
+
+    @Override
+    public void recordOperation(final Operation operation) {
+        operationFlowFileMap.put(operation, flowFile);
+    }
+
+    @Override
+    public void addError(final RowError rowError) {
+        // When this class is used to store results from processing FlowFiles, 
the FlushMode
+        // is set to AUTO_FLUSH_BACKGROUND or MANUAL_FLUSH. In either case, we 
won't know which
+        // FlowFile/Record we are currently processing as the RowErrors are 
obtained from the KuduSession
+        // post-processing of the FlowFile/Record
+        this.pendingRowErrors.add(rowError);
+    }
+
+    @Override
+    public void resolveFlowFileToRowErrorAssociations() {
+        flowFileRowErrorsMap.putAll(pendingRowErrors.stream()
+                .filter(e -> operationFlowFileMap.get(e.getOperation()) != 
null)
+                .collect(
+                        Collectors.groupingBy(e -> 
operationFlowFileMap.get(e.getOperation()))
+                )
+        );
+
+        pendingRowErrors.clear();
+    }
+
+    @Override
+    public boolean hasRowErrorsOrFailures() {
+        if (!flowFileFailures.isEmpty()) {
+            return true;
+        }
+
+        for (final Map.Entry<FlowFile, List<RowError>> entry : 
flowFileRowErrorsMap.entrySet()) {
+            if (!entry.getValue().isEmpty()) {
+                return true;
+            }
+        }
+
+        return false;

Review Comment:
   This can be simplified with:
   ```suggestion
           return flowFileRowErrorsMap.entrySet().stream().anyMatch(entry -> 
!entry.getValue().isEmpty()));
   ```



##########
nifi-nar-bundles/nifi-kudu-bundle/nifi-kudu-processors/src/main/java/org/apache/nifi/processors/kudu/PutKuduResult.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.kudu;
+
+import org.apache.kudu.client.Operation;
+import org.apache.kudu.client.RowError;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.serialization.record.Record;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public abstract class PutKuduResult {
+    protected FlowFile flowFile;
+    protected final Map<FlowFile, Object> flowFileFailures;
+    private final Set<FlowFile> processedFlowFiles;
+    private final Map<FlowFile, Integer> processedRecords;
+
+    public PutKuduResult() {
+        this.flowFile = null;
+
+        this.flowFileFailures = new HashMap<>();
+        this.processedFlowFiles = new HashSet<>();
+        this.processedRecords = new HashMap<>();
+    }
+
+    public void setFlowFile(final FlowFile flowFile) {
+        this.flowFile = flowFile;
+        processedFlowFiles.add(flowFile);
+    }
+
+    public Set<FlowFile> getProcessedFlowFiles() {
+        return this.processedFlowFiles;
+    }
+
+    public int getProcessedRecordsForFlowFile(final FlowFile flowFile) {
+        return this.processedRecords.getOrDefault(flowFile, 0);
+    }
+
+    /**
+     * Increments the number of {@link Record}s that has been successfully 
processed for this {@link FlowFile}
+     */
+    public void incrementProcessedRecordsForFlowFile() {
+        final int newCount = this.processedRecords.getOrDefault(flowFile, 0) + 
1;
+        this.processedRecords.put(flowFile, newCount);
+    }
+
+    /**
+     * Records an {@link Operation} being processed for a specific {@link 
FlowFile}
+     * @param operation the {@link Operation} to record
+     */
+    public abstract void recordOperation(final Operation operation);
+
+    /**
+     * Records a {@link RowError} for the particular {@link FlowFile} that's 
being processed
+     * @param rowError the {@link RowError} to add
+     */
+    public abstract void addError(final RowError rowError);
+
+    /**
+     * Records a {@link List} of {@link RowError}s for the particular {@link 
FlowFile} that's being processed
+     * @param rowErrors the {@link List} of {@link RowError}s to add
+     */
+    public void addErrors(final List<RowError> rowErrors) {
+        for (final RowError rowError : rowErrors) {
+            addError(rowError);
+        }
+    }
+
+    /**
+     * Records a failure (an {@link Exception} or a {@link RowError}) for the 
particular {@link FlowFile} that's being processed.
+     * A failure is defined as anything that stops the processing of the 
records in a {@link FlowFile}
+     * @param failure the {@link Exception} or {@link RowError} to add
+     */
+    public void addFailure(final Object failure) {
+        if (flowFileFailures.containsKey(flowFile)) {
+            throw new IllegalStateException("A failure has already previously 
occurred while processing FlowFile.");
+        }
+        flowFileFailures.put(flowFile, failure);
+    }
+
+
+    /**
+     * Resolves the associations between {@link FlowFile} and the {@link 
RowError}s that occurred
+     * while processing them. This is only applicable in batch sesssion 
flushes, namely when
+     * using the {@code SessionConfiguration.FlushMode.AUTO_FLUSH_BACKGROUND} 
and
+     * {@code SessionConfiguration.FlushMode.MANUAL_FLUSH} flush modes. 
Otherwise, this
+     * function should be a no-op. This function should only be called once 
finished with processing
+     * all {@link FlowFile}s in a batch.
+     */
+    public void resolveFlowFileToRowErrorAssociations() {
+        return;
+    }
+
+    /**
+     * Checks whether there was a failure (i.e. either an {@link Exception} or 
{@link RowError} that happened during processing)
+     * @return {@code true} if there was a {@link Exception} or a {@link 
RowError} that happened during processing, {@code false} otherwise
+     */
+    public abstract boolean hasRowErrorsOrFailures();
+
+    /**
+     * Checks whether the {@link FlowFile} was processed successfully (i.e. no 
{@link Exception}s or
+     * {@link RowError}s occurred while processing the {@link FlowFile}).
+     *
+     * @param flowFile {@link FlowFile} to check
+     * @return {@code true} if the processing the {@link FlowFile} did not 
incur any exceptions, {@code false} otherwise
+     */
+    public boolean isFlowFileProcessedSuccessfully(final FlowFile flowFile) {
+        if (flowFileFailures.containsKey(flowFile)) {
+            return false;
+        }
+
+        return true;

Review Comment:
   Couldn't this be simplified with 
   ```suggestion
          return !flowFileFailures.containsKey(flowFile);
   ```



-- 
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-unsubscr...@nifi.apache.org

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

Reply via email to