sjyang18 commented on a change in pull request #4253:
URL: https://github.com/apache/nifi/pull/4253#discussion_r421138897



##########
File path: 
nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/cosmos/document/PutAzureCosmosDBRecord.java
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.azure.cosmos.document;
+
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.SystemResourceConsideration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SystemResource;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.schema.access.SchemaNotFoundException;
+import org.apache.nifi.serialization.MalformedRecordException;
+import org.apache.nifi.serialization.RecordReader;
+import org.apache.nifi.serialization.RecordReaderFactory;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.util.DataTypeUtils;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import com.azure.cosmos.CosmosClientException;
+import com.azure.cosmos.implementation.ConflictException;
+
+@EventDriven
+@Tags({ "azure", "cosmos", "insert", "record", "put" })
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@CapabilityDescription("This processor is a record-aware processor for 
inserting data into Azure CosmosDB with Core SQL API. It uses a configured 
record reader and " +
+        "schema to read an incoming record set from the body of a flowfile and 
then inserts those records into " +
+        "a configured Cosmos Container.")
+@SystemResourceConsideration(resource = SystemResource.MEMORY)
+public class PutAzureCosmosDBRecord extends AbstractAzureCosmosDBProcessor {
+
+    static final PropertyDescriptor RECORD_READER_FACTORY = new 
PropertyDescriptor.Builder()
+        .name("record-reader")
+        .displayName("Record Reader")
+        .description("Specifies the Controller Service to use for parsing 
incoming data and determining the data's schema")
+        .identifiesControllerService(RecordReaderFactory.class)
+        .required(true)
+        .build();
+
+    static final PropertyDescriptor INSERT_BATCH_SIZE = new 
PropertyDescriptor.Builder()
+        .name("insert-batch-size")
+        .displayName("Insert Batch Size")
+        .description("The number of records to group together for one single 
insert operation against Azure CosmosDB.")
+        .defaultValue("20")
+        .required(false)
+        .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
+        .build();
+
+    private final static Set<Relationship> relationships;
+    private final static List<PropertyDescriptor> propertyDescriptors;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.addAll(descriptors);
+        _propertyDescriptors.add(RECORD_READER_FACTORY);
+        _propertyDescriptors.add(INSERT_BATCH_SIZE);
+        propertyDescriptors = 
Collections.unmodifiableList(_propertyDescriptors);
+
+        final Set<Relationship> _relationships = new HashSet<>();
+        _relationships.add(REL_SUCCESS);
+        _relationships.add(REL_FAILURE);
+        relationships = Collections.unmodifiableSet(_relationships);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return propertyDescriptors;
+    }
+
+    protected void bulkInsert(List<Map<String, Object>> records ) throws 
CosmosClientException{
+        // In the future, this method will be replaced by calling createItems 
API
+        // for example, this.container.createItems(records);
+        // currently, no createItems API available in Azure Cosmos Java SDK
+        for(Map<String, Object> record : records){
+            try {
+                this.container.createItem(record);
+            }catch (ConflictException e) {
+                //  insert with unique id is expeected, but incase test data 
contains data with duplicate id(s).
+                this.container.upsertItem(record);
+            }
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession 
session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final ComponentLog logger = getLogger();
+        final RecordReaderFactory recordParserFactory = 
context.getProperty(RECORD_READER_FACTORY)
+                .asControllerService(RecordReaderFactory.class);
+
+        final String partitionKeyField = 
context.getProperty(PARTITION_KEY).getValue();
+        List<Map<String, Object>> batch = new ArrayList<>();
+        int ceiling = context.getProperty(INSERT_BATCH_SIZE).asInteger();
+        boolean error = false;
+        try (final InputStream inStream = session.read(flowFile);
+             final RecordReader reader = 
recordParserFactory.createRecordReader(flowFile, inStream, getLogger())) {
+
+            RecordSchema schema = reader.getSchema();
+            Record record;
+
+            while((record = reader.nextRecord()) != null) {
+                // Convert each Record to HashMap
+                Map<String, Object> contentMap = (Map<String, Object>) 
DataTypeUtils.convertRecordFieldtoObject(record, 
RecordFieldType.RECORD.getRecordDataType(schema));
+                if(contentMap.containsKey("id")) {
+                    Object val = contentMap.get("id");
+                    if(!(val instanceof String)) {
+                        logger.debug("coverting number id into string...");
+                        contentMap.put("id", contentMap.get("id").toString());
+                    }
+                } else {
+                    contentMap.put("id", flowFile.getAttribute("uuid"));
+                }
+                if(!contentMap.containsKey(partitionKeyField)){
+                    // We set partition key field value with flowfile.uuid if 
not exists
+                    contentMap.put(partitionKeyField, 
flowFile.getAttribute("uuid"));
+                }
+                batch.add(contentMap);
+                if (batch.size() == ceiling) {
+                    bulkInsert(batch);
+                    batch = new ArrayList<>();
+                }
+            }
+            if(batch.size() > 0) {
+                bulkInsert(batch);
+            }
+        } catch (SchemaNotFoundException | IOException | 
MalformedRecordException | CosmosClientException e) {
+            logger.error("PutAzureCosmoDBRecord failed with error:", e);
+            session.transfer(flowFile, REL_FAILURE);
+            error = true;
+        } finally {
+            if (!error) {
+                session.getProvenanceReporter().send(flowFile, 
getURI(context));
+                session.transfer(flowFile, REL_SUCCESS);

Review comment:
       It seems that session.transfer should be called outside try catch block. 
When I moved them inside try and catch block, I am getting the following 
exception. I will keep those two lines in finally block as PutMongoRecord does, 
not inside try catch:
   
   
   [ERROR] 
testOnTriggerWithFlatRecords(org.apache.nifi.processors.azure.cosmos.document.PutAzureCosmosDBRecordTest)
  Time elapsed: 0.256 s  <<< FAILURE!
   java.lang.AssertionError: java.lang.IllegalStateException: 
FlowFile[0,850021980779208.mockFlowFile,0B] already in use for an active 
callback or InputStream created by ProcessSession.read(FlowFile) has not been 
closed
           at 
org.apache.nifi.processors.azure.cosmos.document.PutAzureCosmosDBRecordTest.testOnTriggerWithFlatRecords(PutAzureCosmosDBRecordTest.java:128)
   Caused by: java.lang.IllegalStateException: 
FlowFile[0,850021980779208.mockFlowFile,0B] already in use for an active 
callback or InputStream created by ProcessSession.read(FlowFile) has not been 
closed
   
   [ERROR] 
testOnTriggerWithNestedRecords(org.apache.nifi.processors.azure.cosmos.document.PutAzureCosmosDBRecordTest)
  Time elapsed: 0.013 s  <<< FAILURE!
   java.lang.AssertionError: java.lang.IllegalStateException: 
FlowFile[0,850022330204874.mockFlowFile,0B] already in use for an active 
callback or InputStream created by ProcessSession.read(FlowFile) has not been 
closed
           at 
org.apache.nifi.processors.azure.cosmos.document.PutAzureCosmosDBRecordTest.testOnTriggerWithNestedRecords(PutAzureCosmosDBRecordTest.java:190)
   Caused by: java.lang.IllegalStateException: 
FlowFile[0,850022330204874.mockFlowFile,0B] already in use for an active 
callback or InputStream created by ProcessSession.read(FlowFile) has not been 
closed
   
    




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

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to