[ 
https://issues.apache.org/jira/browse/HAWQ-178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15124186#comment-15124186
 ] 

ASF GitHub Bot commented on HAWQ-178:
-------------------------------------

Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51313817
  
    --- Diff: 
pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java 
---
    @@ -0,0 +1,259 @@
    +package org.apache.hawq.pxf.plugins.json;
    +
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + * 
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + * 
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.api.utilities.Plugin;
    +import org.codehaus.jackson.JsonNode;
    +
    +/**
    + * This JSON resolver for PXF will decode a given object from the {@link 
JsonAccessor} into a row for HAWQ. It will
    + * decode this data into a JsonNode and walk the tree for each column. It 
supports normal value mapping via projections
    + * and JSON array indexing.
    + */
    +public class JsonResolver extends Plugin implements ReadResolver {
    +
    +   private ArrayList<OneField> list = new ArrayList<OneField>();
    +
    +   public JsonResolver(InputData inputData) throws Exception {
    +           super(inputData);
    +   }
    +
    +   @Override
    +   public List<OneField> getFields(OneRow row) throws Exception {
    +           list.clear();
    +
    +           // key is a Text object
    +           JsonNode root = 
JsonInputFormat.decodeLineToJsonNode(row.getKey().toString());
    +
    +           // if we weren't given a null object
    +           if (root != null) {
    +                   // Iterate through the column definition and fetch our 
JSON data
    +                   for (int i = 0; i < inputData.getColumns(); ++i) {
    +
    +                           // Get the current column description
    +                           ColumnDescriptor cd = inputData.getColumn(i);
    +                           DataType columnType = 
DataType.get(cd.columnTypeCode());
    +
    +                           // Get the JSON projections from the column name
    +                           // For example, "user.name" turns into 
["user","name"]
    +                           String[] projs = cd.columnName().split("\\.");
    +
    +                           // Move down the JSON path to the final name
    +                           JsonNode node = getPriorJsonNode(root, projs);
    +
    +                           // If this column is an array index, ex. 
"tweet.hashtags[0]"
    +                           if (isArrayIndex(projs)) {
    +
    +                                   // Get the node name and index
    +                                   String nodeName = getArrayName(projs);
    +                                   int arrayIndex = getArrayIndex(projs);
    +
    +                                   // Move to the array node
    +                                   node = node.get(nodeName);
    +
    +                                   // If this node is null or missing, add 
a null value here
    +                                   if (node == null || 
node.isMissingNode()) {
    +                                           addNullField(columnType);
    +                                   } else if (node.isArray()) {
    +                                           // If the JSON node is an 
array, then add it to our list
    +                                           
addFieldFromJsonArray(columnType, node, arrayIndex);
    +                                   } else {
    +                                           throw new 
InvalidParameterException(nodeName + " is not an array node");
    +                                   }
    +                           } else {
    +                                   // This column is not an array type
    +                                   // Move to the final node
    +                                   node = node.get(projs[projs.length - 
1]);
    +
    +                                   // If this node is null or missing, add 
a null value here
    +                                   if (node == null || 
node.isMissingNode()) {
    +                                           addNullField(columnType);
    +                                   } else {
    +                                           // Else, add the value to the 
record
    +                                           
addFieldFromJsonNode(columnType, node);
    +                                   }
    +                           }
    +                   }
    +           }
    +
    +           return list;
    +   }
    +
    +   /**
    +    * Iterates down the root node to the prior JSON node. This node is used
    +    * 
    +    * @param root
    +    * @param projs
    +    * @return
    +    */
    +   private JsonNode getPriorJsonNode(JsonNode root, String[] projs) {
    +
    +           // Iterate through all the tokens to the desired JSON node
    +           JsonNode node = root;
    +           for (int j = 0; j < projs.length - 1; ++j) {
    +                   node = node.path(projs[j]);
    +           }
    +
    +           return node;
    +   }
    +
    +   /**
    +    * Gets a boolean value indicating if this column is an array index 
column
    +    * 
    +    * @param projs
    +    *            The array of JSON projections
    +    * @throws ArrayIndexOutOfBoundsException
    +    */
    +   private boolean isArrayIndex(String[] projs) {
    +           return projs[projs.length - 1].contains("[") && 
projs[projs.length - 1].contains("]");
    +   }
    +
    +   /**
    +    * Gets the node name from the given String array of JSON projections, 
parsed from the ColumnDescriptor's
    +    * 
    +    * @param projs
    +    *            The array of JSON projections
    +    * @return The name
    +    * @throws ArrayIndexOutOfBoundsException
    +    */
    +   private String getArrayName(String[] projs) {
    +           return projs[projs.length - 1].replaceAll("\\[[0-9]+\\]", "");
    +   }
    +
    +   /**
    +    * Gets the array index from the given String array of JSON 
projections, parsed from the ColumnDescriptor's name
    +    * 
    +    * @param projs
    +    *            The array of JSON projections
    +    * @return The index
    +    * @throws ArrayIndexOutOfBoundsException
    +    */
    +   private int getArrayIndex(String[] projs) {
    +           return Integer.parseInt(projs[projs.length - 
1].substring(projs[projs.length - 1].indexOf('[') + 1,
    --- End diff --
    
    perhaps use a variable for `String arrayElem = projs[projs.length - 1]` - 
will make the code shorter.


> Add JSON plugin support in code base
> ------------------------------------
>
>                 Key: HAWQ-178
>                 URL: https://issues.apache.org/jira/browse/HAWQ-178
>             Project: Apache HAWQ
>          Issue Type: New Feature
>          Components: PXF
>            Reporter: Goden Yao
>            Assignee: Goden Yao
>             Fix For: backlog
>
>         Attachments: PXFJSONPluginforHAWQ2.0andPXF3.0.0.pdf
>
>
> JSON has been a popular format used in HDFS as well as in the community, 
> there has been a few JSON PXF plugins developed by the community and we'd 
> like to see it being incorporated into the code base as an optional package.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to