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

gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/orc.git


The following commit(s) were added to refs/heads/master by this push:
     new 8f802a6  ORC-452: Support converting JSON object to MAP type
8f802a6 is described below

commit 8f802a67a87a5163d7b759de4176a7b4eae2f7a1
Author: Quanlong Huang <[email protected]>
AuthorDate: Fri Jan 18 04:38:21 2019 +0800

    ORC-452: Support converting JSON object to MAP type
    
    Fixes #354
    
    Signed-off-by: Gang Wu <[email protected]>
---
 .../org/apache/orc/tools/convert/JsonReader.java   | 39 ++++++++++++++++++++++
 1 file changed, 39 insertions(+)

diff --git a/java/tools/src/java/org/apache/orc/tools/convert/JsonReader.java 
b/java/tools/src/java/org/apache/orc/tools/convert/JsonReader.java
index 16b5d64..c020d11 100644
--- a/java/tools/src/java/org/apache/orc/tools/convert/JsonReader.java
+++ b/java/tools/src/java/org/apache/orc/tools/convert/JsonReader.java
@@ -20,6 +20,7 @@ package org.apache.orc.tools.convert;
 import com.google.gson.JsonArray;
 import com.google.gson.JsonElement;
 import com.google.gson.JsonObject;
+import com.google.gson.JsonPrimitive;
 import com.google.gson.JsonStreamParser;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -33,6 +34,7 @@ import 
org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.MapColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -52,6 +54,7 @@ import java.nio.charset.StandardCharsets;
 import java.sql.Timestamp;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.zip.GZIPInputStream;
 
 public class JsonReader implements RecordReader {
@@ -222,6 +225,40 @@ public class JsonReader implements RecordReader {
     }
   }
 
+  static class MapColumnConverter implements JsonConverter {
+    private JsonConverter keyConverter;
+    private JsonConverter valueConverter;
+
+    public MapColumnConverter(TypeDescription schema) {
+      TypeDescription keyType = schema.getChildren().get(0);
+      if (keyType.getCategory() != TypeDescription.Category.STRING)
+        throw new IllegalArgumentException("JSON can only support MAP key in 
STRING type: " + schema);
+      keyConverter = createConverter(keyType);
+      valueConverter = createConverter(schema.getChildren().get(1));
+    }
+
+    public void convert(JsonElement value, ColumnVector vect, int row) {
+      if (value == null || value.isJsonNull()) {
+        vect.noNulls = false;
+        vect.isNull[row] = true;
+      } else {
+        MapColumnVector vector = (MapColumnVector) vect;
+        JsonObject obj = value.getAsJsonObject();
+        vector.lengths[row] = obj.entrySet().size();
+        vector.offsets[row] = vector.childCount;
+        vector.childCount += vector.lengths[row];
+        vector.keys.ensureSize(vector.childCount, true);
+        vector.values.ensureSize(vector.childCount, true);
+        int cnt = 0;
+        for (Map.Entry<String, JsonElement> entry : obj.entrySet()) {
+          int offset = (int) vector.offsets[row] + cnt++;
+          keyConverter.convert(new JsonPrimitive(entry.getKey()), vector.keys, 
offset);
+          valueConverter.convert(entry.getValue(), vector.values, offset);
+        }
+      }
+    }
+  }
+
   static JsonConverter createConverter(TypeDescription schema) {
     switch (schema.getCategory()) {
       case BYTE:
@@ -248,6 +285,8 @@ public class JsonReader implements RecordReader {
         return new StructColumnConverter(schema);
       case LIST:
         return new ListColumnConverter(schema);
+      case MAP:
+        return new MapColumnConverter(schema);
       default:
         throw new IllegalArgumentException("Unhandled type " + schema);
     }

Reply via email to