This commit is contained in:
Mark Payne 2015-11-11 08:16:38 -05:00
commit 6c510fae80
3 changed files with 330 additions and 296 deletions

View File

@ -1,164 +1,169 @@
/* /*
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership. * this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0 * 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 not use this file except in compliance with
* the License. You may obtain a copy of the License at * the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.nifi.processors.avro; package org.apache.nifi.processors.avro;
import java.io.BufferedInputStream; import java.io.BufferedInputStream;
import java.io.BufferedOutputStream; import java.io.BufferedOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import org.apache.avro.file.DataFileStream; import org.apache.avro.file.DataFileStream;
import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.nifi.annotation.behavior.InputRequirement; import org.apache.nifi.annotation.behavior.InputRequirement;
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SideEffectFree;
import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.behavior.SupportsBatching;
import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttribute;
import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.CapabilityDescription;
import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyDescriptor;
import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.flowfile.FlowFile;
import org.apache.nifi.flowfile.attributes.CoreAttributes; import org.apache.nifi.flowfile.attributes.CoreAttributes;
import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.AbstractProcessor;
import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessContext;
import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSession;
import org.apache.nifi.processor.ProcessorInitializationContext; import org.apache.nifi.processor.ProcessorInitializationContext;
import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.Relationship;
import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.exception.ProcessException;
import org.apache.nifi.processor.io.StreamCallback; import org.apache.nifi.processor.io.StreamCallback;
@SideEffectFree @SideEffectFree
@SupportsBatching @SupportsBatching
@InputRequirement(Requirement.INPUT_REQUIRED) @InputRequirement(Requirement.INPUT_REQUIRED)
@CapabilityDescription("Converts a Binary Avro record into a JSON object. This processor provides a direct mapping of an Avro field to a JSON field, such " @CapabilityDescription("Converts a Binary Avro record into a JSON object. This processor provides a direct mapping of an Avro field to a JSON field, such "
+ "that the resulting JSON will have the same hierarchical structure as the Avro document. Note that the Avro schema information will be lost, as this " + "that the resulting JSON will have the same hierarchical structure as the Avro document. Note that the Avro schema information will be lost, as this "
+ "is not a translation from binary Avro to JSON formatted Avro. The output JSON is encoded the UTF-8 encoding. If an incoming FlowFile contains a stream of " + "is not a translation from binary Avro to JSON formatted Avro. The output JSON is encoded the UTF-8 encoding. If an incoming FlowFile contains a stream of "
+ "multiple Avro records, the resultant FlowFile will contain a JSON Array containing all of the Avro records or a sequence of JSON Objects") + "multiple Avro records, the resultant FlowFile will contain a JSON Array containing all of the Avro records or a sequence of JSON Objects")
@WritesAttribute(attribute = "mime.type", description = "Sets the mime type to application/json") @WritesAttribute(attribute = "mime.type", description = "Sets the mime type to application/json")
public class ConvertAvroToJSON extends AbstractProcessor { public class ConvertAvroToJSON extends AbstractProcessor {
protected static final String CONTAINER_ARRAY = "array"; protected static final String CONTAINER_ARRAY = "array";
protected static final String CONTAINER_NONE = "none"; protected static final String CONTAINER_NONE = "none";
static final PropertyDescriptor CONTAINER_OPTIONS = new PropertyDescriptor.Builder() private static final byte [] EMPTY_JSON_OBJECT = "{}".getBytes(StandardCharsets.UTF_8);
.name("JSON container options")
.description("Determines how stream of records is exposed: either as a sequence of single Objects (" + CONTAINER_NONE static final PropertyDescriptor CONTAINER_OPTIONS = new PropertyDescriptor.Builder()
+ ") (i.e. writing every Object to a new line), or as an array of Objects (" + CONTAINER_ARRAY + ").") .name("JSON container options")
.allowableValues(CONTAINER_NONE, CONTAINER_ARRAY) .description("Determines how stream of records is exposed: either as a sequence of single Objects (" + CONTAINER_NONE
.required(true) + ") (i.e. writing every Object to a new line), or as an array of Objects (" + CONTAINER_ARRAY + ").")
.defaultValue(CONTAINER_ARRAY) .allowableValues(CONTAINER_NONE, CONTAINER_ARRAY)
.build(); .required(true)
.defaultValue(CONTAINER_ARRAY)
static final Relationship REL_SUCCESS = new Relationship.Builder() .build();
.name("success")
.description("A FlowFile is routed to this relationship after it has been converted to JSON") static final Relationship REL_SUCCESS = new Relationship.Builder()
.build(); .name("success")
static final Relationship REL_FAILURE = new Relationship.Builder() .description("A FlowFile is routed to this relationship after it has been converted to JSON")
.name("failure") .build();
.description("A FlowFile is routed to this relationship if it cannot be parsed as Avro or cannot be converted to JSON for any reason") static final Relationship REL_FAILURE = new Relationship.Builder()
.build(); .name("failure")
.description("A FlowFile is routed to this relationship if it cannot be parsed as Avro or cannot be converted to JSON for any reason")
.build();
private List<PropertyDescriptor> properties;
@Override private List<PropertyDescriptor> properties;
protected void init(ProcessorInitializationContext context) {
super.init(context); @Override
protected void init(ProcessorInitializationContext context) {
final List<PropertyDescriptor> properties = new ArrayList<>(); super.init(context);
properties.add(CONTAINER_OPTIONS);
this.properties = Collections.unmodifiableList(properties); final List<PropertyDescriptor> properties = new ArrayList<>();
} properties.add(CONTAINER_OPTIONS);
this.properties = Collections.unmodifiableList(properties);
@Override }
protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties; @Override
} protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
return properties;
@Override }
public Set<Relationship> getRelationships() {
final Set<Relationship> rels = new HashSet<>(); @Override
rels.add(REL_SUCCESS); public Set<Relationship> getRelationships() {
rels.add(REL_FAILURE); final Set<Relationship> rels = new HashSet<>();
return rels; rels.add(REL_SUCCESS);
} rels.add(REL_FAILURE);
return rels;
@Override }
public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
FlowFile flowFile = session.get(); @Override
if (flowFile == null) { public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException {
return; FlowFile flowFile = session.get();
} if (flowFile == null) {
return;
final String containerOption = context.getProperty(CONTAINER_OPTIONS).getValue(); }
try { final String containerOption = context.getProperty(CONTAINER_OPTIONS).getValue();
flowFile = session.write(flowFile, new StreamCallback() {
@Override try {
public void process(final InputStream rawIn, final OutputStream rawOut) throws IOException { flowFile = session.write(flowFile, new StreamCallback() {
try (final InputStream in = new BufferedInputStream(rawIn); @Override
public void process(final InputStream rawIn, final OutputStream rawOut) throws IOException {
final OutputStream out = new BufferedOutputStream(rawOut); try (final InputStream in = new BufferedInputStream(rawIn);
final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
final OutputStream out = new BufferedOutputStream(rawOut);
final GenericData genericData = GenericData.get(); final DataFileStream<GenericRecord> reader = new DataFileStream<>(in, new GenericDatumReader<GenericRecord>())) {
GenericRecord record = reader.next();
final String json = genericData.toString(record); final GenericData genericData = GenericData.get();
int recordCount = 0; if (reader.hasNext() == false ) {
if (reader.hasNext() && containerOption.equals(CONTAINER_ARRAY)) { out.write(EMPTY_JSON_OBJECT);
out.write('['); return;
} }
int recordCount = 1;
out.write(json.getBytes(StandardCharsets.UTF_8)); GenericRecord reuse = reader.next();
recordCount++; // Only open container if more than one record
if(reader.hasNext() && containerOption.equals(CONTAINER_ARRAY)){
while (reader.hasNext()) { out.write('[');
if (containerOption.equals(CONTAINER_ARRAY)) { }
out.write(','); out.write(genericData.toString(reuse).getBytes(StandardCharsets.UTF_8));
} else {
out.write('\n'); while (reader.hasNext()) {
} if (containerOption.equals(CONTAINER_ARRAY)) {
out.write(',');
final GenericRecord nextRecord = reader.next(record); } else {
out.write(genericData.toString(nextRecord).getBytes(StandardCharsets.UTF_8)); out.write('\n');
recordCount++; }
}
reuse = reader.next(reuse);
if (recordCount > 1 && containerOption.equals(CONTAINER_ARRAY)) { out.write(genericData.toString(reuse).getBytes(StandardCharsets.UTF_8));
out.write(']'); recordCount++;
} }
}
} // Only close container if more than one record
}); if (recordCount > 1 && containerOption.equals(CONTAINER_ARRAY)) {
} catch (final ProcessException pe) { out.write(']');
getLogger().error("Failed to convert {} from Avro to JSON due to {}; transferring to failure", new Object[] {flowFile, pe}); }
session.transfer(flowFile, REL_FAILURE); }
return; }
} });
} catch (final ProcessException pe) {
flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), "application/json"); getLogger().error("Failed to convert {} from Avro to JSON due to {}; transferring to failure", new Object[] {flowFile, pe});
session.transfer(flowFile, REL_SUCCESS); session.transfer(flowFile, REL_FAILURE);
} return;
} }
flowFile = session.putAttribute(flowFile, CoreAttributes.MIME_TYPE.key(), "application/json");
session.transfer(flowFile, REL_SUCCESS);
}
}

View File

@ -218,7 +218,7 @@ public class SplitAvro extends AbstractProcessor {
/** /**
* Splits the incoming Avro datafile into batches of records by reading and de-serializing each record. * Splits the incoming Avro datafile into batches of records by reading and de-serializing each record.
*/ */
private class RecordSplitter implements Splitter { static private class RecordSplitter implements Splitter {
private final int splitSize; private final int splitSize;
private final boolean transferMetadata; private final boolean transferMetadata;
@ -300,7 +300,7 @@ public class SplitAvro extends AbstractProcessor {
/** /**
* Writes a binary Avro Datafile to the OutputStream. * Writes a binary Avro Datafile to the OutputStream.
*/ */
private class DatafileSplitWriter implements SplitWriter { static private class DatafileSplitWriter implements SplitWriter {
private final boolean transferMetadata; private final boolean transferMetadata;
private DataFileWriter<GenericRecord> writer; private DataFileWriter<GenericRecord> writer;
@ -344,7 +344,7 @@ public class SplitAvro extends AbstractProcessor {
/** /**
* Writes bare Avro records to the OutputStream. * Writes bare Avro records to the OutputStream.
*/ */
private class BareRecordSplitWriter implements SplitWriter { static private class BareRecordSplitWriter implements SplitWriter {
private Encoder encoder; private Encoder encoder;
private DatumWriter<GenericRecord> writer; private DatumWriter<GenericRecord> writer;

View File

@ -1,129 +1,158 @@
/* /*
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership. * this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0 * 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 not use this file except in compliance with
* the License. You may obtain a copy of the License at * the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, software * Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, * distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.nifi.processors.avro; package org.apache.nifi.processors.avro;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.avro.file.DataFileWriter; import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericRecord;
import org.apache.avro.io.DatumWriter; import org.apache.avro.io.DatumWriter;
import org.apache.nifi.stream.io.ByteArrayOutputStream; import org.apache.nifi.stream.io.ByteArrayOutputStream;
import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.MockFlowFile;
import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunner;
import org.apache.nifi.util.TestRunners; import org.apache.nifi.util.TestRunners;
import org.junit.Test; import org.junit.Test;
public class TestConvertAvroToJSON { public class TestConvertAvroToJSON {
@Test @Test
public void testSingleAvroMessage() throws IOException { public void testSingleAvroMessage() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON()); final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON());
final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc")); final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc"));
final GenericRecord user1 = new GenericData.Record(schema); final GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa"); user1.put("name", "Alyssa");
user1.put("favorite_number", 256); user1.put("favorite_number", 256);
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema); final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = AvroTestUtil.serializeAvroRecord(schema, datumWriter, user1); final ByteArrayOutputStream out1 = AvroTestUtil.serializeAvroRecord(schema, datumWriter, user1);
runner.enqueue(out1.toByteArray()); runner.enqueue(out1.toByteArray());
runner.run(); runner.run();
runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_SUCCESS, 1); runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ConvertAvroToJSON.REL_SUCCESS).get(0); final MockFlowFile out = runner.getFlowFilesForRelationship(ConvertAvroToJSON.REL_SUCCESS).get(0);
out.assertContentEquals("{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null}"); out.assertContentEquals("{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null}");
} }
@Test @Test
public void testMultipleAvroMessages() throws IOException { public void testMultipleAvroMessages() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON()); final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON());
final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc")); final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc"));
runner.setProperty(ConvertAvroToJSON.CONTAINER_OPTIONS, ConvertAvroToJSON.CONTAINER_ARRAY); runner.setProperty(ConvertAvroToJSON.CONTAINER_OPTIONS, ConvertAvroToJSON.CONTAINER_ARRAY);
final GenericRecord user1 = new GenericData.Record(schema); final GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa"); user1.put("name", "Alyssa");
user1.put("favorite_number", 256); user1.put("favorite_number", 256);
final GenericRecord user2 = new GenericData.Record(schema); final GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "George"); user2.put("name", "George");
user2.put("favorite_number", 1024); user2.put("favorite_number", 1024);
user2.put("favorite_color", "red"); user2.put("favorite_color", "red");
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema); final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = AvroTestUtil.serializeAvroRecord(schema, datumWriter, user1, user2); final ByteArrayOutputStream out1 = AvroTestUtil.serializeAvroRecord(schema, datumWriter, user1, user2);
runner.enqueue(out1.toByteArray()); runner.enqueue(out1.toByteArray());
runner.run(); runner.run();
runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_SUCCESS, 1); runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ConvertAvroToJSON.REL_SUCCESS).get(0); final MockFlowFile out = runner.getFlowFilesForRelationship(ConvertAvroToJSON.REL_SUCCESS).get(0);
out.assertContentEquals("[{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null},{\"name\": \"George\", \"favorite_number\": 1024, \"favorite_color\": \"red\"}]"); out.assertContentEquals("[{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null},{\"name\": \"George\", \"favorite_number\": 1024, \"favorite_color\": \"red\"}]");
} }
@Test @Test
public void testNonJsonHandledProperly() throws IOException { public void testNonJsonHandledProperly() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON()); final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON());
runner.enqueue("hello".getBytes()); runner.enqueue("hello".getBytes());
runner.run(); runner.run();
runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_FAILURE, 1); runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_FAILURE, 1);
} }
private ByteArrayOutputStream serializeAvroRecord(final Schema schema, final DatumWriter<GenericRecord> datumWriter, final GenericRecord... users) throws IOException { private ByteArrayOutputStream serializeAvroRecord(final Schema schema, final DatumWriter<GenericRecord> datumWriter, final GenericRecord... users) throws IOException {
ByteArrayOutputStream out = new ByteArrayOutputStream(); ByteArrayOutputStream out = new ByteArrayOutputStream();
DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter); DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<GenericRecord>(datumWriter);
dataFileWriter.create(schema, out); dataFileWriter.create(schema, out);
for (final GenericRecord user : users) { for (final GenericRecord user : users) {
dataFileWriter.append(user); dataFileWriter.append(user);
} }
dataFileWriter.close(); dataFileWriter.close();
return out; return out;
} }
@Test @Test
public void testMultipleAvroMessagesContainerNone() throws IOException { public void testMultipleAvroMessagesContainerNone() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON()); final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON());
final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc")); final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc"));
runner.setProperty(ConvertAvroToJSON.CONTAINER_OPTIONS, ConvertAvroToJSON.CONTAINER_NONE); runner.setProperty(ConvertAvroToJSON.CONTAINER_OPTIONS, ConvertAvroToJSON.CONTAINER_NONE);
final GenericRecord user1 = new GenericData.Record(schema); final GenericRecord user1 = new GenericData.Record(schema);
user1.put("name", "Alyssa"); user1.put("name", "Alyssa");
user1.put("favorite_number", 256); user1.put("favorite_number", 256);
final GenericRecord user2 = new GenericData.Record(schema); final GenericRecord user2 = new GenericData.Record(schema);
user2.put("name", "George"); user2.put("name", "George");
user2.put("favorite_number", 1024); user2.put("favorite_number", 1024);
user2.put("favorite_color", "red"); user2.put("favorite_color", "red");
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema); final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = serializeAvroRecord(schema, datumWriter, user1, user2); final ByteArrayOutputStream out1 = serializeAvroRecord(schema, datumWriter, user1, user2);
runner.enqueue(out1.toByteArray()); runner.enqueue(out1.toByteArray());
runner.run(); runner.run();
runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_SUCCESS, 1); runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ConvertAvroToJSON.REL_SUCCESS).get(0); final MockFlowFile out = runner.getFlowFilesForRelationship(ConvertAvroToJSON.REL_SUCCESS).get(0);
out.assertContentEquals("{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null}\n{\"name\": \"George\", \"favorite_number\": 1024, \"favorite_color\": \"red\"}"); out.assertContentEquals("{\"name\": \"Alyssa\", \"favorite_number\": 256, \"favorite_color\": null}\n{\"name\": \"George\", \"favorite_number\": 1024, \"favorite_color\": \"red\"}");
} }
}
@Test
public void testEmptyFlowFile() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON());
runner.enqueue(new byte[]{});
runner.run();
runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_FAILURE, 1);
}
@Test
public void testZeroRecords() throws IOException {
final TestRunner runner = TestRunners.newTestRunner(new ConvertAvroToJSON());
final Schema schema = new Schema.Parser().parse(new File("src/test/resources/user.avsc"));
final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
final ByteArrayOutputStream out1 = serializeAvroRecord(schema, datumWriter);
runner.enqueue(out1.toByteArray());
runner.run();
runner.assertAllFlowFilesTransferred(ConvertAvroToJSON.REL_SUCCESS, 1);
final MockFlowFile out = runner.getFlowFilesForRelationship(ConvertAvroToJSON.REL_SUCCESS).get(0);
out.assertContentEquals("{}");
}
}