Fix broken GenericRecord.Arrays in avro parsing (#5115)
* Fix broken GenericRecord.Arrays in avro parsing
* Add whitespace for linting
diff --git a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/GenericRecordAsMap.java b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/GenericRecordAsMap.java
index 97e3c8d..61286fb 100644
--- a/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/GenericRecordAsMap.java
+++ b/extensions-core/avro-extensions/src/main/java/io/druid/data/input/avro/GenericRecordAsMap.java
@@ -28,10 +28,10 @@
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Arrays;
-import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
+import java.util.Collection;
public class GenericRecordAsMap implements Map<String, Object>
{
@@ -45,7 +45,10 @@
@Override
public String apply(Object input)
{
- return String.valueOf(((GenericRecord) input).get(0));
+ if (input instanceof GenericRecord) {
+ return String.valueOf(((GenericRecord) input).get(0));
+ }
+ return String.valueOf(input);
}
};
@@ -102,7 +105,7 @@
public Object get(Object key)
{
Object field = record.get(key.toString());
- if (fromPigAvroStorage && field instanceof GenericData.Array) {
+ if (field instanceof GenericData.Array) {
return Lists.transform((List) field, PIG_AVRO_STORAGE_ARRAY_TO_STRING_INCLUDING_NULL);
}
if (field instanceof ByteBuffer) {