Github user markap14 commented on a diff in the pull request: https://github.com/apache/nifi/pull/459#discussion_r64097658 --- Diff: nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/nifi/processors/avro/ConvertAvroToJSON.java --- @@ -128,49 +141,77 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro // 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; + final String stringSchema = context.getProperty(SCHEMA).getValue(); + final boolean schemaLess = stringSchema != null; + 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>())) { + if (schemaLess) { + if (schema == null) { + schema = new Schema.Parser().parse(stringSchema); + } + try (final InputStream in = new BufferedInputStream(rawIn); + final OutputStream out = new BufferedOutputStream(rawOut)) { + final DatumReader<GenericRecord> reader = new GenericDatumReader<GenericRecord>(schema); + final BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(in, null); + final GenericRecord record = reader.read(null, decoder); + + // Schemaless records are singletons, so both useContainer and wrapSingleRecord + // need to be true before we wrap it with an array + if (useContainer && wrapSingleRecord) { + out.write('['); + } - final GenericData genericData = GenericData.get(); + final byte[] outputBytes = (record == null) ? EMPTY_JSON_OBJECT : record.toString().getBytes(StandardCharsets.UTF_8); --- End diff -- I think we need to keep the use of GenericData here. While record.toString() does in fact convert the Avro object to JSON, the toString() method is not documented as doing so and could change at any time. The GenericData object is documented to convert the object into JSON.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---