You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by mattyb149 <gi...@git.apache.org> on 2018/05/18 04:31:14 UTC

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

GitHub user mattyb149 opened a pull request:

    https://github.com/apache/nifi/pull/2718

    NIFI-5213: Allow AvroReader to process files w embedded schema even when the access strategy is explicit schema

    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [x] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    
    - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [x] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [x] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [x] Have you written or updated unit tests to verify your changes?
    - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
    - [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
    - [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mattyb149/nifi NIFI-5213

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/2718.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2718
    
----
commit 40b9e110abadf802780e2d697aeab336b50094b9
Author: Matthew Burgess <ma...@...>
Date:   2018-05-18T04:29:52Z

    NIFI-5213: Allow AvroReader to process files w embedded schema even when the access strategy is explicit schema

----


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194589636
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    --- End diff --
    
    Sounds about right :) I was playing around with gluing these together and as soon as it "worked" I stopped touching it. Will take a closer look, thanks!


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194590103
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    +        } catch (IOException ioe) {
    +            // Carry on, hopefully a raw Avro file
    +            // Need to be able to re-read the bytes read so far, and the InputStream passed in doesn't support reset. Use the TeeInputStream in
    +            // conjunction with SequenceInputStream to glue the two streams back together for future reading
    +            ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
    +            SequenceInputStream sis = new SequenceInputStream(bais, in);
    +            decoder = DecoderFactory.get().binaryDecoder(sis, null);
    +        }
    +        if (dataFileStream != null) {
    +            // Verify the schemas are the same
    +            Schema embeddedSchema = dataFileStream.getSchema();
    +            if (!embeddedSchema.equals(avroSchema)) {
    +                throw new IOException("Explicit schema does not match embedded schema");
    --- End diff --
    
    I thought schema evolution was supported in other ways such as including optional (possibly missing) fields to support a transition to/from additional/deleted fields, but I admit I don't have my mind wrapped around the whole thing. In this case it was driven by the Avro API, if the file has a schema, there is a much more fluent API to read the records than if it does not. That is not for the case when someone wants to impose a schema on a file that already has a schema; I'm not sure that's a case for schema evolution (i.e. the embedded schema is not correct?), the alternate API is for "raw" Avro files that don't have an embedded schema, and instead need an external one for processing. TBH I don't know how to parse an Avro file that has an embedded schema with their API by imposing an external one. This was the middle ground to allow it as long as the external schema matched the embedded one. Thoughts?


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194575493
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithExplicitSchema.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.avro;
    +
    +import org.apache.avro.Schema;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +
    +public class TestAvroReaderWithExplicitSchema {
    +
    +    @Test
    +    public void testAvroExplicitReaderWithSchemalessFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_schemaless.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    --- End diff --
    
    As-is, this wouldn't test anything that I can tell with that method because all of the errors are caught in that method. Should have some assertions around it IMO.


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194883172
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithExplicitSchema.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.avro;
    +
    +import org.apache.avro.Schema;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +
    +public class TestAvroReaderWithExplicitSchema {
    +
    +    @Test
    +    public void testAvroExplicitReaderWithSchemalessFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_schemaless.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    +    }
    +
    +    @Test
    +    public void testAvroExplicitReaderWithEmbeddedSchemaFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_embed_schema.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    +    }
    +
    +    @Test(expected = IOException.class)
    --- End diff --
    
    No from the constructor. I updated the test to make that more clear, thanks!


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194576777
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    --- End diff --
    
    I don't see where `decoder` is initialized outside of the try block, but it's used in `nextAvroRecord`. Shouldn't you initialize it here so it's guaranteed to be properly initialized when `nextAvroRecord` is called or are you just relying on the first if statement in that method as a null check?


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194575528
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithExplicitSchema.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.avro;
    +
    +import org.apache.avro.Schema;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +
    +public class TestAvroReaderWithExplicitSchema {
    +
    +    @Test
    +    public void testAvroExplicitReaderWithSchemalessFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_schemaless.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    +    }
    +
    +    @Test
    +    public void testAvroExplicitReaderWithEmbeddedSchemaFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_embed_schema.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    --- End diff --
    
    Same deal.


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194575701
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithExplicitSchema.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.avro;
    +
    +import org.apache.avro.Schema;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +
    +public class TestAvroReaderWithExplicitSchema {
    +
    +    @Test
    +    public void testAvroExplicitReaderWithSchemalessFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_schemaless.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    +    }
    +
    +    @Test
    +    public void testAvroExplicitReaderWithEmbeddedSchemaFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_embed_schema.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    +    }
    +
    +    @Test(expected = IOException.class)
    --- End diff --
    
    I assume that comes from this, right?
    
    ```
    if (dataFileStream != null) {
        return dataFileStream.hasNext() ? dataFileStream.next() : null;
    }
    ```


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r189606932
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    +        } catch (IOException ioe) {
    +            // Carry on, hopefully a raw Avro file
    +            // Need to be able to re-read the bytes read so far, and the InputStream passed in doesn't support reset. Use the TeeInputStream in
    +            // conjunction with SequenceInputStream to glue the two streams back together for future reading
    +            ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
    +            SequenceInputStream sis = new SequenceInputStream(bais, in);
    +            decoder = DecoderFactory.get().binaryDecoder(sis, null);
    +        }
    +        if (dataFileStream != null) {
    +            // Verify the schemas are the same
    +            Schema embeddedSchema = dataFileStream.getSchema();
    +            if (!embeddedSchema.equals(avroSchema)) {
    +                throw new IOException("Explicit schema does not match embedded schema");
    --- End diff --
    
    It could be, if SchemaValidationException extended IOException instead of RuntimeException. The processors that end up creating this reader handle IOException but often handle a runtime exception with rollback (I couldn't find anything that explicitly catches SchemaValidationException. Also, although Javadoc is missing for that class, in RecordReader there is Javadoc for a parameter that says:
    
    `throws SchemaValidationException if a Record contains a field that violates the schema and cannot be coerced into the appropriate field type.`
    
    That's not exactly what's going on here, so I thought to keep it an IOException which may be handled by routing to failure rather than a rollback (which would prevent a flow from continuing if an Avro file showed up with an embedded that didn't exactly match the explicit one. It's still best practice to Use Embedded Schema instead of an explicit one, this was just supposed to make it a little easier on the user if they configured it differently.


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194885270
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    --- End diff --
    
    I checked this, it's a bit messy but I wasn't sure how else to do it. `decoder` is only used when parsing a "raw" Avro file, one without an embedded schema. For these changes, there are two scenarios:
    
    1) The input is a raw Avro file. Then line 55 will throw an IOException and `decoder` will be set as it used to be (but this time in the catch). Then in nextAvroRecord it will drop through the `dataFileStream != null` check and use decoder (which is not null).
    2) The input is an Avro file with an embedded schema. Then `dataFileStream` will not be null and nextAvroRecord() will return at line 86, before `decoder` is used.


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194572498
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    +        } catch (IOException ioe) {
    +            // Carry on, hopefully a raw Avro file
    +            // Need to be able to re-read the bytes read so far, and the InputStream passed in doesn't support reset. Use the TeeInputStream in
    +            // conjunction with SequenceInputStream to glue the two streams back together for future reading
    +            ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
    +            SequenceInputStream sis = new SequenceInputStream(bais, in);
    +            decoder = DecoderFactory.get().binaryDecoder(sis, null);
    +        }
    +        if (dataFileStream != null) {
    +            // Verify the schemas are the same
    +            Schema embeddedSchema = dataFileStream.getSchema();
    +            if (!embeddedSchema.equals(avroSchema)) {
    +                throw new IOException("Explicit schema does not match embedded schema");
    --- End diff --
    
    @mattyb149 How does it handle schema evolution in this case? It's possible that the Kafka producer has `Corporate Schema v1` and NiFi is configured with `Corporate Schema v2` and v2 gracefully allows an upgrade from v1 via Avro schema evolution rules. Or am I missing something about that being not really a thing WRT the Record API?


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194589527
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithExplicitSchema.java ---
    @@ -0,0 +1,62 @@
    +/*
    + * 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.avro;
    +
    +import org.apache.avro.Schema;
    +import org.apache.nifi.serialization.SimpleRecordSchema;
    +import org.apache.nifi.serialization.record.RecordSchema;
    +import org.junit.Test;
    +
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.IOException;
    +
    +public class TestAvroReaderWithExplicitSchema {
    +
    +    @Test
    +    public void testAvroExplicitReaderWithSchemalessFile() throws Exception {
    +        File avroFileWithEmbeddedSchema = new File("src/test/resources/avro/avro_schemaless.avro");
    +        FileInputStream fileInputStream = new FileInputStream(avroFileWithEmbeddedSchema);
    +        Schema dataSchema = new Schema.Parser().parse(new File("src/test/resources/avro/avro_schemaless.avsc"));
    +        RecordSchema recordSchema = new SimpleRecordSchema(dataSchema.toString(), AvroTypeUtil.AVRO_SCHEMA_FORMAT, null);
    +
    +        AvroReaderWithExplicitSchema avroReader = new AvroReaderWithExplicitSchema(fileInputStream, recordSchema, dataSchema);
    +        avroReader.nextAvroRecord();
    --- End diff --
    
    Probably, I think I did too much copy-paste in the tests instead of validating the individual things


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by MikeThomsen <gi...@git.apache.org>.
Github user MikeThomsen commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r194695221
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    +        } catch (IOException ioe) {
    +            // Carry on, hopefully a raw Avro file
    +            // Need to be able to re-read the bytes read so far, and the InputStream passed in doesn't support reset. Use the TeeInputStream in
    +            // conjunction with SequenceInputStream to glue the two streams back together for future reading
    +            ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
    +            SequenceInputStream sis = new SequenceInputStream(bais, in);
    +            decoder = DecoderFactory.get().binaryDecoder(sis, null);
    +        }
    +        if (dataFileStream != null) {
    +            // Verify the schemas are the same
    +            Schema embeddedSchema = dataFileStream.getSchema();
    +            if (!embeddedSchema.equals(avroSchema)) {
    +                throw new IOException("Explicit schema does not match embedded schema");
    --- End diff --
    
    I don't think it's a show stopper here. The point of bringing it up was just to see if you had any input on it.


---

[GitHub] nifi pull request #2718: NIFI-5213: Allow AvroReader to process files w embe...

Posted by bdesert <gi...@git.apache.org>.
Github user bdesert commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2718#discussion_r189422180
  
    --- Diff: nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java ---
    @@ -17,33 +17,61 @@
     
     package org.apache.nifi.avro;
     
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
     import java.io.EOFException;
     import java.io.IOException;
     import java.io.InputStream;
    +import java.io.SequenceInputStream;
     
     import org.apache.avro.Schema;
    +import org.apache.avro.file.DataFileStream;
     import org.apache.avro.generic.GenericDatumReader;
     import org.apache.avro.generic.GenericRecord;
     import org.apache.avro.io.BinaryDecoder;
     import org.apache.avro.io.DatumReader;
     import org.apache.avro.io.DecoderFactory;
    -import org.apache.nifi.schema.access.SchemaNotFoundException;
    +import org.apache.commons.io.input.TeeInputStream;
     import org.apache.nifi.serialization.MalformedRecordException;
     import org.apache.nifi.serialization.record.RecordSchema;
     
     public class AvroReaderWithExplicitSchema extends AvroRecordReader {
         private final InputStream in;
         private final RecordSchema recordSchema;
         private final DatumReader<GenericRecord> datumReader;
    -    private final BinaryDecoder decoder;
    +    private BinaryDecoder decoder;
         private GenericRecord genericRecord;
    +    private DataFileStream<GenericRecord> dataFileStream;
     
    -    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException, SchemaNotFoundException {
    +    public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema, final Schema avroSchema) throws IOException {
             this.in = in;
             this.recordSchema = recordSchema;
     
    -        datumReader = new GenericDatumReader<GenericRecord>(avroSchema);
    -        decoder = DecoderFactory.get().binaryDecoder(in, null);
    +        datumReader = new GenericDatumReader<>(avroSchema);
    +        ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +        TeeInputStream teeInputStream = new TeeInputStream(in, baos);
    +        // Try to parse as a DataFileStream, if it works, glue the streams back together and delegate calls to the DataFileStream
    +        try {
    +            dataFileStream = new DataFileStream<>(teeInputStream, new GenericDatumReader<>());
    +        } catch (IOException ioe) {
    +            // Carry on, hopefully a raw Avro file
    +            // Need to be able to re-read the bytes read so far, and the InputStream passed in doesn't support reset. Use the TeeInputStream in
    +            // conjunction with SequenceInputStream to glue the two streams back together for future reading
    +            ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
    +            SequenceInputStream sis = new SequenceInputStream(bais, in);
    +            decoder = DecoderFactory.get().binaryDecoder(sis, null);
    +        }
    +        if (dataFileStream != null) {
    +            // Verify the schemas are the same
    +            Schema embeddedSchema = dataFileStream.getSchema();
    +            if (!embeddedSchema.equals(avroSchema)) {
    +                throw new IOException("Explicit schema does not match embedded schema");
    --- End diff --
    
    Would it be better to throw SchemaValidationException - makes more sense than IOException?


---