2020import com .google .gson .JsonArray ;
2121import com .google .gson .JsonElement ;
2222import com .google .gson .JsonObject ;
23+ import com .google .gson .JsonPrimitive ;
2324import com .google .gson .JsonStreamParser ;
2425import org .apache .hadoop .conf .Configuration ;
2526import org .apache .hadoop .fs .FSDataInputStream ;
3334import org .apache .hadoop .hive .ql .exec .vector .DoubleColumnVector ;
3435import org .apache .hadoop .hive .ql .exec .vector .ListColumnVector ;
3536import org .apache .hadoop .hive .ql .exec .vector .LongColumnVector ;
37+ import org .apache .hadoop .hive .ql .exec .vector .MapColumnVector ;
3638import org .apache .hadoop .hive .ql .exec .vector .StructColumnVector ;
3739import org .apache .hadoop .hive .ql .exec .vector .TimestampColumnVector ;
3840import org .apache .hadoop .hive .ql .exec .vector .VectorizedRowBatch ;
5254import java .sql .Timestamp ;
5355import java .util .Iterator ;
5456import java .util .List ;
57+ import java .util .Map ;
5558import java .util .zip .GZIPInputStream ;
5659
5760public class JsonReader implements RecordReader {
@@ -222,6 +225,40 @@ public void convert(JsonElement value, ColumnVector vect, int row) {
222225 }
223226 }
224227
228+ static class MapColumnConverter implements JsonConverter {
229+ private JsonConverter keyConverter ;
230+ private JsonConverter valueConverter ;
231+
232+ public MapColumnConverter (TypeDescription schema ) {
233+ TypeDescription keyType = schema .getChildren ().get (0 );
234+ if (keyType .getCategory () != TypeDescription .Category .STRING )
235+ throw new IllegalArgumentException ("JSON can only support MAP key in STRING type: " + schema );
236+ keyConverter = createConverter (keyType );
237+ valueConverter = createConverter (schema .getChildren ().get (1 ));
238+ }
239+
240+ public void convert (JsonElement value , ColumnVector vect , int row ) {
241+ if (value == null || value .isJsonNull ()) {
242+ vect .noNulls = false ;
243+ vect .isNull [row ] = true ;
244+ } else {
245+ MapColumnVector vector = (MapColumnVector ) vect ;
246+ JsonObject obj = value .getAsJsonObject ();
247+ vector .lengths [row ] = obj .entrySet ().size ();
248+ vector .offsets [row ] = vector .childCount ;
249+ vector .childCount += vector .lengths [row ];
250+ vector .keys .ensureSize (vector .childCount , true );
251+ vector .values .ensureSize (vector .childCount , true );
252+ int cnt = 0 ;
253+ for (Map .Entry <String , JsonElement > entry : obj .entrySet ()) {
254+ int offset = (int ) vector .offsets [row ] + cnt ++;
255+ keyConverter .convert (new JsonPrimitive (entry .getKey ()), vector .keys , offset );
256+ valueConverter .convert (entry .getValue (), vector .values , offset );
257+ }
258+ }
259+ }
260+ }
261+
225262 static JsonConverter createConverter (TypeDescription schema ) {
226263 switch (schema .getCategory ()) {
227264 case BYTE :
@@ -248,6 +285,8 @@ static JsonConverter createConverter(TypeDescription schema) {
248285 return new StructColumnConverter (schema );
249286 case LIST :
250287 return new ListColumnConverter (schema );
288+ case MAP :
289+ return new MapColumnConverter (schema );
251290 default :
252291 throw new IllegalArgumentException ("Unhandled type " + schema );
253292 }
0 commit comments