This is an automated email from the ASF dual-hosted git repository.

crallen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new e69a2f2  Fix avro parser issue while deserializing sketches  (#6440)
e69a2f2 is described below

commit e69a2f217b378b6a5dd416c791531e9b1cfb208e
Author: Atul Mohan <[email protected]>
AuthorDate: Wed Oct 10 10:53:26 2018 -0500

    Fix avro parser issue while deserializing sketches  (#6440)
    
    * Fix object transform for avro parser
    
    * Remove unwanted space
---
 .../apache/druid/data/input/avro/AvroFlattenerMaker.java   |  3 +--
 .../druid/data/input/AvroHadoopInputRowParserTest.java     |  2 +-
 .../druid/data/input/AvroStreamInputRowParserTest.java     | 14 ++++++++------
 3 files changed, 10 insertions(+), 9 deletions(-)

diff --git 
a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java
 
b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java
index a10b21d..72d7a22 100644
--- 
a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java
+++ 
b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroFlattenerMaker.java
@@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.parsers.ObjectFlatteners;
 
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Set;
@@ -105,7 +104,7 @@ public class AvroFlattenerMaker implements 
ObjectFlatteners.FlattenerMaker<Gener
       if (binaryAsString) {
         return StringUtils.fromUtf8(((ByteBuffer) field).array());
       } else {
-        return Arrays.toString(((ByteBuffer) field).array());
+        return ((ByteBuffer) field).array();
       }
     }
     if (field instanceof Utf8) {
diff --git 
a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java
 
b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java
index 72ddb41..22f0bd7 100644
--- 
a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java
+++ 
b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroHadoopInputRowParserTest.java
@@ -83,7 +83,7 @@ public class AvroHadoopInputRowParserTest
         AvroHadoopInputRowParser.class
     );
     InputRow inputRow = parser2.parseBatch(record).get(0);
-    assertInputRowCorrect(inputRow, DIMENSIONS);
+    assertInputRowCorrect(inputRow, DIMENSIONS, fromPigAvroStorage);
   }
 
 
diff --git 
a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java
 
b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java
index 6301880..a713101 100644
--- 
a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java
+++ 
b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java
@@ -216,7 +216,7 @@ public class AvroStreamInputRowParserTest
 
     InputRow inputRow = 
parser2.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0);
 
-    assertInputRowCorrect(inputRow, DIMENSIONS);
+    assertInputRowCorrect(inputRow, DIMENSIONS, false);
   }
 
   @Test
@@ -257,10 +257,10 @@ public class AvroStreamInputRowParserTest
 
     InputRow inputRow = 
parser2.parseBatch(ByteBuffer.wrap(out.toByteArray())).get(0);
 
-    assertInputRowCorrect(inputRow, DIMENSIONS_SCHEMALESS);
+    assertInputRowCorrect(inputRow, DIMENSIONS_SCHEMALESS, false);
   }
 
-  public static void assertInputRowCorrect(InputRow inputRow, List<String> 
expectedDimensions)
+  public static void assertInputRowCorrect(InputRow inputRow, List<String> 
expectedDimensions, boolean isFromPigAvro)
   {
     assertEquals(expectedDimensions, inputRow.getDimensions());
     assertEquals(1543698L, inputRow.getTimestampFromEpoch());
@@ -307,10 +307,12 @@ public class AvroStreamInputRowParserTest
     );
     assertEquals(Collections.singletonList(SOME_UNION_VALUE), 
inputRow.getDimension("someUnion"));
     assertEquals(Collections.emptyList(), inputRow.getDimension("someNull"));
-    assertEquals(Collections.singletonList(String.valueOf(SOME_FIXED_VALUE)), 
inputRow.getDimension("someFixed"));
+    if (isFromPigAvro) {
+      assertEquals(String.valueOf(SOME_FIXED_VALUE), Arrays.toString((byte[]) 
inputRow.getRaw("someFixed")));
+    }
     assertEquals(
-        Collections.singletonList(Arrays.toString(SOME_BYTES_VALUE.array())),
-        inputRow.getDimension("someBytes")
+        Arrays.toString(SOME_BYTES_VALUE.array()),
+        Arrays.toString((byte[]) (inputRow.getRaw("someBytes")))
     );
     assertEquals(Collections.singletonList(String.valueOf(MyEnum.ENUM1)), 
inputRow.getDimension("someEnum"));
     assertEquals(Collections.singletonList(String.valueOf(SOME_RECORD_VALUE)), 
inputRow.getDimension("someRecord"));


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to