[ 
https://issues.apache.org/jira/browse/NIFI-1234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15036982#comment-15036982
 ] 

ASF GitHub Bot commented on NIFI-1234:
--------------------------------------

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

    https://github.com/apache/nifi/pull/136#discussion_r46500474
  
    --- Diff: 
nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ConvertAvroToJSON.java
 ---
    @@ -116,51 +124,59 @@ public void onTrigger(ProcessContext context, 
ProcessSession session) throws Pro
             }
     
             final String containerOption = 
context.getProperty(CONTAINER_OPTIONS).getValue();
    +        final boolean useContainer = 
containerOption.equals(CONTAINER_ARRAY);
    +        // Wrap a single record (inclusive of no records) only when a 
container is being used
    +        final boolean wrapSingleRecord = 
context.getProperty(WRAP_SINGLE_RECORD).asBoolean() && useContainer;
     
             try {
                 flowFile = session.write(flowFile, new StreamCallback() {
                     @Override
                     public void process(final InputStream rawIn, final 
OutputStream rawOut) throws IOException {
                         try (final InputStream in = new 
BufferedInputStream(rawIn);
    -
    -                        final OutputStream out = new 
BufferedOutputStream(rawOut);
    -                        final DataFileStream<GenericRecord> reader = new 
DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
    +                         final OutputStream out = new 
BufferedOutputStream(rawOut);
    +                         final DataFileStream<GenericRecord> reader = new 
DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
     
                             final GenericData genericData = GenericData.get();
     
    -                        if (reader.hasNext() == false ) {
    -                            out.write(EMPTY_JSON_OBJECT);
    -                            return;
    +                        int recordCount = 0;
    +                        GenericRecord currRecord = null;
    +                        if (reader.hasNext()) {
    +                            currRecord = reader.next();
    +                            recordCount++;
                             }
    -                        int recordCount = 1;
    -                        GenericRecord reuse = reader.next();
    -                        // Only open container if more than one record
    -                        if(reader.hasNext() && 
containerOption.equals(CONTAINER_ARRAY)){
    +
    +                        // Open container if desired output is an array 
format and there are are multiple records or
    +                        // if configured to wrap single record
    +                        if (reader.hasNext() && useContainer || 
wrapSingleRecord) {
                                 out.write('[');
                             }
    -                        
out.write(genericData.toString(reuse).getBytes(StandardCharsets.UTF_8));
    +
    +                        // Determine the initial output record, inclusive 
if we should have an empty set of Avro records
    +                        final byte[] outputBytes = (currRecord == null) ? 
EMPTY_JSON_OBJECT : 
genericData.toString(currRecord).getBytes(StandardCharsets.UTF_8);
    +                        out.write(outputBytes);
     
                             while (reader.hasNext()) {
    -                            if (containerOption.equals(CONTAINER_ARRAY)) {
    +                            if (useContainer) {
                                     out.write(',');
                                 } else {
                                     out.write('\n');
                                 }
     
    -                            reuse = reader.next(reuse);
    -                            
out.write(genericData.toString(reuse).getBytes(StandardCharsets.UTF_8));
    +                            currRecord = reader.next(currRecord);
    +                            
out.write(genericData.toString(currRecord).getBytes(StandardCharsets.UTF_8));
                                 recordCount++;
                             }
     
    -                        // Only close container if more than one record
    -                        if (recordCount > 1 && 
containerOption.equals(CONTAINER_ARRAY)) {
    +                        // Close container if desired output is an array 
format and there are multiple records or if
    +                        // configured to wrap a single record
    +                        if (recordCount > 1 && useContainer || 
wrapSingleRecord) {
    --- End diff --
    
    I think it is correct.
    
    useContainer only applies to the case where there is more than one record.  
Those cases where there are zero or one are driven by wrapSingleRecord which 
incorporates whether or not the container should be used.  Happy to rework this 
(and the grouping) if I can get some better clarity.   Could certainly 
alleviate the line where it is defined earlier.
    
    wrapSingleRecord is defined as:
    final boolean wrapSingleRecord = 
context.getProperty(WRAP_SINGLE_RECORD).asBoolean() && useContainer;



> Inconsistent container behavior in ConvertAvroToJSON
> ----------------------------------------------------
>
>                 Key: NIFI-1234
>                 URL: https://issues.apache.org/jira/browse/NIFI-1234
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>    Affects Versions: 0.4.0
>            Reporter: Douglas Doughty
>            Assignee: Aldrin Piri
>            Priority: Minor
>
> ConvertAvroToJSON has 2 container options: array and none.
> When set to 'array', usage dictates an "array of objects" should be produced. 
>  But, if only a single Avro record is passed in, an array is not created--a 
> JSON object is created.
> I believe it should be wrapped in an array.  This caused errors when chained 
> with SplitJSON (which expects an array).
> Use case scenario:  ExecuteSQL returns an unknown number of results -> 
> ConvertAvroToJSON -> SplitJSON.  If ExecuteSQL returns 1 result, then the 
> operation fails.
> Work around: Set up a failure relationship between SplitJSON and the next 
> processor--the object will be forwarded, but an error will be logged on the 
> SplitJSON processor.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to