Github user ijokarumawak commented on a diff in the pull request:
https://github.com/apache/nifi/pull/1788#discussion_r116895091
--- Diff:
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-content-viewer/src/main/java/org/apache/nifi/web/StandardContentViewerController.java
---
@@ -103,11 +108,27 @@ protected void doGet(HttpServletRequest request,
HttpServletResponse response) t
} else if ("application/avro-binary".equals(contentType)
|| "avro/binary".equals(contentType) ||
"application/avro+binary".equals(contentType)) {
final StringBuilder sb = new StringBuilder();
sb.append("[");
- final DatumReader<GenericData.Record> datumReader =
new GenericDatumReader<>();
+ // Use Avro conversions to display logical type values
in human readable way.
+ final GenericData genericData = new GenericData(){
+ @Override
+ protected void toString(Object datum,
StringBuilder buffer) {
+ // Since these types are not quoted and
produce a malformed JSON string, quote it here.
+ if (datum instanceof LocalDate || datum
instanceof LocalTime || datum instanceof DateTime) {
+
buffer.append("\"").append(datum).append("\"");
+ return;
+ }
+ super.toString(datum, buffer);
+ }
+ };
+ genericData.addLogicalTypeConversion(new
Conversions.DecimalConversion());
+ genericData.addLogicalTypeConversion(new
TimeConversions.DateConversion());
+ genericData.addLogicalTypeConversion(new
TimeConversions.TimeConversion());
+ genericData.addLogicalTypeConversion(new
TimeConversions.TimestampConversion());
+ final DatumReader<GenericData.Record> datumReader =
new GenericDatumReader<>(null, null, genericData);
try (final DataFileStream<GenericData.Record>
dataFileReader = new DataFileStream<>(content.getContentStream(), datumReader))
{
while (dataFileReader.hasNext()) {
final GenericData.Record record =
dataFileReader.next();
- final String formattedRecord =
record.toString();
+ final String formattedRecord =
genericData.toString(record);
--- End diff --
Thanks for your suggestion. I took a look at #1179 and customized
[StandardJsonEncoder](https://github.com/mattyb149/nifi/blob/879efb7572888cf4bd863e1938a3a798bd87ff0c/nifi-nar-bundles/nifi-avro-bundle/nifi-avro-processors/src/main/java/org/apache/avro/io/StandardJsonEncoder.java).
However, it's not clear to me what benefit it provides. GenericData.toString
doesn't add extra layer for union objects as described in NIFI-969. Even with a
union value, GenericData.toString seems to be working as expected (at least for
me).
Since I don't know what should be fixed by applying the
StandardJsonEncoder, I prefer keep using `.toString` unless there is a clear
reason.
---
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 [email protected] or file a JIRA ticket
with INFRA.
---