Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2180#discussion_r160449596
  
    --- Diff: 
nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/RunMongoAggregation.java
 ---
    @@ -0,0 +1,180 @@
    +/*
    + * 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.mongodb;
    +
    +import com.mongodb.BasicDBObject;
    +import com.mongodb.client.AggregateIterable;
    +import com.mongodb.client.MongoCollection;
    +import com.mongodb.client.MongoCursor;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +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.bson.conversions.Bson;
    +import com.fasterxml.jackson.databind.ObjectMapper;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +@Tags({"mongo", "aggregation", "aggregate"})
    +@CapabilityDescription("A processor that runs an aggregation query at 
user-defined intervals.")
    +@InputRequirement(InputRequirement.Requirement.INPUT_ALLOWED)
    +public class RunMongoAggregation extends AbstractMongoProcessor {
    +
    +    private final static Set<Relationship> relationships;
    +    private final static List<PropertyDescriptor> propertyDescriptors;
    +
    +    static final Relationship REL_ORIGINAL = new Relationship.Builder()
    +            .description("The input flowfile gets sent to this 
relationship when it succeeds.")
    +            .name("Original")
    +            .build();
    +
    +
    +    static final List<Bson> buildAggregationQuery(String query) throws 
IOException {
    +        List<Bson> result = new ArrayList<>();
    +
    +        ObjectMapper mapper = new ObjectMapper();
    +        List<Map> values = mapper.readValue(query, List.class);
    +        for (Map val : values) {
    +            result.add(new BasicDBObject(val));
    +        }
    +
    +        return result;
    +    }
    +
    +    public static final Validator AGG_VALIDATOR = (subject, value, 
context) -> {
    +        final ValidationResult.Builder builder = new 
ValidationResult.Builder();
    +        builder.subject(subject).input(value);
    +
    +        if (context.isExpressionLanguageSupported(subject) && 
context.isExpressionLanguagePresent(value)) {
    +            return builder.valid(true).explanation("Contains Expression 
Language").build();
    +        }
    +
    +        String reason = null;
    +        try {
    +            buildAggregationQuery(value);
    +        } catch (final RuntimeException | IOException e) {
    +            reason = e.getLocalizedMessage();
    +        }
    +
    +        return builder.explanation(reason).valid(reason == null).build();
    +    };
    +
    +    static final PropertyDescriptor QUERY = new 
PropertyDescriptor.Builder()
    +            .name("mongo-agg-query")
    +            .displayName("Query")
    +            .expressionLanguageSupported(true)
    +            .description("The aggregation query to be executed.")
    +            .required(true)
    +            .addValidator(AGG_VALIDATOR)
    +            .build();
    +
    +    static {
    +        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
    +        _propertyDescriptors.addAll(descriptors);
    +        _propertyDescriptors.add(QUERY);
    +        _propertyDescriptors.add(BATCH_SIZE);
    +        _propertyDescriptors.add(RESULTS_PER_FLOWFILE);
    +        _propertyDescriptors.add(SSL_CONTEXT_SERVICE);
    +        _propertyDescriptors.add(CLIENT_AUTH);
    +        propertyDescriptors = 
Collections.unmodifiableList(_propertyDescriptors);
    +
    +        final Set<Relationship> _relationships = new HashSet<>();
    +        _relationships.add(REL_SUCCESS);
    +        _relationships.add(REL_ORIGINAL);
    +        relationships = Collections.unmodifiableSet(_relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> 
getSupportedPropertyDescriptors() {
    +        return propertyDescriptors;
    +    }
    +
    +    static String buildBatch(List batch) {
    +        ObjectMapper mapper = new ObjectMapper();
    +        String retVal;
    +        try {
    +            retVal = mapper.writeValueAsString(batch.size() > 1 ? batch : 
batch.get(0));
    +        } catch (Exception e) {
    +            retVal = null;
    +        }
    +
    +        return retVal;
    +    }
    +
    +    @Override
    +    public void onTrigger(ProcessContext context, ProcessSession session) 
throws ProcessException {
    +        FlowFile flowFile = session.get();
    +
    +        String query = 
context.getProperty(QUERY).evaluateAttributeExpressions(flowFile).getValue();
    +        Integer batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        Integer resultsPerFlowfile = 
context.getProperty(RESULTS_PER_FLOWFILE).asInteger();
    +
    +        MongoCollection collection = getCollection(context);
    +        MongoCursor iter = null;
    +
    +        try {
    +            List<Bson> aggQuery = buildAggregationQuery(query);
    +            AggregateIterable it = collection.aggregate(aggQuery);
    +            it.batchSize(batchSize != null ? batchSize : 1);
    +
    +            iter = it.iterator();
    +            List batch = new ArrayList();
    +
    +            while (iter.hasNext()) {
    +                batch.add(iter.next());
    +                if (batch.size() == resultsPerFlowfile) {
    +                    writeBatch(buildBatch(batch), context, session);
    +                    batch = new ArrayList();
    +                }
    +            }
    +
    +            if (batch.size() > 0) {
    +                writeBatch(buildBatch(batch), context, session);
    +            }
    +
    +            session.transfer(flowFile, REL_ORIGINAL);
    +        } catch (IOException i) {
    +            context.yield();
    --- End diff --
    
    If there is an input flow file, an error should be logged, and the flow 
file transferred to a different relationship, rather than rollback. Depending 
on the kinds of errors that can occur, you may want a "retry" relationship 
(along with possible penalization) if the same flow file might succeed later 
(due to connection issues, e.g.) or "failure" if the flow file will always fail 
due to bad data, format, etc.
    
    If there is no input flow file, a yield and rollback should be ok I think.


---

Reply via email to